-
Notifications
You must be signed in to change notification settings - Fork 13.6k
/
ChangeLoggingKeyValueBytesStore.java
107 lines (93 loc) · 3.82 KB
/
ChangeLoggingKeyValueBytesStore.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
/*
* 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.state.internals;
import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.common.utils.Bytes;
import org.apache.kafka.streams.KeyValue;
import org.apache.kafka.streams.processor.ProcessorContext;
import org.apache.kafka.streams.processor.StateStore;
import org.apache.kafka.streams.processor.internals.ProcessorStateManager;
import org.apache.kafka.streams.state.KeyValueIterator;
import org.apache.kafka.streams.state.KeyValueStore;
import org.apache.kafka.streams.state.StateSerdes;
import java.util.List;
public class ChangeLoggingKeyValueBytesStore extends WrappedStateStore.AbstractStateStore implements KeyValueStore<Bytes, byte[]> {
private final KeyValueStore<Bytes, byte[]> inner;
private StoreChangeLogger<Bytes, byte[]> changeLogger;
ChangeLoggingKeyValueBytesStore(final KeyValueStore<Bytes, byte[]> inner) {
super(inner);
this.inner = inner;
}
@Override
public void init(final ProcessorContext context, final StateStore root) {
inner.init(context, root);
final String topic = ProcessorStateManager.storeChangelogTopic(context.applicationId(), inner.name());
this.changeLogger = new StoreChangeLogger<>(inner.name(), context, new StateSerdes<>(topic, Serdes.Bytes(), Serdes.ByteArray()));
// if the inner store is an LRU cache, add the eviction listener to log removed record
if (inner instanceof MemoryLRUCache) {
((MemoryLRUCache<Bytes, byte[]>) inner).whenEldestRemoved(new MemoryLRUCache.EldestEntryRemovalListener<Bytes, byte[]>() {
@Override
public void apply(Bytes key, byte[] value) {
// pass null to indicate removal
changeLogger.logChange(key, null);
}
});
}
}
@Override
public long approximateNumEntries() {
return inner.approximateNumEntries();
}
@Override
public void put(final Bytes key, final byte[] value) {
inner.put(key, value);
changeLogger.logChange(key, value);
}
@Override
public byte[] putIfAbsent(final Bytes key, final byte[] value) {
final byte[] previous = get(key);
if (previous == null) {
put(key, value);
}
return previous;
}
@Override
public void putAll(final List<KeyValue<Bytes, byte[]>> entries) {
inner.putAll(entries);
for (KeyValue<Bytes, byte[]> entry : entries) {
changeLogger.logChange(entry.key, entry.value);
}
}
@Override
public byte[] delete(final Bytes key) {
final byte[] oldValue = inner.delete(key);
changeLogger.logChange(key, null);
return oldValue;
}
@Override
public byte[] get(final Bytes key) {
return inner.get(key);
}
@Override
public KeyValueIterator<Bytes, byte[]> range(final Bytes from, final Bytes to) {
return inner.range(from, to);
}
@Override
public KeyValueIterator<Bytes, byte[]> all() {
return inner.all();
}
}