aliehsaeedii commented on code in PR #21446: URL: https://github.com/apache/kafka/pull/21446#discussion_r2803652938
########## streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStoreWithHeadersTest.java: ########## @@ -0,0 +1,535 @@ +/* + * 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.Serializer; +import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.common.utils.LogCaptureAppender; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.state.KeyValueIterator; + +import org.junit.jupiter.api.Test; +import org.rocksdb.ColumnFamilyDescriptor; +import org.rocksdb.ColumnFamilyHandle; +import org.rocksdb.ColumnFamilyOptions; +import org.rocksdb.DBOptions; +import org.rocksdb.RocksDB; + +import java.io.File; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; + +import static java.util.Arrays.asList; +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class RocksDBTimestampedStoreWithHeadersTest extends RocksDBStoreTest { + + private final Serializer<String> stringSerializer = new StringSerializer(); + + RocksDBStore getRocksDBStore() { + return new RocksDBTimestampedStoreWithHeaders(DB_NAME, METRICS_SCOPE); + } + + @Test + public void shouldOpenNewStoreInRegularMode() { + try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(RocksDBTimestampedStoreWithHeaders.class)) { + rocksDBStore.init(context, rocksDBStore); + + assertTrue(appender.getMessages().contains("Opening store " + DB_NAME + " in regular headers-aware mode")); + } + + try (final KeyValueIterator<Bytes, byte[]> iterator = rocksDBStore.all()) { + assertFalse(iterator.hasNext()); + } + } + + @Test + public void shouldOpenExistingStoreInRegularMode() throws Exception { + // prepare store + rocksDBStore.init(context, rocksDBStore); + rocksDBStore.put(new Bytes("key".getBytes()), "timestampedWithHeaders".getBytes()); + rocksDBStore.close(); + + // re-open store + try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(RocksDBTimestampedStoreWithHeaders.class)) { + rocksDBStore.init(context, rocksDBStore); + + assertTrue(appender.getMessages().contains("Opening store " + DB_NAME + " in regular headers-aware mode")); + } finally { + rocksDBStore.close(); + } + + // verify store + final DBOptions dbOptions = new DBOptions(); + final ColumnFamilyOptions columnFamilyOptions = new ColumnFamilyOptions(); + + final List<ColumnFamilyDescriptor> columnFamilyDescriptors = asList( + new ColumnFamilyDescriptor(RocksDB.DEFAULT_COLUMN_FAMILY, columnFamilyOptions), + new ColumnFamilyDescriptor("keyValueWithTimestampAndHeaders".getBytes(StandardCharsets.UTF_8), columnFamilyOptions)); + final List<ColumnFamilyHandle> columnFamilies = new ArrayList<>(columnFamilyDescriptors.size()); + + RocksDB db = null; + ColumnFamilyHandle defaultColumnFamily = null, headersColumnFamily = null; + try { + db = RocksDB.open( + dbOptions, + new File(new File(context.stateDir(), "rocksdb"), DB_NAME).getAbsolutePath(), + columnFamilyDescriptors, + columnFamilies); + + defaultColumnFamily = columnFamilies.get(0); + headersColumnFamily = columnFamilies.get(1); + + assertNull(db.get(defaultColumnFamily, "key".getBytes())); + assertEquals(0L, db.getLongProperty(defaultColumnFamily, "rocksdb.estimate-num-keys")); + assertEquals(22, db.get(headersColumnFamily, "key".getBytes()).length); + assertEquals(1L, db.getLongProperty(headersColumnFamily, "rocksdb.estimate-num-keys")); + } finally { + // Order of closing must follow: ColumnFamilyHandle > RocksDB > DBOptions > ColumnFamilyOptions + if (defaultColumnFamily != null) { + defaultColumnFamily.close(); + } + if (headersColumnFamily != null) { + headersColumnFamily.close(); + } + if (db != null) { + db.close(); + } + dbOptions.close(); + columnFamilyOptions.close(); + } + } + + @Test + public void shouldMigrateFromTimestampedToHeadersAwareColumnFamily() throws Exception { + prepareOldStore(); + + // Open with RocksDBTimestampedStoreWithHeaders - should detect legacy CF and enter upgrade mode + try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(RocksDBTimestampedStoreWithHeaders.class)) { + rocksDBStore.init(context, rocksDBStore); + + assertTrue(appender.getMessages().contains("Opening store " + DB_NAME + " in upgrade mode")); + } + + assertEquals(7L, rocksDBStore.approximateNumEntries(), "Expected 7 entries in legacy CF and 0 in headers-aware CF before migration"); + + // get() - tests lazy migration on read + + assertNull(rocksDBStore.get(new Bytes("unknown".getBytes())), "Expected null for unknown key"); + assertEquals(7L, rocksDBStore.approximateNumEntries(), "Expected 7 entries on legacy CF, 0 in headers-aware CF"); + + assertEquals(1 + 0 + 8 + 1, rocksDBStore.get(new Bytes("key1".getBytes())).length, + "Expected header-aware format: varint(1) + empty headers(0) + timestamp(8) + value(1) = 10 bytes"); + assertEquals(7L, rocksDBStore.approximateNumEntries(), "Expected 6 entries on legacy CF, 1 in headers-aware CF after migrating key1"); + + // put() - tests migration on write + + rocksDBStore.put(new Bytes("key2".getBytes()), "timestamp+22".getBytes()); + assertEquals(7L, rocksDBStore.approximateNumEntries(), "Expected 5 entries on legacy CF, 2 in headers-aware CF after migrating key2 with put()"); + + rocksDBStore.put(new Bytes("key3".getBytes()), null); + // count is off by one, due to two delete operations (even if one does not delete anything) + assertEquals(5L, rocksDBStore.approximateNumEntries(), "Expected 4 entries on legacy CF, 1 in headers-aware CF after deleting key3 with put()"); + + rocksDBStore.put(new Bytes("key8".getBytes()), "headers+timestamp+88888888".getBytes()); + assertEquals(5L, rocksDBStore.approximateNumEntries(), "Expected 3 entries on legacy CF, 2 in headers-aware CF after adding new key8 with put()"); + + // putIfAbsent() - tests migration on conditional write + + assertEquals(1 + 0 + 8 + 4, + rocksDBStore.putIfAbsent(new Bytes("key4".getBytes()), "headers+timestamp+4444".getBytes()).length, + "Expected header-aware format: varint(1) + empty headers(0) + timestamp(8) + value(4) = 13 bytes"); + assertEquals(5L, rocksDBStore.approximateNumEntries(), "Expected 2 entries on legacy CF, 3 in headers-aware CF after migrating key4 with putIfAbsent()"); + + assertNull(rocksDBStore.putIfAbsent(new Bytes("key11".getBytes()), "headers+timestamp+11111111111".getBytes()), + "Expected null return value for putIfAbsent on non-existing key11, and new key should be added to headers-aware CF"); + assertEquals(5L, rocksDBStore.approximateNumEntries(), "Expected 1 entry on legacy CF, 4 in headers-aware CF after adding new key11 with putIfAbsent()"); + + assertEquals(1 + 0 + 8 + 5, rocksDBStore.putIfAbsent(new Bytes("key5".getBytes()), null).length, + "Expected header-aware format: varint(1) + empty headers(0) + timestamp(8) + value(5) = 14 bytes for putIfAbsent with null on existing key5"); + assertEquals(5L, rocksDBStore.approximateNumEntries(), "Expected 0 entries on legacy CF, 5 in headers-aware CF after migrating key5 with putIfAbsent(null)"); + + assertNull(rocksDBStore.putIfAbsent(new Bytes("key12".getBytes()), null)); + assertEquals(4L, rocksDBStore.approximateNumEntries(), "Expected 0 entries on legacy CF, 4 in headers-aware CF after putIfAbsent with null on non-existing key12"); + + // delete() - tests migration on delete + + assertEquals(1 + 0 + 8 + 6, rocksDBStore.delete(new Bytes("key6".getBytes())).length, + "Expected header-aware format: varint(1) + empty headers(0) + timestamp(8) + value(6) = 15 bytes for delete() on existing key6"); + assertEquals(3L, rocksDBStore.approximateNumEntries(), "Expected 0 entries on legacy CF, 3 in headers-aware CF after deleting key6 with delete()"); + + // iterators should not trigger migration (read-only) + iteratorsShouldNotMigrateData(); + assertEquals(3L, rocksDBStore.approximateNumEntries()); + + rocksDBStore.close(); + + // Verify the final state of both column families + verifyOldAndNewColumnFamily(); + } + + private void iteratorsShouldNotMigrateData() { + // iterating should not migrate any data, but return all keys over both CFs + // Values from legacy CF are converted to header-aware format: [varint][headers][timestamp][value] + try (final KeyValueIterator<Bytes, byte[]> itAll = rocksDBStore.all()) { + { + final KeyValue<Bytes, byte[]> keyValue = itAll.next(); + assertArrayEquals("key1".getBytes(), keyValue.key.get()); + assertEquals(10, keyValue.value.length, "Expected header-aware format: varint(0) + empty headers(0) + timestamp(8) + value(1) = 10 bytes for key1 from legacy CF"); Review Comment: I reponded above. the iterator does not migrate any thing but does the conversion on the fly after iterating if when we call `verifyLegacyTimestampedColumnFamily` and `verifyHeadersColumnFamily` we see that nothing is migrated -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
