/
KTable.java
2369 lines (2312 loc) · 136 KB
/
KTable.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams.kstream;
import org.apache.kafka.common.serialization.Serde;
import org.apache.kafka.common.utils.Bytes;
import org.apache.kafka.streams.KafkaStreams;
import org.apache.kafka.streams.KeyValue;
import org.apache.kafka.streams.StoreQueryParameters;
import org.apache.kafka.streams.StreamsBuilder;
import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.Topology;
import org.apache.kafka.streams.processor.ProcessorContext;
import org.apache.kafka.streams.processor.StateStore;
import org.apache.kafka.streams.processor.StreamPartitioner;
import org.apache.kafka.streams.state.KeyValueBytesStoreSupplier;
import org.apache.kafka.streams.state.KeyValueStore;
import org.apache.kafka.streams.state.ReadOnlyKeyValueStore;
import java.util.function.Function;
/**
* {@code KTable} is an abstraction of a <i>changelog stream</i> from a primary-keyed table.
* Each record in this changelog stream is an update on the primary-keyed table with the record key as the primary key.
* <p>
* A {@code KTable} is either {@link StreamsBuilder#table(String) defined from a single Kafka topic} that is
* consumed message by message or the result of a {@code KTable} transformation.
* An aggregation of a {@link KStream} also yields a {@code KTable}.
* <p>
* A {@code KTable} can be transformed record by record, joined with another {@code KTable} or {@link KStream}, or
* can be re-partitioned and aggregated into a new {@code KTable}.
* <p>
* Some {@code KTable}s have an internal state (a {@link ReadOnlyKeyValueStore}) and are therefore queryable via the
* interactive queries API.
* For example:
* <pre>{@code
* final KTable table = ...
* ...
* final KafkaStreams streams = ...;
* streams.start()
* ...
* final String queryableStoreName = table.queryableStoreName(); // returns null if KTable is not queryable
* ReadOnlyKeyValueStore view = streams.store(queryableStoreName, QueryableStoreTypes.timestampedKeyValueStore());
* view.get(key);
*}</pre>
*<p>
* Records from the source topic that have null keys are dropped.
*
* @param <K> Type of primary keys
* @param <V> Type of value changes
* @see KStream
* @see KGroupedTable
* @see GlobalKTable
* @see StreamsBuilder#table(String)
*/
public interface KTable<K, V> {
/**
* Create a new {@code KTable} that consists of all records of this {@code KTable} which satisfy the given
* predicate, with default serializers, deserializers, and state store.
* All records that do not satisfy the predicate are dropped.
* For each {@code KTable} update, the filter is evaluated based on the current update
* record and then an update record is produced for the result {@code KTable}.
* This is a stateless record-by-record operation.
* <p>
* Note that {@code filter} for a <i>changelog stream</i> works differently than {@link KStream#filter(Predicate)
* record stream filters}, because {@link KeyValue records} with {@code null} values (so-called tombstone records)
* have delete semantics.
* Thus, for tombstones the provided filter predicate is not evaluated but the tombstone record is forwarded
* directly if required (i.e., if there is anything to be deleted).
* Furthermore, for each record that gets dropped (i.e., does not satisfy the given predicate) a tombstone record
* is forwarded.
*
* @param predicate a filter {@link Predicate} that is applied to each record
* @return a {@code KTable} that contains only those records that satisfy the given predicate
* @see #filterNot(Predicate)
*/
KTable<K, V> filter(final Predicate<? super K, ? super V> predicate);
/**
* Create a new {@code KTable} that consists of all records of this {@code KTable} which satisfy the given
* predicate, with default serializers, deserializers, and state store.
* All records that do not satisfy the predicate are dropped.
* For each {@code KTable} update, the filter is evaluated based on the current update
* record and then an update record is produced for the result {@code KTable}.
* This is a stateless record-by-record operation.
* <p>
* Note that {@code filter} for a <i>changelog stream</i> works differently than {@link KStream#filter(Predicate)
* record stream filters}, because {@link KeyValue records} with {@code null} values (so-called tombstone records)
* have delete semantics.
* Thus, for tombstones the provided filter predicate is not evaluated but the tombstone record is forwarded
* directly if required (i.e., if there is anything to be deleted).
* Furthermore, for each record that gets dropped (i.e., does not satisfy the given predicate) a tombstone record
* is forwarded.
*
* @param predicate a filter {@link Predicate} that is applied to each record
* @param named a {@link Named} config used to name the processor in the topology
* @return a {@code KTable} that contains only those records that satisfy the given predicate
* @see #filterNot(Predicate)
*/
KTable<K, V> filter(final Predicate<? super K, ? super V> predicate, final Named named);
/**
* Create a new {@code KTable} that consists of all records of this {@code KTable} which satisfy the given
* predicate, with the {@link Serde key serde}, {@link Serde value serde}, and the underlying
* {@link KeyValueStore materialized state storage} configured in the {@link Materialized} instance.
* All records that do not satisfy the predicate are dropped.
* For each {@code KTable} update, the filter is evaluated based on the current update
* record and then an update record is produced for the result {@code KTable}.
* This is a stateless record-by-record operation.
* <p>
* Note that {@code filter} for a <i>changelog stream</i> works differently than {@link KStream#filter(Predicate)
* record stream filters}, because {@link KeyValue records} with {@code null} values (so-called tombstone records)
* have delete semantics.
* Thus, for tombstones the provided filter predicate is not evaluated but the tombstone record is forwarded
* directly if required (i.e., if there is anything to be deleted).
* Furthermore, for each record that gets dropped (i.e., does not satisfy the given predicate) a tombstone record
* is forwarded.
* <p>
* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via
* {@link KafkaStreams#store(StoreQueryParameters) KafkaStreams#store(...)}:
* <pre>{@code
* KafkaStreams streams = ... // filtering words
* ReadOnlyKeyValueStore<K, ValueAndTimestamp<V>> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<K, ValueAndTimestamp<V>>timestampedKeyValueStore());
* K key = "some-word";
* ValueAndTimestamp<V> valueForKey = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
* }</pre>
* For non-local keys, a custom RPC mechanism must be implemented using {@link KafkaStreams#metadataForAllStreamsClients()} to
* query the value of the key on a parallel running instance of your Kafka Streams application.
* The store name to query with is specified by {@link Materialized#as(String)} or {@link Materialized#as(KeyValueBytesStoreSupplier)}.
* <p>
*
* @param predicate a filter {@link Predicate} that is applied to each record
* @param materialized a {@link Materialized} that describes how the {@link StateStore} for the resulting {@code KTable}
* should be materialized. Cannot be {@code null}
* @return a {@code KTable} that contains only those records that satisfy the given predicate
* @see #filterNot(Predicate, Materialized)
*/
KTable<K, V> filter(final Predicate<? super K, ? super V> predicate,
final Materialized<K, V, KeyValueStore<Bytes, byte[]>> materialized);
/**
* Create a new {@code KTable} that consists of all records of this {@code KTable} which satisfy the given
* predicate, with the {@link Serde key serde}, {@link Serde value serde}, and the underlying
* {@link KeyValueStore materialized state storage} configured in the {@link Materialized} instance.
* All records that do not satisfy the predicate are dropped.
* For each {@code KTable} update, the filter is evaluated based on the current update
* record and then an update record is produced for the result {@code KTable}.
* This is a stateless record-by-record operation.
* <p>
* Note that {@code filter} for a <i>changelog stream</i> works differently than {@link KStream#filter(Predicate)
* record stream filters}, because {@link KeyValue records} with {@code null} values (so-called tombstone records)
* have delete semantics.
* Thus, for tombstones the provided filter predicate is not evaluated but the tombstone record is forwarded
* directly if required (i.e., if there is anything to be deleted).
* Furthermore, for each record that gets dropped (i.e., does not satisfy the given predicate) a tombstone record
* is forwarded.
* <p>
* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via
* {@link KafkaStreams#store(StoreQueryParameters) KafkaStreams#store(...)}:
* <pre>{@code
* KafkaStreams streams = ... // filtering words
* ReadOnlyKeyValueStore<K, ValueAndTimestamp<V>> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<K, ValueAndTimestamp<V>>timestampedKeyValueStore());
* K key = "some-word";
* ValueAndTimestamp<V> valueForKey = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
* }</pre>
* For non-local keys, a custom RPC mechanism must be implemented using {@link KafkaStreams#metadataForAllStreamsClients()} to
* query the value of the key on a parallel running instance of your Kafka Streams application.
* The store name to query with is specified by {@link Materialized#as(String)} or {@link Materialized#as(KeyValueBytesStoreSupplier)}.
* <p>
*
* @param predicate a filter {@link Predicate} that is applied to each record
* @param named a {@link Named} config used to name the processor in the topology
* @param materialized a {@link Materialized} that describes how the {@link StateStore} for the resulting {@code KTable}
* should be materialized. Cannot be {@code null}
* @return a {@code KTable} that contains only those records that satisfy the given predicate
* @see #filterNot(Predicate, Materialized)
*/
KTable<K, V> filter(final Predicate<? super K, ? super V> predicate,
final Named named,
final Materialized<K, V, KeyValueStore<Bytes, byte[]>> materialized);
/**
* Create a new {@code KTable} that consists all records of this {@code KTable} which do <em>not</em> satisfy the
* given predicate, with default serializers, deserializers, and state store.
* All records that <em>do</em> satisfy the predicate are dropped.
* For each {@code KTable} update, the filter is evaluated based on the current update
* record and then an update record is produced for the result {@code KTable}.
* This is a stateless record-by-record operation.
* <p>
* Note that {@code filterNot} for a <i>changelog stream</i> works differently than {@link KStream#filterNot(Predicate)
* record stream filters}, because {@link KeyValue records} with {@code null} values (so-called tombstone records)
* have delete semantics.
* Thus, for tombstones the provided filter predicate is not evaluated but the tombstone record is forwarded
* directly if required (i.e., if there is anything to be deleted).
* Furthermore, for each record that gets dropped (i.e., does satisfy the given predicate) a tombstone record is
* forwarded.
*
* @param predicate a filter {@link Predicate} that is applied to each record
* @return a {@code KTable} that contains only those records that do <em>not</em> satisfy the given predicate
* @see #filter(Predicate)
*/
KTable<K, V> filterNot(final Predicate<? super K, ? super V> predicate);
/**
* Create a new {@code KTable} that consists all records of this {@code KTable} which do <em>not</em> satisfy the
* given predicate, with default serializers, deserializers, and state store.
* All records that <em>do</em> satisfy the predicate are dropped.
* For each {@code KTable} update, the filter is evaluated based on the current update
* record and then an update record is produced for the result {@code KTable}.
* This is a stateless record-by-record operation.
* <p>
* Note that {@code filterNot} for a <i>changelog stream</i> works differently than {@link KStream#filterNot(Predicate)
* record stream filters}, because {@link KeyValue records} with {@code null} values (so-called tombstone records)
* have delete semantics.
* Thus, for tombstones the provided filter predicate is not evaluated but the tombstone record is forwarded
* directly if required (i.e., if there is anything to be deleted).
* Furthermore, for each record that gets dropped (i.e., does satisfy the given predicate) a tombstone record is
* forwarded.
*
* @param predicate a filter {@link Predicate} that is applied to each record
* @param named a {@link Named} config used to name the processor in the topology
* @return a {@code KTable} that contains only those records that do <em>not</em> satisfy the given predicate
* @see #filter(Predicate)
*/
KTable<K, V> filterNot(final Predicate<? super K, ? super V> predicate, final Named named);
/**
* Create a new {@code KTable} that consists all records of this {@code KTable} which do <em>not</em> satisfy the
* given predicate, with the {@link Serde key serde}, {@link Serde value serde}, and the underlying
* {@link KeyValueStore materialized state storage} configured in the {@link Materialized} instance.
* All records that <em>do</em> satisfy the predicate are dropped.
* For each {@code KTable} update, the filter is evaluated based on the current update
* record and then an update record is produced for the result {@code KTable}.
* This is a stateless record-by-record operation.
* <p>
* Note that {@code filterNot} for a <i>changelog stream</i> works differently than {@link KStream#filterNot(Predicate)
* record stream filters}, because {@link KeyValue records} with {@code null} values (so-called tombstone records)
* have delete semantics.
* Thus, for tombstones the provided filter predicate is not evaluated but the tombstone record is forwarded
* directly if required (i.e., if there is anything to be deleted).
* Furthermore, for each record that gets dropped (i.e., does satisfy the given predicate) a tombstone record is
* forwarded.
* <p>
* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via
* {@link KafkaStreams#store(StoreQueryParameters) KafkaStreams#store(...)}:
* <pre>{@code
* KafkaStreams streams = ... // filtering words
* ReadOnlyKeyValueStore<K, ValueAndTimestamp<V>> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<K, ValueAndTimestamp<V>>timestampedKeyValueStore());
* K key = "some-word";
* ValueAndTimestamp<V> valueForKey = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
* }</pre>
* For non-local keys, a custom RPC mechanism must be implemented using {@link KafkaStreams#metadataForAllStreamsClients()} to
* query the value of the key on a parallel running instance of your Kafka Streams application.
* The store name to query with is specified by {@link Materialized#as(String)} or {@link Materialized#as(KeyValueBytesStoreSupplier)}.
* <p>
* @param predicate a filter {@link Predicate} that is applied to each record
* @param materialized a {@link Materialized} that describes how the {@link StateStore} for the resulting {@code KTable}
* should be materialized. Cannot be {@code null}
* @return a {@code KTable} that contains only those records that do <em>not</em> satisfy the given predicate
* @see #filter(Predicate, Materialized)
*/
KTable<K, V> filterNot(final Predicate<? super K, ? super V> predicate,
final Materialized<K, V, KeyValueStore<Bytes, byte[]>> materialized);
/**
* Create a new {@code KTable} that consists all records of this {@code KTable} which do <em>not</em> satisfy the
* given predicate, with the {@link Serde key serde}, {@link Serde value serde}, and the underlying
* {@link KeyValueStore materialized state storage} configured in the {@link Materialized} instance.
* All records that <em>do</em> satisfy the predicate are dropped.
* For each {@code KTable} update, the filter is evaluated based on the current update
* record and then an update record is produced for the result {@code KTable}.
* This is a stateless record-by-record operation.
* <p>
* Note that {@code filterNot} for a <i>changelog stream</i> works differently than {@link KStream#filterNot(Predicate)
* record stream filters}, because {@link KeyValue records} with {@code null} values (so-called tombstone records)
* have delete semantics.
* Thus, for tombstones the provided filter predicate is not evaluated but the tombstone record is forwarded
* directly if required (i.e., if there is anything to be deleted).
* Furthermore, for each record that gets dropped (i.e., does satisfy the given predicate) a tombstone record is
* forwarded.
* <p>
* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via
* {@link KafkaStreams#store(StoreQueryParameters) KafkaStreams#store(...)}:
* <pre>{@code
* KafkaStreams streams = ... // filtering words
* ReadOnlyKeyValueStore<K, ValueAndTimestamp<V>> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<K, ValueAndTimestamp<V>>timestampedKeyValueStore());
* K key = "some-word";
* ValueAndTimestamp<V> valueForKey = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
* }</pre>
* For non-local keys, a custom RPC mechanism must be implemented using {@link KafkaStreams#metadataForAllStreamsClients()} to
* query the value of the key on a parallel running instance of your Kafka Streams application.
* The store name to query with is specified by {@link Materialized#as(String)} or {@link Materialized#as(KeyValueBytesStoreSupplier)}.
* <p>
* @param predicate a filter {@link Predicate} that is applied to each record
* @param named a {@link Named} config used to name the processor in the topology
* @param materialized a {@link Materialized} that describes how the {@link StateStore} for the resulting {@code KTable}
* should be materialized. Cannot be {@code null}
* @return a {@code KTable} that contains only those records that do <em>not</em> satisfy the given predicate
* @see #filter(Predicate, Materialized)
*/
KTable<K, V> filterNot(final Predicate<? super K, ? super V> predicate,
final Named named,
final Materialized<K, V, KeyValueStore<Bytes, byte[]>> materialized);
/**
* Create a new {@code KTable} by transforming the value of each record in this {@code KTable} into a new value
* (with possibly a new type) in the new {@code KTable}, with default serializers, deserializers, and state store.
* For each {@code KTable} update the provided {@link ValueMapper} is applied to the value of the updated record and
* computes a new value for it, resulting in an updated record for the result {@code KTable}.
* Thus, an input record {@code <K,V>} can be transformed into an output record {@code <K:V'>}.
* This is a stateless record-by-record operation.
* <p>
* The example below counts the number of token of the value string.
* <pre>{@code
* KTable<String, String> inputTable = builder.table("topic");
* KTable<String, Integer> outputTable = inputTable.mapValues(value -> value.split(" ").length);
* }</pre>
* <p>
* This operation preserves data co-location with respect to the key.
* Thus, <em>no</em> internal data redistribution is required if a key based operator (like a join) is applied to
* the result {@code KTable}.
* <p>
* Note that {@code mapValues} for a <i>changelog stream</i> works differently than {@link KStream#mapValues(ValueMapper)
* record stream filters}, because {@link KeyValue records} with {@code null} values (so-called tombstone records)
* have delete semantics.
* Thus, for tombstones the provided value-mapper is not evaluated but the tombstone record is forwarded directly to
* delete the corresponding record in the result {@code KTable}.
*
* @param mapper a {@link ValueMapper} that computes a new output value
* @param <VR> the value type of the result {@code KTable}
* @return a {@code KTable} that contains records with unmodified keys and new values (possibly of different type)
*/
<VR> KTable<K, VR> mapValues(final ValueMapper<? super V, ? extends VR> mapper);
/**
* Create a new {@code KTable} by transforming the value of each record in this {@code KTable} into a new value
* (with possibly a new type) in the new {@code KTable}, with default serializers, deserializers, and state store.
* For each {@code KTable} update the provided {@link ValueMapper} is applied to the value of the updated record and
* computes a new value for it, resulting in an updated record for the result {@code KTable}.
* Thus, an input record {@code <K,V>} can be transformed into an output record {@code <K:V'>}.
* This is a stateless record-by-record operation.
* <p>
* The example below counts the number of token of the value string.
* <pre>{@code
* KTable<String, String> inputTable = builder.table("topic");
* KTable<String, Integer> outputTable = inputTable.mapValues(value -> value.split(" ").length, Named.as("countTokenValue"));
* }</pre>
* <p>
* This operation preserves data co-location with respect to the key.
* Thus, <em>no</em> internal data redistribution is required if a key based operator (like a join) is applied to
* the result {@code KTable}.
* <p>
* Note that {@code mapValues} for a <i>changelog stream</i> works differently than {@link KStream#mapValues(ValueMapper)
* record stream filters}, because {@link KeyValue records} with {@code null} values (so-called tombstone records)
* have delete semantics.
* Thus, for tombstones the provided value-mapper is not evaluated but the tombstone record is forwarded directly to
* delete the corresponding record in the result {@code KTable}.
*
* @param mapper a {@link ValueMapper} that computes a new output value
* @param named a {@link Named} config used to name the processor in the topology
* @param <VR> the value type of the result {@code KTable}
* @return a {@code KTable} that contains records with unmodified keys and new values (possibly of different type)
*/
<VR> KTable<K, VR> mapValues(final ValueMapper<? super V, ? extends VR> mapper,
final Named named);
/**
* Create a new {@code KTable} by transforming the value of each record in this {@code KTable} into a new value
* (with possibly a new type) in the new {@code KTable}, with default serializers, deserializers, and state store.
* For each {@code KTable} update the provided {@link ValueMapperWithKey} is applied to the value of the update
* record and computes a new value for it, resulting in an updated record for the result {@code KTable}.
* Thus, an input record {@code <K,V>} can be transformed into an output record {@code <K:V'>}.
* This is a stateless record-by-record operation.
* <p>
* The example below counts the number of token of value and key strings.
* <pre>{@code
* KTable<String, String> inputTable = builder.table("topic");
* KTable<String, Integer> outputTable =
* inputTable.mapValues((readOnlyKey, value) -> readOnlyKey.split(" ").length + value.split(" ").length);
* }</pre>
* <p>
* Note that the key is read-only and should not be modified, as this can lead to corrupt partitioning.
* This operation preserves data co-location with respect to the key.
* Thus, <em>no</em> internal data redistribution is required if a key based operator (like a join) is applied to
* the result {@code KTable}.
* <p>
* Note that {@code mapValues} for a <i>changelog stream</i> works differently than {@link KStream#mapValues(ValueMapperWithKey)
* record stream filters}, because {@link KeyValue records} with {@code null} values (so-called tombstone records)
* have delete semantics.
* Thus, for tombstones the provided value-mapper is not evaluated but the tombstone record is forwarded directly to
* delete the corresponding record in the result {@code KTable}.
*
* @param mapper a {@link ValueMapperWithKey} that computes a new output value
* @param <VR> the value type of the result {@code KTable}
* @return a {@code KTable} that contains records with unmodified keys and new values (possibly of different type)
*/
<VR> KTable<K, VR> mapValues(final ValueMapperWithKey<? super K, ? super V, ? extends VR> mapper);
/**
* Create a new {@code KTable} by transforming the value of each record in this {@code KTable} into a new value
* (with possibly a new type) in the new {@code KTable}, with default serializers, deserializers, and state store.
* For each {@code KTable} update the provided {@link ValueMapperWithKey} is applied to the value of the update
* record and computes a new value for it, resulting in an updated record for the result {@code KTable}.
* Thus, an input record {@code <K,V>} can be transformed into an output record {@code <K:V'>}.
* This is a stateless record-by-record operation.
* <p>
* The example below counts the number of token of value and key strings.
* <pre>{@code
* KTable<String, String> inputTable = builder.table("topic");
* KTable<String, Integer> outputTable =
* inputTable.mapValues((readOnlyKey, value) -> readOnlyKey.split(" ").length + value.split(" ").length, Named.as("countTokenValueAndKey"));
* }</pre>
* <p>
* Note that the key is read-only and should not be modified, as this can lead to corrupt partitioning.
* This operation preserves data co-location with respect to the key.
* Thus, <em>no</em> internal data redistribution is required if a key based operator (like a join) is applied to
* the result {@code KTable}.
* <p>
* Note that {@code mapValues} for a <i>changelog stream</i> works differently than {@link KStream#mapValues(ValueMapperWithKey)
* record stream filters}, because {@link KeyValue records} with {@code null} values (so-called tombstone records)
* have delete semantics.
* Thus, for tombstones the provided value-mapper is not evaluated but the tombstone record is forwarded directly to
* delete the corresponding record in the result {@code KTable}.
*
* @param mapper a {@link ValueMapperWithKey} that computes a new output value
* @param named a {@link Named} config used to name the processor in the topology
* @param <VR> the value type of the result {@code KTable}
* @return a {@code KTable} that contains records with unmodified keys and new values (possibly of different type)
*/
<VR> KTable<K, VR> mapValues(final ValueMapperWithKey<? super K, ? super V, ? extends VR> mapper,
final Named named);
/**
* Create a new {@code KTable} by transforming the value of each record in this {@code KTable} into a new value
* (with possibly a new type) in the new {@code KTable}, with the {@link Serde key serde}, {@link Serde value serde},
* and the underlying {@link KeyValueStore materialized state storage} configured in the {@link Materialized}
* instance.
* For each {@code KTable} update the provided {@link ValueMapper} is applied to the value of the updated record and
* computes a new value for it, resulting in an updated record for the result {@code KTable}.
* Thus, an input record {@code <K,V>} can be transformed into an output record {@code <K:V'>}.
* This is a stateless record-by-record operation.
* <p>
* The example below counts the number of token of the value string.
* <pre>{@code
* KTable<String, String> inputTable = builder.table("topic");
* KTable<String, Integer> outputTable = inputTable.mapValue(new ValueMapper<String, Integer> {
* Integer apply(String value) {
* return value.split(" ").length;
* }
* });
* }</pre>
* <p>
* To query the local {@link KeyValueStore} representing outputTable above it must be obtained via
* {@link KafkaStreams#store(StoreQueryParameters) KafkaStreams#store(...)}:
* For non-local keys, a custom RPC mechanism must be implemented using {@link KafkaStreams#metadataForAllStreamsClients()} to
* query the value of the key on a parallel running instance of your Kafka Streams application.
* The store name to query with is specified by {@link Materialized#as(String)} or {@link Materialized#as(KeyValueBytesStoreSupplier)}.
* <p>
* This operation preserves data co-location with respect to the key.
* Thus, <em>no</em> internal data redistribution is required if a key based operator (like a join) is applied to
* the result {@code KTable}.
* <p>
* Note that {@code mapValues} for a <i>changelog stream</i> works differently than {@link KStream#mapValues(ValueMapper)
* record stream filters}, because {@link KeyValue records} with {@code null} values (so-called tombstone records)
* have delete semantics.
* Thus, for tombstones the provided value-mapper is not evaluated but the tombstone record is forwarded directly to
* delete the corresponding record in the result {@code KTable}.
*
* @param mapper a {@link ValueMapper} that computes a new output value
* @param materialized a {@link Materialized} that describes how the {@link StateStore} for the resulting {@code KTable}
* should be materialized. Cannot be {@code null}
* @param <VR> the value type of the result {@code KTable}
*
* @return a {@code KTable} that contains records with unmodified keys and new values (possibly of different type)
*/
<VR> KTable<K, VR> mapValues(final ValueMapper<? super V, ? extends VR> mapper,
final Materialized<K, VR, KeyValueStore<Bytes, byte[]>> materialized);
/**
* Create a new {@code KTable} by transforming the value of each record in this {@code KTable} into a new value
* (with possibly a new type) in the new {@code KTable}, with the {@link Serde key serde}, {@link Serde value serde},
* and the underlying {@link KeyValueStore materialized state storage} configured in the {@link Materialized}
* instance.
* For each {@code KTable} update the provided {@link ValueMapper} is applied to the value of the updated record and
* computes a new value for it, resulting in an updated record for the result {@code KTable}.
* Thus, an input record {@code <K,V>} can be transformed into an output record {@code <K:V'>}.
* This is a stateless record-by-record operation.
* <p>
* The example below counts the number of token of the value string.
* <pre>{@code
* KTable<String, String> inputTable = builder.table("topic");
* KTable<String, Integer> outputTable = inputTable.mapValue(new ValueMapper<String, Integer> {
* Integer apply(String value) {
* return value.split(" ").length;
* }
* });
* }</pre>
* <p>
* To query the local {@link KeyValueStore} representing outputTable above it must be obtained via
* {@link KafkaStreams#store(StoreQueryParameters) KafkaStreams#store(...)}:
* For non-local keys, a custom RPC mechanism must be implemented using {@link KafkaStreams#metadataForAllStreamsClients()} to
* query the value of the key on a parallel running instance of your Kafka Streams application.
* The store name to query with is specified by {@link Materialized#as(String)} or {@link Materialized#as(KeyValueBytesStoreSupplier)}.
* <p>
* This operation preserves data co-location with respect to the key.
* Thus, <em>no</em> internal data redistribution is required if a key based operator (like a join) is applied to
* the result {@code KTable}.
* <p>
* Note that {@code mapValues} for a <i>changelog stream</i> works differently than {@link KStream#mapValues(ValueMapper)
* record stream filters}, because {@link KeyValue records} with {@code null} values (so-called tombstone records)
* have delete semantics.
* Thus, for tombstones the provided value-mapper is not evaluated but the tombstone record is forwarded directly to
* delete the corresponding record in the result {@code KTable}.
*
* @param mapper a {@link ValueMapper} that computes a new output value
* @param named a {@link Named} config used to name the processor in the topology
* @param materialized a {@link Materialized} that describes how the {@link StateStore} for the resulting {@code KTable}
* should be materialized. Cannot be {@code null}
* @param <VR> the value type of the result {@code KTable}
*
* @return a {@code KTable} that contains records with unmodified keys and new values (possibly of different type)
*/
<VR> KTable<K, VR> mapValues(final ValueMapper<? super V, ? extends VR> mapper,
final Named named,
final Materialized<K, VR, KeyValueStore<Bytes, byte[]>> materialized);
/**
* Create a new {@code KTable} by transforming the value of each record in this {@code KTable} into a new value
* (with possibly a new type) in the new {@code KTable}, with the {@link Serde key serde}, {@link Serde value serde},
* and the underlying {@link KeyValueStore materialized state storage} configured in the {@link Materialized}
* instance.
* For each {@code KTable} update the provided {@link ValueMapperWithKey} is applied to the value of the update
* record and computes a new value for it, resulting in an updated record for the result {@code KTable}.
* Thus, an input record {@code <K,V>} can be transformed into an output record {@code <K:V'>}.
* This is a stateless record-by-record operation.
* <p>
* The example below counts the number of token of value and key strings.
* <pre>{@code
* KTable<String, String> inputTable = builder.table("topic");
* KTable<String, Integer> outputTable = inputTable.mapValue(new ValueMapperWithKey<String, String, Integer> {
* Integer apply(String readOnlyKey, String value) {
* return readOnlyKey.split(" ").length + value.split(" ").length;
* }
* });
* }</pre>
* <p>
* To query the local {@link KeyValueStore} representing outputTable above it must be obtained via
* {@link KafkaStreams#store(StoreQueryParameters)} KafkaStreams#store(...)}:
* For non-local keys, a custom RPC mechanism must be implemented using {@link KafkaStreams#metadataForAllStreamsClients()} to
* query the value of the key on a parallel running instance of your Kafka Streams application.
* The store name to query with is specified by {@link Materialized#as(String)} or {@link Materialized#as(KeyValueBytesStoreSupplier)}.
* <p>
* Note that the key is read-only and should not be modified, as this can lead to corrupt partitioning.
* This operation preserves data co-location with respect to the key.
* Thus, <em>no</em> internal data redistribution is required if a key based operator (like a join) is applied to
* the result {@code KTable}.
* <p>
* Note that {@code mapValues} for a <i>changelog stream</i> works differently than {@link KStream#mapValues(ValueMapper)
* record stream filters}, because {@link KeyValue records} with {@code null} values (so-called tombstone records)
* have delete semantics.
* Thus, for tombstones the provided value-mapper is not evaluated but the tombstone record is forwarded directly to
* delete the corresponding record in the result {@code KTable}.
*
* @param mapper a {@link ValueMapperWithKey} that computes a new output value
* @param materialized a {@link Materialized} that describes how the {@link StateStore} for the resulting {@code KTable}
* should be materialized. Cannot be {@code null}
* @param <VR> the value type of the result {@code KTable}
*
* @return a {@code KTable} that contains records with unmodified keys and new values (possibly of different type)
*/
<VR> KTable<K, VR> mapValues(final ValueMapperWithKey<? super K, ? super V, ? extends VR> mapper,
final Materialized<K, VR, KeyValueStore<Bytes, byte[]>> materialized);
/**
* Create a new {@code KTable} by transforming the value of each record in this {@code KTable} into a new value
* (with possibly a new type) in the new {@code KTable}, with the {@link Serde key serde}, {@link Serde value serde},
* and the underlying {@link KeyValueStore materialized state storage} configured in the {@link Materialized}
* instance.
* For each {@code KTable} update the provided {@link ValueMapperWithKey} is applied to the value of the update
* record and computes a new value for it, resulting in an updated record for the result {@code KTable}.
* Thus, an input record {@code <K,V>} can be transformed into an output record {@code <K:V'>}.
* This is a stateless record-by-record operation.
* <p>
* The example below counts the number of token of value and key strings.
* <pre>{@code
* KTable<String, String> inputTable = builder.table("topic");
* KTable<String, Integer> outputTable = inputTable.mapValue(new ValueMapperWithKey<String, String, Integer> {
* Integer apply(String readOnlyKey, String value) {
* return readOnlyKey.split(" ").length + value.split(" ").length;
* }
* });
* }</pre>
* <p>
* To query the local {@link KeyValueStore} representing outputTable above it must be obtained via
* {@link KafkaStreams#store(StoreQueryParameters) KafkaStreams#store(...)}:
* For non-local keys, a custom RPC mechanism must be implemented using {@link KafkaStreams#metadataForAllStreamsClients()} to
* query the value of the key on a parallel running instance of your Kafka Streams application.
* The store name to query with is specified by {@link Materialized#as(String)} or {@link Materialized#as(KeyValueBytesStoreSupplier)}.
* <p>
* Note that the key is read-only and should not be modified, as this can lead to corrupt partitioning.
* This operation preserves data co-location with respect to the key.
* Thus, <em>no</em> internal data redistribution is required if a key based operator (like a join) is applied to
* the result {@code KTable}.
* <p>
* Note that {@code mapValues} for a <i>changelog stream</i> works differently than {@link KStream#mapValues(ValueMapper)
* record stream filters}, because {@link KeyValue records} with {@code null} values (so-called tombstone records)
* have delete semantics.
* Thus, for tombstones the provided value-mapper is not evaluated but the tombstone record is forwarded directly to
* delete the corresponding record in the result {@code KTable}.
*
* @param mapper a {@link ValueMapperWithKey} that computes a new output value
* @param named a {@link Named} config used to name the processor in the topology
* @param materialized a {@link Materialized} that describes how the {@link StateStore} for the resulting {@code KTable}
* should be materialized. Cannot be {@code null}
* @param <VR> the value type of the result {@code KTable}
*
* @return a {@code KTable} that contains records with unmodified keys and new values (possibly of different type)
*/
<VR> KTable<K, VR> mapValues(final ValueMapperWithKey<? super K, ? super V, ? extends VR> mapper,
final Named named,
final Materialized<K, VR, KeyValueStore<Bytes, byte[]>> materialized);
/**
* Convert this changelog stream to a {@link KStream}.
* <p>
* Note that this is a logical operation and only changes the "interpretation" of the stream, i.e., each record of
* this changelog stream is no longer treated as an updated record (cf. {@link KStream} vs {@code KTable}).
*
* @return a {@link KStream} that contains the same records as this {@code KTable}
*/
KStream<K, V> toStream();
/**
* Convert this changelog stream to a {@link KStream}.
* <p>
* Note that this is a logical operation and only changes the "interpretation" of the stream, i.e., each record of
* this changelog stream is no longer treated as an updated record (cf. {@link KStream} vs {@code KTable}).
*
* @param named a {@link Named} config used to name the processor in the topology
*
* @return a {@link KStream} that contains the same records as this {@code KTable}
*/
KStream<K, V> toStream(final Named named);
/**
* Convert this changelog stream to a {@link KStream} using the given {@link KeyValueMapper} to select the new key.
* <p>
* For example, you can compute the new key as the length of the value string.
* <pre>{@code
* KTable<String, String> table = builder.table("topic");
* KStream<Integer, String> keyedStream = table.toStream(new KeyValueMapper<String, String, Integer> {
* Integer apply(String key, String value) {
* return value.length();
* }
* });
* }</pre>
* Setting a new key might result in an internal data redistribution if a key based operator (like an aggregation or
* join) is applied to the result {@link KStream}.
* <p>
* This operation is equivalent to calling
* {@code table.}{@link #toStream() toStream}{@code ().}{@link KStream#selectKey(KeyValueMapper) selectKey(KeyValueMapper)}.
* <p>
* Note that {@link #toStream()} is a logical operation and only changes the "interpretation" of the stream, i.e.,
* each record of this changelog stream is no longer treated as an updated record (cf. {@link KStream} vs {@code KTable}).
*
* @param mapper a {@link KeyValueMapper} that computes a new key for each record
* @param <KR> the new key type of the result stream
* @return a {@link KStream} that contains the same records as this {@code KTable}
*/
<KR> KStream<KR, V> toStream(final KeyValueMapper<? super K, ? super V, ? extends KR> mapper);
/**
* Convert this changelog stream to a {@link KStream} using the given {@link KeyValueMapper} to select the new key.
* <p>
* For example, you can compute the new key as the length of the value string.
* <pre>{@code
* KTable<String, String> table = builder.table("topic");
* KTable<Integer, String> keyedStream = table.toStream(new KeyValueMapper<String, String, Integer> {
* Integer apply(String key, String value) {
* return value.length();
* }
* });
* }</pre>
* Setting a new key might result in an internal data redistribution if a key based operator (like an aggregation or
* join) is applied to the result {@link KStream}.
* <p>
* This operation is equivalent to calling
* {@code table.}{@link #toStream() toStream}{@code ().}{@link KStream#selectKey(KeyValueMapper) selectKey(KeyValueMapper)}.
* <p>
* Note that {@link #toStream()} is a logical operation and only changes the "interpretation" of the stream, i.e.,
* each record of this changelog stream is no longer treated as an updated record (cf. {@link KStream} vs {@code KTable}).
*
* @param mapper a {@link KeyValueMapper} that computes a new key for each record
* @param named a {@link Named} config used to name the processor in the topology
* @param <KR> the new key type of the result stream
* @return a {@link KStream} that contains the same records as this {@code KTable}
*/
<KR> KStream<KR, V> toStream(final KeyValueMapper<? super K, ? super V, ? extends KR> mapper,
final Named named);
/**
* Suppress some updates from this changelog stream, determined by the supplied {@link Suppressed} configuration.
*
* This controls what updates downstream table and stream operations will receive.
*
* @param suppressed Configuration object determining what, if any, updates to suppress
* @return A new KTable with the desired suppression characteristics.
*/
KTable<K, V> suppress(final Suppressed<? super K> suppressed);
/**
* Create a new {@code KTable} by transforming the value of each record in this {@code KTable} into a new value
* (with possibly a new type), with default serializers, deserializers, and state store.
* A {@link ValueTransformerWithKey} (provided by the given {@link ValueTransformerWithKeySupplier}) is applied to each input
* record value and computes a new value for it.
* Thus, an input record {@code <K,V>} can be transformed into an output record {@code <K:V'>}.
* This is similar to {@link #mapValues(ValueMapperWithKey)}, but more flexible, allowing access to additional state-stores,
* and access to the {@link ProcessorContext}.
* Furthermore, via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long)} the processing progress can be observed and additional
* periodic actions can be performed.
* <p>
* If the downstream topology uses aggregation functions, (e.g. {@link KGroupedTable#reduce}, {@link KGroupedTable#aggregate}, etc),
* care must be taken when dealing with state, (either held in state-stores or transformer instances), to ensure correct aggregate results.
* In contrast, if the resulting KTable is materialized, (cf. {@link #transformValues(ValueTransformerWithKeySupplier, Materialized, String...)}),
* such concerns are handled for you.
* <p>
* In order to assign a state, the state must be created and registered beforehand:
* <pre>{@code
* // create store
* StoreBuilder<KeyValueStore<String,String>> keyValueStoreBuilder =
* Stores.keyValueStoreBuilder(Stores.persistentKeyValueStore("myValueTransformState"),
* Serdes.String(),
* Serdes.String());
* // register store
* builder.addStateStore(keyValueStoreBuilder);
*
* KTable outputTable = inputTable.transformValues(new ValueTransformerWithKeySupplier() { ... }, "myValueTransformState");
* }</pre>
* <p>
* Within the {@link ValueTransformerWithKey}, the state is obtained via the
* {@link ProcessorContext}.
* To trigger periodic actions via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) punctuate()},
* a schedule must be registered.
* <pre>{@code
* new ValueTransformerWithKeySupplier() {
* ValueTransformerWithKey get() {
* return new ValueTransformerWithKey() {
* private KeyValueStore<String, String> state;
*
* void init(ProcessorContext context) {
* this.state = (KeyValueStore<String, String>)context.getStateStore("myValueTransformState");
* context.schedule(Duration.ofSeconds(1), PunctuationType.WALL_CLOCK_TIME, new Punctuator(..)); // punctuate each 1000ms, can access this.state
* }
*
* NewValueType transform(K readOnlyKey, V value) {
* // can access this.state and use read-only key
* return new NewValueType(readOnlyKey); // or null
* }
*
* void close() {
* // can access this.state
* }
* }
* }
* }
* }</pre>
* <p>
* Note that the key is read-only and should not be modified, as this can lead to corrupt partitioning.
* Setting a new value preserves data co-location with respect to the key.
*
* @param transformerSupplier a instance of {@link ValueTransformerWithKeySupplier} that generates a
* {@link ValueTransformerWithKey}.
* At least one transformer instance will be created per streaming task.
* Transformers do not need to be thread-safe.
* @param stateStoreNames the names of the state stores used by the processor
* @param <VR> the value type of the result table
* @return a {@code KTable} that contains records with unmodified key and new values (possibly of different type)
* @see #mapValues(ValueMapper)
* @see #mapValues(ValueMapperWithKey)
*/
<VR> KTable<K, VR> transformValues(final ValueTransformerWithKeySupplier<? super K, ? super V, ? extends VR> transformerSupplier,
final String... stateStoreNames);
/**
* Create a new {@code KTable} by transforming the value of each record in this {@code KTable} into a new value
* (with possibly a new type), with default serializers, deserializers, and state store.
* A {@link ValueTransformerWithKey} (provided by the given {@link ValueTransformerWithKeySupplier}) is applied to each input
* record value and computes a new value for it.
* Thus, an input record {@code <K,V>} can be transformed into an output record {@code <K:V'>}.
* This is similar to {@link #mapValues(ValueMapperWithKey)}, but more flexible, allowing access to additional state-stores,
* and access to the {@link ProcessorContext}.
* Furthermore, via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long)} the processing progress can be observed and additional
* periodic actions can be performed.
* <p>
* If the downstream topology uses aggregation functions, (e.g. {@link KGroupedTable#reduce}, {@link KGroupedTable#aggregate}, etc),
* care must be taken when dealing with state, (either held in state-stores or transformer instances), to ensure correct aggregate results.
* In contrast, if the resulting KTable is materialized, (cf. {@link #transformValues(ValueTransformerWithKeySupplier, Materialized, String...)}),
* such concerns are handled for you.
* <p>
* In order to assign a state, the state must be created and registered beforehand:
* <pre>{@code
* // create store
* StoreBuilder<KeyValueStore<String,String>> keyValueStoreBuilder =
* Stores.keyValueStoreBuilder(Stores.persistentKeyValueStore("myValueTransformState"),
* Serdes.String(),
* Serdes.String());
* // register store
* builder.addStateStore(keyValueStoreBuilder);
*
* KTable outputTable = inputTable.transformValues(new ValueTransformerWithKeySupplier() { ... }, "myValueTransformState");
* }</pre>
* <p>
* Within the {@link ValueTransformerWithKey}, the state is obtained via the
* {@link ProcessorContext}.
* To trigger periodic actions via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) punctuate()},
* a schedule must be registered.
* <pre>{@code
* new ValueTransformerWithKeySupplier() {
* ValueTransformerWithKey get() {
* return new ValueTransformerWithKey() {
* private KeyValueStore<String, String> state;
*
* void init(ProcessorContext context) {
* this.state = (KeyValueStore<String, String>)context.getStateStore("myValueTransformState");
* context.schedule(Duration.ofSeconds(1), PunctuationType.WALL_CLOCK_TIME, new Punctuator(..)); // punctuate each 1000ms, can access this.state
* }
*
* NewValueType transform(K readOnlyKey, V value) {
* // can access this.state and use read-only key
* return new NewValueType(readOnlyKey); // or null
* }
*
* void close() {
* // can access this.state
* }
* }
* }
* }
* }</pre>
* <p>
* Note that the key is read-only and should not be modified, as this can lead to corrupt partitioning.
* Setting a new value preserves data co-location with respect to the key.
*
* @param transformerSupplier a instance of {@link ValueTransformerWithKeySupplier} that generates a
* {@link ValueTransformerWithKey}.
* At least one transformer instance will be created per streaming task.
* Transformers do not need to be thread-safe.
* @param named a {@link Named} config used to name the processor in the topology
* @param stateStoreNames the names of the state stores used by the processor
* @param <VR> the value type of the result table
* @return a {@code KTable} that contains records with unmodified key and new values (possibly of different type)
* @see #mapValues(ValueMapper)
* @see #mapValues(ValueMapperWithKey)
*/
<VR> KTable<K, VR> transformValues(final ValueTransformerWithKeySupplier<? super K, ? super V, ? extends VR> transformerSupplier,
final Named named,
final String... stateStoreNames);
/**
* Create a new {@code KTable} by transforming the value of each record in this {@code KTable} into a new value
* (with possibly a new type), with the {@link Serde key serde}, {@link Serde value serde}, and the underlying
* {@link KeyValueStore materialized state storage} configured in the {@link Materialized} instance.
* A {@link ValueTransformerWithKey} (provided by the given {@link ValueTransformerWithKeySupplier}) is applied to each input
* record value and computes a new value for it.
* This is similar to {@link #mapValues(ValueMapperWithKey)}, but more flexible, allowing stateful, rather than stateless,
* record-by-record operation, access to additional state-stores, and access to the {@link ProcessorContext}.
* Furthermore, via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long)} the processing progress can be observed and additional
* periodic actions can be performed.
* The resulting {@code KTable} is materialized into another state store (additional to the provided state store names)
* as specified by the user via {@link Materialized} parameter, and is queryable through its given name.
* <p>
* In order to assign a state, the state must be created and registered beforehand:
* <pre>{@code
* // create store
* StoreBuilder<KeyValueStore<String,String>> keyValueStoreBuilder =
* Stores.keyValueStoreBuilder(Stores.persistentKeyValueStore("myValueTransformState"),
* Serdes.String(),
* Serdes.String());
* // register store
* builder.addStateStore(keyValueStoreBuilder);
*
* KTable outputTable = inputTable.transformValues(
* new ValueTransformerWithKeySupplier() { ... },
* Materialized.<String, String, KeyValueStore<Bytes, byte[]>>as("outputTable")
* .withKeySerde(Serdes.String())
* .withValueSerde(Serdes.String()),
* "myValueTransformState");
* }</pre>
* <p>
* Within the {@link ValueTransformerWithKey}, the state is obtained via the
* {@link ProcessorContext}.
* To trigger periodic actions via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) punctuate()},
* a schedule must be registered.
* <pre>{@code
* new ValueTransformerWithKeySupplier() {
* ValueTransformerWithKey get() {
* return new ValueTransformerWithKey() {
* private KeyValueStore<String, String> state;
*
* void init(ProcessorContext context) {
* this.state = (KeyValueStore<String, String>)context.getStateStore("myValueTransformState");
* context.schedule(Duration.ofSeconds(1), PunctuationType.WALL_CLOCK_TIME, new Punctuator(..)); // punctuate each 1000ms, can access this.state
* }
*
* NewValueType transform(K readOnlyKey, V value) {
* // can access this.state and use read-only key
* return new NewValueType(readOnlyKey); // or null
* }
*
* void close() {
* // can access this.state
* }
* }
* }
* }
* }</pre>
* <p>
* Note that the key is read-only and should not be modified, as this can lead to corrupt partitioning.
* Setting a new value preserves data co-location with respect to the key.
*
* @param transformerSupplier a instance of {@link ValueTransformerWithKeySupplier} that generates a
* {@link ValueTransformerWithKey}.
* At least one transformer instance will be created per streaming task.
* Transformers do not need to be thread-safe.
* @param materialized an instance of {@link Materialized} used to describe how the state store of the
* resulting table should be materialized.
* Cannot be {@code null}
* @param stateStoreNames the names of the state stores used by the processor
* @param <VR> the value type of the result table
* @return a {@code KTable} that contains records with unmodified key and new values (possibly of different type)
* @see #mapValues(ValueMapper)
* @see #mapValues(ValueMapperWithKey)
*/
<VR> KTable<K, VR> transformValues(final ValueTransformerWithKeySupplier<? super K, ? super V, ? extends VR> transformerSupplier,
final Materialized<K, VR, KeyValueStore<Bytes, byte[]>> materialized,
final String... stateStoreNames);
/**
* Create a new {@code KTable} by transforming the value of each record in this {@code KTable} into a new value
* (with possibly a new type), with the {@link Serde key serde}, {@link Serde value serde}, and the underlying
* {@link KeyValueStore materialized state storage} configured in the {@link Materialized} instance.
* A {@link ValueTransformerWithKey} (provided by the given {@link ValueTransformerWithKeySupplier}) is applied to each input
* record value and computes a new value for it.
* This is similar to {@link #mapValues(ValueMapperWithKey)}, but more flexible, allowing stateful, rather than stateless,
* record-by-record operation, access to additional state-stores, and access to the {@link ProcessorContext}.
* Furthermore, via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long)} the processing progress can be observed and additional
* periodic actions can be performed.
* The resulting {@code KTable} is materialized into another state store (additional to the provided state store names)
* as specified by the user via {@link Materialized} parameter, and is queryable through its given name.
* <p>
* In order to assign a state, the state must be created and registered beforehand:
* <pre>{@code
* // create store
* StoreBuilder<KeyValueStore<String,String>> keyValueStoreBuilder =
* Stores.keyValueStoreBuilder(Stores.persistentKeyValueStore("myValueTransformState"),
* Serdes.String(),
* Serdes.String());
* // register store
* builder.addStateStore(keyValueStoreBuilder);
*
* KTable outputTable = inputTable.transformValues(
* new ValueTransformerWithKeySupplier() { ... },
* Materialized.<String, String, KeyValueStore<Bytes, byte[]>>as("outputTable")
* .withKeySerde(Serdes.String())
* .withValueSerde(Serdes.String()),
* "myValueTransformState");
* }</pre>
* <p>
* Within the {@link ValueTransformerWithKey}, the state is obtained via the
* {@link ProcessorContext}.
* To trigger periodic actions via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) punctuate()},
* a schedule must be registered.
* <pre>{@code
* new ValueTransformerWithKeySupplier() {
* ValueTransformerWithKey get() {
* return new ValueTransformerWithKey() {
* private KeyValueStore<String, String> state;
*
* void init(ProcessorContext context) {
* this.state = (KeyValueStore<String, String>)context.getStateStore("myValueTransformState");
* context.schedule(Duration.ofSeconds(1), PunctuationType.WALL_CLOCK_TIME, new Punctuator(..)); // punctuate each 1000ms, can access this.state
* }
*
* NewValueType transform(K readOnlyKey, V value) {
* // can access this.state and use read-only key