Skip to content

Commit

Permalink
java / jni io_uring support (facebook#9224)
Browse files Browse the repository at this point in the history
Summary:
Existing multiGet() in java calls multi_get_helper() which then calls DB::std::vector MultiGet(). This doesn't take advantage of io_uring.

This change adds another JNI level method that runs a parallel code path using the DB::void MultiGet(), using ByteBuffers at the JNI level. We call it multiGetDirect(). In addition to using the io_uring path, this code internally returns pinned slices which we can copy out of into our direct byte buffers; this should reduce the overall number of copies in the code path to/from Java. Some jmh benchmark runs (100k keys, 1000 key multiGet) suggest that for value sizes > 1k, we see about a 20% performance improvement, although performance is slightly reduced for small value sizes, there's a little bit more overhead in the JNI methods.

Closes facebook#8407

Pull Request resolved: facebook#9224

Reviewed By: mrambacher

Differential Revision: D32951754

Pulled By: jay-zhuang

fbshipit-source-id: 1f70df7334be2b6c42a9c8f92725f67c71631690
  • Loading branch information
alanpaxton authored and facebook-github-bot committed Dec 16, 2021
1 parent 7ac3a5d commit c1ec0b2
Show file tree
Hide file tree
Showing 10 changed files with 1,083 additions and 70 deletions.
1 change: 1 addition & 0 deletions java/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -112,6 +112,7 @@ set(JAVA_MAIN_CLASSES
src/main/java/org/rocksdb/BlockBasedTableConfig.java
src/main/java/org/rocksdb/BloomFilter.java
src/main/java/org/rocksdb/BuiltinComparator.java
src/main/java/org/rocksdb/ByteBufferGetStatus.java
src/main/java/org/rocksdb/Cache.java
src/main/java/org/rocksdb/CassandraCompactionFilter.java
src/main/java/org/rocksdb/CassandraValueMergeOperator.java
Expand Down
6 changes: 6 additions & 0 deletions java/jmh/README.md
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,12 @@ These are micro-benchmarks for RocksJava functionality, using [JMH (Java Microbe

**Note**: This uses a specific build of RocksDB that is set in the `<version>` element of the `dependencies` section of the `pom.xml` file. If you are testing local changes you should build and install a SNAPSHOT version of rocksdbjni, and update the `pom.xml` of rocksdbjni-jmh file to test with this.

For instance, this is how to install the OSX jar you just built for 6.26.0

```bash
$ mvn install:install-file -Dfile=./java/target/rocksdbjni-6.26.0-SNAPSHOT-osx.jar -DgroupId=org.rocksdb -DartifactId=rocksdbjni -Dversion=6.26.0-SNAPSHOT -Dpackaging=jar
```

```bash
$ mvn package
```
Expand Down
2 changes: 1 addition & 1 deletion java/jmh/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,7 @@
<dependency>
<groupId>org.rocksdb</groupId>
<artifactId>rocksdbjni</artifactId>
<version>6.6.0-SNAPSHOT</version>
<version>6.27.0-SNAPSHOT</version>
</dependency>

<dependency>
Expand Down
100 changes: 87 additions & 13 deletions java/jmh/src/main/java/org/rocksdb/jmh/MultiGetBenchmarks.java
Original file line number Diff line number Diff line change
Expand Up @@ -6,23 +6,26 @@
*/
package org.rocksdb.jmh;

import org.openjdk.jmh.annotations.*;
import org.rocksdb.*;
import org.rocksdb.util.FileUtils;
import static org.rocksdb.util.KVUtils.ba;
import static org.rocksdb.util.KVUtils.keys;

import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.concurrent.atomic.AtomicInteger;
import org.openjdk.jmh.annotations.*;
import org.openjdk.jmh.runner.Runner;
import org.openjdk.jmh.runner.RunnerException;
import org.openjdk.jmh.runner.options.OptionsBuilder;
import org.rocksdb.*;
import org.rocksdb.util.FileUtils;

import static org.rocksdb.util.KVUtils.ba;
import static org.rocksdb.util.KVUtils.keys;

@State(Scope.Benchmark)
@State(Scope.Thread)
public class MultiGetBenchmarks {

@Param({
"no_column_family",
"1_column_family",
Expand All @@ -31,8 +34,7 @@ public class MultiGetBenchmarks {
})
String columnFamilyTestType;

@Param("100000")
int keyCount;
@Param({"10000", "25000", "100000"}) int keyCount;

@Param({
"10",
Expand All @@ -42,6 +44,9 @@ public class MultiGetBenchmarks {
})
int multiGetSize;

@Param({"16", "64", "250", "1000", "4000", "16000"}) int valueSize;
@Param({"16"}) int keySize; // big enough

Path dbDir;
DBOptions options;
int cfs = 0; // number of column families
Expand Down Expand Up @@ -85,7 +90,8 @@ public void setup() throws IOException, RocksDBException {
// store initial data for retrieving via get
for (int i = 0; i < cfs; i++) {
for (int j = 0; j < keyCount; j++) {
db.put(cfHandles[i], ba("key" + j), ba("value" + j));
final byte[] paddedValue = Arrays.copyOf(ba("value" + j), valueSize);
db.put(cfHandles[i], ba("key" + j), paddedValue);
}
}

Expand Down Expand Up @@ -149,10 +155,78 @@ private int next(final int inc, final int limit) {
}
}

ByteBuffer keysBuffer;
ByteBuffer valuesBuffer;

List<ByteBuffer> valueBuffersList;
List<ByteBuffer> keyBuffersList;

@Setup
public void allocateSliceBuffers() {
keysBuffer = ByteBuffer.allocateDirect(keyCount * valueSize);
valuesBuffer = ByteBuffer.allocateDirect(keyCount * valueSize);
valueBuffersList = new ArrayList<>();
keyBuffersList = new ArrayList<>();
for (int i = 0; i < keyCount; i++) {
valueBuffersList.add(valuesBuffer.slice());
valuesBuffer.position(i * valueSize);
keyBuffersList.add(keysBuffer.slice());
keysBuffer.position(i * keySize);
}
}

@TearDown
public void freeSliceBuffers() {
valueBuffersList.clear();
}

@Benchmark
public List<byte[]> multiGet10() throws RocksDBException {
final int fromKeyIdx = next(multiGetSize, keyCount);
final List<byte[]> keys = keys(fromKeyIdx, fromKeyIdx + multiGetSize);
return db.multiGetAsList(keys);
if (fromKeyIdx >= 0) {
final List<byte[]> keys = keys(fromKeyIdx, fromKeyIdx + multiGetSize);
final List<byte[]> valueResults = db.multiGetAsList(keys);
for (final byte[] result : valueResults) {
if (result.length != valueSize)
throw new RuntimeException("Test valueSize assumption wrong");
}
}
return new ArrayList<>();
}

@Benchmark
public List<RocksDB.MultiGetInstance> multiGetDirect10() throws RocksDBException {
final int fromKeyIdx = next(multiGetSize, keyCount);
if (fromKeyIdx >= 0) {
final List<ByteBuffer> keys = keys(keyBuffersList, fromKeyIdx, fromKeyIdx + multiGetSize);
final List<RocksDB.MultiGetInstance> results = db.multiGetByteBuffers(
keys, valueBuffersList.subList(fromKeyIdx, fromKeyIdx + multiGetSize));
for (final RocksDB.MultiGetInstance result : results) {
if (result.status.getCode() != Status.Code.Ok)
throw new RuntimeException("Test status assumption wrong");
if (result.valueSize != valueSize)
throw new RuntimeException("Test valueSize assumption wrong");
}
return results;
}
return new ArrayList<>();
}

public static void main(final String[] args) throws RunnerException {
final org.openjdk.jmh.runner.options.Options opt =
new OptionsBuilder()
.include(MultiGetBenchmarks.class.getSimpleName())
.forks(1)
.jvmArgs("-ea")
.warmupIterations(1)
.measurementIterations(2)
.forks(2)
.param("columnFamilyTestType=", "1_column_family")
.param("multiGetSize=", "10", "1000")
.param("keyCount=", "1000")
.output("jmh_output")
.build();

new Runner(opt).run();
}
}
18 changes: 16 additions & 2 deletions java/jmh/src/main/java/org/rocksdb/util/KVUtils.java
Original file line number Diff line number Diff line change
Expand Up @@ -6,11 +6,12 @@
*/
package org.rocksdb.util;

import static java.nio.charset.StandardCharsets.UTF_8;

import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.List;

import static java.nio.charset.StandardCharsets.UTF_8;

public final class KVUtils {

/**
Expand Down Expand Up @@ -55,4 +56,17 @@ public static List<byte[]> keys(final int from, final int to) {
}
return keys;
}

public static List<ByteBuffer> keys(
final List<ByteBuffer> keyBuffers, final int from, final int to) {
final List<ByteBuffer> keys = new ArrayList<>(to - from);
for (int i = from; i < to; i++) {
final ByteBuffer key = keyBuffers.get(i);
key.clear();
key.put(ba("key" + i));
key.flip();
keys.add(key);
}
return keys;
}
}
Loading

0 comments on commit c1ec0b2

Please sign in to comment.