Skip to content
Draft
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
28 commits
Select commit Hold shift + click to select a range
f822b30
HDDS-14225. Upgrade RocksDB from 7.7.3 to 10.4.2
smengcl Feb 24, 2026
d7e5170
findbugs
smengcl Feb 24, 2026
e42ca37
Patch rocks-native.patch
smengcl Feb 24, 2026
6c960eb
Address comment https://github.com/apache/ozone/pull/9813#pullrequest…
smengcl Mar 9, 2026
471e6dc
Merge branch 'master' into rocksdb-v10-upgrade
smengcl Mar 9, 2026
7260d5b
Fix RocksDB JNI 10.4.2 runtime classpath
smengcl Mar 9, 2026
24dd415
Fix snapshot existence checks after RocksDB 9.10.0 keyMayExist behavi…
smengcl Mar 9, 2026
7b49c19
Handle RocksDB keyMayExist buffer semantics and db restore
smengcl Mar 10, 2026
21bc915
Switch to self-built rocksdbjni 10.10.1 snapshot fat jar
smengcl Mar 10, 2026
7270ac0
Adjust RDBTableStore metrics tests for keyMayExist fallback
smengcl Mar 10, 2026
84da109
Implement getSkipCache in InMemoryTestTable
smengcl Mar 10, 2026
f67327d
Fix replicas-test.sh
smengcl Mar 11, 2026
8ab8f58
Fix replicas-test.sh attempt 2
smengcl Mar 11, 2026
b3715f0
Fix replicas-test.sh attempt 3
smengcl Mar 11, 2026
602e0db
Fix replicas-test.sh attempt 4
smengcl Mar 11, 2026
9ef409e
Fix replicas-test.sh attempt 5
smengcl Mar 11, 2026
d416db1
Fix replicas-test.sh attempt 6
smengcl Mar 11, 2026
1044f33
Handle already-stopped datanodes in replicas-test.sh
smengcl Mar 12, 2026
4c971a3
Use upstream rocksdbjni 10.5.1 for testing
smengcl Mar 12, 2026
ab967ec
Revert "Use upstream rocksdbjni 10.5.1 for testing"
smengcl Mar 13, 2026
763440d
Restore replicas-test.sh state offline
smengcl Mar 15, 2026
ed1d278
Use official rocksdbjni 10.10.1 release
smengcl Mar 30, 2026
7f3abff
Sort pom
smengcl Mar 31, 2026
9b6f43f
Remove unneeded pom changes
smengcl Apr 20, 2026
e2f4ca9
Restore pom line
smengcl Apr 20, 2026
0f389c7
Merge remote-tracking branch 'asf' into rocksdb-v10-upgrade
smengcl Apr 20, 2026
626b417
Fix replicas-test
smengcl Apr 20, 2026
7b47f38
Use 10.10.1.1, which presumably has some packaging side fixes
smengcl Apr 23, 2026
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -87,6 +87,7 @@ public ManagedBlockBasedTableConfig getBlockBasedTableConfig() {
ManagedBlockBasedTableConfig config = new ManagedBlockBasedTableConfig();
config.setBlockCache(new ManagedLRUCache(blockCacheSize))
.setBlockSize(blockSize)
.setFormatVersion(BLOCK_BASED_TABLE_FORMAT_VERSION)
Comment thread
smengcl marked this conversation as resolved.
.setPinL0FilterAndIndexBlocksInCache(true)
.setFilterPolicy(new ManagedBloomFilter());
return config;
Expand Down Expand Up @@ -145,6 +146,9 @@ public ManagedBlockBasedTableConfig getBlockBasedTableConfig() {
}
};

// Keep SST/block-based table format stable across RocksDB upgrades.
private static final int BLOCK_BASED_TABLE_FORMAT_VERSION = 5;

