Skip to content

Commit 59e5682

Browse files
apourchetAntoine Pourchet
andauthored
Windowed Table: Unit testing (#252)
* Windowed Table: Unit testing Fixed the BSON parsing bug that was hidden by the KS state store caching mechanism. Without the proper @BsonCreator constructor, our document retrieval and parsing from MongoDB were never getting their ID fields populated correctly. * fixed style * added ranged fetch test * better test and KVDoc fix --------- Co-authored-by: Antoine Pourchet <antoine@antoine-mbp-responsive.local>
1 parent aab11f7 commit 59e5682

File tree

4 files changed

+164
-3
lines changed

4 files changed

+164
-3
lines changed

kafka-client/src/main/java/dev/responsive/kafka/internal/db/mongo/KVDoc.java

Lines changed: 9 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,8 @@
1919
import java.util.Arrays;
2020
import java.util.Date;
2121
import java.util.Objects;
22-
import org.bson.codecs.pojo.annotations.BsonId;
22+
import org.bson.codecs.pojo.annotations.BsonCreator;
23+
import org.bson.codecs.pojo.annotations.BsonProperty;
2324

2425
public class KVDoc {
2526

@@ -29,7 +30,6 @@ public class KVDoc {
2930
public static final String EPOCH = "epoch";
3031
public static final String TOMBSTONE_TS = "tombstoneTs";
3132

32-
@BsonId
3333
byte[] id;
3434
byte[] value;
3535
long epoch;
@@ -38,7 +38,13 @@ public class KVDoc {
3838
public KVDoc() {
3939
}
4040

41-
public KVDoc(final byte[] key, final byte[] value, final long epoch) {
41+
@BsonCreator
42+
public KVDoc(
43+
@BsonProperty(ID) byte[] id,
44+
@BsonProperty(VALUE) byte[] value,
45+
@BsonProperty(EPOCH) long epoch
46+
) {
47+
this.id = id;
4248
this.value = value;
4349
this.epoch = epoch;
4450
}

kafka-client/src/main/java/dev/responsive/kafka/internal/db/mongo/MongoWindowedTable.java

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -442,6 +442,9 @@ public KeyValueIterator<WindowedKey, byte[]> fetch(
442442

443443
for (final var segment : partitioner.segmenter().range(kafkaPartition, timeFrom, timeTo)) {
444444
final var segmentWindows = partitionSegments.segmentWindows.get(segment);
445+
if (segmentWindows == null) {
446+
continue;
447+
}
445448
final FindIterable<WindowDoc> fetchResults = segmentWindows.find(
446449
Filters.and(
447450
Filters.gte(WindowDoc.ID, compositeKey(key, timeFrom)),

kafka-client/src/main/java/dev/responsive/kafka/internal/db/mongo/WindowDoc.java

Lines changed: 13 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,8 @@
2323
import java.util.Arrays;
2424
import java.util.Objects;
2525
import org.apache.kafka.common.utils.Bytes;
26+
import org.bson.codecs.pojo.annotations.BsonCreator;
27+
import org.bson.codecs.pojo.annotations.BsonProperty;
2628

2729
public class WindowDoc {
2830

@@ -41,6 +43,17 @@ public class WindowDoc {
4143
public WindowDoc() {
4244
}
4345

46+
@BsonCreator
47+
public WindowDoc(
48+
@BsonProperty(ID) BasicDBObject id,
49+
@BsonProperty(VALUE) byte[] value,
50+
@BsonProperty(EPOCH) long epoch
51+
) {
52+
this.id = id;
53+
this.value = value;
54+
this.epoch = epoch;
55+
}
56+
4457
public BasicDBObject getKey() {
4558
return id;
4659
}
Lines changed: 139 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,139 @@
1+
/*
2+
* Copyright 2024 Responsive Computing, Inc.
3+
*
4+
* Licensed under the Apache License, Version 2.0 (the "License");
5+
* you may not use this file except in compliance with the License.
6+
* You may obtain a copy of the License at
7+
*
8+
* http://www.apache.org/licenses/LICENSE-2.0
9+
*
10+
* Unless required by applicable law or agreed to in writing, software
11+
* distributed under the License is distributed on an "AS IS" BASIS,
12+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13+
* See the License for the specific language governing permissions and
14+
* limitations under the License.
15+
*/
16+
17+
package dev.responsive.kafka.internal.db.mongo;
18+
19+
import static dev.responsive.kafka.api.config.ResponsiveConfig.STORAGE_HOSTNAME_CONFIG;
20+
import static org.hamcrest.MatcherAssert.assertThat;
21+
22+
import com.mongodb.client.MongoClient;
23+
import dev.responsive.kafka.api.config.StorageBackend;
24+
import dev.responsive.kafka.internal.db.partitioning.WindowSegmentPartitioner;
25+
import dev.responsive.kafka.internal.utils.SessionUtil;
26+
import dev.responsive.kafka.internal.utils.WindowedKey;
27+
import dev.responsive.kafka.testutils.ResponsiveConfigParam;
28+
import dev.responsive.kafka.testutils.ResponsiveExtension;
29+
import java.util.ArrayList;
30+
import java.util.Map;
31+
import org.apache.kafka.common.utils.Bytes;
32+
import org.apache.kafka.streams.KeyValue;
33+
import org.hamcrest.Matchers;
34+
import org.junit.jupiter.api.BeforeEach;
35+
import org.junit.jupiter.api.Test;
36+
import org.junit.jupiter.api.TestInfo;
37+
import org.junit.jupiter.api.extension.RegisterExtension;
38+
39+
class MongoWindowTableTest {
40+
41+
@RegisterExtension
42+
public static final ResponsiveExtension EXT = new ResponsiveExtension(StorageBackend.MONGO_DB);
43+
private static final CollectionCreationOptions UNSHARDED = new CollectionCreationOptions(
44+
false,
45+
0
46+
);
47+
private static final byte[] DEFAULT_VALUE = new byte[] {1};
48+
49+
private String name;
50+
private MongoClient client;
51+
52+
@BeforeEach
53+
public void before(
54+
final TestInfo info,
55+
@ResponsiveConfigParam final Map<String, Object> props
56+
) {
57+
name = info.getDisplayName().replace("()", "");
58+
59+
final String mongoConnection = (String) props.get(STORAGE_HOSTNAME_CONFIG);
60+
client = SessionUtil.connect(mongoConnection, null, null);
61+
}
62+
63+
@Test
64+
public void shouldSucceedSimpleSetGet() {
65+
// Given:
66+
final WindowSegmentPartitioner partitioner = new WindowSegmentPartitioner(10_000L, 1_000L);
67+
final var segment = partitioner.segmenter().activeSegments(0, 100).get(0);
68+
69+
final MongoWindowedTable table = new MongoWindowedTable(client, name, partitioner,
70+
false, UNSHARDED
71+
);
72+
final var flushManager = table.init(0);
73+
flushManager.updateOffsetAndStreamTime(0, 100);
74+
flushManager.createSegment(segment);
75+
76+
// When:
77+
final var byteKey = Bytes.wrap("key".getBytes());
78+
var writer = flushManager.createWriter(segment);
79+
writer.insert(
80+
new WindowedKey(byteKey, 0),
81+
DEFAULT_VALUE,
82+
table.localEpoch(0)
83+
);
84+
writer.flush();
85+
86+
// Then:
87+
var value = table.fetch(0, byteKey, 0);
88+
assertThat(value, Matchers.equalTo(DEFAULT_VALUE));
89+
value = table.fetch(0, byteKey, 100);
90+
assertThat(value, Matchers.nullValue());
91+
value = table.fetch(0, Bytes.wrap("other".getBytes()), 0);
92+
assertThat(value, Matchers.nullValue());
93+
}
94+
95+
@Test
96+
public void shouldSucceedRangeSetGet() {
97+
// Given:
98+
final WindowSegmentPartitioner partitioner = new WindowSegmentPartitioner(10_000L, 1_000L);
99+
final var segment = partitioner.segmenter().activeSegments(0, 100).get(0);
100+
101+
final MongoWindowedTable table = new MongoWindowedTable(client, name, partitioner,
102+
false, UNSHARDED
103+
);
104+
final var flushManager = table.init(0);
105+
flushManager.updateOffsetAndStreamTime(0, 6_000);
106+
flushManager.createSegment(segment);
107+
108+
// When:
109+
final var byteKey = Bytes.wrap("key".getBytes());
110+
final var windowedKey1 = new WindowedKey(byteKey, 500);
111+
final var windowedKey2 = new WindowedKey(byteKey, 5_000);
112+
var writer = flushManager.createWriter(segment);
113+
writer.insert(
114+
windowedKey1,
115+
DEFAULT_VALUE,
116+
table.localEpoch(0)
117+
);
118+
writer.insert(
119+
windowedKey2,
120+
DEFAULT_VALUE,
121+
table.localEpoch(0)
122+
);
123+
writer.flush();
124+
125+
// Then:
126+
var it = table.fetch(0, byteKey, 100, 6_000);
127+
var kvs = new ArrayList<KeyValue<WindowedKey, byte[]>>();
128+
it.forEachRemaining(kvs::add);
129+
130+
assertThat(kvs, Matchers.hasSize(2));
131+
assertThat(kvs.get(0).key.key, Matchers.equalTo(windowedKey1.key));
132+
assertThat(kvs.get(0).key.windowStartMs, Matchers.equalTo(windowedKey1.windowStartMs));
133+
assertThat(kvs.get(0).value, Matchers.equalTo(DEFAULT_VALUE));
134+
135+
assertThat(kvs.get(1).key.key, Matchers.equalTo(windowedKey2.key));
136+
assertThat(kvs.get(1).key.windowStartMs, Matchers.equalTo(windowedKey2.windowStartMs));
137+
assertThat(kvs.get(1).value, Matchers.equalTo(DEFAULT_VALUE));
138+
}
139+
}

0 commit comments

Comments
 (0)