[GitHub] [kafka] mjsax commented on a diff in pull request #13143: KAFKA-14491: [3/N] Add logical key value segments

2023-02-02 Thread via GitHub


mjsax commented on code in PR #13143:
URL: https://github.com/apache/kafka/pull/13143#discussion_r1095367325


##
streams/src/test/java/org/apache/kafka/streams/state/internals/LogicalKeyValueSegmentsTest.java:
##
@@ -0,0 +1,168 @@
+/*
+ * 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 static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.util.List;
+import org.apache.kafka.common.metrics.Metrics;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.streams.processor.internals.MockStreamsMetrics;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.apache.kafka.test.InternalMockProcessorContext;
+import org.apache.kafka.test.MockRecordCollector;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class LogicalKeyValueSegmentsTest {
+
+private static final long SEGMENT_INTERVAL = 100L;
+private static final long RETENTION_PERIOD = 4 * SEGMENT_INTERVAL;
+private static final String STORE_NAME = "logical-segments";
+private static final String METRICS_SCOPE = "metrics-scope";
+private static final String DB_FILE_DIR = "rocksdb";
+
+private InternalMockProcessorContext context;
+
+private LogicalKeyValueSegments segments;
+
+@Before
+public void setUp() {
+context = new InternalMockProcessorContext<>(
+TestUtils.tempDirectory(),
+Serdes.String(),
+Serdes.Long(),
+new MockRecordCollector(),
+new ThreadCache(new LogContext("testCache "), 0, new 
MockStreamsMetrics(new Metrics()))
+);
+segments = new LogicalKeyValueSegments(
+STORE_NAME,
+DB_FILE_DIR,
+RETENTION_PERIOD,
+SEGMENT_INTERVAL,
+new RocksDBMetricsRecorder(METRICS_SCOPE, STORE_NAME)
+);
+segments.openExisting(context, -1L);
+}
+
+@After
+public void tearDown() {
+segments.close();
+}
+
+@Test
+public void shouldGetSegmentIdsFromTimestamp() {
+assertEquals(0, segments.segmentId(0));
+assertEquals(1, segments.segmentId(SEGMENT_INTERVAL));
+assertEquals(2, segments.segmentId(2 * SEGMENT_INTERVAL));
+assertEquals(3, segments.segmentId(3 * SEGMENT_INTERVAL));
+}
+
+@Test
+public void shouldCreateSegments() {
+final LogicalKeyValueSegment segment1 = 
segments.getOrCreateSegmentIfLive(0, context, -1L);
+final LogicalKeyValueSegment segment2 = 
segments.getOrCreateSegmentIfLive(1, context, -1L);
+final LogicalKeyValueSegment segment3 = 
segments.getOrCreateSegmentIfLive(2, context, -1L);
+
+final File rocksdbDir = new File(new File(context.stateDir(), 
DB_FILE_DIR), STORE_NAME);
+assertTrue(rocksdbDir.isDirectory());
+
+assertTrue(segment1.isOpen());
+assertTrue(segment2.isOpen());
+assertTrue(segment3.isOpen());
+}
+
+@Test
+public void shouldNotCreateSegmentThatIsAlreadyExpired() {
+final long streamTime = updateStreamTimeAndCreateSegment(7);
+assertNull(segments.getOrCreateSegmentIfLive(0, context, streamTime));
+}
+
+@Test
+public void shouldCleanupSegmentsThatHaveExpired() {

Review Comment:
   >  If you think it's worth it, I can remove these tests from here and also 
from KeyValueSegmentsTest.java, and create a dummy AbstractSegments 
implementation to add an AbstractSegmentsTest.java. I'd like to do that as a 
follow-up PR instead of as part of this change, though.
   
   Sounds cleaner to me. And yes, follow up PR is preferable.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: 

[GitHub] [kafka] mjsax commented on a diff in pull request #13143: KAFKA-14491: [3/N] Add logical key value segments

2023-02-02 Thread via GitHub


mjsax commented on code in PR #13143:
URL: https://github.com/apache/kafka/pull/13143#discussion_r1095365120


##
streams/src/test/java/org/apache/kafka/streams/state/internals/LogicalKeyValueSegmentTest.java:
##
@@ -0,0 +1,270 @@
+/*
+ * 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 static java.nio.charset.StandardCharsets.UTF_8;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.errors.InvalidStateStoreException;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.state.KeyValueIterator;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.apache.kafka.test.InternalMockProcessorContext;
+import org.apache.kafka.test.StreamsTestUtils;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class LogicalKeyValueSegmentTest {
+
+private static final String STORE_NAME = "physical-rocks";
+private static final String METRICS_SCOPE = "metrics-scope";
+private static final String DB_FILE_DIR = "rocksdb";
+private static final Serializer STRING_SERIALIZER = new 
StringSerializer();
+private static final Deserializer STRING_DESERIALIZER = new 
StringDeserializer();
+
+private RocksDBStore physicalStore;
+
+private LogicalKeyValueSegment segment1;
+private LogicalKeyValueSegment segment2;
+
+@Before
+public void setUp() {
+physicalStore = new RocksDBStore(STORE_NAME, DB_FILE_DIR, new 
RocksDBMetricsRecorder(METRICS_SCOPE, STORE_NAME), false);
+physicalStore.init((StateStoreContext) new 
InternalMockProcessorContext<>(
+TestUtils.tempDirectory(),
+Serdes.String(),
+Serdes.String(),
+new StreamsConfig(StreamsTestUtils.getStreamsConfig())
+), physicalStore);
+
+segment1 = new LogicalKeyValueSegment(1, "segment-1", physicalStore);
+segment2 = new LogicalKeyValueSegment(2, "segment-2", physicalStore);
+}
+
+@After
+public void tearDown() {
+segment1.close();
+segment2.close();
+physicalStore.close();
+}
+
+@Test
+public void shouldPut() {
+final KeyValue kv0 = new KeyValue<>("1", "a");
+final KeyValue kv1 = new KeyValue<>("2", "b");
+
+segment1.put(new Bytes(kv0.key.getBytes(UTF_8)), 
kv0.value.getBytes(UTF_8));
+segment1.put(new Bytes(kv1.key.getBytes(UTF_8)), 
kv1.value.getBytes(UTF_8));
+segment2.put(new Bytes(kv0.key.getBytes(UTF_8)), 
kv0.value.getBytes(UTF_8));
+segment2.put(new Bytes(kv1.key.getBytes(UTF_8)), 
kv1.value.getBytes(UTF_8));
+
+assertEquals("a", getAndDeserialize(segment1, "1"));

Review Comment:
   I see your point, but the test does not really achieve this, as we put the 
same data into both segment? To test "segment isolation" we would need to put 4 
different record (2 per segment) and test both positive (put on s1 allows use 
to get on s1) and negative (put on s1, does not allow get on s2 to see the 
data)?
   
   Might apply to other tests, too?



##
streams/src/test/java/org/apache/kafka/streams/state/internals/LogicalKeyValueSegmentsTest.java:
##
@@ -0,0 +1,168 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this 

[GitHub] [kafka] mjsax commented on a diff in pull request #13143: KAFKA-14491: [3/N] Add logical key value segments

2023-02-01 Thread via GitHub


mjsax commented on code in PR #13143:
URL: https://github.com/apache/kafka/pull/13143#discussion_r1093764504


##
streams/src/test/java/org/apache/kafka/streams/state/internals/LogicalKeyValueSegmentTest.java:
##
@@ -0,0 +1,270 @@
+/*
+ * 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 static java.nio.charset.StandardCharsets.UTF_8;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.errors.InvalidStateStoreException;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.state.KeyValueIterator;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.apache.kafka.test.InternalMockProcessorContext;
+import org.apache.kafka.test.StreamsTestUtils;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class LogicalKeyValueSegmentTest {
+
+private static final String STORE_NAME = "physical-rocks";
+private static final String METRICS_SCOPE = "metrics-scope";
+private static final String DB_FILE_DIR = "rocksdb";
+private static final Serializer STRING_SERIALIZER = new 
StringSerializer();
+private static final Deserializer STRING_DESERIALIZER = new 
StringDeserializer();
+
+private RocksDBStore physicalStore;
+
+private LogicalKeyValueSegment segment1;
+private LogicalKeyValueSegment segment2;
+
+@Before
+public void setUp() {
+physicalStore = new RocksDBStore(STORE_NAME, DB_FILE_DIR, new 
RocksDBMetricsRecorder(METRICS_SCOPE, STORE_NAME), false);
+physicalStore.init((StateStoreContext) new 
InternalMockProcessorContext<>(
+TestUtils.tempDirectory(),
+Serdes.String(),
+Serdes.String(),
+new StreamsConfig(StreamsTestUtils.getStreamsConfig())
+), physicalStore);
+
+segment1 = new LogicalKeyValueSegment(1, "segment-1", physicalStore);
+segment2 = new LogicalKeyValueSegment(2, "segment-2", physicalStore);
+}
+
+@After
+public void tearDown() {
+segment1.close();
+segment2.close();
+physicalStore.close();
+}
+
+@Test
+public void shouldPut() {
+final KeyValue kv0 = new KeyValue<>("1", "a");
+final KeyValue kv1 = new KeyValue<>("2", "b");
+
+segment1.put(new Bytes(kv0.key.getBytes(UTF_8)), 
kv0.value.getBytes(UTF_8));

Review Comment:
   Why use `getBytes` here and the `StringSerializer` below?



##
streams/src/test/java/org/apache/kafka/streams/state/internals/LogicalKeyValueSegmentTest.java:
##
@@ -0,0 +1,270 @@
+/*
+ * 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
+ * 

[GitHub] [kafka] mjsax commented on a diff in pull request #13143: KAFKA-14491: [3/N] Add logical key value segments

2023-01-23 Thread via GitHub


mjsax commented on code in PR #13143:
URL: https://github.com/apache/kafka/pull/13143#discussion_r1084834388


##
streams/src/main/java/org/apache/kafka/streams/state/internals/LogicalKeyValueSegment.java:
##
@@ -0,0 +1,297 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import org.apache.kafka.common.serialization.BytesSerializer;
+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.state.KeyValueIterator;
+import org.rocksdb.RocksDBException;
+import org.rocksdb.WriteBatch;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This "logical segment" is a segment which shares its underlying physical 
store with other
+ * logical segments. Each segment uses a unique, fixed-length key prefix 
derived from the
+ * segment ID when writing to the shared physical store.
+ */
+class LogicalKeyValueSegment implements Comparable, 
Segment {
+private static final Logger log = 
LoggerFactory.getLogger(LogicalKeyValueSegment.class);
+
+public final long id;
+private final String name;
+private final RocksDBStore physicalStore;
+private final PrefixKeyFormatter prefixKeyFormatter;
+
+private volatile boolean open = false;
+final Set> openIterators = 
Collections.synchronizedSet(new HashSet<>());
+
+LogicalKeyValueSegment(final long id,
+   final String name,
+   final RocksDBStore physicalStore) {
+this.id = id;
+this.name = name;
+this.physicalStore = Objects.requireNonNull(physicalStore);
+
+this.prefixKeyFormatter = new 
PrefixKeyFormatter(serializeLongToBytes(id));
+}
+
+void openDB() {

Review Comment:
   Seems `openDB` does not do much -- do we actually need it (compare my other 
comments about the `open` flag)?



##
streams/src/main/java/org/apache/kafka/streams/state/internals/LogicalKeyValueSegment.java:
##
@@ -0,0 +1,297 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import org.apache.kafka.common.serialization.BytesSerializer;
+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.state.KeyValueIterator;
+import org.rocksdb.RocksDBException;
+import org.rocksdb.WriteBatch;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This "logical segment" is a segment which shares its underlying physical 
store with other
+ * logical segments. Each segment uses a unique, fixed-length key prefix 
derived from the
+ * segment ID when writing to the shared physical store.
+ */
+class