public static long toLong(double value) {
BigDecimal temp = BigDecimal.valueOf(value);
return temp.longValue();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -99,16 +99,16 @@ public boolean isEmpty() throws RocksDatabaseException {
@Override
public boolean isExist(byte[] key) throws RocksDatabaseException {
rdbMetrics.incNumDBKeyMayExistChecks();
final Supplier<byte[]> holder = db.keyMayExist(family, key);
if (holder == null) {
return false; // definitely not exists
}
final byte[] value = holder.get();
if (value != null) {
return true; // definitely exists
final Supplier<byte[]> valueSupplier = db.keyMayExist(family, key);
if (valueSupplier != null) {
final byte[] value = valueSupplier.get();
if (value != null) {
return true; // definitely exists
}
}

// inconclusive: the key may or may not exist
// keyMayExist can be a false-negative in some RocksDB setups (for example,
// snapshot/checkpoint DBs). Verify via point-get to preserve correctness.
final boolean exists = get(key) != null;
Comment on lines +110 to 112
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I am trying to understand more about this comment or how we can reach this stage.

The keyMayExist documentation says that it will never return a false-negative.

Are we trying to say here that a key may not exist in the snapshot/checkpoint DB but may exist in the active DB? In that case the get(key) is still running on the same db which would be the snapshot DB instead of the active DB and again not find the key.

If the key definitely does not exist in the database, then this method
returns false, otherwise it returns true if the key might exist.
That is to say that this method is probabilistic and may return false
positives, but never a false negative.

if (!exists) {
rdbMetrics.incNumDBKeyMayExistMisses();
Expand Down Expand Up @@ -141,15 +141,15 @@ public byte[] getSkipCache(byte[] bytes) throws RocksDatabaseException {
@Override
public byte[] getIfExist(byte[] key) throws RocksDatabaseException {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

does this API provide definitive result?
if not, the javadoc needs to update

rdbMetrics.incNumDBKeyGetIfExistChecks();
final Supplier<byte[]> value = db.keyMayExist(family, key);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

from commit history:
24dd415

Fix snapshot existence checks after RocksDB 9.10.0 keyMayExist behavior change
RocksDB 9.10.0 changed DB::KeyMayExist behavior semantics to follow its function comment. Ozone snapshot code paths were treating keyMayExist/getIfExist misses as definitive, which could misclassify existing keys in snapshot-related flows under RocksDB >= 9.10.0 and cause test failures.

Treat keyMayExist/getIfExist as hints and fall back to point reads before deciding not-found:

  • RDBTable: verify with get()/get(ByteBuffer) on inconclusive keyMayExist
  • TypedTable: in codec-buffer isExist path, fallback to full get before returning false
  • KeyManagerImpl and ReclaimableRenameEntryFilter: use getSkipCache fallback for snapshot rename lookups

RocksDB changelog for reference:
https://github.com/facebook/rocksdb/releases/tag/v9.10.0

Behavior Changes
DB::KeyMayExist() now follows its function comment, which means value parameter can be null, and it will be set only if value_found is passed in.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@smengcl can you put the above keyMayExist() behavior change in the PR description.
The goal is to ensure the Ozone API does not change semantics after RocksDB update.

if (value == null) {
return null; // definitely not exists
}
if (value.get() != null) {
return value.get(); // definitely exists
final Supplier<byte[]> valueSupplier = db.keyMayExist(family, key);
if (valueSupplier != null) {
final byte[] value = valueSupplier.get();
if (value != null) {
return value; // definitely exists
}
}

// inconclusive: the key may or may not exist
// keyMayExist is treated as a hint only; confirm via point-get.
rdbMetrics.incNumDBKeyGetIfExistGets();
final byte[] val = get(key);
if (val == null) {
Expand All @@ -160,19 +160,21 @@ public byte[] getIfExist(byte[] key) throws RocksDatabaseException {

Integer getIfExist(ByteBuffer key, ByteBuffer outValue) throws RocksDatabaseException {
rdbMetrics.incNumDBKeyGetIfExistChecks();
// keyMayExist may change key buffer position; never reuse the same
// ByteBuffer instance for fallback point-get.
final Supplier<Integer> value = db.keyMayExist(
family, key, outValue.duplicate());
if (value == null) {
return null; // definitely not exists
}
if (value.get() != null) {
// definitely exists, return value size.
return value.get();
family, key.duplicate(), outValue.duplicate());
if (value != null) {
final Integer length = value.get();
if (length != null) {
// definitely exists, return value size.
return length;
}
}

// inconclusive: the key may or may not exist
// keyMayExist is treated as a hint only; confirm via point-get.
rdbMetrics.incNumDBKeyGetIfExistGets();
final Integer val = get(key, outValue);
final Integer val = get(key.duplicate(), outValue);
if (val == null) {
rdbMetrics.incNumDBKeyGetIfExistMisses();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -626,8 +626,11 @@ Supplier<byte[]> keyMayExist(ColumnFamily family, byte[] key)
Supplier<Integer> keyMayExist(ColumnFamily family,
ByteBuffer key, ByteBuffer out) throws RocksDatabaseException {
try (UncheckedAutoCloseable ignored = acquire()) {
// keyMayExist may advance the input ByteBuffer position in native code.
// Always pass a duplicate so callers can safely reuse the original key
// buffer for a follow-up point-get.
final KeyMayExist result = db.get().keyMayExist(
family.getHandle(), key, out);
family.getHandle(), key.duplicate(), out);
switch (result.exists) {
case kNotExist: return null;
case kExistsWithValue: return () -> result.valueLength;
Expand Down Expand Up @@ -888,6 +891,12 @@ public void deleteFilesNotMatchingPrefix(TablePrefixInfo prefixInfo) throws Rock
boolean isKeyWithPrefixPresent = RocksDiffUtils.isKeyWithPrefixPresent(
prefixForColumnFamily, firstDbKey, lastDbKey);
if (!isKeyWithPrefixPresent) {
ColumnFamilyHandle handle = getColumnFamilyHandle(sstFileColumnFamily);
if (handle == null) {
LOG.warn("Skipping sst file deletion for {}: no handle found for column family {}",
liveFileMetaData.fileName(), sstFileColumnFamily);
continue;
}
LOG.info("Deleting sst file: {} with start key: {} and end key: {} "
+ "corresponding to column family {} from db: {}. "
+ "Prefix for the column family: {}.",
Expand All @@ -896,7 +905,7 @@ public void deleteFilesNotMatchingPrefix(TablePrefixInfo prefixInfo) throws Rock
StringUtils.bytes2String(liveFileMetaData.columnFamilyName()),
db.get().getName(),
prefixForColumnFamily);
db.deleteFile(liveFileMetaData);
db.deleteFile(handle, liveFileMetaData);
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -179,11 +179,17 @@ public boolean isExist(KEY key) throws RocksDatabaseException, CodecException {
} else if (keyCodec.supportCodecBuffer()) {
// keyCodec.supportCodecBuffer() is enough since value is not needed.
try (CodecBuffer inKey = keyCodec.toDirectCodecBuffer(key)) {
// Use zero capacity buffer since value is not needed.
// Try the lightweight "existence only" check first.
try (CodecBuffer outValue = CodecBuffer.getEmptyBuffer()) {
return getFromTableIfExist(inKey, outValue) != null;
if (getFromTableIfExist(inKey, outValue) != null) {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

shouldn't this Ozone API be definitive?

return true;
}
}
}
// keyMayExist/getIfExist can still be false-negative in some RocksDB
// configurations (observed with snapshot/checkpoint DBs), so confirm with
// a regular point lookup before returning false.
return getFromTable(key) != null;
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is this really necessary? This means an additional lookup.

} else {
return rawTable.isExist(encodeKey(key));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -80,6 +80,11 @@ public VALUE getIfExist(KEY key) {
return map.get(key);
}

@Override
public VALUE getSkipCache(KEY key) {
return map.get(key);
}

@Override
public void delete(KEY key) {
map.remove(key);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -100,13 +100,13 @@ Answer<Integer> newAnswer(String name, byte... b) {
public void testForEachRemaining() throws Exception {
when(rocksIteratorMock.isValid())
.thenReturn(true, true, true, true, true, true, true, false);
when(rocksIteratorMock.key(any()))
when(rocksIteratorMock.key(any(ByteBuffer.class)))
.then(newAnswerInt("key1", 0x00))
.then(newAnswerInt("key2", 0x00))
.then(newAnswerInt("key3", 0x01))
.then(newAnswerInt("key4", 0x02))
.thenThrow(new NoSuchElementException());
when(rocksIteratorMock.value(any()))
when(rocksIteratorMock.value(any(ByteBuffer.class)))
.then(newAnswerInt("val1", 0x7f))
.then(newAnswerInt("val2", 0x7f))
.then(newAnswerInt("val3", 0x7e))
Expand Down Expand Up @@ -152,8 +152,8 @@ public void testNextCallsIsValidThenGetsTheValueAndStepsToNext()
}

verifier.verify(rocksIteratorMock).isValid();
verifier.verify(rocksIteratorMock).key(any());
verifier.verify(rocksIteratorMock).value(any());
verifier.verify(rocksIteratorMock).key(any(ByteBuffer.class));
verifier.verify(rocksIteratorMock).value(any(ByteBuffer.class));
verifier.verify(rocksIteratorMock).next();

CodecTestUtil.gc();
Expand Down Expand Up @@ -192,9 +192,9 @@ public void testSeekToLastSeeks() throws Exception {
@Test
public void testSeekReturnsTheActualKey() throws Exception {
when(rocksIteratorMock.isValid()).thenReturn(true);
when(rocksIteratorMock.key(any()))
when(rocksIteratorMock.key(any(ByteBuffer.class)))
.then(newAnswerInt("key1", 0x00));
when(rocksIteratorMock.value(any()))
when(rocksIteratorMock.value(any(ByteBuffer.class)))
.then(newAnswerInt("val1", 0x7f));

try (RDBStoreCodecBufferIterator i = newIterator();
Expand All @@ -208,8 +208,8 @@ public void testSeekReturnsTheActualKey() throws Exception {
verifier.verify(rocksIteratorMock, times(1))
.seek(any(ByteBuffer.class));
verifier.verify(rocksIteratorMock, times(1)).isValid();
verifier.verify(rocksIteratorMock, times(1)).key(any());
verifier.verify(rocksIteratorMock, times(1)).value(any());
verifier.verify(rocksIteratorMock, times(1)).key(any(ByteBuffer.class));
verifier.verify(rocksIteratorMock, times(1)).value(any(ByteBuffer.class));
assertArrayEquals(new byte[]{0x00}, val.getKey().getArray());
assertArrayEquals(new byte[]{0x7f}, val.getValue().getArray());
}
Expand All @@ -220,7 +220,7 @@ public void testSeekReturnsTheActualKey() throws Exception {
@Test
public void testGettingTheKeyIfIteratorIsValid() throws Exception {
when(rocksIteratorMock.isValid()).thenReturn(true);
when(rocksIteratorMock.key(any()))
when(rocksIteratorMock.key(any(ByteBuffer.class)))
.then(newAnswerInt("key1", 0x00));

byte[] key = null;
Expand All @@ -233,7 +233,7 @@ public void testGettingTheKeyIfIteratorIsValid() throws Exception {
InOrder verifier = inOrder(rocksIteratorMock);

verifier.verify(rocksIteratorMock, times(1)).isValid();
verifier.verify(rocksIteratorMock, times(1)).key(any());
verifier.verify(rocksIteratorMock, times(1)).key(any(ByteBuffer.class));
assertArrayEquals(new byte[]{0x00}, key);

CodecTestUtil.gc();
Expand All @@ -242,9 +242,9 @@ public void testGettingTheKeyIfIteratorIsValid() throws Exception {
@Test
public void testGettingTheValueIfIteratorIsValid() throws Exception {
when(rocksIteratorMock.isValid()).thenReturn(true);
when(rocksIteratorMock.key(any()))
when(rocksIteratorMock.key(any(ByteBuffer.class)))
.then(newAnswerInt("key1", 0x00));
when(rocksIteratorMock.value(any()))
when(rocksIteratorMock.value(any(ByteBuffer.class)))
.then(newAnswerInt("val1", 0x7f));

byte[] key = null;
Expand All @@ -260,7 +260,7 @@ public void testGettingTheValueIfIteratorIsValid() throws Exception {
InOrder verifier = inOrder(rocksIteratorMock);

verifier.verify(rocksIteratorMock, times(1)).isValid();
verifier.verify(rocksIteratorMock, times(1)).key(any());
verifier.verify(rocksIteratorMock, times(1)).key(any(ByteBuffer.class));
assertArrayEquals(new byte[]{0x00}, key);
assertArrayEquals(new byte[]{0x7f}, value);

Expand All @@ -272,7 +272,7 @@ public void testRemovingFromDBActuallyDeletesFromTable() throws Exception {
final byte[] testKey = new byte[10];
ThreadLocalRandom.current().nextBytes(testKey);
when(rocksIteratorMock.isValid()).thenReturn(true);
when(rocksIteratorMock.key(any()))
when(rocksIteratorMock.key(any(ByteBuffer.class)))
.then(newAnswer("key1", testKey));

try (RDBStoreCodecBufferIterator i = newIterator(null)) {
Expand Down Expand Up @@ -320,7 +320,7 @@ public void testNullPrefixedIterator() throws Exception {
when(rocksIteratorMock.isValid()).thenReturn(true);
assertTrue(i.hasNext());
verify(rocksIteratorMock, times(1)).isValid();
verify(rocksIteratorMock, times(0)).key(any());
verify(rocksIteratorMock, times(0)).key(any(ByteBuffer.class));

i.seekToLast();
verify(rocksIteratorMock, times(1)).seekToLast();
Expand All @@ -343,11 +343,11 @@ public void testNormalPrefixedIterator() throws Exception {
clearInvocations(rocksIteratorMock);

when(rocksIteratorMock.isValid()).thenReturn(true);
when(rocksIteratorMock.key(any()))
when(rocksIteratorMock.key(any(ByteBuffer.class)))
.then(newAnswer("key1", prefixBytes));
assertTrue(i.hasNext());
verify(rocksIteratorMock, times(1)).isValid();
verify(rocksIteratorMock, times(1)).key(any());
verify(rocksIteratorMock, times(1)).key(any(ByteBuffer.class));

Exception e =
assertThrows(Exception.class, () -> i.seekToLast(), "Prefixed iterator does not support seekToLast");
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,68 @@
/*
* 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.hadoop.hdds.utils.db;

import static java.nio.charset.StandardCharsets.UTF_8;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;

import java.nio.ByteBuffer;
import java.util.function.Supplier;
import org.apache.hadoop.hdds.utils.db.RocksDatabase.ColumnFamily;
import org.junit.jupiter.api.Test;

/**
* Unit tests for {@link RDBTable}.
*/
public class TestRDBTable {

@Test
public void testGetIfExistByteBufferFallbackUsesFreshKeyBuffer()
throws Exception {
RocksDatabase db = mock(RocksDatabase.class);
ColumnFamily columnFamily = mock(ColumnFamily.class);
RDBMetrics metrics = mock(RDBMetrics.class);
RDBTable table = new RDBTable(db, columnFamily, metrics);

byte[] keyBytes = "key-1".getBytes(UTF_8);
ByteBuffer key = ByteBuffer.wrap(keyBytes);
ByteBuffer outValue = ByteBuffer.allocate(64);

when(db.keyMayExist(eq(columnFamily), any(ByteBuffer.class),
any(ByteBuffer.class))).thenAnswer(invocation -> {
// Simulate JNI calls that advance the key position.
ByteBuffer keyBuffer = invocation.getArgument(1);
keyBuffer.position(keyBuffer.limit());
return (Supplier<Integer>) () -> null;
});

when(db.get(eq(columnFamily), any(ByteBuffer.class), any(ByteBuffer.class)))
.thenAnswer(invocation -> {
ByteBuffer keyBuffer = invocation.getArgument(1);
return keyBuffer.remaining() == keyBytes.length ? 0 : null;
});

Integer result = table.getIfExist(key, outValue);
assertEquals(0, result);
assertEquals(0, key.position(), "caller key buffer position must be unchanged");
}
}

Original file line number Diff line number Diff line change
Expand Up @@ -368,8 +368,8 @@ public void testIsExist() throws Exception {

RDBMetrics rdbMetrics = rdbStore.getMetrics();
assertEquals(3, rdbMetrics.getNumDBKeyMayExistChecks());
assertEquals(0, rdbMetrics.getNumDBKeyMayExistMisses());
assertEquals(2, rdbMetrics.getNumDBKeyGets());
assertEquals(2, rdbMetrics.getNumDBKeyMayExistMisses());
assertEquals(4, rdbMetrics.getNumDBKeyGets());

// Reinsert key for further testing.
testTable.put(key, value);
Expand Down Expand Up @@ -437,9 +437,8 @@ public void testGetIfExist() throws Exception {
RDBMetrics rdbMetrics = rdbStore.getMetrics();
assertEquals(3, rdbMetrics.getNumDBKeyGetIfExistChecks());

assertEquals(0, rdbMetrics.getNumDBKeyGetIfExistMisses());

assertEquals(0, rdbMetrics.getNumDBKeyGetIfExistGets());
assertEquals(2, rdbMetrics.getNumDBKeyGetIfExistMisses());
assertEquals(2, rdbMetrics.getNumDBKeyGetIfExistGets());

// Reinsert key for further testing.
testTable.put(key, value);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,16 @@
public class ManagedBloomFilter extends BloomFilter {
private final UncheckedAutoCloseable leakTracker = track(this);

@Override
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks unnecessary to me

public boolean equals(Object obj) {
return super.equals(obj);
}

@Override
public int hashCode() {
return super.hashCode();
}

@Override
public void close() {
try {
Expand Down
Loading