Skip to content

Commit 091a201

Browse files
autophagytillrohrmann
authored andcommitted
[FLINK-14482][rocksdb] Bump FRocksDB version to 6.20.3
This closes apache#16794.
1 parent d1b15ba commit 091a201

File tree

8 files changed

+38
-13
lines changed

8 files changed

+38
-13
lines changed

docs/layouts/shortcodes/generated/rocksdb_configurable_configuration.html

+1-1
Original file line numberDiff line numberDiff line change
@@ -48,7 +48,7 @@
4848
<td><h5>state.backend.rocksdb.compaction.style</h5></td>
4949
<td style="word-wrap: break-word;">(none)</td>
5050
<td><p>Enum</p></td>
51-
<td>The specified compaction style for DB. Candidate compaction style is LEVEL, FIFO or UNIVERSAL, and RocksDB choose 'LEVEL' as default style.<br /><br />Possible values:<ul><li>"LEVEL"</li><li>"UNIVERSAL"</li><li>"FIFO"</li></ul></td>
51+
<td>The specified compaction style for DB. Candidate compaction style is LEVEL, FIFO, UNIVERSAL or NONE, and RocksDB choose 'LEVEL' as default style.<br /><br />Possible values:<ul><li>"LEVEL"</li><li>"UNIVERSAL"</li><li>"FIFO"</li><li>"NONE"</li></ul></td>
5252
</tr>
5353
<tr>
5454
<td><h5>state.backend.rocksdb.files.open</h5></td>

flink-dist/src/main/resources/META-INF/NOTICE

