[RocksJava] Column family support
This commit includes the support for the following functionalities: - Single Get/Put operations - WriteBatch operations - Single iterator functionality - Open database with column families - Open database with column families Read/Only - Create column family - Drop column family - Properties of column families - Listing of column families - Fully backwards comptabile implementation - Multi Iterator support - MultiGet - KeyMayExist - Option to create missing column families on open In addition there is are two new Tests: - Test of ColumnFamily functionality - Test of Read only feature to open subsets of column families - Basic test to test the KeyMayExist feature What is not supported currently using RocksJava: - Custom ColumnFamilyOptions The following targets work as expected: - make rocksdbjava - make jtest Test environment: Ubuntu 14.04(LTS, x64), Java 1.7.0_65(OpenJDK IcedTea 2.5.2), g++ 4.8.2, kernel 3.13.0-35-generix
This commit is contained in:
parent
5908d08a0b
commit
18004d2f20
@ -1,4 +1,4 @@
|
||||
NATIVE_JAVA_CLASSES = org.rocksdb.RocksDB org.rocksdb.Options org.rocksdb.WriteBatch org.rocksdb.WriteBatchInternal org.rocksdb.WriteBatchTest org.rocksdb.WriteOptions org.rocksdb.BackupableDB org.rocksdb.BackupableDBOptions org.rocksdb.Statistics org.rocksdb.RocksIterator org.rocksdb.VectorMemTableConfig org.rocksdb.SkipListMemTableConfig org.rocksdb.HashLinkedListMemTableConfig org.rocksdb.HashSkipListMemTableConfig org.rocksdb.PlainTableConfig org.rocksdb.BlockBasedTableConfig org.rocksdb.ReadOptions org.rocksdb.Filter org.rocksdb.BloomFilter org.rocksdb.RestoreOptions org.rocksdb.RestoreBackupableDB org.rocksdb.RocksEnv org.rocksdb.GenericRateLimiterConfig
|
||||
NATIVE_JAVA_CLASSES = org.rocksdb.RocksDB org.rocksdb.Options org.rocksdb.WriteBatch org.rocksdb.WriteBatchInternal org.rocksdb.WriteBatchTest org.rocksdb.WriteOptions org.rocksdb.BackupableDB org.rocksdb.BackupableDBOptions org.rocksdb.Statistics org.rocksdb.RocksIterator org.rocksdb.VectorMemTableConfig org.rocksdb.SkipListMemTableConfig org.rocksdb.HashLinkedListMemTableConfig org.rocksdb.HashSkipListMemTableConfig org.rocksdb.PlainTableConfig org.rocksdb.BlockBasedTableConfig org.rocksdb.ReadOptions org.rocksdb.Filter org.rocksdb.BloomFilter org.rocksdb.RestoreOptions org.rocksdb.RestoreBackupableDB org.rocksdb.RocksEnv org.rocksdb.GenericRateLimiterConfig org.rocksdb.ColumnFamilyHandle
|
||||
|
||||
ROCKSDB_MAJOR = $(shell egrep "ROCKSDB_MAJOR.[0-9]" ../include/rocksdb/version.h | cut -d ' ' -f 3)
|
||||
ROCKSDB_MINOR = $(shell egrep "ROCKSDB_MINOR.[0-9]" ../include/rocksdb/version.h | cut -d ' ' -f 3)
|
||||
@ -35,13 +35,18 @@ sample: java
|
||||
@rm -rf /tmp/rocksdbjni_not_found
|
||||
|
||||
test: java
|
||||
@rm -rf /tmp/rocksdbjni_*
|
||||
javac org/rocksdb/test/*.java
|
||||
java -ea -Djava.library.path=.:../ -cp "$(ROCKSDB_JAR):.:./*" org.rocksdb.WriteBatchTest
|
||||
java -ea -Djava.library.path=.:../ -cp "$(ROCKSDB_JAR):.:./*" org.rocksdb.test.BackupableDBTest
|
||||
java -ea -Djava.library.path=.:../ -cp "$(ROCKSDB_JAR):.:./*" org.rocksdb.test.ColumnFamilyTest
|
||||
java -ea -Djava.library.path=.:../ -cp "$(ROCKSDB_JAR):.:./*" org.rocksdb.test.FilterTest
|
||||
java -ea -Djava.library.path=.:../ -cp "$(ROCKSDB_JAR):.:./*" org.rocksdb.test.OptionsTest
|
||||
java -ea -Djava.library.path=.:../ -cp "$(ROCKSDB_JAR):.:./*" org.rocksdb.test.KeyMayExistTest
|
||||
#java -ea -Djava.library.path=.:../ -cp "$(ROCKSDB_JAR):.:./*" org.rocksdb.test.OptionsTest
|
||||
java -ea -Djava.library.path=.:../ -cp "$(ROCKSDB_JAR):.:./*" org.rocksdb.test.ReadOnlyTest
|
||||
java -ea -Djava.library.path=.:../ -cp "$(ROCKSDB_JAR):.:./*" org.rocksdb.test.ReadOptionsTest
|
||||
java -ea -Djava.library.path=.:../ -cp "$(ROCKSDB_JAR):.:./*" org.rocksdb.test.StatisticsCollectorTest
|
||||
@rm -rf /tmp/rocksdbjni_*
|
||||
|
||||
db_bench: java
|
||||
javac org/rocksdb/benchmark/*.java
|
||||
|
@ -55,6 +55,7 @@ public class RocksDBSample {
|
||||
assert(options.maxBackgroundCompactions() == 10);
|
||||
assert(options.compressionType() == CompressionType.SNAPPY_COMPRESSION);
|
||||
assert(options.compactionStyle() == CompactionStyle.UNIVERSAL);
|
||||
|
||||
try {
|
||||
assert(options.memTableFactoryName().equals("SkipListFactory"));
|
||||
options.setMemTableConfig(
|
||||
@ -87,6 +88,7 @@ public class RocksDBSample {
|
||||
} catch (RocksDBException e) {
|
||||
assert(false);
|
||||
}
|
||||
|
||||
Filter bloomFilter = new BloomFilter(10);
|
||||
BlockBasedTableConfig table_options = new BlockBasedTableConfig();
|
||||
table_options.setBlockCacheSize(64 * SizeUnit.KB)
|
||||
|
32
java/org/rocksdb/ColumnFamilyHandle.java
Normal file
32
java/org/rocksdb/ColumnFamilyHandle.java
Normal file
@ -0,0 +1,32 @@
|
||||
// Copyright (c) 2014, Facebook, Inc. All rights reserved.
|
||||
// This source code is licensed under the BSD-style license found in the
|
||||
// LICENSE file in the root directory of this source tree. An additional grant
|
||||
// of patent rights can be found in the PATENTS file in the same directory.
|
||||
|
||||
package org.rocksdb;
|
||||
|
||||
/**
|
||||
* ColumnFamilyHandle class to hold handles to underlying rocksdb
|
||||
* ColumnFamily Pointers.
|
||||
*/
|
||||
public class ColumnFamilyHandle extends RocksObject {
|
||||
ColumnFamilyHandle(long nativeHandle) {
|
||||
super();
|
||||
nativeHandle_ = nativeHandle;
|
||||
}
|
||||
|
||||
/**
|
||||
* Deletes underlying C++ filter pointer.
|
||||
*
|
||||
* Note that this function should be called only after all
|
||||
* RocksDB instances referencing the filter are closed.
|
||||
* Otherwise an undefined behavior will occur.
|
||||
*/
|
||||
@Override protected void disposeInternal() {
|
||||
assert(isInitialized());
|
||||
disposeInternal(nativeHandle_);
|
||||
}
|
||||
|
||||
private native void disposeInternal(long handle);
|
||||
|
||||
}
|
@ -57,6 +57,21 @@ public class Options extends RocksObject {
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* <p>If true, missing column families will be automatically created</p>
|
||||
*
|
||||
* <p>Default: false</p>
|
||||
*
|
||||
* @param flag
|
||||
* @return true if missing column families shall be created automatically
|
||||
* on open.
|
||||
*/
|
||||
public Options setCreateMissingColumnFamilies(boolean flag) {
|
||||
assert(isInitialized());
|
||||
setCreateMissingColumnFamilies(nativeHandle_, flag);
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Use the specified object to interact with the environment,
|
||||
* e.g. to read/write files, schedule background work, etc.
|
||||
@ -87,6 +102,19 @@ public class Options extends RocksObject {
|
||||
return createIfMissing(nativeHandle_);
|
||||
}
|
||||
|
||||
/**
|
||||
* Return true if the create_missing_column_families flag is set
|
||||
* to true. If true column families be created if missing.
|
||||
*
|
||||
* @return true if the createMissingColumnFamilies is set to
|
||||
* true.
|
||||
* @see #setCreateMissingColumnFamilies(boolean)
|
||||
*/
|
||||
public boolean createMissingColumnFamilies() {
|
||||
assert(isInitialized());
|
||||
return createIfMissing(nativeHandle_);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set {@link org.rocksdb.Options.BuiltinComparator} to be used with RocksDB.
|
||||
*
|
||||
@ -781,7 +809,7 @@ public class Options extends RocksObject {
|
||||
private native void setWalTtlSeconds(long handle, long walTtlSeconds);
|
||||
|
||||
/**
|
||||
* {@link #walTtlSeconds()} and {@link #walSizeLimitMB()} affect how archived logs
|
||||
* {@link #walTtlSeconds()} and {@code #walSizeLimitMB()} affect how archived logs
|
||||
* will be deleted.
|
||||
* <ol>
|
||||
* <li>If both set to 0, logs will be deleted asap and will not get into
|
||||
@ -1515,8 +1543,6 @@ public class Options extends RocksObject {
|
||||
* and total file size for level-3 will be 2GB.
|
||||
* by default 'maxBytesForLevelBase' is 10MB.
|
||||
*
|
||||
* @return maxBytesForLevelBase the upper-bound of the total size of
|
||||
* leve-1 files in bytes.
|
||||
* @return the reference to the current option.
|
||||
* @see #setMaxBytesForLevelMultiplier(int)
|
||||
*/
|
||||
@ -2227,6 +2253,9 @@ public class Options extends RocksObject {
|
||||
private native boolean createIfMissing(long handle);
|
||||
private native void setWriteBufferSize(long handle, long writeBufferSize)
|
||||
throws RocksDBException;
|
||||
private native void setCreateMissingColumnFamilies(
|
||||
long handle, boolean flag);
|
||||
private native boolean createMissingColumnFamilies(long handle);
|
||||
private native long writeBufferSize(long handle);
|
||||
private native void setMaxWriteBufferNumber(
|
||||
long handle, int maxWriteBufferNumber);
|
||||
|
@ -5,13 +5,9 @@
|
||||
|
||||
package org.rocksdb;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.HashMap;
|
||||
import java.io.Closeable;
|
||||
import java.util.*;
|
||||
import java.io.IOException;
|
||||
import org.rocksdb.util.Environment;
|
||||
import org.rocksdb.NativeLibraryLoader;
|
||||
|
||||
/**
|
||||
* A RocksDB is a persistent ordered map from keys to values. It is safe for
|
||||
@ -84,7 +80,7 @@ public class RocksDB extends RocksObject {
|
||||
err = e;
|
||||
}
|
||||
}
|
||||
if (success == false) {
|
||||
if (!success) {
|
||||
throw err;
|
||||
}
|
||||
}
|
||||
@ -95,11 +91,11 @@ public class RocksDB extends RocksObject {
|
||||
* set to true.
|
||||
*
|
||||
* @param path the path to the rocksdb.
|
||||
* @return a rocksdb instance on success, null if the specified rocksdb can
|
||||
* not be opened.
|
||||
* @return a {@link RocksDB} instance on success, null if the specified
|
||||
* {@link RocksDB} can not be opened.
|
||||
*
|
||||
* @throws org.rocksdb.RocksDBException
|
||||
* @see Options#setCreateIfMissing(boolean)
|
||||
* @see org.rocksdb.Options#createIfMissing()
|
||||
*/
|
||||
public static RocksDB open(String path) throws RocksDBException {
|
||||
// This allows to use the rocksjni default Options instead of
|
||||
@ -110,16 +106,61 @@ public class RocksDB extends RocksObject {
|
||||
|
||||
/**
|
||||
* The factory constructor of RocksDB that opens a RocksDB instance given
|
||||
* the path to the database using the specified options and db path.
|
||||
*
|
||||
* the path to the database using the specified options and db path and a list
|
||||
* of column family names.
|
||||
* <p>
|
||||
* If opened in read write mode every existing column family name must be passed
|
||||
* within the list to this method.</p>
|
||||
* <p>
|
||||
* If opened in read-only mode only a subset of existing column families must
|
||||
* be passed to this method.</p>
|
||||
* <p>
|
||||
* Options instance *should* not be disposed before all DBs using this options
|
||||
* instance have been closed. If user doesn't call options dispose explicitly,
|
||||
* then this options instance will be GC'd automatically.
|
||||
* then this options instance will be GC'd automatically</p>
|
||||
* <p>
|
||||
* ColumnFamily handles are disposed when the RocksDB instance is disposed.
|
||||
* </p>
|
||||
*
|
||||
* @param path the path to the rocksdb.
|
||||
* @param columnFamilyNames list of column family names
|
||||
* @param columnFamilyHandles will be filled with ColumnFamilyHandle instances
|
||||
* on open.
|
||||
* @return a {@link RocksDB} instance on success, null if the specified
|
||||
* {@link RocksDB} can not be opened.
|
||||
*
|
||||
* @throws org.rocksdb.RocksDBException
|
||||
* @see Options#setCreateIfMissing(boolean)
|
||||
*/
|
||||
public static RocksDB open(String path, List<String> columnFamilyNames,
|
||||
List<ColumnFamilyHandle> columnFamilyHandles) throws RocksDBException {
|
||||
// This allows to use the rocksjni default Options instead of
|
||||
// the c++ one.
|
||||
Options options = new Options();
|
||||
return open(options, path, columnFamilyNames, columnFamilyHandles);
|
||||
}
|
||||
|
||||
/**
|
||||
* The factory constructor of RocksDB that opens a RocksDB instance given
|
||||
* the path to the database using the specified options and db path.
|
||||
*
|
||||
* <p>
|
||||
* Options instance *should* not be disposed before all DBs using this options
|
||||
* instance have been closed. If user doesn't call options dispose explicitly,
|
||||
* then this options instance will be GC'd automatically.</p>
|
||||
* <p>
|
||||
* Options instance can be re-used to open multiple DBs if DB statistics is
|
||||
* not used. If DB statistics are required, then its recommended to open DB
|
||||
* with new Options instance as underlying native statistics instance does not
|
||||
* use any locks to prevent concurrent updates.
|
||||
* use any locks to prevent concurrent updates.</p>
|
||||
*
|
||||
* @param options {@link org.rocksdb.Options} instance.
|
||||
* @param path the path to the rocksdb.
|
||||
* @return a {@link RocksDB} instance on success, null if the specified
|
||||
* {@link RocksDB} can not be opened.
|
||||
*
|
||||
* @throws org.rocksdb.RocksDBException
|
||||
* @see Options#setCreateIfMissing(boolean)
|
||||
*/
|
||||
public static RocksDB open(Options options, String path)
|
||||
throws RocksDBException {
|
||||
@ -133,6 +174,169 @@ public class RocksDB extends RocksObject {
|
||||
return db;
|
||||
}
|
||||
|
||||
/**
|
||||
* The factory constructor of RocksDB that opens a RocksDB instance given
|
||||
* the path to the database using the specified options and db path and a list
|
||||
* of column family names.
|
||||
* <p>
|
||||
* If opened in read write mode every existing column family name must be passed
|
||||
* within the list to this method.</p>
|
||||
* <p>
|
||||
* If opened in read-only mode only a subset of existing column families must
|
||||
* be passed to this method.</p>
|
||||
* <p>
|
||||
* Options instance *should* not be disposed before all DBs using this options
|
||||
* instance have been closed. If user doesn't call options dispose explicitly,
|
||||
* then this options instance will be GC'd automatically.</p>
|
||||
* <p>
|
||||
* Options instance can be re-used to open multiple DBs if DB statistics is
|
||||
* not used. If DB statistics are required, then its recommended to open DB
|
||||
* with new Options instance as underlying native statistics instance does not
|
||||
* use any locks to prevent concurrent updates.</p>
|
||||
* <p>
|
||||
* ColumnFamily handles are disposed when the RocksDB instance is disposed.</p>
|
||||
*
|
||||
* @param options {@link org.rocksdb.Options} instance.
|
||||
* @param path the path to the rocksdb.
|
||||
* @param columnFamilyNames list of column family names
|
||||
* @param columnFamilyHandles will be filled with ColumnFamilyHandle instances
|
||||
* on open.
|
||||
* @return a {@link RocksDB} instance on success, null if the specified
|
||||
* {@link RocksDB} can not be opened.
|
||||
*
|
||||
* @throws org.rocksdb.RocksDBException
|
||||
* @see Options#setCreateIfMissing(boolean)
|
||||
*/
|
||||
public static RocksDB open(Options options, String path, List<String> columnFamilyNames,
|
||||
List<ColumnFamilyHandle> columnFamilyHandles)
|
||||
throws RocksDBException {
|
||||
RocksDB db = new RocksDB();
|
||||
List<Long> cfReferences = db.open(options.nativeHandle_, path,
|
||||
columnFamilyNames, columnFamilyNames.size());
|
||||
for (int i=0; i<columnFamilyNames.size(); i++) {
|
||||
columnFamilyHandles.add(new ColumnFamilyHandle(cfReferences.get(i)));
|
||||
}
|
||||
db.storeOptionsInstance(options);
|
||||
return db;
|
||||
}
|
||||
|
||||
/**
|
||||
* The factory constructor of RocksDB that opens a RocksDB instance in
|
||||
* Read-Only mode given the path to the database using the default
|
||||
* options.
|
||||
*
|
||||
* @param path the path to the RocksDB.
|
||||
* @return a {@link RocksDB} instance on success, null if the specified
|
||||
* {@link RocksDB} can not be opened.
|
||||
* @throws RocksDBException
|
||||
*/
|
||||
public static RocksDB openReadOnly(String path)
|
||||
throws RocksDBException {
|
||||
// This allows to use the rocksjni default Options instead of
|
||||
// the c++ one.
|
||||
Options options = new Options();
|
||||
return openReadOnly(options, path);
|
||||
}
|
||||
|
||||
/**
|
||||
* The factory constructor of RocksDB that opens a RocksDB instance in
|
||||
* Read-Only mode given the path to the database using the default
|
||||
* options.
|
||||
*
|
||||
* @param path the path to the RocksDB.
|
||||
* @param columnFamilyNames list of column family names
|
||||
* @param columnFamilyHandles will be filled with ColumnFamilyHandle instances
|
||||
* on open.
|
||||
* @return a {@link RocksDB} instance on success, null if the specified
|
||||
* {@link RocksDB} can not be opened.
|
||||
* @throws RocksDBException
|
||||
*/
|
||||
public static RocksDB openReadOnly(String path, List<String> columnFamilyNames,
|
||||
List<ColumnFamilyHandle> columnFamilyHandles) throws RocksDBException {
|
||||
// This allows to use the rocksjni default Options instead of
|
||||
// the c++ one.
|
||||
Options options = new Options();
|
||||
return openReadOnly(options, path, columnFamilyNames, columnFamilyHandles);
|
||||
}
|
||||
|
||||
/**
|
||||
* The factory constructor of RocksDB that opens a RocksDB instance in
|
||||
* Read-Only mode given the path to the database using the specified
|
||||
* options and db path.
|
||||
*
|
||||
* Options instance *should* not be disposed before all DBs using this options
|
||||
* instance have been closed. If user doesn't call options dispose explicitly,
|
||||
* then this options instance will be GC'd automatically.
|
||||
*
|
||||
* @param options {@link Options} instance.
|
||||
* @param path the path to the RocksDB.
|
||||
* @return a {@link RocksDB} instance on success, null if the specified
|
||||
* {@link RocksDB} can not be opened.
|
||||
* @throws RocksDBException
|
||||
*/
|
||||
public static RocksDB openReadOnly(Options options, String path)
|
||||
throws RocksDBException {
|
||||
// when non-default Options is used, keeping an Options reference
|
||||
// in RocksDB can prevent Java to GC during the life-time of
|
||||
// the currently-created RocksDB.
|
||||
RocksDB db = new RocksDB();
|
||||
db.openROnly(options.nativeHandle_, path);
|
||||
|
||||
db.storeOptionsInstance(options);
|
||||
return db;
|
||||
}
|
||||
|
||||
/**
|
||||
* The factory constructor of RocksDB that opens a RocksDB instance in
|
||||
* Read-Only mode given the path to the database using the specified
|
||||
* options and db path.
|
||||
*
|
||||
* <p>This open method allows to open RocksDB using a subset of available
|
||||
* column families</p>
|
||||
* <p>Options instance *should* not be disposed before all DBs using this
|
||||
* options instance have been closed. If user doesn't call options dispose
|
||||
* explicitly,then this options instance will be GC'd automatically.</p>
|
||||
*
|
||||
* @param options {@link Options} instance.
|
||||
* @param path the path to the RocksDB.
|
||||
* @param columnFamilyNames list of column family names
|
||||
* @param columnFamilyHandles will be filled with ColumnFamilyHandle instances
|
||||
* on open.
|
||||
* @return a {@link RocksDB} instance on success, null if the specified
|
||||
* {@link RocksDB} can not be opened.
|
||||
* @throws RocksDBException
|
||||
*/
|
||||
public static RocksDB openReadOnly(Options options, String path,
|
||||
List<String> columnFamilyNames, List<ColumnFamilyHandle> columnFamilyHandles)
|
||||
throws RocksDBException {
|
||||
// when non-default Options is used, keeping an Options reference
|
||||
// in RocksDB can prevent Java to GC during the life-time of
|
||||
// the currently-created RocksDB.
|
||||
RocksDB db = new RocksDB();
|
||||
List<Long> cfReferences = db.openROnly(options.nativeHandle_, path,
|
||||
columnFamilyNames, columnFamilyNames.size());
|
||||
for (int i=0; i<columnFamilyNames.size(); i++) {
|
||||
columnFamilyHandles.add(new ColumnFamilyHandle(cfReferences.get(i)));
|
||||
}
|
||||
|
||||
db.storeOptionsInstance(options);
|
||||
return db;
|
||||
}
|
||||
/**
|
||||
* Static method to determine all available column families for a
|
||||
* rocksdb database identified by path
|
||||
*
|
||||
* @param options Options for opening the database
|
||||
* @param path Absolute path to rocksdb database
|
||||
* @return List<byte[]> List containing the column family names
|
||||
*
|
||||
* @throws RocksDBException
|
||||
*/
|
||||
public static List<byte[]> listColumnFamilies(Options options, String path)
|
||||
throws RocksDBException {
|
||||
return RocksDB.listColumnFamilies(options.nativeHandle_, path);
|
||||
}
|
||||
|
||||
private void storeOptionsInstance(Options options) {
|
||||
options_ = options;
|
||||
}
|
||||
@ -155,16 +359,39 @@ public class RocksDB extends RocksObject {
|
||||
*
|
||||
* @param key the specified key to be inserted.
|
||||
* @param value the value associated with the specified key.
|
||||
*
|
||||
* @see RocksDBException
|
||||
*/
|
||||
public void put(byte[] key, byte[] value) throws RocksDBException {
|
||||
put(nativeHandle_, key, key.length, value, value.length);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the database entry for "key" to "value" in the specified
|
||||
* column family.
|
||||
*
|
||||
* @param columnFamilyHandle {@link org.rocksdb.ColumnFamilyHandle}
|
||||
* instance
|
||||
* @param key the specified key to be inserted.
|
||||
* @param value the value associated with the specified key.
|
||||
*
|
||||
* throws IllegalArgumentException if column family is not present
|
||||
*
|
||||
* @see RocksDBException
|
||||
*/
|
||||
public void put(ColumnFamilyHandle columnFamilyHandle, byte[] key,
|
||||
byte[] value) throws RocksDBException {
|
||||
put(nativeHandle_, key, key.length, value, value.length,
|
||||
columnFamilyHandle.nativeHandle_);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the database entry for "key" to "value".
|
||||
*
|
||||
* @param key the specified key to be inserted.
|
||||
* @param value the value associated with the specified key.
|
||||
*
|
||||
* @see RocksDBException
|
||||
*/
|
||||
public void put(WriteOptions writeOpts, byte[] key, byte[] value)
|
||||
throws RocksDBException {
|
||||
@ -172,8 +399,73 @@ public class RocksDB extends RocksObject {
|
||||
key, key.length, value, value.length);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the database entry for "key" to "value" for the specified
|
||||
* column family.
|
||||
*
|
||||
* @param columnFamilyHandle {@link org.rocksdb.ColumnFamilyHandle}
|
||||
* instance
|
||||
* @param key the specified key to be inserted.
|
||||
* @param value the value associated with the specified key.
|
||||
*
|
||||
* throws IllegalArgumentException if column family is not present
|
||||
*
|
||||
* @see RocksDBException
|
||||
* @see IllegalArgumentException
|
||||
*/
|
||||
public void put(ColumnFamilyHandle columnFamilyHandle, WriteOptions writeOpts,
|
||||
byte[] key, byte[] value) throws RocksDBException {
|
||||
put(nativeHandle_, writeOpts.nativeHandle_, key, key.length, value, value.length,
|
||||
columnFamilyHandle.nativeHandle_);
|
||||
}
|
||||
|
||||
/**
|
||||
* If the key definitely does not exist in the database, then this method
|
||||
* returns false, else true.
|
||||
*
|
||||
* This check is potentially lighter-weight than invoking DB::Get(). One way
|
||||
* to make this lighter weight is to avoid doing any IOs.
|
||||
*
|
||||
* @param columnFamilyHandle {@link ColumnFamilyHandle} instnace
|
||||
* @param key byte array of a key to search for
|
||||
* @param value StringBuffer instance which is a out parameter if a value is
|
||||
* found in block-cache.
|
||||
* @return boolean value indicating if key does not exist or might exist.
|
||||
*/
|
||||
public boolean keyMayExist(ColumnFamilyHandle columnFamilyHandle,
|
||||
byte[] key, StringBuffer value){
|
||||
return keyMayExist(key, key.length, columnFamilyHandle.nativeHandle_,
|
||||
value);
|
||||
}
|
||||
|
||||
/**
|
||||
* If the key definitely does not exist in the database, then this method
|
||||
* returns false, else true.
|
||||
*
|
||||
* This check is potentially lighter-weight than invoking DB::Get(). One way
|
||||
* to make this lighter weight is to avoid doing any IOs.
|
||||
*
|
||||
* @param readOptions {@link ReadOptions} instance
|
||||
* @param columnFamilyHandle {@link ColumnFamilyHandle} instnace
|
||||
* @param key byte array of a key to search for
|
||||
* @param value StringBuffer instance which is a out parameter if a value is
|
||||
* found in block-cache.
|
||||
* @return boolean value indicating if key does not exist or might exist.
|
||||
*/
|
||||
public boolean keyMayExist(ReadOptions readOptions,
|
||||
ColumnFamilyHandle columnFamilyHandle, byte[] key, StringBuffer value){
|
||||
return keyMayExist(readOptions.nativeHandle_,
|
||||
key, key.length, columnFamilyHandle.nativeHandle_,
|
||||
value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Apply the specified updates to the database.
|
||||
*
|
||||
* @param writeOpts WriteOptions instance
|
||||
* @param updates WriteBatch instance
|
||||
*
|
||||
* @see RocksDBException
|
||||
*/
|
||||
public void write(WriteOptions writeOpts, WriteBatch updates)
|
||||
throws RocksDBException {
|
||||
@ -181,7 +473,7 @@ public class RocksDB extends RocksObject {
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the value associated with the specified key.
|
||||
* Get the value associated with the specified key within column family
|
||||
*
|
||||
* @param key the key to retrieve the value.
|
||||
* @param value the out-value to receive the retrieved value.
|
||||
@ -191,11 +483,35 @@ public class RocksDB extends RocksObject {
|
||||
* input buffer {@code value} is insufficient and partial result will
|
||||
* be returned. RocksDB.NOT_FOUND will be returned if the value not
|
||||
* found.
|
||||
*
|
||||
* @see RocksDBException
|
||||
*/
|
||||
public int get(byte[] key, byte[] value) throws RocksDBException {
|
||||
return get(nativeHandle_, key, key.length, value, value.length);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the value associated with the specified key within column family.
|
||||
*
|
||||
* @param columnFamilyHandle {@link org.rocksdb.ColumnFamilyHandle}
|
||||
* instance
|
||||
* @param key the key to retrieve the value.
|
||||
* @param value the out-value to receive the retrieved value.
|
||||
* @return The size of the actual value that matches the specified
|
||||
* {@code key} in byte. If the return value is greater than the
|
||||
* length of {@code value}, then it indicates that the size of the
|
||||
* input buffer {@code value} is insufficient and partial result will
|
||||
* be returned. RocksDB.NOT_FOUND will be returned if the value not
|
||||
* found.
|
||||
*
|
||||
* @throws RocksDBException
|
||||
*/
|
||||
public int get(ColumnFamilyHandle columnFamilyHandle, byte[] key, byte[] value)
|
||||
throws RocksDBException, IllegalArgumentException {
|
||||
return get(nativeHandle_, key, key.length, value, value.length,
|
||||
columnFamilyHandle.nativeHandle_);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the value associated with the specified key.
|
||||
*
|
||||
@ -207,12 +523,35 @@ public class RocksDB extends RocksObject {
|
||||
* input buffer {@code value} is insufficient and partial result will
|
||||
* be returned. RocksDB.NOT_FOUND will be returned if the value not
|
||||
* found.
|
||||
*
|
||||
* @throws RocksDBException
|
||||
*/
|
||||
public int get(ReadOptions opt, byte[] key, byte[] value)
|
||||
throws RocksDBException {
|
||||
return get(nativeHandle_, opt.nativeHandle_,
|
||||
key, key.length, value, value.length);
|
||||
}
|
||||
/**
|
||||
* Get the value associated with the specified key within column family.
|
||||
*
|
||||
* @param columnFamilyHandle {@link org.rocksdb.ColumnFamilyHandle}
|
||||
* instance
|
||||
* @param key the key to retrieve the value.
|
||||
* @param value the out-value to receive the retrieved value.
|
||||
* @return The size of the actual value that matches the specified
|
||||
* {@code key} in byte. If the return value is greater than the
|
||||
* length of {@code value}, then it indicates that the size of the
|
||||
* input buffer {@code value} is insufficient and partial result will
|
||||
* be returned. RocksDB.NOT_FOUND will be returned if the value not
|
||||
* found.
|
||||
*
|
||||
* @throws RocksDBException
|
||||
*/
|
||||
public int get(ColumnFamilyHandle columnFamilyHandle, ReadOptions opt, byte[] key,
|
||||
byte[] value) throws RocksDBException {
|
||||
return get(nativeHandle_, opt.nativeHandle_, key, key.length, value,
|
||||
value.length, columnFamilyHandle.nativeHandle_);
|
||||
}
|
||||
|
||||
/**
|
||||
* The simplified version of get which returns a new byte array storing
|
||||
@ -223,12 +562,30 @@ public class RocksDB extends RocksObject {
|
||||
* @return a byte array storing the value associated with the input key if
|
||||
* any. null if it does not find the specified key.
|
||||
*
|
||||
* @see RocksDBException
|
||||
* @throws RocksDBException
|
||||
*/
|
||||
public byte[] get(byte[] key) throws RocksDBException {
|
||||
return get(nativeHandle_, key, key.length);
|
||||
}
|
||||
|
||||
/**
|
||||
* The simplified version of get which returns a new byte array storing
|
||||
* the value associated with the specified input key if any. null will be
|
||||
* returned if the specified key is not found.
|
||||
*
|
||||
* @param columnFamilyHandle {@link org.rocksdb.ColumnFamilyHandle}
|
||||
* instance
|
||||
* @param key the key retrieve the value.
|
||||
* @return a byte array storing the value associated with the input key if
|
||||
* any. null if it does not find the specified key.
|
||||
*
|
||||
* @throws RocksDBException
|
||||
*/
|
||||
public byte[] get(ColumnFamilyHandle columnFamilyHandle, byte[] key)
|
||||
throws RocksDBException {
|
||||
return get(nativeHandle_, key, key.length, columnFamilyHandle.nativeHandle_);
|
||||
}
|
||||
|
||||
/**
|
||||
* The simplified version of get which returns a new byte array storing
|
||||
* the value associated with the specified input key if any. null will be
|
||||
@ -239,12 +596,32 @@ public class RocksDB extends RocksObject {
|
||||
* @return a byte array storing the value associated with the input key if
|
||||
* any. null if it does not find the specified key.
|
||||
*
|
||||
* @see RocksDBException
|
||||
* @throws RocksDBException
|
||||
*/
|
||||
public byte[] get(ReadOptions opt, byte[] key) throws RocksDBException {
|
||||
return get(nativeHandle_, opt.nativeHandle_, key, key.length);
|
||||
}
|
||||
|
||||
/**
|
||||
* The simplified version of get which returns a new byte array storing
|
||||
* the value associated with the specified input key if any. null will be
|
||||
* returned if the specified key is not found.
|
||||
*
|
||||
* @param columnFamilyHandle {@link org.rocksdb.ColumnFamilyHandle}
|
||||
* instance
|
||||
* @param key the key retrieve the value.
|
||||
* @param opt Read options.
|
||||
* @return a byte array storing the value associated with the input key if
|
||||
* any. null if it does not find the specified key.
|
||||
*
|
||||
* @throws RocksDBException
|
||||
*/
|
||||
public byte[] get(ColumnFamilyHandle columnFamilyHandle, ReadOptions opt,
|
||||
byte[] key) throws RocksDBException {
|
||||
return get(nativeHandle_, opt.nativeHandle_, key, key.length,
|
||||
columnFamilyHandle.nativeHandle_);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a map of keys for which values were found in DB.
|
||||
*
|
||||
@ -252,7 +629,7 @@ public class RocksDB extends RocksObject {
|
||||
* @return Map where key of map is the key passed by user and value for map
|
||||
* entry is the corresponding value in DB.
|
||||
*
|
||||
* @see RocksDBException
|
||||
* @throws RocksDBException
|
||||
*/
|
||||
public Map<byte[], byte[]> multiGet(List<byte[]> keys)
|
||||
throws RocksDBException {
|
||||
@ -273,6 +650,43 @@ public class RocksDB extends RocksObject {
|
||||
return keyValueMap;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a map of keys for which values were found in DB.
|
||||
* <p>
|
||||
* Note: Every key needs to have a related column family name in
|
||||
* {@code columnFamilyHandleList}.
|
||||
* </p>
|
||||
*
|
||||
* @param columnFamilyHandleList {@link java.util.List} containing
|
||||
* {@link org.rocksdb.ColumnFamilyHandle} instances.
|
||||
* @param keys List of keys for which values need to be retrieved.
|
||||
* @return Map where key of map is the key passed by user and value for map
|
||||
* entry is the corresponding value in DB.
|
||||
*
|
||||
* @throws RocksDBException
|
||||
* @throws IllegalArgumentException
|
||||
*/
|
||||
public Map<byte[], byte[]> multiGet(List<ColumnFamilyHandle> columnFamilyHandleList,
|
||||
List<byte[]> keys) throws RocksDBException, IllegalArgumentException {
|
||||
assert(keys.size() != 0);
|
||||
// Check if key size equals cfList size. If not a exception must be
|
||||
// thrown. If not a Segmentation fault happens.
|
||||
if (keys.size()!=columnFamilyHandleList.size()) {
|
||||
throw new IllegalArgumentException(
|
||||
"For each key there must be a ColumnFamilyHandle.");
|
||||
}
|
||||
List<byte[]> values = multiGet(nativeHandle_, keys, keys.size(),
|
||||
columnFamilyHandleList);
|
||||
|
||||
Map<byte[], byte[]> keyValueMap = new HashMap<byte[], byte[]>();
|
||||
for(int i = 0; i < values.size(); i++) {
|
||||
if (values.get(i) == null) {
|
||||
continue;
|
||||
}
|
||||
keyValueMap.put(keys.get(i), values.get(i));
|
||||
}
|
||||
return keyValueMap;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a map of keys for which values were found in DB.
|
||||
@ -282,7 +696,7 @@ public class RocksDB extends RocksObject {
|
||||
* @return Map where key of map is the key passed by user and value for map
|
||||
* entry is the corresponding value in DB.
|
||||
*
|
||||
* @see RocksDBException
|
||||
* @throws RocksDBException
|
||||
*/
|
||||
public Map<byte[], byte[]> multiGet(ReadOptions opt, List<byte[]> keys)
|
||||
throws RocksDBException {
|
||||
@ -303,10 +717,56 @@ public class RocksDB extends RocksObject {
|
||||
return keyValueMap;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a map of keys for which values were found in DB.
|
||||
* <p>
|
||||
* Note: Every key needs to have a related column family name in
|
||||
* {@code columnFamilyHandleList}.
|
||||
* </p>
|
||||
*
|
||||
* @param opt Read options.
|
||||
* @param columnFamilyHandleList {@link java.util.List} containing
|
||||
* {@link org.rocksdb.ColumnFamilyHandle} instances.
|
||||
* @param keys of keys for which values need to be retrieved.
|
||||
* @return Map where key of map is the key passed by user and value for map
|
||||
* entry is the corresponding value in DB.
|
||||
*
|
||||
* @throws RocksDBException
|
||||
* @throws java.lang.IllegalArgumentException
|
||||
*/
|
||||
public Map<byte[], byte[]> multiGet(ReadOptions opt,
|
||||
List<ColumnFamilyHandle> columnFamilyHandleList, List<byte[]> keys)
|
||||
throws RocksDBException {
|
||||
assert(keys.size() != 0);
|
||||
// Check if key size equals cfList size. If not a exception must be
|
||||
// thrown. If not a Segmentation fault happens.
|
||||
if (keys.size()!=columnFamilyHandleList.size()){
|
||||
throw new IllegalArgumentException(
|
||||
"For each key there must be a ColumnFamilyHandle.");
|
||||
}
|
||||
|
||||
List<byte[]> values = multiGet(nativeHandle_, opt.nativeHandle_,
|
||||
keys, keys.size(), columnFamilyHandleList);
|
||||
|
||||
Map<byte[], byte[]> keyValueMap = new HashMap<byte[], byte[]>();
|
||||
for(int i = 0; i < values.size(); i++) {
|
||||
if(values.get(i) == null) {
|
||||
continue;
|
||||
}
|
||||
keyValueMap.put(keys.get(i), values.get(i));
|
||||
}
|
||||
|
||||
return keyValueMap;
|
||||
}
|
||||
|
||||
/**
|
||||
* Remove the database entry (if any) for "key". Returns OK on
|
||||
* success, and a non-OK status on error. It is not an error if "key"
|
||||
* did not exist in the database.
|
||||
*
|
||||
* @param key Key to delete within database
|
||||
*
|
||||
* @throws RocksDBException
|
||||
*/
|
||||
public void remove(byte[] key) throws RocksDBException {
|
||||
remove(nativeHandle_, key, key.length);
|
||||
@ -316,27 +776,102 @@ public class RocksDB extends RocksObject {
|
||||
* Remove the database entry (if any) for "key". Returns OK on
|
||||
* success, and a non-OK status on error. It is not an error if "key"
|
||||
* did not exist in the database.
|
||||
*
|
||||
* @param columnFamilyHandle {@link org.rocksdb.ColumnFamilyHandle}
|
||||
* instance
|
||||
* @param key Key to delete within database
|
||||
*
|
||||
* @throws RocksDBException
|
||||
*/
|
||||
public void remove(ColumnFamilyHandle columnFamilyHandle, byte[] key)
|
||||
throws RocksDBException {
|
||||
remove(nativeHandle_, key, key.length, columnFamilyHandle.nativeHandle_);
|
||||
}
|
||||
|
||||
/**
|
||||
* Remove the database entry (if any) for "key". Returns OK on
|
||||
* success, and a non-OK status on error. It is not an error if "key"
|
||||
* did not exist in the database.
|
||||
*
|
||||
* @param writeOpt WriteOptions to be used with delete operation
|
||||
* @param key Key to delete within database
|
||||
*
|
||||
* @throws RocksDBException
|
||||
*/
|
||||
public void remove(WriteOptions writeOpt, byte[] key)
|
||||
throws RocksDBException {
|
||||
remove(nativeHandle_, writeOpt.nativeHandle_, key, key.length);
|
||||
}
|
||||
|
||||
/**
|
||||
* Remove the database entry (if any) for "key". Returns OK on
|
||||
* success, and a non-OK status on error. It is not an error if "key"
|
||||
* did not exist in the database.
|
||||
*
|
||||
* @param columnFamilyHandle {@link org.rocksdb.ColumnFamilyHandle}
|
||||
* instance
|
||||
* @param writeOpt WriteOptions to be used with delete operation
|
||||
* @param key Key to delete within database
|
||||
*
|
||||
* @throws RocksDBException
|
||||
*/
|
||||
public void remove(ColumnFamilyHandle columnFamilyHandle, WriteOptions writeOpt,
|
||||
byte[] key) throws RocksDBException {
|
||||
remove(nativeHandle_, writeOpt.nativeHandle_, key, key.length,
|
||||
columnFamilyHandle.nativeHandle_);
|
||||
}
|
||||
|
||||
/**
|
||||
* DB implements can export properties about their state
|
||||
* via this method on a per column family level.
|
||||
*
|
||||
* <p>If {@code property} is a valid property understood by this DB
|
||||
* implementation, fills {@code value} with its current value and
|
||||
* returns true. Otherwise returns false.</p>
|
||||
*
|
||||
* <p>Valid property names include:
|
||||
* <ul>
|
||||
* <li>"rocksdb.num-files-at-level<N>" - return the number of files at level <N>,
|
||||
* where <N> is an ASCII representation of a level number (e.g. "0").</li>
|
||||
* <li>"rocksdb.stats" - returns a multi-line string that describes statistics
|
||||
* about the internal operation of the DB.</li>
|
||||
* <li>"rocksdb.sstables" - returns a multi-line string that describes all
|
||||
* of the sstables that make up the db contents.</li>
|
||||
*</ul></p>
|
||||
*
|
||||
* @param columnFamilyHandle {@link org.rocksdb.ColumnFamilyHandle}
|
||||
* instance
|
||||
* @param property to be fetched. See above for examples
|
||||
* @return property value
|
||||
*
|
||||
* @throws RocksDBException
|
||||
*/
|
||||
public String getProperty(ColumnFamilyHandle columnFamilyHandle, String property)
|
||||
throws RocksDBException {
|
||||
return getProperty0(nativeHandle_, columnFamilyHandle.nativeHandle_, property,
|
||||
property.length());
|
||||
}
|
||||
|
||||
/**
|
||||
* DB implementations can export properties about their state
|
||||
via this method. If "property" is a valid property understood by this
|
||||
DB implementation, fills "*value" with its current value and returns
|
||||
true. Otherwise returns false.
|
||||
|
||||
|
||||
Valid property names include:
|
||||
|
||||
"rocksdb.num-files-at-level<N>" - return the number of files at level <N>,
|
||||
where <N> is an ASCII representation of a level number (e.g. "0").
|
||||
"rocksdb.stats" - returns a multi-line string that describes statistics
|
||||
about the internal operation of the DB.
|
||||
"rocksdb.sstables" - returns a multi-line string that describes all
|
||||
of the sstables that make up the db contents.
|
||||
* via this method. If "property" is a valid property understood by this
|
||||
* DB implementation, fills "*value" with its current value and returns
|
||||
* true. Otherwise returns false.
|
||||
*
|
||||
* <p>Valid property names include:
|
||||
* <ul>
|
||||
* <li>"rocksdb.num-files-at-level<N>" - return the number of files at level <N>,
|
||||
* where <N> is an ASCII representation of a level number (e.g. "0").</li>
|
||||
* <li>"rocksdb.stats" - returns a multi-line string that describes statistics
|
||||
* about the internal operation of the DB.</li>
|
||||
* <li>"rocksdb.sstables" - returns a multi-line string that describes all
|
||||
* of the sstables that make up the db contents.</li>
|
||||
*</ul></p>
|
||||
*
|
||||
* @param property to be fetched. See above for examples
|
||||
* @return property value
|
||||
*
|
||||
* @throws RocksDBException
|
||||
*/
|
||||
public String getProperty(String property) throws RocksDBException {
|
||||
return getProperty0(nativeHandle_, property, property.length());
|
||||
@ -356,6 +891,77 @@ public class RocksDB extends RocksObject {
|
||||
return new RocksIterator(iterator0(nativeHandle_));
|
||||
}
|
||||
|
||||
/**
|
||||
* Return a heap-allocated iterator over the contents of the database.
|
||||
* The result of newIterator() is initially invalid (caller must
|
||||
* call one of the Seek methods on the iterator before using it).
|
||||
*
|
||||
* Caller should close the iterator when it is no longer needed.
|
||||
* The returned iterator should be closed before this db is closed.
|
||||
*
|
||||
* @param columnFamilyHandle {@link org.rocksdb.ColumnFamilyHandle}
|
||||
* instance
|
||||
* @return instance of iterator object.
|
||||
*/
|
||||
public RocksIterator newIterator(ColumnFamilyHandle columnFamilyHandle) {
|
||||
return new RocksIterator(iterator0(nativeHandle_, columnFamilyHandle.nativeHandle_));
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns iterators from a consistent database state across multiple
|
||||
* column families. Iterators are heap allocated and need to be deleted
|
||||
* before the db is deleted
|
||||
*
|
||||
* @param columnFamilyHandleList {@link java.util.List} containing
|
||||
* {@link org.rocksdb.ColumnFamilyHandle} instances.
|
||||
* @return {@link java.util.List} containing {@link org.rocksdb.RocksIterator}
|
||||
* instances
|
||||
*
|
||||
* @throws RocksDBException
|
||||
*/
|
||||
public List<RocksIterator> newIterators(
|
||||
List<ColumnFamilyHandle> columnFamilyHandleList) throws RocksDBException {
|
||||
List<RocksIterator> iterators =
|
||||
new ArrayList<RocksIterator>(columnFamilyHandleList.size());
|
||||
|
||||
long[] iteratorRefs = iterators(nativeHandle_, columnFamilyHandleList);
|
||||
for (int i=0; i<columnFamilyHandleList.size(); i++){
|
||||
iterators.add(new RocksIterator(iteratorRefs[i]));
|
||||
}
|
||||
return iterators;
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new column family with the name columnFamilyName and
|
||||
* allocates a ColumnFamilyHandle within an internal structure.
|
||||
* The ColumnFamilyHandle is automatically disposed with DB disposal.
|
||||
*
|
||||
* @param columnFamilyName Name of column family to be created.
|
||||
* @return {@link org.rocksdb.ColumnFamilyHandle} instance
|
||||
* @see RocksDBException
|
||||
*/
|
||||
public ColumnFamilyHandle createColumnFamily(String columnFamilyName)
|
||||
throws RocksDBException {
|
||||
return new ColumnFamilyHandle(createColumnFamily(nativeHandle_,
|
||||
columnFamilyName));
|
||||
}
|
||||
|
||||
/**
|
||||
* Drops the column family identified by columnFamilyName. Internal
|
||||
* handles to this column family will be disposed. If the column family
|
||||
* is not known removal will fail.
|
||||
*
|
||||
* @param columnFamilyHandle {@link org.rocksdb.ColumnFamilyHandle}
|
||||
* instance
|
||||
*
|
||||
* @throws RocksDBException
|
||||
*/
|
||||
public void dropColumnFamily(ColumnFamilyHandle columnFamilyHandle)
|
||||
throws RocksDBException, IllegalArgumentException {
|
||||
// throws RocksDBException if something goes wrong
|
||||
dropColumnFamily(nativeHandle_, columnFamilyHandle.nativeHandle_);
|
||||
}
|
||||
|
||||
/**
|
||||
* Private constructor.
|
||||
*/
|
||||
@ -366,39 +972,90 @@ public class RocksDB extends RocksObject {
|
||||
// native methods
|
||||
protected native void open(
|
||||
long optionsHandle, String path) throws RocksDBException;
|
||||
protected native List<Long> open(long optionsHandle, String path,
|
||||
List<String> columnFamilyNames, int columnFamilyNamesLength)
|
||||
throws RocksDBException;
|
||||
protected native static List<byte[]> listColumnFamilies(
|
||||
long optionsHandle, String path) throws RocksDBException;
|
||||
protected native void openROnly(
|
||||
long optionsHandle, String path) throws RocksDBException;
|
||||
protected native List<Long> openROnly(
|
||||
long optionsHandle, String path, List<String> columnFamilyNames,
|
||||
int columnFamilyNamesLength) throws RocksDBException;
|
||||
protected native void put(
|
||||
long handle, byte[] key, int keyLen,
|
||||
byte[] value, int valueLen) throws RocksDBException;
|
||||
protected native void put(
|
||||
long handle, byte[] key, int keyLen,
|
||||
byte[] value, int valueLen, long cfHandle) throws RocksDBException;
|
||||
protected native void put(
|
||||
long handle, long writeOptHandle,
|
||||
byte[] key, int keyLen,
|
||||
byte[] value, int valueLen) throws RocksDBException;
|
||||
protected native void put(
|
||||
long handle, long writeOptHandle,
|
||||
byte[] key, int keyLen,
|
||||
byte[] value, int valueLen, long cfHandle) throws RocksDBException;
|
||||
protected native void write(
|
||||
long writeOptHandle, long batchHandle) throws RocksDBException;
|
||||
protected native boolean keyMayExist(byte[] key, int keyLen,
|
||||
long cfHandle, StringBuffer stringBuffer);
|
||||
protected native boolean keyMayExist(long optionsHandle, byte[] key, int keyLen,
|
||||
long cfHandle, StringBuffer stringBuffer);
|
||||
protected native int get(
|
||||
long handle, byte[] key, int keyLen,
|
||||
byte[] value, int valueLen) throws RocksDBException;
|
||||
protected native int get(
|
||||
long handle, byte[] key, int keyLen,
|
||||
byte[] value, int valueLen, long cfHandle) throws RocksDBException;
|
||||
protected native int get(
|
||||
long handle, long readOptHandle, byte[] key, int keyLen,
|
||||
byte[] value, int valueLen) throws RocksDBException;
|
||||
protected native int get(
|
||||
long handle, long readOptHandle, byte[] key, int keyLen,
|
||||
byte[] value, int valueLen, long cfHandle) throws RocksDBException;
|
||||
protected native List<byte[]> multiGet(
|
||||
long dbHandle, List<byte[]> keys, int keysCount);
|
||||
protected native List<byte[]> multiGet(
|
||||
long dbHandle, List<byte[]> keys, int keysCount, List<ColumnFamilyHandle>
|
||||
cfHandles);
|
||||
protected native List<byte[]> multiGet(
|
||||
long dbHandle, long rOptHandle, List<byte[]> keys, int keysCount);
|
||||
protected native List<byte[]> multiGet(
|
||||
long dbHandle, long rOptHandle, List<byte[]> keys, int keysCount,
|
||||
List<ColumnFamilyHandle> cfHandles);
|
||||
protected native byte[] get(
|
||||
long handle, byte[] key, int keyLen) throws RocksDBException;
|
||||
protected native byte[] get(
|
||||
long handle, byte[] key, int keyLen, long cfHandle) throws RocksDBException;
|
||||
protected native byte[] get(
|
||||
long handle, long readOptHandle,
|
||||
byte[] key, int keyLen) throws RocksDBException;
|
||||
protected native byte[] get(
|
||||
long handle, long readOptHandle,
|
||||
byte[] key, int keyLen, long cfHandle) throws RocksDBException;
|
||||
protected native void remove(
|
||||
long handle, byte[] key, int keyLen) throws RocksDBException;
|
||||
protected native void remove(
|
||||
long handle, byte[] key, int keyLen, long cfHandle) throws RocksDBException;
|
||||
protected native void remove(
|
||||
long handle, long writeOptHandle,
|
||||
byte[] key, int keyLen) throws RocksDBException;
|
||||
protected native void remove(
|
||||
long handle, long writeOptHandle,
|
||||
byte[] key, int keyLen, long cfHandle) throws RocksDBException;
|
||||
protected native String getProperty0(long nativeHandle,
|
||||
String property, int propertyLength) throws RocksDBException;
|
||||
protected native long iterator0(long optHandle);
|
||||
protected native String getProperty0(long nativeHandle, long cfHandle,
|
||||
String property, int propertyLength) throws RocksDBException;
|
||||
protected native long iterator0(long handle);
|
||||
protected native long iterator0(long handle, long cfHandle);
|
||||
protected native long[] iterators(long handle,
|
||||
List<ColumnFamilyHandle> columnFamilyNames) throws RocksDBException;
|
||||
private native void disposeInternal(long handle);
|
||||
|
||||
private native long createColumnFamily(long handle, String name) throws RocksDBException;
|
||||
private native void dropColumnFamily(long handle, long cfHandle) throws RocksDBException;
|
||||
|
||||
protected Options options_;
|
||||
}
|
||||
|
@ -47,6 +47,16 @@ public class WriteBatch extends RocksObject {
|
||||
put(key, key.length, value, value.length);
|
||||
}
|
||||
|
||||
/**
|
||||
* Store the mapping "key->value" within given column
|
||||
* family.
|
||||
*/
|
||||
public void put(ColumnFamilyHandle columnFamilyHandle,
|
||||
byte[] key, byte[] value) {
|
||||
put(key, key.length, value, value.length,
|
||||
columnFamilyHandle.nativeHandle_);
|
||||
}
|
||||
|
||||
/**
|
||||
* Merge "value" with the existing value of "key" in the database.
|
||||
* "key->merge(existing, value)"
|
||||
@ -55,6 +65,16 @@ public class WriteBatch extends RocksObject {
|
||||
merge(key, key.length, value, value.length);
|
||||
}
|
||||
|
||||
/**
|
||||
* Merge "value" with the existing value of "key" in given column family.
|
||||
* "key->merge(existing, value)"
|
||||
*/
|
||||
public void merge(ColumnFamilyHandle columnFamilyHandle,
|
||||
byte[] key, byte[] value) {
|
||||
merge(key, key.length, value, value.length,
|
||||
columnFamilyHandle.nativeHandle_);
|
||||
}
|
||||
|
||||
/**
|
||||
* If the database contains a mapping for "key", erase it. Else do nothing.
|
||||
*/
|
||||
@ -62,6 +82,13 @@ public class WriteBatch extends RocksObject {
|
||||
remove(key, key.length);
|
||||
}
|
||||
|
||||
/**
|
||||
* If column family contains a mapping for "key", erase it. Else do nothing.
|
||||
*/
|
||||
public void remove(ColumnFamilyHandle columnFamilyHandle, byte[] key) {
|
||||
remove(key, key.length, columnFamilyHandle.nativeHandle_);
|
||||
}
|
||||
|
||||
/**
|
||||
* Append a blob of arbitrary size to the records in this batch. The blob will
|
||||
* be stored in the transaction log but not in any other file. In particular,
|
||||
@ -94,9 +121,17 @@ public class WriteBatch extends RocksObject {
|
||||
private native void newWriteBatch(int reserved_bytes);
|
||||
private native void put(byte[] key, int keyLen,
|
||||
byte[] value, int valueLen);
|
||||
private native void put(byte[] key, int keyLen,
|
||||
byte[] value, int valueLen,
|
||||
long cfHandle);
|
||||
private native void merge(byte[] key, int keyLen,
|
||||
byte[] value, int valueLen);
|
||||
private native void merge(byte[] key, int keyLen,
|
||||
byte[] value, int valueLen,
|
||||
long cfHandle);
|
||||
private native void remove(byte[] key, int keyLen);
|
||||
private native void remove(byte[] key, int keyLen,
|
||||
long cfHandle);
|
||||
private native void putLogData(byte[] blob, int blobLen);
|
||||
private native void disposeInternal(long handle);
|
||||
}
|
||||
|
@ -8,8 +8,8 @@ package org.rocksdb.test;
|
||||
import org.rocksdb.*;
|
||||
|
||||
public class BackupableDBTest {
|
||||
static final String db_path = "/tmp/backupablejni_db";
|
||||
static final String backup_path = "/tmp/backupablejni_db_backup";
|
||||
static final String db_path = "/tmp/rocksdbjni_backupable_db_test";
|
||||
static final String backup_path = "/tmp/rocksdbjni_backupable_db_backup_test";
|
||||
static {
|
||||
RocksDB.loadLibrary();
|
||||
}
|
||||
|
282
java/org/rocksdb/test/ColumnFamilyTest.java
Normal file
282
java/org/rocksdb/test/ColumnFamilyTest.java
Normal file
@ -0,0 +1,282 @@
|
||||
// Copyright (c) 2014, Facebook, Inc. All rights reserved.
|
||||
// This source code is licensed under the BSD-style license found in the
|
||||
// LICENSE file in the root directory of this source tree. An additional grant
|
||||
// of patent rights can be found in the PATENTS file in the same directory.
|
||||
|
||||
package org.rocksdb.test;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Map;
|
||||
import org.rocksdb.*;
|
||||
|
||||
public class ColumnFamilyTest {
|
||||
static final String db_path = "/tmp/rocksdbjni_columnfamily_test";
|
||||
static {
|
||||
RocksDB.loadLibrary();
|
||||
}
|
||||
|
||||
public static void main(String[] args) {
|
||||
|
||||
RocksDB db = null;
|
||||
Options options = new Options();
|
||||
options.setCreateIfMissing(true);
|
||||
try {
|
||||
db = RocksDB.open(options, db_path);
|
||||
} catch (RocksDBException e) {
|
||||
assert(false);
|
||||
}
|
||||
// Test listColumnFamilies
|
||||
List<byte[]> columnFamilyNames;
|
||||
try {
|
||||
columnFamilyNames = RocksDB.listColumnFamilies(options, db_path);
|
||||
if (columnFamilyNames != null && columnFamilyNames.size() > 0) {
|
||||
assert(columnFamilyNames.size() == 1);
|
||||
assert(new String(columnFamilyNames.get(0)).equals("default"));
|
||||
} else {
|
||||
assert(false);
|
||||
}
|
||||
} catch (RocksDBException e) {
|
||||
assert(false);
|
||||
}
|
||||
|
||||
// Test createColumnFamily
|
||||
try {
|
||||
db.createColumnFamily("new_cf");
|
||||
} catch (RocksDBException e) {
|
||||
assert(false);
|
||||
}
|
||||
|
||||
if (db != null) {
|
||||
db.close();
|
||||
}
|
||||
|
||||
// Test listColumnFamilies after create "new_cf"
|
||||
try {
|
||||
columnFamilyNames = RocksDB.listColumnFamilies(options, db_path);
|
||||
if (columnFamilyNames != null && columnFamilyNames.size() > 0) {
|
||||
assert(columnFamilyNames.size() == 2);
|
||||
assert(new String(columnFamilyNames.get(0)).equals("default"));
|
||||
assert(new String(columnFamilyNames.get(1)).equals("new_cf"));
|
||||
} else {
|
||||
assert(false);
|
||||
}
|
||||
} catch (RocksDBException e) {
|
||||
assert(false);
|
||||
}
|
||||
|
||||
// Test open database with column family names
|
||||
List<String> cfNames = new ArrayList<String>();
|
||||
List<ColumnFamilyHandle> columnFamilyHandleList =
|
||||
new ArrayList<ColumnFamilyHandle>();
|
||||
cfNames.add("default");
|
||||
cfNames.add("new_cf");
|
||||
|
||||
try {
|
||||
db = RocksDB.open(options, db_path, cfNames, columnFamilyHandleList);
|
||||
assert(columnFamilyHandleList.size() == 2);
|
||||
db.put("dfkey1".getBytes(), "dfvalue".getBytes());
|
||||
db.put(columnFamilyHandleList.get(0), "dfkey2".getBytes(),
|
||||
"dfvalue".getBytes());
|
||||
db.put(columnFamilyHandleList.get(1), "newcfkey1".getBytes(),
|
||||
"newcfvalue".getBytes());
|
||||
|
||||
String retVal = new String(db.get(columnFamilyHandleList.get(1),
|
||||
"newcfkey1".getBytes()));
|
||||
assert(retVal.equals("newcfvalue"));
|
||||
assert( (db.get(columnFamilyHandleList.get(1),
|
||||
"dfkey1".getBytes())) == null);
|
||||
db.remove(columnFamilyHandleList.get(1), "newcfkey1".getBytes());
|
||||
assert( (db.get(columnFamilyHandleList.get(1),
|
||||
"newcfkey1".getBytes())) == null);
|
||||
db.remove("dfkey2".getBytes());
|
||||
assert( (db.get(columnFamilyHandleList.get(0),
|
||||
"dfkey2".getBytes())) == null);
|
||||
} catch (RocksDBException e) {
|
||||
assert(false);
|
||||
}
|
||||
|
||||
// Test create write to and drop ColumnFamily
|
||||
ColumnFamilyHandle tmpColumnFamilyHandle = null;
|
||||
try {
|
||||
tmpColumnFamilyHandle = db.createColumnFamily("tmpCF");
|
||||
db.put(tmpColumnFamilyHandle, "key".getBytes(), "value".getBytes());
|
||||
db.dropColumnFamily(tmpColumnFamilyHandle);
|
||||
tmpColumnFamilyHandle.dispose();
|
||||
} catch (Exception e) {
|
||||
assert(false);
|
||||
}
|
||||
|
||||
// Put to disposed column family tmpColumnFamilyHandle must fail
|
||||
try {
|
||||
db.put(tmpColumnFamilyHandle, "key".getBytes(), "value".getBytes());
|
||||
assert(false);
|
||||
} catch (RocksDBException e) {
|
||||
assert(true);
|
||||
}
|
||||
|
||||
// Remove to disposed column family tmpColumnFamilyHandle must fail
|
||||
try {
|
||||
db.remove(tmpColumnFamilyHandle, "key".getBytes());
|
||||
assert(false);
|
||||
} catch (RocksDBException e) {
|
||||
assert(true);
|
||||
}
|
||||
|
||||
// Get on a disposed column family tmpColumnFamilyHandle must fail
|
||||
try {
|
||||
db.get(tmpColumnFamilyHandle, "key".getBytes());
|
||||
assert(false);
|
||||
} catch (RocksDBException e) {
|
||||
assert(true);
|
||||
}
|
||||
|
||||
// Test WriteBatch
|
||||
try {
|
||||
WriteBatch writeBatch = new WriteBatch();
|
||||
WriteOptions writeOpt = new WriteOptions();
|
||||
writeBatch.put("key".getBytes(), "value".getBytes());
|
||||
writeBatch.put(columnFamilyHandleList.get(1), "newcfkey".getBytes(),
|
||||
"value".getBytes());
|
||||
writeBatch.put(columnFamilyHandleList.get(1), "newcfkey2".getBytes(),
|
||||
"value2".getBytes());
|
||||
writeBatch.remove("xyz".getBytes());
|
||||
writeBatch.remove(columnFamilyHandleList.get(1), "xyz".getBytes());
|
||||
db.write(writeOpt, writeBatch);
|
||||
writeBatch.dispose();
|
||||
assert(db.get(columnFamilyHandleList.get(1),
|
||||
"xyz".getBytes()) == null);
|
||||
assert(new String(db.get(columnFamilyHandleList.get(1),
|
||||
"newcfkey".getBytes())).equals("value"));
|
||||
assert(new String(db.get(columnFamilyHandleList.get(1),
|
||||
"newcfkey2".getBytes())).equals("value2"));
|
||||
assert(new String(db.get("key".getBytes())).equals("value"));
|
||||
} catch (Exception e) {
|
||||
e.printStackTrace();
|
||||
assert(false);
|
||||
}
|
||||
|
||||
// Test iterator on column family
|
||||
try {
|
||||
RocksIterator rocksIterator = db.newIterator(
|
||||
columnFamilyHandleList.get(1));
|
||||
rocksIterator.seekToFirst();
|
||||
Map<String, String> refMap = new HashMap<String, String>();
|
||||
refMap.put("newcfkey", "value");
|
||||
refMap.put("newcfkey2", "value2");
|
||||
int i = 0;
|
||||
while(rocksIterator.isValid()) {
|
||||
i++;
|
||||
refMap.get(new String(rocksIterator.key())).equals(
|
||||
new String(rocksIterator.value()));
|
||||
rocksIterator.next();
|
||||
}
|
||||
assert(i == 2);
|
||||
rocksIterator.dispose();
|
||||
} catch(Exception e) {
|
||||
assert(false);
|
||||
}
|
||||
|
||||
// Test property handling on column families
|
||||
try {
|
||||
assert(db.getProperty("rocksdb.estimate-num-keys") != null);
|
||||
assert(db.getProperty("rocksdb.stats") != null);
|
||||
assert(db.getProperty(columnFamilyHandleList.get(0),
|
||||
"rocksdb.sstables") != null);
|
||||
assert(db.getProperty(columnFamilyHandleList.get(1),
|
||||
"rocksdb.estimate-num-keys") != null);
|
||||
assert(db.getProperty(columnFamilyHandleList.get(1),
|
||||
"rocksdb.stats") != null);
|
||||
assert(db.getProperty(columnFamilyHandleList.get(1),
|
||||
"rocksdb.sstables") != null);
|
||||
} catch(Exception e) {
|
||||
assert(false);
|
||||
}
|
||||
|
||||
// MultiGet test
|
||||
List<ColumnFamilyHandle> cfCustomList = new ArrayList<ColumnFamilyHandle>();
|
||||
try {
|
||||
List<byte[]> keys = new ArrayList<byte[]>();
|
||||
keys.add("key".getBytes());
|
||||
keys.add("newcfkey".getBytes());
|
||||
Map<byte[], byte[]> retValues = db.multiGet(columnFamilyHandleList,keys);
|
||||
assert(retValues.size() == 2);
|
||||
assert(new String(retValues.get(keys.get(0)))
|
||||
.equals("value"));
|
||||
assert(new String(retValues.get(keys.get(1)))
|
||||
.equals("value"));
|
||||
|
||||
cfCustomList.add(columnFamilyHandleList.get(0));
|
||||
cfCustomList.add(columnFamilyHandleList.get(0));
|
||||
retValues = db.multiGet(cfCustomList, keys);
|
||||
assert(retValues.size() == 1);
|
||||
assert(new String(retValues.get(keys.get(0)))
|
||||
.equals("value"));
|
||||
} catch (RocksDBException e) {
|
||||
assert(false);
|
||||
} catch (IllegalArgumentException e) {
|
||||
assert(false);
|
||||
}
|
||||
|
||||
// Test multiget without correct number of column
|
||||
// families
|
||||
try {
|
||||
List<byte[]> keys = new ArrayList<byte[]>();
|
||||
keys.add("key".getBytes());
|
||||
keys.add("newcfkey".getBytes());
|
||||
cfCustomList.remove(1);
|
||||
db.multiGet(cfCustomList, keys);
|
||||
assert(false);
|
||||
} catch (RocksDBException e) {
|
||||
assert(false);
|
||||
} catch (IllegalArgumentException e) {
|
||||
assert(true);
|
||||
}
|
||||
|
||||
try {
|
||||
// iterate over default key/value pairs
|
||||
List<RocksIterator> iterators =
|
||||
db.newIterators(columnFamilyHandleList);
|
||||
assert(iterators.size() == 2);
|
||||
RocksIterator iter = iterators.get(0);
|
||||
iter.seekToFirst();
|
||||
Map<String,String> defRefMap = new HashMap<String, String>();
|
||||
defRefMap.put("dfkey1", "dfvalue");
|
||||
defRefMap.put("key", "value");
|
||||
while (iter.isValid()) {
|
||||
defRefMap.get(new String(iter.key())).equals(
|
||||
new String(iter.value()));
|
||||
iter.next();
|
||||
}
|
||||
// iterate over new_cf key/value pairs
|
||||
Map<String,String> cfRefMap = new HashMap<String, String>();
|
||||
cfRefMap.put("newcfkey", "value");
|
||||
cfRefMap.put("newcfkey2", "value2");
|
||||
iter = iterators.get(1);
|
||||
iter.seekToFirst();
|
||||
while (iter.isValid()) {
|
||||
cfRefMap.get(new String(iter.key())).equals(
|
||||
new String(iter.value()));
|
||||
iter.next();
|
||||
}
|
||||
// free iterators
|
||||
for (RocksIterator iterator : iterators) {
|
||||
iterator.dispose();
|
||||
}
|
||||
assert(true);
|
||||
} catch (RocksDBException e) {
|
||||
assert(false);
|
||||
}
|
||||
|
||||
System.out.println("Passed ColumnFamilyTest");
|
||||
// free cf handles before database close
|
||||
for (ColumnFamilyHandle columnFamilyHandle : columnFamilyHandleList) {
|
||||
columnFamilyHandle.dispose();
|
||||
}
|
||||
// close database
|
||||
db.close();
|
||||
// be sure to dispose c++ pointers
|
||||
options.dispose();
|
||||
}
|
||||
}
|
52
java/org/rocksdb/test/KeyMayExistTest.java
Normal file
52
java/org/rocksdb/test/KeyMayExistTest.java
Normal file
@ -0,0 +1,52 @@
|
||||
// Copyright (c) 2014, Facebook, Inc. All rights reserved.
|
||||
// This source code is licensed under the BSD-style license found in the
|
||||
// LICENSE file in the root directory of this source tree. An additional grant
|
||||
// of patent rights can be found in the PATENTS file in the same directory.
|
||||
package org.rocksdb.test;
|
||||
|
||||
import org.rocksdb.ColumnFamilyHandle;
|
||||
import org.rocksdb.Options;
|
||||
import org.rocksdb.RocksDB;
|
||||
import org.rocksdb.RocksDBException;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
public class KeyMayExistTest {
|
||||
static final String DB_PATH = "/tmp/rocksdbjni_keymayexit_test";
|
||||
static {
|
||||
RocksDB.loadLibrary();
|
||||
}
|
||||
|
||||
public static void main(String[] args){
|
||||
RocksDB db;
|
||||
Options options = new Options();
|
||||
options.setCreateIfMissing(true)
|
||||
.setCreateMissingColumnFamilies(true);
|
||||
try {
|
||||
// open database using cf names
|
||||
List<String> cfNames = new ArrayList<String>();
|
||||
List<ColumnFamilyHandle> columnFamilyHandleList =
|
||||
new ArrayList<ColumnFamilyHandle>();
|
||||
cfNames.add("default");
|
||||
cfNames.add("new_cf");
|
||||
db = RocksDB.open(options, DB_PATH, cfNames, columnFamilyHandleList);
|
||||
assert(columnFamilyHandleList.size()==2);
|
||||
|
||||
db.put("key".getBytes(), "value".getBytes());
|
||||
StringBuffer retValue = new StringBuffer();
|
||||
if (db.keyMayExist(columnFamilyHandleList.get(0), "key".getBytes(),
|
||||
retValue)) {
|
||||
assert(retValue.toString().equals("value"));
|
||||
} else {
|
||||
assert(false);
|
||||
}
|
||||
assert(db.keyMayExist(columnFamilyHandleList.get(1), "key".getBytes(),
|
||||
retValue) == false);
|
||||
System.out.println("Passed KeyMayExistTest");
|
||||
}catch (RocksDBException e){
|
||||
e.printStackTrace();
|
||||
assert(false);
|
||||
}
|
||||
}
|
||||
}
|
@ -26,6 +26,12 @@ public class OptionsTest {
|
||||
assert(opt.createIfMissing() == boolValue);
|
||||
}
|
||||
|
||||
{ // CreateMissingColumnFamilies test
|
||||
boolean boolValue = rand.nextBoolean();
|
||||
opt.setCreateMissingColumnFamilies(boolValue);
|
||||
assert(opt.createMissingColumnFamilies() == boolValue);
|
||||
}
|
||||
|
||||
{ // ErrorIfExists test
|
||||
boolean boolValue = rand.nextBoolean();
|
||||
opt.setErrorIfExists(boolValue);
|
||||
|
126
java/org/rocksdb/test/ReadOnlyTest.java
Normal file
126
java/org/rocksdb/test/ReadOnlyTest.java
Normal file
@ -0,0 +1,126 @@
|
||||
// Copyright (c) 2014, Facebook, Inc. All rights reserved.
|
||||
// This source code is licensed under the BSD-style license found in the
|
||||
// LICENSE file in the root directory of this source tree. An additional grant
|
||||
// of patent rights can be found in the PATENTS file in the same directory.
|
||||
package org.rocksdb.test;
|
||||
|
||||
import org.rocksdb.*;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
public class ReadOnlyTest {
|
||||
static final String DB_PATH = "/tmp/rocksdbjni_readonly_test";
|
||||
static {
|
||||
RocksDB.loadLibrary();
|
||||
}
|
||||
|
||||
public static void main(String[] args){
|
||||
RocksDB db = null, db2 = null, db3 = null;
|
||||
List<ColumnFamilyHandle> columnFamilyHandleList =
|
||||
new ArrayList<ColumnFamilyHandle>();
|
||||
List<ColumnFamilyHandle> db2ColumnFamilyHandleList =
|
||||
new ArrayList<ColumnFamilyHandle>();
|
||||
List<ColumnFamilyHandle> db3ColumnFamilyHandleList =
|
||||
new ArrayList<ColumnFamilyHandle>();
|
||||
Options options = new Options();
|
||||
options.setCreateIfMissing(true);
|
||||
try {
|
||||
db = RocksDB.open(options, DB_PATH);
|
||||
db.put("key".getBytes(), "value".getBytes());
|
||||
db2 = RocksDB.openReadOnly(DB_PATH);
|
||||
assert("value".equals(new String(db2.get("key".getBytes()))));
|
||||
db.close();
|
||||
db2.close();
|
||||
|
||||
|
||||
List<String> cfNames = new ArrayList<String>();
|
||||
cfNames.add("default");
|
||||
|
||||
db = RocksDB.open(DB_PATH, cfNames, columnFamilyHandleList);
|
||||
columnFamilyHandleList.add(db.createColumnFamily("new_cf"));
|
||||
columnFamilyHandleList.add(db.createColumnFamily("new_cf2"));
|
||||
db.put(columnFamilyHandleList.get(2), "key2".getBytes(),
|
||||
"value2".getBytes());
|
||||
|
||||
db2 = RocksDB.openReadOnly(DB_PATH, cfNames, db2ColumnFamilyHandleList);
|
||||
assert(db2.get("key2".getBytes())==null);
|
||||
assert(db2.get(columnFamilyHandleList.get(0), "key2".getBytes())==null);
|
||||
|
||||
List<String> cfNewName = new ArrayList<String>();
|
||||
cfNewName.add("default");
|
||||
cfNewName.add("new_cf2");
|
||||
db3 = RocksDB.openReadOnly(DB_PATH, cfNewName, db3ColumnFamilyHandleList);
|
||||
assert(new String(db3.get(db3ColumnFamilyHandleList.get(1),
|
||||
"key2".getBytes())).equals("value2"));
|
||||
}catch (RocksDBException e){
|
||||
e.printStackTrace();
|
||||
assert(false);
|
||||
}
|
||||
// test that put fails in readonly mode
|
||||
try {
|
||||
db2.put("key".getBytes(), "value".getBytes());
|
||||
assert(false);
|
||||
} catch (RocksDBException e) {
|
||||
assert(true);
|
||||
}
|
||||
try {
|
||||
db3.put(db3ColumnFamilyHandleList.get(1),
|
||||
"key".getBytes(), "value".getBytes());
|
||||
assert(false);
|
||||
} catch (RocksDBException e) {
|
||||
assert(true);
|
||||
}
|
||||
// test that remove fails in readonly mode
|
||||
try {
|
||||
db2.remove("key".getBytes());
|
||||
assert(false);
|
||||
} catch (RocksDBException e) {
|
||||
assert(true);
|
||||
}
|
||||
try {
|
||||
db3.remove(db3ColumnFamilyHandleList.get(1),
|
||||
"key".getBytes());
|
||||
assert(false);
|
||||
} catch (RocksDBException e) {
|
||||
assert(true);
|
||||
}
|
||||
// test that write fails in readonly mode
|
||||
WriteBatch wb = new WriteBatch();
|
||||
wb.put("key".getBytes(), "value".getBytes());
|
||||
try {
|
||||
db2.write(new WriteOptions(), wb);
|
||||
assert(false);
|
||||
} catch (RocksDBException e) {
|
||||
assert(true);
|
||||
}
|
||||
wb.dispose();
|
||||
wb = new WriteBatch();
|
||||
wb.put(db3ColumnFamilyHandleList.get(1),
|
||||
"key".getBytes(), "value".getBytes());
|
||||
try {
|
||||
db3.write(new WriteOptions(), wb);
|
||||
assert(false);
|
||||
} catch (RocksDBException e) {
|
||||
assert(true);
|
||||
}
|
||||
wb.dispose();
|
||||
// cleanup c++ pointers
|
||||
for (ColumnFamilyHandle columnFamilyHandle :
|
||||
columnFamilyHandleList) {
|
||||
columnFamilyHandle.dispose();
|
||||
}
|
||||
db.close();
|
||||
for (ColumnFamilyHandle columnFamilyHandle :
|
||||
db2ColumnFamilyHandleList) {
|
||||
columnFamilyHandle.dispose();
|
||||
}
|
||||
db2.close();
|
||||
for (ColumnFamilyHandle columnFamilyHandle :
|
||||
db3ColumnFamilyHandleList) {
|
||||
columnFamilyHandle.dispose();
|
||||
}
|
||||
db3.close();
|
||||
System.out.println("Passed ReadOnlyTest");
|
||||
}
|
||||
}
|
@ -9,7 +9,7 @@ import java.util.Collections;
|
||||
import org.rocksdb.*;
|
||||
|
||||
public class StatisticsCollectorTest {
|
||||
static final String db_path = "/tmp/backupablejni_db";
|
||||
static final String db_path = "/tmp/rocksdbjni_statistics_collector_test";
|
||||
static {
|
||||
RocksDB.loadLibrary();
|
||||
}
|
||||
@ -19,7 +19,7 @@ public class StatisticsCollectorTest {
|
||||
Options opt = new Options().createStatistics().setCreateIfMissing(true);
|
||||
Statistics stats = opt.statisticsPtr();
|
||||
|
||||
RocksDB db = RocksDB.open(db_path);
|
||||
RocksDB db = RocksDB.open(opt, db_path);
|
||||
|
||||
StatsCallbackMock callback = new StatsCallbackMock();
|
||||
StatsCollectorInput statsInput = new StatsCollectorInput(stats, callback);
|
||||
|
25
java/rocksjni/columnfamilyhandle.cc
Normal file
25
java/rocksjni/columnfamilyhandle.cc
Normal file
@ -0,0 +1,25 @@
|
||||
// Copyright (c) 2014, Facebook, Inc. All rights reserved.
|
||||
// This source code is licensed under the BSD-style license found in the
|
||||
// LICENSE file in the root directory of this source tree. An additional grant
|
||||
// of patent rights can be found in the PATENTS file in the same directory.
|
||||
//
|
||||
// This file implements the "bridge" between Java and C++ and enables
|
||||
// calling c++ rocksdb::Iterator methods from Java side.
|
||||
|
||||
#include <stdio.h>
|
||||
#include <stdlib.h>
|
||||
#include <jni.h>
|
||||
|
||||
#include "include/org_rocksdb_ColumnFamilyHandle.h"
|
||||
#include "rocksjni/portal.h"
|
||||
|
||||
/*
|
||||
* Class: org_rocksdb_ColumnFamilyHandle
|
||||
* Method: disposeInternal
|
||||
* Signature: (J)V
|
||||
*/
|
||||
void Java_org_rocksdb_ColumnFamilyHandle_disposeInternal(
|
||||
JNIEnv* env, jobject jobj, jlong handle) {
|
||||
auto it = reinterpret_cast<rocksdb::ColumnFamilyHandle*>(handle);
|
||||
delete it;
|
||||
}
|
@ -8,7 +8,7 @@
|
||||
#include <stdio.h>
|
||||
#include <stdlib.h>
|
||||
#include <jni.h>
|
||||
#include <string>
|
||||
#include <strings.h>
|
||||
#include <memory>
|
||||
|
||||
#include "include/org_rocksdb_Options.h"
|
||||
@ -64,6 +64,28 @@ jboolean Java_org_rocksdb_Options_createIfMissing(
|
||||
return reinterpret_cast<rocksdb::Options*>(jhandle)->create_if_missing;
|
||||
}
|
||||
|
||||
/*
|
||||
* Class: org_rocksdb_Options
|
||||
* Method: setCreateMissingColumnFamilies
|
||||
* Signature: (JZ)V
|
||||
*/
|
||||
void Java_org_rocksdb_Options_setCreateMissingColumnFamilies(
|
||||
JNIEnv* env, jobject jobj, jlong jhandle, jboolean flag) {
|
||||
reinterpret_cast<rocksdb::Options*>
|
||||
(jhandle)->create_missing_column_families = flag;
|
||||
}
|
||||
|
||||
/*
|
||||
* Class: org_rocksdb_Options
|
||||
* Method: createMissingColumnFamilies
|
||||
* Signature: (J)Z
|
||||
*/
|
||||
jboolean Java_org_rocksdb_Options_createMissingColumnFamilies(
|
||||
JNIEnv* env, jobject jobj, jlong jhandle) {
|
||||
return reinterpret_cast<rocksdb::Options*>
|
||||
(jhandle)->create_missing_column_families;
|
||||
}
|
||||
|
||||
/*
|
||||
* Class: org_rocksdb_Options
|
||||
* Method: useReverseBytewiseComparator
|
||||
|
@ -327,6 +327,39 @@ class FilterJni {
|
||||
}
|
||||
};
|
||||
|
||||
class ColumnFamilyHandleJni {
|
||||
public:
|
||||
// Get the java class id of org.rocksdb.ColumnFamilyHandle.
|
||||
static jclass getJClass(JNIEnv* env) {
|
||||
jclass jclazz = env->FindClass("org/rocksdb/ColumnFamilyHandle");
|
||||
assert(jclazz != nullptr);
|
||||
return jclazz;
|
||||
}
|
||||
|
||||
// Get the field id of the member variable of org.rocksdb.ColumnFamilyHandle.
|
||||
// that stores the pointer to rocksdb::ColumnFamilyHandle.
|
||||
static jfieldID getHandleFieldID(JNIEnv* env) {
|
||||
static jfieldID fid = env->GetFieldID(
|
||||
getJClass(env), "nativeHandle_", "J");
|
||||
assert(fid != nullptr);
|
||||
return fid;
|
||||
}
|
||||
|
||||
// Get the pointer to rocksdb::ColumnFamilyHandle.
|
||||
static rocksdb::ColumnFamilyHandle* getHandle(JNIEnv* env, jobject jobj) {
|
||||
return reinterpret_cast<rocksdb::ColumnFamilyHandle*>(
|
||||
env->GetLongField(jobj, getHandleFieldID(env)));
|
||||
}
|
||||
|
||||
// Pass the rocksdb::ColumnFamilyHandle pointer to the java side.
|
||||
static void setHandle(
|
||||
JNIEnv* env, jobject jobj, const rocksdb::ColumnFamilyHandle* op) {
|
||||
env->SetLongField(
|
||||
jobj, getHandleFieldID(env),
|
||||
reinterpret_cast<jlong>(op));
|
||||
}
|
||||
};
|
||||
|
||||
class ListJni {
|
||||
public:
|
||||
// Get the java class id of java.util.List.
|
||||
|
@ -25,7 +25,7 @@
|
||||
* Method: open
|
||||
* Signature: (JLjava/lang/String;)V
|
||||
*/
|
||||
void Java_org_rocksdb_RocksDB_open(
|
||||
void Java_org_rocksdb_RocksDB_open__JLjava_lang_String_2(
|
||||
JNIEnv* env, jobject jdb, jlong jopt_handle, jstring jdb_path) {
|
||||
auto opt = reinterpret_cast<rocksdb::Options*>(jopt_handle);
|
||||
rocksdb::DB* db = nullptr;
|
||||
@ -40,12 +40,215 @@ void Java_org_rocksdb_RocksDB_open(
|
||||
rocksdb::RocksDBExceptionJni::ThrowNew(env, s);
|
||||
}
|
||||
|
||||
/*
|
||||
* Class: org_rocksdb_RocksDB
|
||||
* Method: openROnly
|
||||
* Signature: (JLjava/lang/String;)V
|
||||
*/
|
||||
void Java_org_rocksdb_RocksDB_openROnly__JLjava_lang_String_2(
|
||||
JNIEnv* env, jobject jdb, jlong jopt_handle, jstring jdb_path) {
|
||||
auto opt = reinterpret_cast<rocksdb::Options*>(jopt_handle);
|
||||
rocksdb::DB* db = nullptr;
|
||||
const char* db_path = env->GetStringUTFChars(jdb_path, 0);
|
||||
rocksdb::Status s = rocksdb::DB::OpenForReadOnly(*opt,
|
||||
db_path, &db);
|
||||
env->ReleaseStringUTFChars(jdb_path, db_path);
|
||||
|
||||
if (s.ok()) {
|
||||
rocksdb::RocksDBJni::setHandle(env, jdb, db);
|
||||
return;
|
||||
}
|
||||
rocksdb::RocksDBExceptionJni::ThrowNew(env, s);
|
||||
}
|
||||
|
||||
/*
|
||||
* Class: org_rocksdb_RocksDB
|
||||
* Method: openROnly
|
||||
* Signature: (JLjava/lang/String;Ljava/util/List;I)Ljava/util/List;
|
||||
*/
|
||||
jobject
|
||||
Java_org_rocksdb_RocksDB_openROnly__JLjava_lang_String_2Ljava_util_List_2I(
|
||||
JNIEnv* env, jobject jdb, jlong jopt_handle, jstring jdb_path,
|
||||
jobject jcfname_list, jint jcfname_count) {
|
||||
auto opt = reinterpret_cast<rocksdb::Options*>(jopt_handle);
|
||||
rocksdb::DB* db = nullptr;
|
||||
const char* db_path = env->GetStringUTFChars(jdb_path, 0);
|
||||
|
||||
std::vector<const char*> cfnames_to_free;
|
||||
std::vector<jstring> jcfnames_for_free;
|
||||
|
||||
std::vector<rocksdb::ColumnFamilyDescriptor> column_families;
|
||||
std::vector<rocksdb::ColumnFamilyHandle* > handles;
|
||||
// get iterator for cfnames
|
||||
jobject iteratorObj = env->CallObjectMethod(
|
||||
jcfname_list, rocksdb::ListJni::getIteratorMethod(env));
|
||||
|
||||
// iterate over cfnames and convert cfnames to
|
||||
// ColumnFamilyDescriptor instances
|
||||
while (env->CallBooleanMethod(
|
||||
iteratorObj, rocksdb::ListJni::getHasNextMethod(env)) == JNI_TRUE) {
|
||||
jstring jstr = (jstring) env->CallObjectMethod(iteratorObj,
|
||||
rocksdb::ListJni::getNextMethod(env));
|
||||
const char* cfname = env->GetStringUTFChars(jstr, 0);
|
||||
|
||||
// free allocated cfnames after call to open
|
||||
cfnames_to_free.push_back(cfname);
|
||||
jcfnames_for_free.push_back(jstr);
|
||||
column_families.push_back(rocksdb::ColumnFamilyDescriptor(cfname,
|
||||
rocksdb::ColumnFamilyOptions()));
|
||||
}
|
||||
|
||||
rocksdb::Status s = rocksdb::DB::OpenForReadOnly(*opt,
|
||||
db_path, column_families, &handles, &db);
|
||||
env->ReleaseStringUTFChars(jdb_path, db_path);
|
||||
// free jbyte allocations
|
||||
for (std::vector<jbyte*>::size_type i = 0;
|
||||
i != cfnames_to_free.size(); i++) {
|
||||
// free cfnames
|
||||
env->ReleaseStringUTFChars(jcfnames_for_free[i], cfnames_to_free[i]);
|
||||
}
|
||||
|
||||
jobject jcfhandle_list = nullptr;
|
||||
// check if open operation was successful
|
||||
if (s.ok()) {
|
||||
rocksdb::RocksDBJni::setHandle(env, jdb, db);
|
||||
jclass jclazz = env->FindClass("java/util/ArrayList");
|
||||
jmethodID mid = rocksdb::ListJni::getArrayListConstructorMethodId(
|
||||
env, jclazz);
|
||||
jobject jcfhandle_list = env->NewObject(jclazz, mid, handles.size());
|
||||
// insert in java list
|
||||
for (std::vector<rocksdb::ColumnFamilyHandle*>::size_type i = 0;
|
||||
i != handles.size(); i++) {
|
||||
// jlong must be converted to Long due to collections restrictions
|
||||
jclass jclazz = env->FindClass("java/lang/Long");
|
||||
jmethodID mid = env->GetMethodID(jclazz, "<init>", "(J)V");
|
||||
jobject obj = env->NewObject(jclazz, mid,
|
||||
reinterpret_cast<jlong>(handles[i]));
|
||||
env->CallBooleanMethod(jcfhandle_list,
|
||||
rocksdb::ListJni::getListAddMethodId(env), obj);
|
||||
}
|
||||
|
||||
return jcfhandle_list;
|
||||
}
|
||||
rocksdb::RocksDBExceptionJni::ThrowNew(env, s);
|
||||
return jcfhandle_list;
|
||||
}
|
||||
|
||||
/*
|
||||
* Class: org_rocksdb_RocksDB
|
||||
* Method: open
|
||||
* Signature: (JLjava/lang/String;Ljava/util/List;I)Ljava/util/List;
|
||||
*/
|
||||
jobject Java_org_rocksdb_RocksDB_open__JLjava_lang_String_2Ljava_util_List_2I(
|
||||
JNIEnv* env, jobject jdb, jlong jopt_handle, jstring jdb_path,
|
||||
jobject jcfname_list, jint jcfname_count) {
|
||||
auto opt = reinterpret_cast<rocksdb::Options*>(jopt_handle);
|
||||
rocksdb::DB* db = nullptr;
|
||||
const char* db_path = env->GetStringUTFChars(jdb_path, 0);
|
||||
|
||||
std::vector<const char*> cfnames_to_free;
|
||||
std::vector<jstring> jcfnames_for_free;
|
||||
|
||||
std::vector<rocksdb::ColumnFamilyDescriptor> column_families;
|
||||
std::vector<rocksdb::ColumnFamilyHandle* > handles;
|
||||
// get iterator for cfnames
|
||||
jobject iteratorObj = env->CallObjectMethod(
|
||||
jcfname_list, rocksdb::ListJni::getIteratorMethod(env));
|
||||
|
||||
// iterate over cfnames and convert cfnames to
|
||||
// ColumnFamilyDescriptor instances
|
||||
while (env->CallBooleanMethod(
|
||||
iteratorObj, rocksdb::ListJni::getHasNextMethod(env)) == JNI_TRUE) {
|
||||
jstring jstr = (jstring) env->CallObjectMethod(iteratorObj,
|
||||
rocksdb::ListJni::getNextMethod(env));
|
||||
const char* cfname = env->GetStringUTFChars(jstr, 0);
|
||||
|
||||
// free allocated cfnames after call to open
|
||||
cfnames_to_free.push_back(cfname);
|
||||
jcfnames_for_free.push_back(jstr);
|
||||
column_families.push_back(rocksdb::ColumnFamilyDescriptor(cfname,
|
||||
rocksdb::ColumnFamilyOptions()));
|
||||
}
|
||||
|
||||
rocksdb::Status s = rocksdb::DB::Open(*opt, db_path, column_families,
|
||||
&handles, &db);
|
||||
env->ReleaseStringUTFChars(jdb_path, db_path);
|
||||
// free jbyte allocations
|
||||
for (std::vector<jbyte*>::size_type i = 0;
|
||||
i != cfnames_to_free.size(); i++) {
|
||||
// free cfnames
|
||||
env->ReleaseStringUTFChars(jcfnames_for_free[i], cfnames_to_free[i]);
|
||||
}
|
||||
|
||||
jobject jcfhandle_list = nullptr;
|
||||
// check if open operation was successful
|
||||
if (s.ok()) {
|
||||
rocksdb::RocksDBJni::setHandle(env, jdb, db);
|
||||
jclass jclazz = env->FindClass("java/util/ArrayList");
|
||||
jmethodID mid = rocksdb::ListJni::getArrayListConstructorMethodId(
|
||||
env, jclazz);
|
||||
jobject jcfhandle_list = env->NewObject(jclazz, mid, handles.size());
|
||||
// insert in java list
|
||||
for (std::vector<rocksdb::ColumnFamilyHandle*>::size_type i = 0;
|
||||
i != handles.size(); i++) {
|
||||
// jlong must be converted to Long due to collections restrictions
|
||||
jclass jclazz = env->FindClass("java/lang/Long");
|
||||
jmethodID mid = env->GetMethodID(jclazz, "<init>", "(J)V");
|
||||
jobject obj = env->NewObject(jclazz, mid,
|
||||
reinterpret_cast<jlong>(handles[i]));
|
||||
env->CallBooleanMethod(jcfhandle_list,
|
||||
rocksdb::ListJni::getListAddMethodId(env), obj);
|
||||
}
|
||||
|
||||
return jcfhandle_list;
|
||||
}
|
||||
rocksdb::RocksDBExceptionJni::ThrowNew(env, s);
|
||||
return jcfhandle_list;
|
||||
}
|
||||
|
||||
//////////////////////////////////////////////////////////////////////////////
|
||||
// rocksdb::DB::ListColumnFamilies
|
||||
|
||||
/*
|
||||
* Class: org_rocksdb_RocksDB
|
||||
* Method: listColumnFamilies
|
||||
* Signature: (JLjava/lang/String;)Ljava/util/List;
|
||||
*/
|
||||
jobject Java_org_rocksdb_RocksDB_listColumnFamilies(
|
||||
JNIEnv* env, jclass jclazz, jlong jopt_handle, jstring jdb_path) {
|
||||
std::vector<std::string> column_family_names;
|
||||
auto opt = reinterpret_cast<rocksdb::Options*>(jopt_handle);
|
||||
const char* db_path = env->GetStringUTFChars(jdb_path, 0);
|
||||
jobject jvalue_list = nullptr;
|
||||
|
||||
rocksdb::Status s = rocksdb::DB::ListColumnFamilies(*opt, db_path,
|
||||
&column_family_names);
|
||||
env->ReleaseStringUTFChars(jdb_path, db_path);
|
||||
if (s.ok()) {
|
||||
// Don't reuse class pointer
|
||||
jclass jclazz = env->FindClass("java/util/ArrayList");
|
||||
jmethodID mid = rocksdb::ListJni::getArrayListConstructorMethodId(env,
|
||||
jclazz);
|
||||
jvalue_list = env->NewObject(jclazz, mid, column_family_names.size());
|
||||
|
||||
for (std::vector<std::string>::size_type i = 0;
|
||||
i < column_family_names.size(); i++) {
|
||||
jbyteArray jvalue = env->NewByteArray(column_family_names[i].size());
|
||||
env->SetByteArrayRegion(jvalue, 0, column_family_names[i].size(),
|
||||
reinterpret_cast<const jbyte*>(column_family_names[i].c_str()));
|
||||
env->CallBooleanMethod(jvalue_list,
|
||||
rocksdb::ListJni::getListAddMethodId(env), jvalue);
|
||||
}
|
||||
}
|
||||
return jvalue_list;
|
||||
}
|
||||
|
||||
//////////////////////////////////////////////////////////////////////////////
|
||||
// rocksdb::DB::Put
|
||||
|
||||
void rocksdb_put_helper(
|
||||
JNIEnv* env, rocksdb::DB* db, const rocksdb::WriteOptions& write_options,
|
||||
jbyteArray jkey, jint jkey_len,
|
||||
rocksdb::ColumnFamilyHandle* cf_handle, jbyteArray jkey, jint jkey_len,
|
||||
jbyteArray jvalue, jint jvalue_len) {
|
||||
|
||||
jbyte* key = env->GetByteArrayElements(jkey, 0);
|
||||
@ -53,7 +256,13 @@ void rocksdb_put_helper(
|
||||
rocksdb::Slice key_slice(reinterpret_cast<char*>(key), jkey_len);
|
||||
rocksdb::Slice value_slice(reinterpret_cast<char*>(value), jvalue_len);
|
||||
|
||||
rocksdb::Status s = db->Put(write_options, key_slice, value_slice);
|
||||
rocksdb::Status s;
|
||||
if (cf_handle != nullptr) {
|
||||
s = db->Put(write_options, cf_handle, key_slice, value_slice);
|
||||
} else {
|
||||
// backwards compatibility
|
||||
s = db->Put(write_options, key_slice, value_slice);
|
||||
}
|
||||
|
||||
// trigger java unref on key and value.
|
||||
// by passing JNI_ABORT, it will simply release the reference without
|
||||
@ -80,10 +289,31 @@ void Java_org_rocksdb_RocksDB_put__J_3BI_3BI(
|
||||
static const rocksdb::WriteOptions default_write_options =
|
||||
rocksdb::WriteOptions();
|
||||
|
||||
rocksdb_put_helper(env, db, default_write_options,
|
||||
rocksdb_put_helper(env, db, default_write_options, nullptr,
|
||||
jkey, jkey_len,
|
||||
jvalue, jvalue_len);
|
||||
}
|
||||
/*
|
||||
* Class: org_rocksdb_RocksDB
|
||||
* Method: put
|
||||
* Signature: (J[BI[BIJ)V
|
||||
*/
|
||||
void Java_org_rocksdb_RocksDB_put__J_3BI_3BIJ(
|
||||
JNIEnv* env, jobject jdb, jlong jdb_handle,
|
||||
jbyteArray jkey, jint jkey_len,
|
||||
jbyteArray jvalue, jint jvalue_len, jlong jcf_handle) {
|
||||
auto db = reinterpret_cast<rocksdb::DB*>(jdb_handle);
|
||||
static const rocksdb::WriteOptions default_write_options =
|
||||
rocksdb::WriteOptions();
|
||||
auto cf_handle = reinterpret_cast<rocksdb::ColumnFamilyHandle*>(jcf_handle);
|
||||
if (cf_handle != nullptr) {
|
||||
rocksdb_put_helper(env, db, default_write_options, cf_handle,
|
||||
jkey, jkey_len, jvalue, jvalue_len);
|
||||
} else {
|
||||
rocksdb::RocksDBExceptionJni::ThrowNew(env,
|
||||
rocksdb::Status::InvalidArgument("Invalid ColumnFamilyHandle."));
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
* Class: org_rocksdb_RocksDB
|
||||
@ -99,11 +329,34 @@ void Java_org_rocksdb_RocksDB_put__JJ_3BI_3BI(
|
||||
auto write_options = reinterpret_cast<rocksdb::WriteOptions*>(
|
||||
jwrite_options_handle);
|
||||
|
||||
rocksdb_put_helper(env, db, *write_options,
|
||||
rocksdb_put_helper(env, db, *write_options, nullptr,
|
||||
jkey, jkey_len,
|
||||
jvalue, jvalue_len);
|
||||
}
|
||||
|
||||
/*
|
||||
* Class: org_rocksdb_RocksDB
|
||||
* Method: put
|
||||
* Signature: (JJ[BI[BIJ)V
|
||||
*/
|
||||
void Java_org_rocksdb_RocksDB_put__JJ_3BI_3BIJ(
|
||||
JNIEnv* env, jobject jdb,
|
||||
jlong jdb_handle, jlong jwrite_options_handle,
|
||||
jbyteArray jkey, jint jkey_len,
|
||||
jbyteArray jvalue, jint jvalue_len, jlong jcf_handle) {
|
||||
auto db = reinterpret_cast<rocksdb::DB*>(jdb_handle);
|
||||
auto write_options = reinterpret_cast<rocksdb::WriteOptions*>(
|
||||
jwrite_options_handle);
|
||||
auto cf_handle = reinterpret_cast<rocksdb::ColumnFamilyHandle*>(jcf_handle);
|
||||
if (cf_handle != nullptr) {
|
||||
rocksdb_put_helper(env, db, *write_options, cf_handle,
|
||||
jkey, jkey_len, jvalue, jvalue_len);
|
||||
} else {
|
||||
rocksdb::RocksDBExceptionJni::ThrowNew(env,
|
||||
rocksdb::Status::InvalidArgument("Invalid ColumnFamilyHandle."));
|
||||
}
|
||||
}
|
||||
|
||||
//////////////////////////////////////////////////////////////////////////////
|
||||
// rocksdb::DB::Write
|
||||
/*
|
||||
@ -126,20 +379,94 @@ void Java_org_rocksdb_RocksDB_write(
|
||||
}
|
||||
}
|
||||
|
||||
//////////////////////////////////////////////////////////////////////////////
|
||||
// rocksdb::DB::KeyMayExist
|
||||
jboolean key_may_exist_helper(JNIEnv* env, rocksdb::DB* db,
|
||||
const rocksdb::ReadOptions& read_opt,
|
||||
rocksdb::ColumnFamilyHandle* cf_handle, jbyteArray jkey, jint jkey_len,
|
||||
jobject jvalue) {
|
||||
std::string value;
|
||||
bool value_found = false;
|
||||
jboolean isCopy;
|
||||
jbyte* key = env->GetByteArrayElements(jkey, &isCopy);
|
||||
rocksdb::Slice key_slice(reinterpret_cast<char*>(key), jkey_len);
|
||||
bool keyMaxExist = db->KeyMayExist(read_opt, cf_handle, key_slice,
|
||||
&value, &value_found);
|
||||
if (value_found && !value.empty()) {
|
||||
jclass clazz = env->GetObjectClass(jvalue);
|
||||
jmethodID mid = env->GetMethodID(clazz, "append",
|
||||
"(Ljava/lang/String;)Ljava/lang/StringBuffer;");
|
||||
jstring new_value_str = env->NewStringUTF(value.c_str());
|
||||
env->CallObjectMethod(jvalue, mid, new_value_str);
|
||||
}
|
||||
env->ReleaseByteArrayElements(jkey, key, JNI_ABORT);
|
||||
return static_cast<jboolean>(keyMaxExist);
|
||||
}
|
||||
|
||||
/*
|
||||
* Class: org_rocksdb_RocksDB
|
||||
* Method: keyMayExist
|
||||
* Signature: ([BIJLjava/lang/StringBuffer;)Z
|
||||
*/
|
||||
jboolean Java_org_rocksdb_RocksDB_keyMayExist___3BIJLjava_lang_StringBuffer_2(
|
||||
JNIEnv* env, jobject jdb, jbyteArray jkey, jint jkey_len,
|
||||
jlong jcf_handle, jobject jvalue) {
|
||||
rocksdb::DB* db = rocksdb::RocksDBJni::getHandle(env, jdb);
|
||||
auto cf_handle = reinterpret_cast<rocksdb::ColumnFamilyHandle*>(
|
||||
jcf_handle);
|
||||
if (cf_handle != nullptr) {
|
||||
return key_may_exist_helper(env, db, rocksdb::ReadOptions(),
|
||||
cf_handle, jkey, jkey_len, jvalue);
|
||||
} else {
|
||||
rocksdb::RocksDBExceptionJni::ThrowNew(env,
|
||||
rocksdb::Status::InvalidArgument("Invalid ColumnFamilyHandle."));
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
/*
|
||||
* Class: org_rocksdb_RocksDB
|
||||
* Method: keyMayExist
|
||||
* Signature: (J[BIJLjava/lang/StringBuffer;)Z
|
||||
*/
|
||||
jboolean Java_org_rocksdb_RocksDB_keyMayExist__J_3BIJLjava_lang_StringBuffer_2(
|
||||
JNIEnv* env, jobject jdb, jlong jread_options_handle,
|
||||
jbyteArray jkey, jint jkey_len, jlong jcf_handle, jobject jvalue) {
|
||||
rocksdb::DB* db = rocksdb::RocksDBJni::getHandle(env, jdb);
|
||||
auto& read_options = *reinterpret_cast<rocksdb::ReadOptions*>(
|
||||
jread_options_handle);
|
||||
auto cf_handle = reinterpret_cast<rocksdb::ColumnFamilyHandle*>(
|
||||
jcf_handle);
|
||||
if (cf_handle != nullptr) {
|
||||
return key_may_exist_helper(env, db, read_options, cf_handle,
|
||||
jkey, jkey_len, jvalue);
|
||||
} else {
|
||||
rocksdb::RocksDBExceptionJni::ThrowNew(env,
|
||||
rocksdb::Status::InvalidArgument("Invalid ColumnFamilyHandle."));
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
//////////////////////////////////////////////////////////////////////////////
|
||||
// rocksdb::DB::Get
|
||||
|
||||
jbyteArray rocksdb_get_helper(
|
||||
JNIEnv* env, rocksdb::DB* db, const rocksdb::ReadOptions& read_opt,
|
||||
jbyteArray jkey, jint jkey_len) {
|
||||
rocksdb::ColumnFamilyHandle* column_family_handle, jbyteArray jkey,
|
||||
jint jkey_len) {
|
||||
jboolean isCopy;
|
||||
jbyte* key = env->GetByteArrayElements(jkey, &isCopy);
|
||||
rocksdb::Slice key_slice(
|
||||
reinterpret_cast<char*>(key), jkey_len);
|
||||
|
||||
std::string value;
|
||||
rocksdb::Status s = db->Get(
|
||||
read_opt, key_slice, &value);
|
||||
rocksdb::Status s;
|
||||
if (column_family_handle != nullptr) {
|
||||
s = db->Get(read_opt, column_family_handle, key_slice, &value);
|
||||
} else {
|
||||
// backwards compatibility
|
||||
s = db->Get(read_opt, key_slice, &value);
|
||||
}
|
||||
|
||||
// trigger java unref on key.
|
||||
// by passing JNI_ABORT, it will simply release the reference without
|
||||
@ -172,10 +499,31 @@ jbyteArray Java_org_rocksdb_RocksDB_get__J_3BI(
|
||||
jbyteArray jkey, jint jkey_len) {
|
||||
return rocksdb_get_helper(env,
|
||||
reinterpret_cast<rocksdb::DB*>(jdb_handle),
|
||||
rocksdb::ReadOptions(),
|
||||
rocksdb::ReadOptions(), nullptr,
|
||||
jkey, jkey_len);
|
||||
}
|
||||
|
||||
/*
|
||||
* Class: org_rocksdb_RocksDB
|
||||
* Method: get
|
||||
* Signature: (J[BIJ)[B
|
||||
*/
|
||||
jbyteArray Java_org_rocksdb_RocksDB_get__J_3BIJ(
|
||||
JNIEnv* env, jobject jdb, jlong jdb_handle,
|
||||
jbyteArray jkey, jint jkey_len, jlong jcf_handle) {
|
||||
auto db_handle = reinterpret_cast<rocksdb::DB*>(jdb_handle);
|
||||
auto cf_handle = reinterpret_cast<rocksdb::ColumnFamilyHandle*>(jcf_handle);
|
||||
if (cf_handle != nullptr) {
|
||||
return rocksdb_get_helper(env, db_handle, rocksdb::ReadOptions(),
|
||||
cf_handle, jkey, jkey_len);
|
||||
} else {
|
||||
rocksdb::RocksDBExceptionJni::ThrowNew(env,
|
||||
rocksdb::Status::InvalidArgument("Invalid ColumnFamilyHandle."));
|
||||
// will never be evaluated
|
||||
return env->NewByteArray(0);
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
* Class: org_rocksdb_RocksDB
|
||||
* Method: get
|
||||
@ -186,14 +534,36 @@ jbyteArray Java_org_rocksdb_RocksDB_get__JJ_3BI(
|
||||
jbyteArray jkey, jint jkey_len) {
|
||||
return rocksdb_get_helper(env,
|
||||
reinterpret_cast<rocksdb::DB*>(jdb_handle),
|
||||
*reinterpret_cast<rocksdb::ReadOptions*>(jropt_handle),
|
||||
*reinterpret_cast<rocksdb::ReadOptions*>(jropt_handle), nullptr,
|
||||
jkey, jkey_len);
|
||||
}
|
||||
|
||||
/*
|
||||
* Class: org_rocksdb_RocksDB
|
||||
* Method: get
|
||||
* Signature: (JJ[BIJ)[B
|
||||
*/
|
||||
jbyteArray Java_org_rocksdb_RocksDB_get__JJ_3BIJ(
|
||||
JNIEnv* env, jobject jdb, jlong jdb_handle, jlong jropt_handle,
|
||||
jbyteArray jkey, jint jkey_len, jlong jcf_handle) {
|
||||
auto db_handle = reinterpret_cast<rocksdb::DB*>(jdb_handle);
|
||||
auto& ro_opt = *reinterpret_cast<rocksdb::ReadOptions*>(jropt_handle);
|
||||
auto cf_handle = reinterpret_cast<rocksdb::ColumnFamilyHandle*>(jcf_handle);
|
||||
if (cf_handle != nullptr) {
|
||||
return rocksdb_get_helper(env, db_handle, ro_opt, cf_handle,
|
||||
jkey, jkey_len);
|
||||
} else {
|
||||
rocksdb::RocksDBExceptionJni::ThrowNew(env,
|
||||
rocksdb::Status::InvalidArgument("Invalid ColumnFamilyHandle."));
|
||||
// will never be evaluated
|
||||
return env->NewByteArray(0);
|
||||
}
|
||||
}
|
||||
|
||||
jint rocksdb_get_helper(
|
||||
JNIEnv* env, rocksdb::DB* db, const rocksdb::ReadOptions& read_options,
|
||||
jbyteArray jkey, jint jkey_len,
|
||||
jbyteArray jvalue, jint jvalue_len) {
|
||||
rocksdb::ColumnFamilyHandle* column_family_handle, jbyteArray jkey,
|
||||
jint jkey_len, jbyteArray jvalue, jint jvalue_len) {
|
||||
static const int kNotFound = -1;
|
||||
static const int kStatusError = -2;
|
||||
|
||||
@ -204,8 +574,13 @@ jint rocksdb_get_helper(
|
||||
// TODO(yhchiang): we might save one memory allocation here by adding
|
||||
// a DB::Get() function which takes preallocated jbyte* as input.
|
||||
std::string cvalue;
|
||||
rocksdb::Status s = db->Get(
|
||||
read_options, key_slice, &cvalue);
|
||||
rocksdb::Status s;
|
||||
if (column_family_handle != nullptr) {
|
||||
s = db->Get(read_options, column_family_handle, key_slice, &cvalue);
|
||||
} else {
|
||||
// backwards compatibility
|
||||
s = db->Get(read_options, key_slice, &cvalue);
|
||||
}
|
||||
|
||||
// trigger java unref on key.
|
||||
// by passing JNI_ABORT, it will simply release the reference without
|
||||
@ -236,11 +611,31 @@ jint rocksdb_get_helper(
|
||||
return cvalue_len;
|
||||
}
|
||||
|
||||
// cf multi get
|
||||
jobject multi_get_helper(JNIEnv* env, jobject jdb, rocksdb::DB* db,
|
||||
const rocksdb::ReadOptions& rOpt, jobject jkey_list, jint jkeys_count) {
|
||||
const rocksdb::ReadOptions& rOpt, jobject jkey_list, jint jkeys_count,
|
||||
jobject jcfhandle_list) {
|
||||
std::vector<rocksdb::Slice> keys;
|
||||
std::vector<jbyte*> keys_to_free;
|
||||
std::vector<rocksdb::ColumnFamilyHandle*> cf_handles;
|
||||
|
||||
if (jcfhandle_list != nullptr) {
|
||||
// get cf iterator
|
||||
jobject cfIteratorObj = env->CallObjectMethod(
|
||||
jcfhandle_list, rocksdb::ListJni::getIteratorMethod(env));
|
||||
|
||||
// iterate over keys and convert java byte array to slice
|
||||
while (env->CallBooleanMethod(
|
||||
cfIteratorObj, rocksdb::ListJni::getHasNextMethod(env)) == JNI_TRUE) {
|
||||
jobject jobj = (jbyteArray) env->CallObjectMethod(
|
||||
cfIteratorObj, rocksdb::ListJni::getNextMethod(env));
|
||||
rocksdb::ColumnFamilyHandle* cfHandle =
|
||||
rocksdb::ColumnFamilyHandleJni::getHandle(env, jobj);
|
||||
cf_handles.push_back(cfHandle);
|
||||
}
|
||||
}
|
||||
|
||||
// Process key list
|
||||
// get iterator
|
||||
jobject iteratorObj = env->CallObjectMethod(
|
||||
jkey_list, rocksdb::ListJni::getIteratorMethod(env));
|
||||
@ -263,7 +658,12 @@ jobject multi_get_helper(JNIEnv* env, jobject jdb, rocksdb::DB* db,
|
||||
}
|
||||
|
||||
std::vector<std::string> values;
|
||||
std::vector<rocksdb::Status> s = db->MultiGet(rOpt, keys, &values);
|
||||
std::vector<rocksdb::Status> s;
|
||||
if (cf_handles.size() == 0) {
|
||||
s = db->MultiGet(rOpt, keys, &values);
|
||||
} else {
|
||||
s = db->MultiGet(rOpt, cf_handles, keys, &values);
|
||||
}
|
||||
|
||||
// Don't reuse class pointer
|
||||
jclass jclazz = env->FindClass("java/util/ArrayList");
|
||||
@ -285,13 +685,11 @@ jobject multi_get_helper(JNIEnv* env, jobject jdb, rocksdb::DB* db,
|
||||
jvalue_list, rocksdb::ListJni::getListAddMethodId(env), nullptr);
|
||||
}
|
||||
}
|
||||
|
||||
// free up allocated byte arrays
|
||||
for (std::vector<jbyte*>::size_type i = 0; i != keys_to_free.size(); i++) {
|
||||
delete[] keys_to_free[i];
|
||||
}
|
||||
keys_to_free.clear();
|
||||
|
||||
return jvalue_list;
|
||||
}
|
||||
|
||||
@ -304,7 +702,20 @@ jobject Java_org_rocksdb_RocksDB_multiGet__JLjava_util_List_2I(
|
||||
JNIEnv* env, jobject jdb, jlong jdb_handle,
|
||||
jobject jkey_list, jint jkeys_count) {
|
||||
return multi_get_helper(env, jdb, reinterpret_cast<rocksdb::DB*>(jdb_handle),
|
||||
rocksdb::ReadOptions(), jkey_list, jkeys_count);
|
||||
rocksdb::ReadOptions(), jkey_list, jkeys_count, nullptr);
|
||||
}
|
||||
|
||||
/*
|
||||
* Class: org_rocksdb_RocksDB
|
||||
* Method: multiGet
|
||||
* Signature: (JLjava/util/List;ILjava/util/List;)Ljava/util/List;
|
||||
*/
|
||||
jobject
|
||||
Java_org_rocksdb_RocksDB_multiGet__JLjava_util_List_2ILjava_util_List_2(
|
||||
JNIEnv* env, jobject jdb, jlong jdb_handle,
|
||||
jobject jkey_list, jint jkeys_count, jobject jcfhandle_list) {
|
||||
return multi_get_helper(env, jdb, reinterpret_cast<rocksdb::DB*>(jdb_handle),
|
||||
rocksdb::ReadOptions(), jkey_list, jkeys_count, jcfhandle_list);
|
||||
}
|
||||
|
||||
/*
|
||||
@ -317,7 +728,22 @@ jobject Java_org_rocksdb_RocksDB_multiGet__JJLjava_util_List_2I(
|
||||
jlong jropt_handle, jobject jkey_list, jint jkeys_count) {
|
||||
return multi_get_helper(env, jdb, reinterpret_cast<rocksdb::DB*>(jdb_handle),
|
||||
*reinterpret_cast<rocksdb::ReadOptions*>(jropt_handle), jkey_list,
|
||||
jkeys_count);
|
||||
jkeys_count, nullptr);
|
||||
}
|
||||
|
||||
/*
|
||||
* Class: org_rocksdb_RocksDB
|
||||
* Method: multiGet
|
||||
* Signature: (JJLjava/util/List;ILjava/util/List;)Ljava/util/List;
|
||||
*/
|
||||
jobject
|
||||
Java_org_rocksdb_RocksDB_multiGet__JJLjava_util_List_2ILjava_util_List_2(
|
||||
JNIEnv* env, jobject jdb, jlong jdb_handle,
|
||||
jlong jropt_handle, jobject jkey_list, jint jkeys_count,
|
||||
jobject jcfhandle_list) {
|
||||
return multi_get_helper(env, jdb, reinterpret_cast<rocksdb::DB*>(jdb_handle),
|
||||
*reinterpret_cast<rocksdb::ReadOptions*>(jropt_handle), jkey_list,
|
||||
jkeys_count, jcfhandle_list);
|
||||
}
|
||||
|
||||
/*
|
||||
@ -331,10 +757,32 @@ jint Java_org_rocksdb_RocksDB_get__J_3BI_3BI(
|
||||
jbyteArray jvalue, jint jvalue_len) {
|
||||
return rocksdb_get_helper(env,
|
||||
reinterpret_cast<rocksdb::DB*>(jdb_handle),
|
||||
rocksdb::ReadOptions(),
|
||||
rocksdb::ReadOptions(), nullptr,
|
||||
jkey, jkey_len, jvalue, jvalue_len);
|
||||
}
|
||||
|
||||
/*
|
||||
* Class: org_rocksdb_RocksDB
|
||||
* Method: get
|
||||
* Signature: (J[BI[BIJ)I
|
||||
*/
|
||||
jint Java_org_rocksdb_RocksDB_get__J_3BI_3BIJ(
|
||||
JNIEnv* env, jobject jdb, jlong jdb_handle,
|
||||
jbyteArray jkey, jint jkey_len,
|
||||
jbyteArray jvalue, jint jvalue_len, jlong jcf_handle) {
|
||||
auto db_handle = reinterpret_cast<rocksdb::DB*>(jdb_handle);
|
||||
auto cf_handle = reinterpret_cast<rocksdb::ColumnFamilyHandle*>(jcf_handle);
|
||||
if (cf_handle != nullptr) {
|
||||
return rocksdb_get_helper(env, db_handle, rocksdb::ReadOptions(), cf_handle,
|
||||
jkey, jkey_len, jvalue, jvalue_len);
|
||||
} else {
|
||||
rocksdb::RocksDBExceptionJni::ThrowNew(env,
|
||||
rocksdb::Status::InvalidArgument("Invalid ColumnFamilyHandle."));
|
||||
// will never be evaluated
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
* Class: org_rocksdb_RocksDB
|
||||
* Method: get
|
||||
@ -347,19 +795,46 @@ jint Java_org_rocksdb_RocksDB_get__JJ_3BI_3BI(
|
||||
return rocksdb_get_helper(env,
|
||||
reinterpret_cast<rocksdb::DB*>(jdb_handle),
|
||||
*reinterpret_cast<rocksdb::ReadOptions*>(jropt_handle),
|
||||
jkey, jkey_len, jvalue, jvalue_len);
|
||||
nullptr, jkey, jkey_len, jvalue, jvalue_len);
|
||||
}
|
||||
|
||||
/*
|
||||
* Class: org_rocksdb_RocksDB
|
||||
* Method: get
|
||||
* Signature: (JJ[BI[BIJ)I
|
||||
*/
|
||||
jint Java_org_rocksdb_RocksDB_get__JJ_3BI_3BIJ(
|
||||
JNIEnv* env, jobject jdb, jlong jdb_handle, jlong jropt_handle,
|
||||
jbyteArray jkey, jint jkey_len,
|
||||
jbyteArray jvalue, jint jvalue_len, jlong jcf_handle) {
|
||||
auto db_handle = reinterpret_cast<rocksdb::DB*>(jdb_handle);
|
||||
auto& ro_opt = *reinterpret_cast<rocksdb::ReadOptions*>(jropt_handle);
|
||||
auto cf_handle = reinterpret_cast<rocksdb::ColumnFamilyHandle*>(jcf_handle);
|
||||
if (cf_handle != nullptr) {
|
||||
return rocksdb_get_helper(env, db_handle, ro_opt, cf_handle, jkey,
|
||||
jkey_len, jvalue, jvalue_len);
|
||||
} else {
|
||||
rocksdb::RocksDBExceptionJni::ThrowNew(env,
|
||||
rocksdb::Status::InvalidArgument("Invalid ColumnFamilyHandle."));
|
||||
// will never be evaluated
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
//////////////////////////////////////////////////////////////////////////////
|
||||
// rocksdb::DB::Delete()
|
||||
void rocksdb_remove_helper(
|
||||
JNIEnv* env, rocksdb::DB* db, const rocksdb::WriteOptions& write_options,
|
||||
jbyteArray jkey, jint jkey_len) {
|
||||
rocksdb::ColumnFamilyHandle* cf_handle, jbyteArray jkey, jint jkey_len) {
|
||||
jbyte* key = env->GetByteArrayElements(jkey, 0);
|
||||
rocksdb::Slice key_slice(reinterpret_cast<char*>(key), jkey_len);
|
||||
|
||||
rocksdb::Status s = db->Delete(write_options, key_slice);
|
||||
|
||||
rocksdb::Status s;
|
||||
if (cf_handle != nullptr) {
|
||||
s = db->Delete(write_options, cf_handle, key_slice);
|
||||
} else {
|
||||
// backwards compatibility
|
||||
s = db->Delete(write_options, key_slice);
|
||||
}
|
||||
// trigger java unref on key and value.
|
||||
// by passing JNI_ABORT, it will simply release the reference without
|
||||
// copying the result back to the java byte array.
|
||||
@ -382,24 +857,63 @@ void Java_org_rocksdb_RocksDB_remove__J_3BI(
|
||||
auto db = reinterpret_cast<rocksdb::DB*>(jdb_handle);
|
||||
static const rocksdb::WriteOptions default_write_options =
|
||||
rocksdb::WriteOptions();
|
||||
|
||||
rocksdb_remove_helper(env, db, default_write_options, jkey, jkey_len);
|
||||
rocksdb_remove_helper(env, db, default_write_options, nullptr,
|
||||
jkey, jkey_len);
|
||||
}
|
||||
|
||||
/*
|
||||
* Class: org_rocksdb_RocksDB
|
||||
* Method: remove
|
||||
* Signature: (JJ[BI)V
|
||||
* Signature: (J[BIJ)V
|
||||
*/
|
||||
void Java_org_rocksdb_RocksDB_remove__J_3BIJ(
|
||||
JNIEnv* env, jobject jdb, jlong jdb_handle,
|
||||
jbyteArray jkey, jint jkey_len, jlong jcf_handle) {
|
||||
auto db = reinterpret_cast<rocksdb::DB*>(jdb_handle);
|
||||
static const rocksdb::WriteOptions default_write_options =
|
||||
rocksdb::WriteOptions();
|
||||
auto cf_handle = reinterpret_cast<rocksdb::ColumnFamilyHandle*>(jcf_handle);
|
||||
if (cf_handle != nullptr) {
|
||||
rocksdb_remove_helper(env, db, default_write_options, cf_handle,
|
||||
jkey, jkey_len);
|
||||
} else {
|
||||
rocksdb::RocksDBExceptionJni::ThrowNew(env,
|
||||
rocksdb::Status::InvalidArgument("Invalid ColumnFamilyHandle."));
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
* Class: org_rocksdb_RocksDB
|
||||
* Method: remove
|
||||
* Signature: (JJ[BIJ)V
|
||||
*/
|
||||
void Java_org_rocksdb_RocksDB_remove__JJ_3BI(
|
||||
JNIEnv* env, jobject jdb, jlong jdb_handle,
|
||||
jlong jwrite_options, jbyteArray jkey, jint jkey_len) {
|
||||
auto db = reinterpret_cast<rocksdb::DB*>(jdb_handle);
|
||||
auto write_options = reinterpret_cast<rocksdb::WriteOptions*>(jwrite_options);
|
||||
|
||||
rocksdb_remove_helper(env, db, *write_options, jkey, jkey_len);
|
||||
rocksdb_remove_helper(env, db, *write_options, nullptr, jkey, jkey_len);
|
||||
}
|
||||
|
||||
/*
|
||||
* Class: org_rocksdb_RocksDB
|
||||
* Method: remove
|
||||
* Signature: (JJ[BIJ)V
|
||||
*/
|
||||
void Java_org_rocksdb_RocksDB_remove__JJ_3BIJ(
|
||||
JNIEnv* env, jobject jdb, jlong jdb_handle,
|
||||
jlong jwrite_options, jbyteArray jkey, jint jkey_len,
|
||||
jlong jcf_handle) {
|
||||
auto db = reinterpret_cast<rocksdb::DB*>(jdb_handle);
|
||||
auto write_options = reinterpret_cast<rocksdb::WriteOptions*>(jwrite_options);
|
||||
auto cf_handle = reinterpret_cast<rocksdb::ColumnFamilyHandle*>(jcf_handle);
|
||||
if (cf_handle != nullptr) {
|
||||
rocksdb_remove_helper(env, db, *write_options, cf_handle, jkey, jkey_len);
|
||||
} else {
|
||||
rocksdb::RocksDBExceptionJni::ThrowNew(env,
|
||||
rocksdb::Status::InvalidArgument("Invalid ColumnFamilyHandle."));
|
||||
}
|
||||
}
|
||||
//////////////////////////////////////////////////////////////////////////////
|
||||
// rocksdb::DB::~DB()
|
||||
|
||||
@ -418,19 +932,111 @@ void Java_org_rocksdb_RocksDB_disposeInternal(
|
||||
* Method: iterator0
|
||||
* Signature: (J)J
|
||||
*/
|
||||
jlong Java_org_rocksdb_RocksDB_iterator0(
|
||||
jlong Java_org_rocksdb_RocksDB_iterator0__J(
|
||||
JNIEnv* env, jobject jdb, jlong db_handle) {
|
||||
auto db = reinterpret_cast<rocksdb::DB*>(db_handle);
|
||||
rocksdb::Iterator* iterator = db->NewIterator(rocksdb::ReadOptions());
|
||||
return reinterpret_cast<jlong>(iterator);
|
||||
}
|
||||
|
||||
/*
|
||||
* Class: org_rocksdb_RocksDB
|
||||
* Method: iterator0
|
||||
* Signature: (JJ)J
|
||||
*/
|
||||
jlong Java_org_rocksdb_RocksDB_iterator0__JJ(
|
||||
JNIEnv* env, jobject jdb, jlong db_handle, jlong jcf_handle) {
|
||||
auto db = reinterpret_cast<rocksdb::DB*>(db_handle);
|
||||
auto cf_handle = reinterpret_cast<rocksdb::ColumnFamilyHandle*>(jcf_handle);
|
||||
rocksdb::Iterator* iterator = db->NewIterator(rocksdb::ReadOptions(),
|
||||
cf_handle);
|
||||
return reinterpret_cast<jlong>(iterator);
|
||||
}
|
||||
|
||||
/*
|
||||
* Class: org_rocksdb_RocksDB
|
||||
* Method: iterators
|
||||
* Signature: (JLjava/util/List;)[J
|
||||
*/
|
||||
jlongArray Java_org_rocksdb_RocksDB_iterators(
|
||||
JNIEnv* env, jobject jdb, jlong db_handle, jobject jcfhandle_list) {
|
||||
auto db = reinterpret_cast<rocksdb::DB*>(db_handle);
|
||||
std::vector<rocksdb::ColumnFamilyHandle*> cf_handles;
|
||||
std::vector<rocksdb::Iterator*> iterators;
|
||||
|
||||
if (jcfhandle_list != nullptr) {
|
||||
// get cf iterator
|
||||
jobject cfIteratorObj = env->CallObjectMethod(
|
||||
jcfhandle_list, rocksdb::ListJni::getIteratorMethod(env));
|
||||
|
||||
// iterate over keys and convert java byte array to slice
|
||||
while (env->CallBooleanMethod(
|
||||
cfIteratorObj, rocksdb::ListJni::getHasNextMethod(env)) == JNI_TRUE) {
|
||||
jobject jobj = (jbyteArray) env->CallObjectMethod(
|
||||
cfIteratorObj, rocksdb::ListJni::getNextMethod(env));
|
||||
rocksdb::ColumnFamilyHandle* cfHandle =
|
||||
rocksdb::ColumnFamilyHandleJni::getHandle(env, jobj);
|
||||
cf_handles.push_back(cfHandle);
|
||||
}
|
||||
}
|
||||
|
||||
rocksdb::Status s = db->NewIterators(rocksdb::ReadOptions(),
|
||||
cf_handles, &iterators);
|
||||
if (s.ok()) {
|
||||
jlongArray jLongArray = env->NewLongArray(iterators.size());
|
||||
for (std::vector<rocksdb::Iterator*>::size_type i = 0;
|
||||
i < iterators.size(); i++) {
|
||||
env->SetLongArrayRegion(jLongArray, i, 1,
|
||||
reinterpret_cast<const jlong*>(&iterators[i]));
|
||||
}
|
||||
return jLongArray;
|
||||
}
|
||||
rocksdb::RocksDBExceptionJni::ThrowNew(env, s);
|
||||
return env->NewLongArray(0);
|
||||
}
|
||||
|
||||
/*
|
||||
* Class: org_rocksdb_RocksDB
|
||||
* Method: createColumnFamily
|
||||
* Signature: (JLjava/lang/String;)J;
|
||||
*/
|
||||
jlong Java_org_rocksdb_RocksDB_createColumnFamily(
|
||||
JNIEnv* env, jobject jdb, jlong jdb_handle, jstring jcfname) {
|
||||
rocksdb::ColumnFamilyHandle* handle;
|
||||
const char* cfname = env->GetStringUTFChars(jcfname, 0);
|
||||
auto db_handle = reinterpret_cast<rocksdb::DB*>(jdb_handle);
|
||||
rocksdb::Status s = db_handle->CreateColumnFamily(
|
||||
rocksdb::ColumnFamilyOptions(), cfname, &handle);
|
||||
env->ReleaseStringUTFChars(jcfname, cfname);
|
||||
|
||||
if (s.ok()) {
|
||||
return reinterpret_cast<jlong>(handle);
|
||||
}
|
||||
rocksdb::RocksDBExceptionJni::ThrowNew(env, s);
|
||||
return 0;
|
||||
}
|
||||
|
||||
/*
|
||||
* Class: org_rocksdb_RocksDB
|
||||
* Method: dropColumnFamily
|
||||
* Signature: (JJ)V;
|
||||
*/
|
||||
void Java_org_rocksdb_RocksDB_dropColumnFamily(
|
||||
JNIEnv* env, jobject jdb, jlong jdb_handle, jlong jcf_handle) {
|
||||
auto cf_handle = reinterpret_cast<rocksdb::ColumnFamilyHandle*>(jcf_handle);
|
||||
auto db_handle = reinterpret_cast<rocksdb::DB*>(jdb_handle);
|
||||
rocksdb::Status s = db_handle->DropColumnFamily(cf_handle);
|
||||
if (!s.ok()) {
|
||||
rocksdb::RocksDBExceptionJni::ThrowNew(env, s);
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
* Class: org_rocksdb_RocksDB
|
||||
* Method: getProperty0
|
||||
* Signature: (JLjava/lang/String;I)Ljava/lang/String;
|
||||
*/
|
||||
jstring Java_org_rocksdb_RocksDB_getProperty0(
|
||||
jstring Java_org_rocksdb_RocksDB_getProperty0__JLjava_lang_String_2I(
|
||||
JNIEnv* env, jobject jdb, jlong db_handle, jstring jproperty,
|
||||
jint jproperty_len) {
|
||||
auto db = reinterpret_cast<rocksdb::DB*>(db_handle);
|
||||
@ -448,3 +1054,28 @@ jstring Java_org_rocksdb_RocksDB_getProperty0(
|
||||
|
||||
return env->NewStringUTF(property_value.data());
|
||||
}
|
||||
|
||||
/*
|
||||
* Class: org_rocksdb_RocksDB
|
||||
* Method: getProperty0
|
||||
* Signature: (JJLjava/lang/String;I)Ljava/lang/String;
|
||||
*/
|
||||
jstring Java_org_rocksdb_RocksDB_getProperty0__JJLjava_lang_String_2I(
|
||||
JNIEnv* env, jobject jdb, jlong db_handle, jlong jcf_handle,
|
||||
jstring jproperty, jint jproperty_len) {
|
||||
auto db = reinterpret_cast<rocksdb::DB*>(db_handle);
|
||||
auto cf_handle = reinterpret_cast<rocksdb::ColumnFamilyHandle*>(jcf_handle);
|
||||
|
||||
const char* property = env->GetStringUTFChars(jproperty, 0);
|
||||
rocksdb::Slice property_slice(property, jproperty_len);
|
||||
|
||||
std::string property_value;
|
||||
bool retCode = db->GetProperty(cf_handle, property_slice, &property_value);
|
||||
env->ReleaseStringUTFChars(jproperty, property);
|
||||
|
||||
if (!retCode) {
|
||||
rocksdb::RocksDBExceptionJni::ThrowNew(env, rocksdb::Status::NotFound());
|
||||
}
|
||||
|
||||
return env->NewStringUTF(property_value.data());
|
||||
}
|
||||
|
@ -60,14 +60,13 @@ void Java_org_rocksdb_WriteBatch_clear(JNIEnv* env, jobject jobj) {
|
||||
}
|
||||
|
||||
/*
|
||||
* Class: org_rocksdb_WriteBatch
|
||||
* Method: put
|
||||
* Signature: ([BI[BI)V
|
||||
* Helper for WriteBatch put operations
|
||||
*/
|
||||
void Java_org_rocksdb_WriteBatch_put(
|
||||
void write_batch_put_helper(
|
||||
JNIEnv* env, jobject jobj,
|
||||
jbyteArray jkey, jint jkey_len,
|
||||
jbyteArray jvalue, jint jvalue_len) {
|
||||
jbyteArray jvalue, jint jvalue_len,
|
||||
rocksdb::ColumnFamilyHandle* cf_handle) {
|
||||
rocksdb::WriteBatch* wb = rocksdb::WriteBatchJni::getHandle(env, jobj);
|
||||
assert(wb != nullptr);
|
||||
|
||||
@ -75,7 +74,64 @@ void Java_org_rocksdb_WriteBatch_put(
|
||||
jbyte* value = env->GetByteArrayElements(jvalue, nullptr);
|
||||
rocksdb::Slice key_slice(reinterpret_cast<char*>(key), jkey_len);
|
||||
rocksdb::Slice value_slice(reinterpret_cast<char*>(value), jvalue_len);
|
||||
wb->Put(key_slice, value_slice);
|
||||
if (cf_handle != nullptr) {
|
||||
wb->Put(cf_handle, key_slice, value_slice);
|
||||
} else {
|
||||
// backwards compatibility
|
||||
wb->Put(key_slice, value_slice);
|
||||
}
|
||||
env->ReleaseByteArrayElements(jkey, key, JNI_ABORT);
|
||||
env->ReleaseByteArrayElements(jvalue, value, JNI_ABORT);
|
||||
}
|
||||
|
||||
/*
|
||||
* Class: org_rocksdb_WriteBatch
|
||||
* Method: put
|
||||
* Signature: ([BI[BI)V
|
||||
*/
|
||||
void Java_org_rocksdb_WriteBatch_put___3BI_3BI(
|
||||
JNIEnv* env, jobject jobj,
|
||||
jbyteArray jkey, jint jkey_len,
|
||||
jbyteArray jvalue, jint jvalue_len) {
|
||||
write_batch_put_helper(env, jobj, jkey, jkey_len, jvalue,
|
||||
jvalue_len, nullptr);
|
||||
}
|
||||
|
||||
/*
|
||||
* Class: org_rocksdb_WriteBatch
|
||||
* Method: put
|
||||
* Signature: ([BI[BIJ)V
|
||||
*/
|
||||
void Java_org_rocksdb_WriteBatch_put___3BI_3BIJ(
|
||||
JNIEnv* env, jobject jobj,
|
||||
jbyteArray jkey, jint jkey_len,
|
||||
jbyteArray jvalue, jint jvalue_len, jlong jcf_handle) {
|
||||
auto cf_handle = reinterpret_cast<rocksdb::ColumnFamilyHandle*>(jcf_handle);
|
||||
write_batch_put_helper(env, jobj, jkey, jkey_len, jvalue,
|
||||
jvalue_len, cf_handle);
|
||||
}
|
||||
|
||||
/*
|
||||
* Helper for write batch merge operations
|
||||
*/
|
||||
void write_batch_merge_helper(
|
||||
JNIEnv* env, jobject jobj,
|
||||
jbyteArray jkey, jint jkey_len,
|
||||
jbyteArray jvalue, jint jvalue_len,
|
||||
rocksdb::ColumnFamilyHandle* cf_handle) {
|
||||
rocksdb::WriteBatch* wb = rocksdb::WriteBatchJni::getHandle(env, jobj);
|
||||
assert(wb != nullptr);
|
||||
|
||||
jbyte* key = env->GetByteArrayElements(jkey, nullptr);
|
||||
jbyte* value = env->GetByteArrayElements(jvalue, nullptr);
|
||||
rocksdb::Slice key_slice(reinterpret_cast<char*>(key), jkey_len);
|
||||
rocksdb::Slice value_slice(reinterpret_cast<char*>(value), jvalue_len);
|
||||
if (cf_handle != nullptr) {
|
||||
wb->Merge(cf_handle, key_slice, value_slice);
|
||||
} else {
|
||||
// backwards compatibility
|
||||
wb->Merge(key_slice, value_slice);
|
||||
}
|
||||
env->ReleaseByteArrayElements(jkey, key, JNI_ABORT);
|
||||
env->ReleaseByteArrayElements(jvalue, value, JNI_ABORT);
|
||||
}
|
||||
@ -85,20 +141,46 @@ void Java_org_rocksdb_WriteBatch_put(
|
||||
* Method: merge
|
||||
* Signature: ([BI[BI)V
|
||||
*/
|
||||
JNIEXPORT void JNICALL Java_org_rocksdb_WriteBatch_merge(
|
||||
void Java_org_rocksdb_WriteBatch_merge___3BI_3BI(
|
||||
JNIEnv* env, jobject jobj,
|
||||
jbyteArray jkey, jint jkey_len,
|
||||
jbyteArray jvalue, jint jvalue_len) {
|
||||
write_batch_merge_helper(env, jobj, jkey, jkey_len,
|
||||
jvalue, jvalue_len, nullptr);
|
||||
}
|
||||
|
||||
/*
|
||||
* Class: org_rocksdb_WriteBatch
|
||||
* Method: merge
|
||||
* Signature: ([BI[BIJ)V
|
||||
*/
|
||||
void Java_org_rocksdb_WriteBatch_merge___3BI_3BIJ(
|
||||
JNIEnv* env, jobject jobj,
|
||||
jbyteArray jkey, jint jkey_len,
|
||||
jbyteArray jvalue, jint jvalue_len, jlong jcf_handle) {
|
||||
auto cf_handle = reinterpret_cast<rocksdb::ColumnFamilyHandle*>(jcf_handle);
|
||||
write_batch_merge_helper(env, jobj, jkey, jkey_len,
|
||||
jvalue, jvalue_len, cf_handle);
|
||||
}
|
||||
|
||||
/*
|
||||
* Helper for write batch remove operations
|
||||
*/
|
||||
void write_batch_remove_helper(
|
||||
JNIEnv* env, jobject jobj,
|
||||
jbyteArray jkey, jint jkey_len,
|
||||
rocksdb::ColumnFamilyHandle* cf_handle) {
|
||||
rocksdb::WriteBatch* wb = rocksdb::WriteBatchJni::getHandle(env, jobj);
|
||||
assert(wb != nullptr);
|
||||
|
||||
jbyte* key = env->GetByteArrayElements(jkey, nullptr);
|
||||
jbyte* value = env->GetByteArrayElements(jvalue, nullptr);
|
||||
rocksdb::Slice key_slice(reinterpret_cast<char*>(key), jkey_len);
|
||||
rocksdb::Slice value_slice(reinterpret_cast<char*>(value), jvalue_len);
|
||||
wb->Merge(key_slice, value_slice);
|
||||
if (cf_handle != nullptr) {
|
||||
wb->Delete(cf_handle, key_slice);
|
||||
} else {
|
||||
wb->Delete(key_slice);
|
||||
}
|
||||
env->ReleaseByteArrayElements(jkey, key, JNI_ABORT);
|
||||
env->ReleaseByteArrayElements(jvalue, value, JNI_ABORT);
|
||||
}
|
||||
|
||||
/*
|
||||
@ -106,16 +188,22 @@ JNIEXPORT void JNICALL Java_org_rocksdb_WriteBatch_merge(
|
||||
* Method: remove
|
||||
* Signature: ([BI)V
|
||||
*/
|
||||
JNIEXPORT void JNICALL Java_org_rocksdb_WriteBatch_remove(
|
||||
void Java_org_rocksdb_WriteBatch_remove___3BI(
|
||||
JNIEnv* env, jobject jobj,
|
||||
jbyteArray jkey, jint jkey_len) {
|
||||
rocksdb::WriteBatch* wb = rocksdb::WriteBatchJni::getHandle(env, jobj);
|
||||
assert(wb != nullptr);
|
||||
write_batch_remove_helper(env, jobj, jkey, jkey_len, nullptr);
|
||||
}
|
||||
|
||||
jbyte* key = env->GetByteArrayElements(jkey, nullptr);
|
||||
rocksdb::Slice key_slice(reinterpret_cast<char*>(key), jkey_len);
|
||||
wb->Delete(key_slice);
|
||||
env->ReleaseByteArrayElements(jkey, key, JNI_ABORT);
|
||||
/*
|
||||
* Class: org_rocksdb_WriteBatch
|
||||
* Method: remove
|
||||
* Signature: ([BIJ)V
|
||||
*/
|
||||
void Java_org_rocksdb_WriteBatch_remove___3BIJ(
|
||||
JNIEnv* env, jobject jobj,
|
||||
jbyteArray jkey, jint jkey_len, jlong jcf_handle) {
|
||||
auto cf_handle = reinterpret_cast<rocksdb::ColumnFamilyHandle*>(jcf_handle);
|
||||
write_batch_remove_helper(env, jobj, jkey, jkey_len, cf_handle);
|
||||
}
|
||||
|
||||
/*
|
||||
|
Loading…
Reference in New Issue
Block a user