Skip to content

Commit 0963189

Browse files
committed
[core] Load RocksDB library in advance and catch exceptions
1 parent 3450061 commit 0963189

File tree

2 files changed

+63
-0
lines changed

2 files changed

+63
-0
lines changed

paimon-core/src/main/java/org/apache/paimon/lookup/rocksdb/RocksDBStateFactory.java

Lines changed: 11 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -29,6 +29,8 @@
2929
import org.rocksdb.RocksDB;
3030
import org.rocksdb.RocksDBException;
3131
import org.rocksdb.TtlDB;
32+
import org.slf4j.Logger;
33+
import org.slf4j.LoggerFactory;
3234

3335
import javax.annotation.Nullable;
3436

@@ -39,6 +41,8 @@
3941
/** Factory to create state. */
4042
public class RocksDBStateFactory implements StateFactory {
4143

44+
private static final Logger LOG = LoggerFactory.getLogger(RocksDBStateFactory.class);
45+
4246
public static final String MERGE_OPERATOR_NAME = "stringappendtest";
4347

4448
private final Options options;
@@ -50,6 +54,13 @@ public class RocksDBStateFactory implements StateFactory {
5054
public RocksDBStateFactory(
5155
String path, org.apache.paimon.options.Options conf, @Nullable Duration ttlSecs)
5256
throws IOException {
57+
try {
58+
RocksDB.loadLibrary();
59+
} catch (Throwable e) {
60+
LOG.error("Fail to load RocksDB library.", e);
61+
throw new IOException("Fail to load RocksDB library.", e);
62+
}
63+
5364
DBOptions dbOptions =
5465
RocksDBOptions.createDBOptions(
5566
new DBOptions()

paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/lookup/RocksDBListStateTest.java

Lines changed: 52 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -33,12 +33,17 @@
3333

3434
import org.junit.jupiter.api.Test;
3535
import org.junit.jupiter.api.io.TempDir;
36+
import org.mockito.MockedStatic;
37+
import org.mockito.Mockito;
38+
import org.rocksdb.RocksDB;
3639

40+
import java.io.IOException;
3741
import java.nio.file.Path;
3842
import java.util.ArrayList;
3943
import java.util.List;
4044

4145
import static org.assertj.core.api.Assertions.assertThat;
46+
import static org.assertj.core.api.Assertions.assertThatThrownBy;
4247

4348
/** Test for {@link RocksDBListState}. */
4449
public class RocksDBListStateTest {
@@ -71,6 +76,53 @@ void test() throws Exception {
7176
factory.close();
7277
}
7378

79+
@Test
80+
void testRocksDBLibraryLoadSuccess() throws Exception {
81+
// Test that RocksDBStateFactory can be created successfully when RocksDB library loads
82+
// properly
83+
RocksDBStateFactory factory =
84+
new RocksDBStateFactory(tempDir.toString(), new Options(), null);
85+
86+
// Verify that the factory is created and can be used
87+
assertThat(factory).isNotNull();
88+
assertThat(factory.db()).isNotNull();
89+
assertThat(factory.path()).isEqualTo(tempDir.toString());
90+
91+
factory.close();
92+
}
93+
94+
@Test
95+
void testRocksDBLibraryLoadFailure() {
96+
// Test that IOException is thrown when RocksDB library fails to load
97+
try (MockedStatic<RocksDB> mockedRocksDB = Mockito.mockStatic(RocksDB.class)) {
98+
mockedRocksDB
99+
.when(RocksDB::loadLibrary)
100+
.thenThrow(new RuntimeException("Failed to load RocksDB library"));
101+
102+
assertThatThrownBy(
103+
() -> new RocksDBStateFactory(tempDir.toString(), new Options(), null))
104+
.isInstanceOf(IOException.class)
105+
.hasMessage("Fail to load RocksDB library.")
106+
.hasCauseInstanceOf(RuntimeException.class);
107+
}
108+
}
109+
110+
@Test
111+
void testRocksDBLibraryLoadWithDifferentExceptionTypes() {
112+
// Test that IOException is thrown for different types of exceptions during library loading
113+
try (MockedStatic<RocksDB> mockedRocksDB = Mockito.mockStatic(RocksDB.class)) {
114+
mockedRocksDB
115+
.when(RocksDB::loadLibrary)
116+
.thenThrow(new UnsatisfiedLinkError("Native library not found"));
117+
118+
assertThatThrownBy(
119+
() -> new RocksDBStateFactory(tempDir.toString(), new Options(), null))
120+
.isInstanceOf(IOException.class)
121+
.hasMessage("Fail to load RocksDB library.")
122+
.hasCauseInstanceOf(UnsatisfiedLinkError.class);
123+
}
124+
}
125+
74126
public GenericRow row(String value) {
75127
return GenericRow.of(bs(value));
76128
}

0 commit comments

Comments
 (0)