+1-1
Original file line numberDiff line numberDiff line change
@@ -6,7 +6,7 @@ The Apache Software Foundation (http://www.apache.org/).
66

77
This project bundles the following dependencies under the Apache Software License 2.0 (http://www.apache.org/licenses/LICENSE-2.0.txt)
88

9-
- com.data-artisans:frocksdbjni:5.17.2-artisans-2.0
9+
- com.ververica:frocksdbjni:6.20.3-ververica-1.0
1010
- com.google.code.findbugs:jsr305:1.3.9
1111
- com.twitter:chill-java:0.7.6
1212
- com.twitter:chill_2.11:0.7.6

flink-state-backends/flink-statebackend-rocksdb/pom.xml

+1-1
Original file line numberDiff line numberDiff line change
@@ -57,7 +57,7 @@ under the License.
5757
<dependency>
5858
<groupId>com.ververica</groupId>
5959
<artifactId>frocksdbjni</artifactId>
60-
<version>5.17.2-ververica-2.1</version>
60+
<version>6.20.3-ververica-1.0</version>
6161
</dependency>
6262

6363
<!-- test dependencies -->

flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBConfigurableOptions.java

+7-2
Original file line numberDiff line numberDiff line change
@@ -31,6 +31,7 @@
3131
import static org.apache.flink.configuration.description.LinkElement.link;
3232
import static org.rocksdb.CompactionStyle.FIFO;
3333
import static org.rocksdb.CompactionStyle.LEVEL;
34+
import static org.rocksdb.CompactionStyle.NONE;
3435
import static org.rocksdb.CompactionStyle.UNIVERSAL;
3536
import static org.rocksdb.InfoLogLevel.DEBUG_LEVEL;
3637
import static org.rocksdb.InfoLogLevel.ERROR_LEVEL;
@@ -102,9 +103,13 @@ public class RocksDBConfigurableOptions implements Serializable {
102103
.noDefaultValue()
103104
.withDescription(
104105
String.format(
105-
"The specified compaction style for DB. Candidate compaction style is %s, %s or %s, "
106+
"The specified compaction style for DB. Candidate compaction style is %s, %s, %s or %s, "
106107
+ "and RocksDB choose '%s' as default style.",
107-
LEVEL.name(), FIFO.name(), UNIVERSAL.name(), LEVEL.name()));
108+
LEVEL.name(),
109+
FIFO.name(),
110+
UNIVERSAL.name(),
111+
NONE.name(),
112+
LEVEL.name()));
108113

109114
public static final ConfigOption<Boolean> USE_DYNAMIC_LEVEL_SIZE =
110115
key("state.backend.rocksdb.compaction.level.use-dynamic-size")

flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBResourceContainer.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -250,7 +250,7 @@ private boolean overwriteFilterIfExist(BlockBasedTableConfig blockBasedTableConf
250250
@VisibleForTesting
251251
static Filter getFilterFromBlockBasedTableConfig(BlockBasedTableConfig blockBasedTableConfig)
252252
throws NoSuchFieldException, IllegalAccessException {
253-
Field filterField = blockBasedTableConfig.getClass().getDeclaredField("filter_");
253+
Field filterField = blockBasedTableConfig.getClass().getDeclaredField("filterPolicy");
254254
filterField.setAccessible(true);
255255
Object filter = filterField.get(blockBasedTableConfig);
256256
filterField.setAccessible(false);

flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksIteratorWrapper.java

+19
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,7 @@
2727
import javax.annotation.Nonnull;
2828

2929
import java.io.Closeable;
30+
import java.nio.ByteBuffer;
3031

3132
/**
3233
* This is a wrapper around {@link RocksIterator} to check the iterator status for all the methods
@@ -74,6 +75,18 @@ public void seekForPrev(byte[] target) {
7475
status();
7576
}
7677

78+
@Override
79+
public void seek(ByteBuffer target) {
80+
iterator.seek(target);
81+
status();
82+
}
83+
84+
@Override
85+
public void seekForPrev(ByteBuffer target) {
86+
iterator.seekForPrev(target);
87+
status();
88+
}
89+
7790
@Override
7891
public void next() {
7992
iterator.next();
@@ -95,6 +108,12 @@ public void status() {
95108
}
96109
}
97110

111+
@Override
112+
public void refresh() throws RocksDBException {
113+
iterator.refresh();
114+
status();
115+
}
116+
98117
public byte[] key() {
99118
return iterator.key();
100119
}

flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBInitTest.java

+5-4
Original file line numberDiff line numberDiff line change
@@ -28,7 +28,7 @@
2828
import org.powermock.api.mockito.PowerMockito;
2929
import org.powermock.core.classloader.annotations.PrepareForTest;
3030
import org.powermock.modules.junit4.PowerMockRunner;
31-
import org.rocksdb.RocksDB;
31+
import org.rocksdb.NativeLibraryLoader;
3232
import org.slf4j.Logger;
3333
import org.slf4j.LoggerFactory;
3434

@@ -39,7 +39,7 @@
3939

4040
/** Tests for {@link EmbeddedRocksDBStateBackend} on initialization. */
4141
@RunWith(PowerMockRunner.class)
42-
@PrepareForTest({RocksDB.class})
42+
@PrepareForTest({NativeLibraryLoader.class})
4343
public class RocksDBInitTest {
4444

4545
private static final Logger LOG = LoggerFactory.getLogger(RocksDBInitTest.class);
@@ -57,8 +57,9 @@ public void testResetInitFlag() throws Exception {
5757

5858
@Test
5959
public void testTempLibFolderDeletedOnFail() throws Exception {
60-
PowerMockito.spy(RocksDB.class);
61-
PowerMockito.when(RocksDB.class, "loadLibrary").thenThrow(new ExpectedTestException());
60+
PowerMockito.spy(NativeLibraryLoader.class);
61+
PowerMockito.when(NativeLibraryLoader.class, "getInstance")
62+
.thenThrow(new ExpectedTestException());
6263

6364
File tempFolder = temporaryFolder.newFolder();
6465
try {

flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBResourceContainerTest.java

+3-3
Original file line numberDiff line numberDiff line change
@@ -173,15 +173,15 @@ private Cache getBlockCache(ColumnFamilyOptions columnOptions) {
173173
}
174174
Field cacheField = null;
175175
try {
176-
cacheField = BlockBasedTableConfig.class.getDeclaredField("blockCache_");
176+
cacheField = BlockBasedTableConfig.class.getDeclaredField("blockCache");
177177
} catch (NoSuchFieldException e) {
178-
fail("blockCache_ is not defined");
178+
fail("blockCache is not defined");
179179
}
180180
cacheField.setAccessible(true);
181181
try {
182182
return (Cache) cacheField.get(blockBasedTableConfig);
183183
} catch (IllegalAccessException e) {
184-
fail("Cannot access blockCache_ field.");
184+
fail("Cannot access blockCache field.");
185185
return null;
186186
}
187187
}

0 commit comments

Comments
 (0)