From 39464a990ff8bdc9a8560469641dec04c13eefe7 Mon Sep 17 00:00:00 2001 From: fyrz Date: Tue, 28 Oct 2014 22:38:08 +0100 Subject: [PATCH] [RocksJava] Options Refactoring 3.6 Summary: Options extends now two interfaces DBOptionsInterface and ColumnFamilyOptionsInterface. There are also further improvements to the Options bindings: Optimize methods were ported to Java. (OptimizeForPointLookup, OptimizeLevelCompaction, OptimizeUniversalCompaction). To align BuiltinComparator with every other Enum it was moved to a separate file. Test Plan: make rocksdbjava make jtest --- java/Makefile | 2 +- java/org/rocksdb/BuiltinComparator.java | 20 + .../rocksdb/ColumnFamilyOptionsInterface.java | 979 ++++++++ java/org/rocksdb/DBOptionsInterface.java | 763 ++++++ java/org/rocksdb/Options.java | 1993 ++++------------ java/org/rocksdb/RateLimiterConfig.java | 4 +- java/org/rocksdb/test/OptionsTest.java | 13 +- java/rocksjni/options.cc | 2061 ++++++++++++++++- 8 files changed, 4133 insertions(+), 1702 deletions(-) create mode 100644 java/org/rocksdb/BuiltinComparator.java create mode 100644 java/org/rocksdb/ColumnFamilyOptionsInterface.java create mode 100644 java/org/rocksdb/DBOptionsInterface.java diff --git a/java/Makefile b/java/Makefile index 765ed44fc..04eac63dd 100644 --- a/java/Makefile +++ b/java/Makefile @@ -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.ComparatorOptions org.rocksdb.AbstractComparator org.rocksdb.Comparator org.rocksdb.DirectComparator org.rocksdb.AbstractSlice org.rocksdb.Slice org.rocksdb.DirectSlice org.rocksdb.RestoreOptions org.rocksdb.RestoreBackupableDB org.rocksdb.RocksEnv org.rocksdb.GenericRateLimiterConfig org.rocksdb.ColumnFamilyHandle org.rocksdb.MergeOperator org.rocksdb.StringAppendOperator +NATIVE_JAVA_CLASSES = org.rocksdb.RocksDB org.rocksdb.Options org.rocksdb.DBOptions org.rocksdb.ColumnFamilyOptions 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.ComparatorOptions org.rocksdb.AbstractComparator org.rocksdb.Comparator org.rocksdb.DirectComparator org.rocksdb.AbstractSlice org.rocksdb.Slice org.rocksdb.DirectSlice org.rocksdb.RestoreOptions org.rocksdb.RestoreBackupableDB org.rocksdb.RocksEnv org.rocksdb.GenericRateLimiterConfig org.rocksdb.ColumnFamilyHandle org.rocksdb.MergeOperator org.rocksdb.StringAppendOperator org.rocksdb.ComparatorOptions org.rocksdb.AbstractComparator org.rocksdb.Comparator org.rocksdb.DirectComparator org.rocksdb.AbstractSlice org.rocksdb.Slice org.rocksdb.DirectSlice 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) diff --git a/java/org/rocksdb/BuiltinComparator.java b/java/org/rocksdb/BuiltinComparator.java new file mode 100644 index 000000000..ee92e8dd9 --- /dev/null +++ b/java/org/rocksdb/BuiltinComparator.java @@ -0,0 +1,20 @@ +// 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; + +/** + * Builtin RocksDB comparators + * + *
    + *
  1. BYTEWISE_COMPARATOR - Sorts all keys in ascending bytewise + * order.
  2. + *
  3. REVERSE_BYTEWISE_COMPARATOR - Sorts all keys in descending bytewise + * order
  4. + *
+ */ +public enum BuiltinComparator { + BYTEWISE_COMPARATOR, REVERSE_BYTEWISE_COMPARATOR +} diff --git a/java/org/rocksdb/ColumnFamilyOptionsInterface.java b/java/org/rocksdb/ColumnFamilyOptionsInterface.java new file mode 100644 index 000000000..827fe8c64 --- /dev/null +++ b/java/org/rocksdb/ColumnFamilyOptionsInterface.java @@ -0,0 +1,979 @@ +// 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; + +public interface ColumnFamilyOptionsInterface { + + /** + * Use this if you don't need to keep the data sorted, i.e. you'll never use + * an iterator, only Put() and Get() API calls + * + * @param blockCacheSizeMb Block cache size in MB + * @return the instance of the current Object. + */ + Object optimizeForPointLookup(long blockCacheSizeMb); + + /** + *

Default values for some parameters in ColumnFamilyOptions are not + * optimized for heavy workloads and big datasets, which means you might + * observe write stalls under some conditions. As a starting point for tuning + * RocksDB options, use the following for level style compaction.

+ * + *

Make sure to also call IncreaseParallelism(), which will provide the + * biggest performance gains.

+ *

Note: we might use more memory than memtable_memory_budget during high + * write rate period

+ * + * @return the instance of the current Object. + */ + Object optimizeLevelStyleCompaction(); + + /** + *

Default values for some parameters in ColumnFamilyOptions are not + * optimized for heavy workloads and big datasets, which means you might + * observe write stalls under some conditions. As a starting point for tuning + * RocksDB options, use the following for level style compaction.

+ * + *

Make sure to also call IncreaseParallelism(), which will provide the + * biggest performance gains.

+ *

Note: we might use more memory than memtable_memory_budget during high + * write rate period

+ * + * @param memtableMemoryBudget memory budget in bytes + * @return the instance of the current Object. + */ + Object optimizeLevelStyleCompaction(long memtableMemoryBudget); + + /** + *

Default values for some parameters in ColumnFamilyOptions are not + * optimized for heavy workloads and big datasets, which means you might + * observe write stalls under some conditions. As a starting point for tuning + * RocksDB options, use the following for universal style compaction.

+ * + *

Universal style compaction is focused on reducing Write Amplification + * Factor for big data sets, but increases Space Amplification.

+ * + *

Make sure to also call IncreaseParallelism(), which will provide the + * biggest performance gains.

+ * + *

Note: we might use more memory than memtable_memory_budget during high + * write rate period

+ * + * @return the instance of the current Object. + */ + Object optimizeUniversalStyleCompaction(); + + /** + *

Default values for some parameters in ColumnFamilyOptions are not + * optimized for heavy workloads and big datasets, which means you might + * observe write stalls under some conditions. As a starting point for tuning + * RocksDB options, use the following for universal style compaction.

+ * + *

Universal style compaction is focused on reducing Write Amplification + * Factor for big data sets, but increases Space Amplification.

+ * + *

Make sure to also call IncreaseParallelism(), which will provide the + * biggest performance gains.

+ * + *

Note: we might use more memory than memtable_memory_budget during high + * write rate period

+ * + * @param memtableMemoryBudget memory budget in bytes + * @return the instance of the current Object. + */ + Object optimizeUniversalStyleCompaction(long memtableMemoryBudget); + + /** + * Set {@link BuiltinComparator} to be used with RocksDB. + * + * Note: Comparator can be set once upon database creation. + * + * Default: BytewiseComparator. + * @param builtinComparator a {@link BuiltinComparator} type. + * @return the instance of the current Object. + */ + Object setComparator(BuiltinComparator builtinComparator); + + /** + * Use the specified comparator for key ordering. + * + * Comparator should not be disposed before options instances using this comparator is + * disposed. If dispose() function is not called, then comparator object will be + * GC'd automatically. + * + * Comparator instance can be re-used in multiple options instances. + * + * @param comparator java instance. + * @return the instance of the current Object. + */ + Object setComparator(AbstractComparator comparator); + + /** + *

Set the merge operator to be used for merging two merge operands + * of the same key. The merge function is invoked during + * compaction and at lookup time, if multiple key/value pairs belonging + * to the same key are found in the database.

+ * + * @param name the name of the merge function, as defined by + * the MergeOperators factory (see utilities/MergeOperators.h) + * The merge function is specified by name and must be one of the + * standard merge operators provided by RocksDB. The available + * operators are "put", "uint64add", "stringappend" and "stringappendtest". + * @return the instance of the current Object. + */ + public Object setMergeOperatorName(String name); + + /** + *

Set the merge operator to be used for merging two different key/value + * pairs that share the same key. The merge function is invoked during + * compaction and at lookup time, if multiple key/value pairs belonging + * to the same key are found in the database.

+ * + * @param mergeOperator {@link MergeOperator} instance. + * @return the instance of the current Object. + */ + public Object setMergeOperator(MergeOperator mergeOperator); + + /** + * Amount of data to build up in memory (backed by an unsorted log + * on disk) before converting to a sorted on-disk file. + * + * Larger values increase performance, especially during bulk loads. + * Up to {@code max_write_buffer_number} write buffers may be held in memory + * at the same time, so you may wish to adjust this parameter + * to control memory usage. + * + * Also, a larger write buffer will result in a longer recovery time + * the next time the database is opened. + * + * Default: 4MB + * @param writeBufferSize the size of write buffer. + * @return the instance of the current Object. + * @throws org.rocksdb.RocksDBException + */ + Object setWriteBufferSize(long writeBufferSize) + throws RocksDBException; + + /** + * Return size of write buffer size. + * + * @return size of write buffer. + * @see #setWriteBufferSize(long) + */ + long writeBufferSize(); + + /** + * The maximum number of write buffers that are built up in memory. + * The default is 2, so that when 1 write buffer is being flushed to + * storage, new writes can continue to the other write buffer. + * Default: 2 + * + * @param maxWriteBufferNumber maximum number of write buffers. + * @return the instance of the current Object. + */ + Object setMaxWriteBufferNumber( + int maxWriteBufferNumber); + + /** + * Returns maximum number of write buffers. + * + * @return maximum number of write buffers. + * @see #setMaxWriteBufferNumber(int) + */ + int maxWriteBufferNumber(); + + /** + * The minimum number of write buffers that will be merged together + * before writing to storage. If set to 1, then + * all write buffers are flushed to L0 as individual files and this increases + * read amplification because a get request has to check in all of these + * files. Also, an in-memory merge may result in writing lesser + * data to storage if there are duplicate records in each of these + * individual write buffers. Default: 1 + * + * @param minWriteBufferNumberToMerge the minimum number of write buffers + * that will be merged together. + * @return the reference to the current option. + */ + Object setMinWriteBufferNumberToMerge( + int minWriteBufferNumberToMerge); + + /** + * The minimum number of write buffers that will be merged together + * before writing to storage. If set to 1, then + * all write buffers are flushed to L0 as individual files and this increases + * read amplification because a get request has to check in all of these + * files. Also, an in-memory merge may result in writing lesser + * data to storage if there are duplicate records in each of these + * individual write buffers. Default: 1 + * + * @return the minimum number of write buffers that will be merged together. + */ + int minWriteBufferNumberToMerge(); + + /** + * This prefix-extractor uses the first n bytes of a key as its prefix. + * + * In some hash-based memtable representation such as HashLinkedList + * and HashSkipList, prefixes are used to partition the keys into + * several buckets. Prefix extractor is used to specify how to + * extract the prefix given a key. + * + * @param n use the first n bytes of a key as its prefix. + */ + Object useFixedLengthPrefixExtractor(int n); + + /** + * Compress blocks using the specified compression algorithm. This + * parameter can be changed dynamically. + * + * Default: SNAPPY_COMPRESSION, which gives lightweight but fast compression. + * + * @param compressionType Compression Type. + * @return the reference to the current option. + */ + Object setCompressionType(CompressionType compressionType); + + /** + * Compress blocks using the specified compression algorithm. This + * parameter can be changed dynamically. + * + * Default: SNAPPY_COMPRESSION, which gives lightweight but fast compression. + * + * @return Compression type. + */ + CompressionType compressionType(); + + /** + * Set the number of levels for this database + * If level-styled compaction is used, then this number determines + * the total number of levels. + * + * @param numLevels the number of levels. + * @return the reference to the current option. + */ + Object setNumLevels(int numLevels); + + /** + * If level-styled compaction is used, then this number determines + * the total number of levels. + * + * @return the number of levels. + */ + int numLevels(); + + /** + * Number of files to trigger level-0 compaction. A value < 0 means that + * level-0 compaction will not be triggered by number of files at all. + * Default: 4 + * + * @param numFiles the number of files in level-0 to trigger compaction. + * @return the reference to the current option. + */ + Object setLevelZeroFileNumCompactionTrigger( + int numFiles); + + /** + * The number of files in level 0 to trigger compaction from level-0 to + * level-1. A value < 0 means that level-0 compaction will not be + * triggered by number of files at all. + * Default: 4 + * + * @return the number of files in level 0 to trigger compaction. + */ + int levelZeroFileNumCompactionTrigger(); + + /** + * Soft limit on number of level-0 files. We start slowing down writes at this + * point. A value < 0 means that no writing slow down will be triggered by + * number of files in level-0. + * + * @param numFiles soft limit on number of level-0 files. + * @return the reference to the current option. + */ + Object setLevelZeroSlowdownWritesTrigger( + int numFiles); + + /** + * Soft limit on the number of level-0 files. We start slowing down writes + * at this point. A value < 0 means that no writing slow down will be + * triggered by number of files in level-0. + * + * @return the soft limit on the number of level-0 files. + */ + int levelZeroSlowdownWritesTrigger(); + + /** + * Maximum number of level-0 files. We stop writes at this point. + * + * @param numFiles the hard limit of the number of level-0 files. + * @return the reference to the current option. + */ + Object setLevelZeroStopWritesTrigger(int numFiles); + + /** + * Maximum number of level-0 files. We stop writes at this point. + * + * @return the hard limit of the number of level-0 file. + */ + int levelZeroStopWritesTrigger(); + + /** + * The highest level to which a new compacted memtable is pushed if it + * does not create overlap. We try to push to level 2 to avoid the + * relatively expensive level 0=>1 compactions and to avoid some + * expensive manifest file operations. We do not push all the way to + * the largest level since that can generate a lot of wasted disk + * space if the same key space is being repeatedly overwritten. + * + * @param maxMemCompactionLevel the highest level to which a new compacted + * mem-table will be pushed. + * @return the reference to the current option. + */ + Object setMaxMemCompactionLevel( + int maxMemCompactionLevel); + + /** + * The highest level to which a new compacted memtable is pushed if it + * does not create overlap. We try to push to level 2 to avoid the + * relatively expensive level 0=>1 compactions and to avoid some + * expensive manifest file operations. We do not push all the way to + * the largest level since that can generate a lot of wasted disk + * space if the same key space is being repeatedly overwritten. + * + * @return the highest level where a new compacted memtable will be pushed. + */ + int maxMemCompactionLevel(); + + /** + * The target file size for compaction. + * This targetFileSizeBase determines a level-1 file size. + * Target file size for level L can be calculated by + * targetFileSizeBase * (targetFileSizeMultiplier ^ (L-1)) + * For example, if targetFileSizeBase is 2MB and + * target_file_size_multiplier is 10, then each file on level-1 will + * be 2MB, and each file on level 2 will be 20MB, + * and each file on level-3 will be 200MB. + * by default targetFileSizeBase is 2MB. + * + * @param targetFileSizeBase the target size of a level-0 file. + * @return the reference to the current option. + * + * @see #setTargetFileSizeMultiplier(int) + */ + Object setTargetFileSizeBase(long targetFileSizeBase); + + /** + * The target file size for compaction. + * This targetFileSizeBase determines a level-1 file size. + * Target file size for level L can be calculated by + * targetFileSizeBase * (targetFileSizeMultiplier ^ (L-1)) + * For example, if targetFileSizeBase is 2MB and + * target_file_size_multiplier is 10, then each file on level-1 will + * be 2MB, and each file on level 2 will be 20MB, + * and each file on level-3 will be 200MB. + * by default targetFileSizeBase is 2MB. + * + * @return the target size of a level-0 file. + * + * @see #targetFileSizeMultiplier() + */ + long targetFileSizeBase(); + + /** + * targetFileSizeMultiplier defines the size ratio between a + * level-L file and level-(L+1) file. + * By default target_file_size_multiplier is 1, meaning + * files in different levels have the same target. + * + * @param multiplier the size ratio between a level-(L+1) file + * and level-L file. + * @return the reference to the current option. + */ + Object setTargetFileSizeMultiplier(int multiplier); + + /** + * targetFileSizeMultiplier defines the size ratio between a + * level-(L+1) file and level-L file. + * By default targetFileSizeMultiplier is 1, meaning + * files in different levels have the same target. + * + * @return the size ratio between a level-(L+1) file and level-L file. + */ + int targetFileSizeMultiplier(); + + /** + * The upper-bound of the total size of level-1 files in bytes. + * Maximum number of bytes for level L can be calculated as + * (maxBytesForLevelBase) * (maxBytesForLevelMultiplier ^ (L-1)) + * For example, if maxBytesForLevelBase is 20MB, and if + * max_bytes_for_level_multiplier is 10, total data size for level-1 + * will be 20MB, total file size for level-2 will be 200MB, + * and total file size for level-3 will be 2GB. + * by default 'maxBytesForLevelBase' is 10MB. + * + * @return the reference to the current option. + * @see #setMaxBytesForLevelMultiplier(int) + */ + Object setMaxBytesForLevelBase( + long maxBytesForLevelBase); + + /** + * The upper-bound of the total size of level-1 files in bytes. + * Maximum number of bytes for level L can be calculated as + * (maxBytesForLevelBase) * (maxBytesForLevelMultiplier ^ (L-1)) + * For example, if maxBytesForLevelBase is 20MB, and if + * max_bytes_for_level_multiplier is 10, total data size for level-1 + * will be 20MB, total file size for level-2 will be 200MB, + * and total file size for level-3 will be 2GB. + * by default 'maxBytesForLevelBase' is 10MB. + * + * @return the upper-bound of the total size of leve-1 files in bytes. + * @see #maxBytesForLevelMultiplier() + */ + long maxBytesForLevelBase(); + + /** + * The ratio between the total size of level-(L+1) files and the total + * size of level-L files for all L. + * DEFAULT: 10 + * + * @param multiplier the ratio between the total size of level-(L+1) + * files and the total size of level-L files for all L. + * @return the reference to the current option. + * @see #setMaxBytesForLevelBase(long) + */ + Object setMaxBytesForLevelMultiplier(int multiplier); + + /** + * The ratio between the total size of level-(L+1) files and the total + * size of level-L files for all L. + * DEFAULT: 10 + * + * @return the ratio between the total size of level-(L+1) files and + * the total size of level-L files for all L. + * @see #maxBytesForLevelBase() + */ + int maxBytesForLevelMultiplier(); + + /** + * Maximum number of bytes in all compacted files. We avoid expanding + * the lower level file set of a compaction if it would make the + * total compaction cover more than + * (expanded_compaction_factor * targetFileSizeLevel()) many bytes. + * + * @param expandedCompactionFactor the maximum number of bytes in all + * compacted files. + * @return the reference to the current option. + * @see #setSourceCompactionFactor(int) + */ + Object setExpandedCompactionFactor(int expandedCompactionFactor); + + /** + * Maximum number of bytes in all compacted files. We avoid expanding + * the lower level file set of a compaction if it would make the + * total compaction cover more than + * (expanded_compaction_factor * targetFileSizeLevel()) many bytes. + * + * @return the maximum number of bytes in all compacted files. + * @see #sourceCompactionFactor() + */ + int expandedCompactionFactor(); + + /** + * Maximum number of bytes in all source files to be compacted in a + * single compaction run. We avoid picking too many files in the + * source level so that we do not exceed the total source bytes + * for compaction to exceed + * (source_compaction_factor * targetFileSizeLevel()) many bytes. + * Default:1, i.e. pick maxfilesize amount of data as the source of + * a compaction. + * + * @param sourceCompactionFactor the maximum number of bytes in all + * source files to be compacted in a single compaction run. + * @return the reference to the current option. + * @see #setExpandedCompactionFactor(int) + */ + Object setSourceCompactionFactor(int sourceCompactionFactor); + + /** + * Maximum number of bytes in all source files to be compacted in a + * single compaction run. We avoid picking too many files in the + * source level so that we do not exceed the total source bytes + * for compaction to exceed + * (source_compaction_factor * targetFileSizeLevel()) many bytes. + * Default:1, i.e. pick maxfilesize amount of data as the source of + * a compaction. + * + * @return the maximum number of bytes in all source files to be compactedo. + * @see #expandedCompactionFactor() + */ + int sourceCompactionFactor(); + + /** + * Control maximum bytes of overlaps in grandparent (i.e., level+2) before we + * stop building a single file in a level->level+1 compaction. + * + * @param maxGrandparentOverlapFactor maximum bytes of overlaps in + * "grandparent" level. + * @return the reference to the current option. + */ + Object setMaxGrandparentOverlapFactor( + int maxGrandparentOverlapFactor); + + /** + * Control maximum bytes of overlaps in grandparent (i.e., level+2) before we + * stop building a single file in a level->level+1 compaction. + * + * @return maximum bytes of overlaps in "grandparent" level. + */ + int maxGrandparentOverlapFactor(); + + /** + * Puts are delayed 0-1 ms when any level has a compaction score that exceeds + * soft_rate_limit. This is ignored when == 0.0. + * CONSTRAINT: soft_rate_limit <= hard_rate_limit. If this constraint does not + * hold, RocksDB will set soft_rate_limit = hard_rate_limit + * Default: 0 (disabled) + * + * @param softRateLimit the soft-rate-limit of a compaction score + * for put delay. + * @return the reference to the current option. + */ + Object setSoftRateLimit(double softRateLimit); + + /** + * Puts are delayed 0-1 ms when any level has a compaction score that exceeds + * soft_rate_limit. This is ignored when == 0.0. + * CONSTRAINT: soft_rate_limit <= hard_rate_limit. If this constraint does not + * hold, RocksDB will set soft_rate_limit = hard_rate_limit + * Default: 0 (disabled) + * + * @return soft-rate-limit for put delay. + */ + double softRateLimit(); + + /** + * Puts are delayed 1ms at a time when any level has a compaction score that + * exceeds hard_rate_limit. This is ignored when <= 1.0. + * Default: 0 (disabled) + * + * @param hardRateLimit the hard-rate-limit of a compaction score for put + * delay. + * @return the reference to the current option. + */ + Object setHardRateLimit(double hardRateLimit); + + /** + * Puts are delayed 1ms at a time when any level has a compaction score that + * exceeds hard_rate_limit. This is ignored when <= 1.0. + * Default: 0 (disabled) + * + * @return the hard-rate-limit of a compaction score for put delay. + */ + double hardRateLimit(); + + /** + * The maximum time interval a put will be stalled when hard_rate_limit + * is enforced. If 0, then there is no limit. + * Default: 1000 + * + * @param rateLimitDelayMaxMilliseconds the maximum time interval a put + * will be stalled. + * @return the reference to the current option. + */ + Object setRateLimitDelayMaxMilliseconds( + int rateLimitDelayMaxMilliseconds); + + /** + * The maximum time interval a put will be stalled when hard_rate_limit + * is enforced. If 0, then there is no limit. + * Default: 1000 + * + * @return the maximum time interval a put will be stalled when + * hard_rate_limit is enforced. + */ + int rateLimitDelayMaxMilliseconds(); + + /** + * The size of one block in arena memory allocation. + * If <= 0, a proper value is automatically calculated (usually 1/10 of + * writer_buffer_size). + * + * There are two additonal restriction of the The specified size: + * (1) size should be in the range of [4096, 2 << 30] and + * (2) be the multiple of the CPU word (which helps with the memory + * alignment). + * + * We'll automatically check and adjust the size number to make sure it + * conforms to the restrictions. + * Default: 0 + * + * @param arenaBlockSize the size of an arena block + * @return the reference to the current option. + * @throws org.rocksdb.RocksDBException + */ + Object setArenaBlockSize(long arenaBlockSize) + throws RocksDBException; + + /** + * The size of one block in arena memory allocation. + * If <= 0, a proper value is automatically calculated (usually 1/10 of + * writer_buffer_size). + * + * There are two additonal restriction of the The specified size: + * (1) size should be in the range of [4096, 2 << 30] and + * (2) be the multiple of the CPU word (which helps with the memory + * alignment). + * + * We'll automatically check and adjust the size number to make sure it + * conforms to the restrictions. + * Default: 0 + * + * @return the size of an arena block + */ + long arenaBlockSize(); + + /** + * Disable automatic compactions. Manual compactions can still + * be issued on this column family + * + * @param disableAutoCompactions true if auto-compactions are disabled. + * @return the reference to the current option. + */ + Object setDisableAutoCompactions(boolean disableAutoCompactions); + + /** + * Disable automatic compactions. Manual compactions can still + * be issued on this column family + * + * @return true if auto-compactions are disabled. + */ + boolean disableAutoCompactions(); + + /** + * Purge duplicate/deleted keys when a memtable is flushed to storage. + * Default: true + * + * @param purgeRedundantKvsWhileFlush true if purging keys is disabled. + * @return the reference to the current option. + */ + Object setPurgeRedundantKvsWhileFlush( + boolean purgeRedundantKvsWhileFlush); + + /** + * Purge duplicate/deleted keys when a memtable is flushed to storage. + * Default: true + * + * @return true if purging keys is disabled. + */ + boolean purgeRedundantKvsWhileFlush(); + + /** + * Set compaction style for DB. + * + * Default: LEVEL. + * + * @param compactionStyle Compaction style. + * @return the reference to the current option. + */ + Object setCompactionStyle(CompactionStyle compactionStyle); + + /** + * Compaction style for DB. + * + * @return Compaction style. + */ + CompactionStyle compactionStyle(); + + /** + * If true, compaction will verify checksum on every read that happens + * as part of compaction + * Default: true + * + * @param verifyChecksumsInCompaction true if compaction verifies + * checksum on every read. + * @return the reference to the current option. + */ + Object setVerifyChecksumsInCompaction( + boolean verifyChecksumsInCompaction); + + /** + * If true, compaction will verify checksum on every read that happens + * as part of compaction + * Default: true + * + * @return true if compaction verifies checksum on every read. + */ + boolean verifyChecksumsInCompaction(); + + /** + * Use KeyMayExist API to filter deletes when this is true. + * If KeyMayExist returns false, i.e. the key definitely does not exist, then + * the delete is a noop. KeyMayExist only incurs in-memory look up. + * This optimization avoids writing the delete to storage when appropriate. + * Default: false + * + * @param filterDeletes true if filter-deletes behavior is on. + * @return the reference to the current option. + */ + Object setFilterDeletes(boolean filterDeletes); + + /** + * Use KeyMayExist API to filter deletes when this is true. + * If KeyMayExist returns false, i.e. the key definitely does not exist, then + * the delete is a noop. KeyMayExist only incurs in-memory look up. + * This optimization avoids writing the delete to storage when appropriate. + * Default: false + * + * @return true if filter-deletes behavior is on. + */ + boolean filterDeletes(); + + /** + * An iteration->Next() sequentially skips over keys with the same + * user-key unless this option is set. This number specifies the number + * of keys (with the same userkey) that will be sequentially + * skipped before a reseek is issued. + * Default: 8 + * + * @param maxSequentialSkipInIterations the number of keys could + * be skipped in a iteration. + * @return the reference to the current option. + */ + Object setMaxSequentialSkipInIterations(long maxSequentialSkipInIterations); + + /** + * An iteration->Next() sequentially skips over keys with the same + * user-key unless this option is set. This number specifies the number + * of keys (with the same userkey) that will be sequentially + * skipped before a reseek is issued. + * Default: 8 + * + * @return the number of keys could be skipped in a iteration. + */ + long maxSequentialSkipInIterations(); + + /** + * Set the config for mem-table. + * + * @param config the mem-table config. + * @return the instance of the current Object. + * @throws org.rocksdb.RocksDBException + */ + Object setMemTableConfig(MemTableConfig config) + throws RocksDBException; + + /** + * Returns the name of the current mem table representation. + * Memtable format can be set using setTableFormatConfig. + * + * @return the name of the currently-used memtable factory. + * @see #setTableFormatConfig(org.rocksdb.TableFormatConfig) + */ + String memTableFactoryName(); + + /** + * Set the config for table format. + * + * @param config the table format config. + * @return the reference of the current Options. + */ + Object setTableFormatConfig(TableFormatConfig config); + + /** + * @return the name of the currently used table factory. + */ + String tableFactoryName(); + + /** + * Allows thread-safe inplace updates. + * If inplace_callback function is not set, + * Put(key, new_value) will update inplace the existing_value iff + * * key exists in current memtable + * * new sizeof(new_value) <= sizeof(existing_value) + * * existing_value for that key is a put i.e. kTypeValue + * If inplace_callback function is set, check doc for inplace_callback. + * Default: false. + * + * @param inplaceUpdateSupport true if thread-safe inplace updates + * are allowed. + * @return the reference to the current option. + */ + Object setInplaceUpdateSupport(boolean inplaceUpdateSupport); + + /** + * Allows thread-safe inplace updates. + * If inplace_callback function is not set, + * Put(key, new_value) will update inplace the existing_value iff + * * key exists in current memtable + * * new sizeof(new_value) <= sizeof(existing_value) + * * existing_value for that key is a put i.e. kTypeValue + * If inplace_callback function is set, check doc for inplace_callback. + * Default: false. + * + * @return true if thread-safe inplace updates are allowed. + */ + boolean inplaceUpdateSupport(); + + /** + * Number of locks used for inplace update + * Default: 10000, if inplace_update_support = true, else 0. + * + * @param inplaceUpdateNumLocks the number of locks used for + * inplace updates. + * @return the reference to the current option. + * @throws org.rocksdb.RocksDBException + */ + Object setInplaceUpdateNumLocks(long inplaceUpdateNumLocks) + throws RocksDBException; + + /** + * Number of locks used for inplace update + * Default: 10000, if inplace_update_support = true, else 0. + * + * @return the number of locks used for inplace update. + */ + long inplaceUpdateNumLocks(); + + /** + * Sets the number of bits used in the prefix bloom filter. + * + * This value will be used only when a prefix-extractor is specified. + * + * @param memtablePrefixBloomBits the number of bits used in the + * prefix bloom filter. + * @return the reference to the current option. + */ + Object setMemtablePrefixBloomBits(int memtablePrefixBloomBits); + + /** + * Returns the number of bits used in the prefix bloom filter. + * + * This value will be used only when a prefix-extractor is specified. + * + * @return the number of bloom-bits. + * @see #useFixedLengthPrefixExtractor(int) + */ + int memtablePrefixBloomBits(); + + /** + * The number of hash probes per key used in the mem-table. + * + * @param memtablePrefixBloomProbes the number of hash probes per key. + * @return the reference to the current option. + */ + Object setMemtablePrefixBloomProbes(int memtablePrefixBloomProbes); + + /** + * The number of hash probes per key used in the mem-table. + * + * @return the number of hash probes per key. + */ + int memtablePrefixBloomProbes(); + + /** + * Control locality of bloom filter probes to improve cache miss rate. + * This option only applies to memtable prefix bloom and plaintable + * prefix bloom. It essentially limits the max number of cache lines each + * bloom filter check can touch. + * This optimization is turned off when set to 0. The number should never + * be greater than number of probes. This option can boost performance + * for in-memory workload but should use with care since it can cause + * higher false positive rate. + * Default: 0 + * + * @param bloomLocality the level of locality of bloom-filter probes. + * @return the reference to the current option. + */ + Object setBloomLocality(int bloomLocality); + + /** + * Control locality of bloom filter probes to improve cache miss rate. + * This option only applies to memtable prefix bloom and plaintable + * prefix bloom. It essentially limits the max number of cache lines each + * bloom filter check can touch. + * This optimization is turned off when set to 0. The number should never + * be greater than number of probes. This option can boost performance + * for in-memory workload but should use with care since it can cause + * higher false positive rate. + * Default: 0 + * + * @return the level of locality of bloom-filter probes. + * @see #setMemtablePrefixBloomProbes(int) + */ + int bloomLocality(); + + /** + * Maximum number of successive merge operations on a key in the memtable. + * + * When a merge operation is added to the memtable and the maximum number of + * successive merges is reached, the value of the key will be calculated and + * inserted into the memtable instead of the merge operation. This will + * ensure that there are never more than max_successive_merges merge + * operations in the memtable. + * + * Default: 0 (disabled) + * + * @param maxSuccessiveMerges the maximum number of successive merges. + * @return the reference to the current option. + * @throws org.rocksdb.RocksDBException + */ + Object setMaxSuccessiveMerges(long maxSuccessiveMerges) + throws RocksDBException; + + /** + * Maximum number of successive merge operations on a key in the memtable. + * + * When a merge operation is added to the memtable and the maximum number of + * successive merges is reached, the value of the key will be calculated and + * inserted into the memtable instead of the merge operation. This will + * ensure that there are never more than max_successive_merges merge + * operations in the memtable. + * + * Default: 0 (disabled) + * + * @return the maximum number of successive merges. + */ + long maxSuccessiveMerges(); + + /** + * The number of partial merge operands to accumulate before partial + * merge will be performed. Partial merge will not be called + * if the list of values to merge is less than min_partial_merge_operands. + * + * If min_partial_merge_operands < 2, then it will be treated as 2. + * + * Default: 2 + * + * @param minPartialMergeOperands min partial merge operands + * @return the reference to the current option. + */ + Object setMinPartialMergeOperands(int minPartialMergeOperands); + + /** + * The number of partial merge operands to accumulate before partial + * merge will be performed. Partial merge will not be called + * if the list of values to merge is less than min_partial_merge_operands. + * + * If min_partial_merge_operands < 2, then it will be treated as 2. + * + * Default: 2 + * + * @return min partial merge operands + */ + int minPartialMergeOperands(); + + /** + * Default memtable memory budget used with the following methods: + * + *
    + *
  1. {@link #optimizeLevelStyleCompaction()}
  2. + *
  3. {@link #optimizeUniversalStyleCompaction()}
  4. + *
+ */ + long DEFAULT_COMPACTION_MEMTABLE_MEMORY_BUDGET = 512 * 1024 * 1024; +} diff --git a/java/org/rocksdb/DBOptionsInterface.java b/java/org/rocksdb/DBOptionsInterface.java new file mode 100644 index 000000000..35c65eed2 --- /dev/null +++ b/java/org/rocksdb/DBOptionsInterface.java @@ -0,0 +1,763 @@ +// 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; + +public interface DBOptionsInterface { + + /** + * If this value is set to true, then the database will be created + * if it is missing during {@code RocksDB.open()}. + * Default: false + * + * @param flag a flag indicating whether to create a database the + * specified database in {@link RocksDB#open(org.rocksdb.Options, String)} operation + * is missing. + * @return the instance of the current Options + * @see RocksDB#open(org.rocksdb.Options, String) + */ + Object setCreateIfMissing(boolean flag); + + /** + * Return true if the create_if_missing flag is set to true. + * If true, the database will be created if it is missing. + * + * @return true if the createIfMissing option is set to true. + * @see #setCreateIfMissing(boolean) + */ + boolean createIfMissing(); + + /** + *

If true, missing column families will be automatically created

+ * + *

Default: false

+ * + * @param flag a flag indicating if missing column families shall be + * created automatically. + * @return true if missing column families shall be created automatically + * on open. + */ + Object setCreateMissingColumnFamilies(boolean flag); + + /** + * 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) + */ + boolean createMissingColumnFamilies(); + + /** + * If true, an error will be thrown during RocksDB.open() if the + * database already exists. + * Default: false + * + * @param errorIfExists if true, an exception will be thrown + * during {@code RocksDB.open()} if the database already exists. + * @return the reference to the current option. + * @see RocksDB#open(org.rocksdb.Options, String) + */ + Object setErrorIfExists(boolean errorIfExists); + + /** + * If true, an error will be thrown during RocksDB.open() if the + * database already exists. + * + * @return if true, an error is raised when the specified database + * already exists before open. + */ + boolean errorIfExists(); + + /** + * If true, the implementation will do aggressive checking of the + * data it is processing and will stop early if it detects any + * errors. This may have unforeseen ramifications: for example, a + * corruption of one DB entry may cause a large number of entries to + * become unreadable or for the entire DB to become unopenable. + * If any of the writes to the database fails (Put, Delete, Merge, Write), + * the database will switch to read-only mode and fail all other + * Write operations. + * Default: true + * + * @param paranoidChecks a flag to indicate whether paranoid-check + * is on. + * @return the reference to the current option. + */ + Object setParanoidChecks(boolean paranoidChecks); + + /** + * If true, the implementation will do aggressive checking of the + * data it is processing and will stop early if it detects any + * errors. This may have unforeseen ramifications: for example, a + * corruption of one DB entry may cause a large number of entries to + * become unreadable or for the entire DB to become unopenable. + * If any of the writes to the database fails (Put, Delete, Merge, Write), + * the database will switch to read-only mode and fail all other + * Write operations. + * + * @return a boolean indicating whether paranoid-check is on. + */ + boolean paranoidChecks(); + + /** + * Use to control write rate of flush and compaction. Flush has higher + * priority than compaction. Rate limiting is disabled if nullptr. + * Default: nullptr + * + * @param config rate limiter config. + * @return the instance of the current Object. + */ + Object setRateLimiterConfig(RateLimiterConfig config); + + /** + * Number of open files that can be used by the DB. You may need to + * increase this if your database has a large working set. Value -1 means + * files opened are always kept open. You can estimate number of files based + * on {@code target_file_size_base} and {@code target_file_size_multiplier} + * for level-based compaction. For universal-style compaction, you can usually + * set it to -1. + * Default: 5000 + * + * @param maxOpenFiles the maximum number of open files. + * @return the reference to the current DBOptions. + */ + Object setMaxOpenFiles(int maxOpenFiles); + + /** + * Number of open files that can be used by the DB. You may need to + * increase this if your database has a large working set. Value -1 means + * files opened are always kept open. You can estimate number of files based + * on {@code target_file_size_base} and {@code target_file_size_multiplier} + * for level-based compaction. For universal-style compaction, you can usually + * set it to -1. + * + * @return the maximum number of open files. + */ + int maxOpenFiles(); + + /** + *

Once write-ahead logs exceed this size, we will start forcing the + * flush of column families whose memtables are backed by the oldest live + * WAL file (i.e. the ones that are causing all the space amplification). + *

+ *

If set to 0 (default), we will dynamically choose the WAL size limit to + * be [sum of all write_buffer_size * max_write_buffer_number] * 2

+ *

Default: 0

+ */ + Object setMaxTotalWalSize(long maxTotalWalSize); + + /** + *

Returns the max total wal size. Once write-ahead logs exceed this size, + * we will start forcing the flush of column families whose memtables are + * backed by the oldest live WAL file (i.e. the ones that are causing all + * the space amplification).

+ * + *

If set to 0 (default), we will dynamically choose the WAL size limit + * to be [sum of all write_buffer_size * max_write_buffer_number] * 2 + *

+ * + * @return max total wal size + */ + long maxTotalWalSize(); + + /** + *

Creates statistics object which collects metrics about database operations. + * Statistics objects should not be shared between DB instances as + * it does not use any locks to prevent concurrent updates.

+ * + * @return the instance of the current Object. + * @see RocksDB#open(org.rocksdb.Options, String) + */ + Object createStatistics(); + + /** + *

Returns statistics object. Calls {@link #createStatistics()} if + * C++ returns {@code nullptr} for statistics.

+ * + * @return the instance of the statistics object. + * @see #createStatistics() + */ + Statistics statisticsPtr(); + + /** + *

If true, then the contents of manifest and data files are + * not synced to stable storage. Their contents remain in the + * OS buffers till theOS decides to flush them.

+ * + *

This option is good for bulk-loading of data.

+ * + *

Once the bulk-loading is complete, please issue a sync to + * the OS to flush all dirty buffers to stable storage.

+ * + *

Default: false

+ * + * @param disableDataSync a boolean flag to specify whether to + * disable data sync. + * @return the reference to the current DBOptions. + */ + Object setDisableDataSync(boolean disableDataSync); + + /** + * If true, then the contents of data files are not synced + * to stable storage. Their contents remain in the OS buffers till the + * OS decides to flush them. This option is good for bulk-loading + * of data. Once the bulk-loading is complete, please issue a + * sync to the OS to flush all dirty buffers to stable storage. + * + * @return if true, then data-sync is disabled. + */ + boolean disableDataSync(); + + /** + *

If true, then every store to stable storage will issue a fsync.

+ *

If false, then every store to stable storage will issue a fdatasync. + * This parameter should be set to true while storing data to + * filesystem like ext3 that can lose files after a reboot.

+ *

Default: false

+ * + * @param useFsync a boolean flag to specify whether to use fsync + * @return the instance of the current Object. + */ + Object setUseFsync(boolean useFsync); + + /** + *

If true, then every store to stable storage will issue a fsync.

+ *

If false, then every store to stable storage will issue a fdatasync. + * This parameter should be set to true while storing data to + * filesystem like ext3 that can lose files after a reboot.

+ * + * @return boolean value indicating if fsync is used. + */ + boolean useFsync(); + + /** + * This specifies the info LOG dir. + * If it is empty, the log files will be in the same dir as data. + * If it is non empty, the log files will be in the specified dir, + * and the db data dir's absolute path will be used as the log file + * name's prefix. + * + * @param dbLogDir the path to the info log directory + * @return the instance of the current Object. + */ + Object setDbLogDir(String dbLogDir); + + /** + * Returns the directory of info log. + * + * If it is empty, the log files will be in the same dir as data. + * If it is non empty, the log files will be in the specified dir, + * and the db data dir's absolute path will be used as the log file + * name's prefix. + * + * @return the path to the info log directory + */ + String dbLogDir(); + + /** + * This specifies the absolute dir path for write-ahead logs (WAL). + * If it is empty, the log files will be in the same dir as data, + * dbname is used as the data dir by default + * If it is non empty, the log files will be in kept the specified dir. + * When destroying the db, + * all log files in wal_dir and the dir itself is deleted + * + * @param walDir the path to the write-ahead-log directory. + * @return the instance of the current Object. + */ + Object setWalDir(String walDir); + + /** + * Returns the path to the write-ahead-logs (WAL) directory. + * + * If it is empty, the log files will be in the same dir as data, + * dbname is used as the data dir by default + * If it is non empty, the log files will be in kept the specified dir. + * When destroying the db, + * all log files in wal_dir and the dir itself is deleted + * + * @return the path to the write-ahead-logs (WAL) directory. + */ + String walDir(); + + /** + * The periodicity when obsolete files get deleted. The default + * value is 6 hours. The files that get out of scope by compaction + * process will still get automatically delete on every compaction, + * regardless of this setting + * + * @param micros the time interval in micros + * @return the instance of the current Object. + */ + Object setDeleteObsoleteFilesPeriodMicros(long micros); + + /** + * The periodicity when obsolete files get deleted. The default + * value is 6 hours. The files that get out of scope by compaction + * process will still get automatically delete on every compaction, + * regardless of this setting + * + * @return the time interval in micros when obsolete files will be deleted. + */ + long deleteObsoleteFilesPeriodMicros(); + + /** + * Specifies the maximum number of concurrent background compaction jobs, + * submitted to the default LOW priority thread pool. + * If you're increasing this, also consider increasing number of threads in + * LOW priority thread pool. For more information, see + * Default: 1 + * + * @param maxBackgroundCompactions the maximum number of background + * compaction jobs. + * @return the instance of the current Object. + * + * @see RocksEnv#setBackgroundThreads(int) + * @see RocksEnv#setBackgroundThreads(int, int) + * @see #maxBackgroundFlushes() + */ + Object setMaxBackgroundCompactions(int maxBackgroundCompactions); + + /** + * Returns the maximum number of concurrent background compaction jobs, + * submitted to the default LOW priority thread pool. + * When increasing this number, we may also want to consider increasing + * number of threads in LOW priority thread pool. + * Default: 1 + * + * @return the maximum number of concurrent background compaction jobs. + * @see RocksEnv#setBackgroundThreads(int) + * @see RocksEnv#setBackgroundThreads(int, int) + */ + int maxBackgroundCompactions(); + + /** + * Specifies the maximum number of concurrent background flush jobs. + * If you're increasing this, also consider increasing number of threads in + * HIGH priority thread pool. For more information, see + * Default: 1 + * + * @param maxBackgroundFlushes number of max concurrent flush jobs + * @return the instance of the current Object. + * + * @see RocksEnv#setBackgroundThreads(int) + * @see RocksEnv#setBackgroundThreads(int, int) + * @see #maxBackgroundCompactions() + */ + Object setMaxBackgroundFlushes(int maxBackgroundFlushes); + + /** + * Returns the maximum number of concurrent background flush jobs. + * If you're increasing this, also consider increasing number of threads in + * HIGH priority thread pool. For more information, see + * Default: 1 + * + * @return the maximum number of concurrent background flush jobs. + * @see RocksEnv#setBackgroundThreads(int) + * @see RocksEnv#setBackgroundThreads(int, int) + */ + int maxBackgroundFlushes(); + + /** + * Specifies the maximum size of a info log file. If the current log file + * is larger than `max_log_file_size`, a new info log file will + * be created. + * If 0, all logs will be written to one log file. + * + * @param maxLogFileSize the maximum size of a info log file. + * @return the instance of the current Object. + * @throws org.rocksdb.RocksDBException + */ + Object setMaxLogFileSize(long maxLogFileSize) + throws RocksDBException; + + /** + * Returns the maximum size of a info log file. If the current log file + * is larger than this size, a new info log file will be created. + * If 0, all logs will be written to one log file. + * + * @return the maximum size of the info log file. + */ + long maxLogFileSize(); + + /** + * Specifies the time interval for the info log file to roll (in seconds). + * If specified with non-zero value, log file will be rolled + * if it has been active longer than `log_file_time_to_roll`. + * Default: 0 (disabled) + * + * @param logFileTimeToRoll the time interval in seconds. + * @return the instance of the current Object. + * @throws org.rocksdb.RocksDBException + */ + Object setLogFileTimeToRoll(long logFileTimeToRoll) + throws RocksDBException; + + /** + * Returns the time interval for the info log file to roll (in seconds). + * If specified with non-zero value, log file will be rolled + * if it has been active longer than `log_file_time_to_roll`. + * Default: 0 (disabled) + * + * @return the time interval in seconds. + */ + long logFileTimeToRoll(); + + /** + * Specifies the maximum number of info log files to be kept. + * Default: 1000 + * + * @param keepLogFileNum the maximum number of info log files to be kept. + * @return the instance of the current Object. + * @throws org.rocksdb.RocksDBException + */ + Object setKeepLogFileNum(long keepLogFileNum) + throws RocksDBException; + + /** + * Returns the maximum number of info log files to be kept. + * Default: 1000 + * + * @return the maximum number of info log files to be kept. + */ + long keepLogFileNum(); + + /** + * Manifest file is rolled over on reaching this limit. + * The older manifest file be deleted. + * The default value is MAX_INT so that roll-over does not take place. + * + * @param maxManifestFileSize the size limit of a manifest file. + * @return the instance of the current Object. + */ + Object setMaxManifestFileSize(long maxManifestFileSize); + + /** + * Manifest file is rolled over on reaching this limit. + * The older manifest file be deleted. + * The default value is MAX_INT so that roll-over does not take place. + * + * @return the size limit of a manifest file. + */ + long maxManifestFileSize(); + + /** + * Number of shards used for table cache. + * + * @param tableCacheNumshardbits the number of chards + * @return the instance of the current Object. + */ + Object setTableCacheNumshardbits(int tableCacheNumshardbits); + + /** + * Number of shards used for table cache. + * + * @return the number of shards used for table cache. + */ + int tableCacheNumshardbits(); + + /** + * During data eviction of table's LRU cache, it would be inefficient + * to strictly follow LRU because this piece of memory will not really + * be released unless its refcount falls to zero. Instead, make two + * passes: the first pass will release items with refcount = 1, + * and if not enough space releases after scanning the number of + * elements specified by this parameter, we will remove items in LRU + * order. + * + * @param limit scan count limit + * @return the instance of the current Object. + */ + Object setTableCacheRemoveScanCountLimit(int limit); + + /** + * During data eviction of table's LRU cache, it would be inefficient + * to strictly follow LRU because this piece of memory will not really + * be released unless its refcount falls to zero. Instead, make two + * passes: the first pass will release items with refcount = 1, + * and if not enough space releases after scanning the number of + * elements specified by this parameter, we will remove items in LRU + * order. + * + * @return scan count limit + */ + int tableCacheRemoveScanCountLimit(); + + /** + * {@link #walTtlSeconds()} and {@link #walSizeLimitMB()} affect how archived logs + * will be deleted. + *
    + *
  1. If both set to 0, logs will be deleted asap and will not get into + * the archive.
  2. + *
  3. If WAL_ttl_seconds is 0 and WAL_size_limit_MB is not 0, + * WAL files will be checked every 10 min and if total size is greater + * then WAL_size_limit_MB, they will be deleted starting with the + * earliest until size_limit is met. All empty files will be deleted.
  4. + *
  5. If WAL_ttl_seconds is not 0 and WAL_size_limit_MB is 0, then + * WAL files will be checked every WAL_ttl_secondsi / 2 and those that + * are older than WAL_ttl_seconds will be deleted.
  6. + *
  7. If both are not 0, WAL files will be checked every 10 min and both + * checks will be performed with ttl being first.
  8. + * + * @param walTtlSeconds the ttl seconds + * @return the instance of the current Object. + * @see #setWalSizeLimitMB(long) + */ + Object setWalTtlSeconds(long walTtlSeconds); + + /** + * WalTtlSeconds() and walSizeLimitMB() affect how archived logs + * will be deleted. + *
      + *
    1. If both set to 0, logs will be deleted asap and will not get into + * the archive.
    2. + *
    3. If WAL_ttl_seconds is 0 and WAL_size_limit_MB is not 0, + * WAL files will be checked every 10 min and if total size is greater + * then WAL_size_limit_MB, they will be deleted starting with the + * earliest until size_limit is met. All empty files will be deleted.
    4. + *
    5. If WAL_ttl_seconds is not 0 and WAL_size_limit_MB is 0, then + * WAL files will be checked every WAL_ttl_secondsi / 2 and those that + * are older than WAL_ttl_seconds will be deleted.
    6. + *
    7. If both are not 0, WAL files will be checked every 10 min and both + * checks will be performed with ttl being first.
    8. + *
    + * + * @return the wal-ttl seconds + * @see #walSizeLimitMB() + */ + long walTtlSeconds(); + + /** + * WalTtlSeconds() and walSizeLimitMB() affect how archived logs + * will be deleted. + *
      + *
    1. If both set to 0, logs will be deleted asap and will not get into + * the archive.
    2. + *
    3. If WAL_ttl_seconds is 0 and WAL_size_limit_MB is not 0, + * WAL files will be checked every 10 min and if total size is greater + * then WAL_size_limit_MB, they will be deleted starting with the + * earliest until size_limit is met. All empty files will be deleted.
    4. + *
    5. If WAL_ttl_seconds is not 0 and WAL_size_limit_MB is 0, then + * WAL files will be checked every WAL_ttl_secondsi / 2 and those that + * are older than WAL_ttl_seconds will be deleted.
    6. + *
    7. If both are not 0, WAL files will be checked every 10 min and both + * checks will be performed with ttl being first.
    8. + * + * @param sizeLimitMB size limit in mega-bytes. + * @return the instance of the current Object. + * @see #setWalSizeLimitMB(long) + */ + Object setWalSizeLimitMB(long sizeLimitMB); + + /** + * {@link #walTtlSeconds()} and {@code #walSizeLimitMB()} affect how archived logs + * will be deleted. + *
        + *
      1. If both set to 0, logs will be deleted asap and will not get into + * the archive.
      2. + *
      3. If WAL_ttl_seconds is 0 and WAL_size_limit_MB is not 0, + * WAL files will be checked every 10 min and if total size is greater + * then WAL_size_limit_MB, they will be deleted starting with the + * earliest until size_limit is met. All empty files will be deleted.
      4. + *
      5. If WAL_ttl_seconds is not 0 and WAL_size_limit_MB is 0, then + * WAL files will be checked every WAL_ttl_seconds i / 2 and those that + * are older than WAL_ttl_seconds will be deleted.
      6. + *
      7. If both are not 0, WAL files will be checked every 10 min and both + * checks will be performed with ttl being first.
      8. + *
      + * @return size limit in mega-bytes. + * @see #walSizeLimitMB() + */ + long walSizeLimitMB(); + + /** + * Number of bytes to preallocate (via fallocate) the manifest + * files. Default is 4mb, which is reasonable to reduce random IO + * as well as prevent overallocation for mounts that preallocate + * large amounts of data (such as xfs's allocsize option). + * + * @param size the size in byte + * @return the instance of the current Object. + * @throws org.rocksdb.RocksDBException + */ + Object setManifestPreallocationSize(long size) + throws RocksDBException; + + /** + * Number of bytes to preallocate (via fallocate) the manifest + * files. Default is 4mb, which is reasonable to reduce random IO + * as well as prevent overallocation for mounts that preallocate + * large amounts of data (such as xfs's allocsize option). + * + * @return size in bytes. + */ + long manifestPreallocationSize(); + + /** + * Data being read from file storage may be buffered in the OS + * Default: true + * + * @param allowOsBuffer if true, then OS buffering is allowed. + * @return the instance of the current Object. + */ + Object setAllowOsBuffer(boolean allowOsBuffer); + + /** + * Data being read from file storage may be buffered in the OS + * Default: true + * + * @return if true, then OS buffering is allowed. + */ + boolean allowOsBuffer(); + + /** + * Allow the OS to mmap file for reading sst tables. + * Default: false + * + * @param allowMmapReads true if mmap reads are allowed. + * @return the instance of the current Object. + */ + Object setAllowMmapReads(boolean allowMmapReads); + + /** + * Allow the OS to mmap file for reading sst tables. + * Default: false + * + * @return true if mmap reads are allowed. + */ + boolean allowMmapReads(); + + /** + * Allow the OS to mmap file for writing. Default: false + * + * @param allowMmapWrites true if mmap writes are allowd. + * @return the instance of the current Object. + */ + Object setAllowMmapWrites(boolean allowMmapWrites); + + /** + * Allow the OS to mmap file for writing. Default: false + * + * @return true if mmap writes are allowed. + */ + boolean allowMmapWrites(); + + /** + * Disable child process inherit open files. Default: true + * + * @param isFdCloseOnExec true if child process inheriting open + * files is disabled. + * @return the instance of the current Object. + */ + Object setIsFdCloseOnExec(boolean isFdCloseOnExec); + + /** + * Disable child process inherit open files. Default: true + * + * @return true if child process inheriting open files is disabled. + */ + boolean isFdCloseOnExec(); + + /** + * Skip log corruption error on recovery (If client is ok with + * losing most recent changes) + * Default: false + * + * @param skip true if log corruption errors are skipped during recovery. + * @return the instance of the current Object. + */ + Object setSkipLogErrorOnRecovery(boolean skip); + + /** + * Skip log corruption error on recovery (If client is ok with + * losing most recent changes) + * Default: false + * + * @return true if log corruption errors are skipped during recovery. + */ + boolean skipLogErrorOnRecovery(); + + /** + * if not zero, dump rocksdb.stats to LOG every stats_dump_period_sec + * Default: 3600 (1 hour) + * + * @param statsDumpPeriodSec time interval in seconds. + * @return the instance of the current Object. + */ + Object setStatsDumpPeriodSec(int statsDumpPeriodSec); + + /** + * If not zero, dump rocksdb.stats to LOG every stats_dump_period_sec + * Default: 3600 (1 hour) + * + * @return time interval in seconds. + */ + int statsDumpPeriodSec(); + + /** + * If set true, will hint the underlying file system that the file + * access pattern is random, when a sst file is opened. + * Default: true + * + * @param adviseRandomOnOpen true if hinting random access is on. + * @return the instance of the current Object. + */ + Object setAdviseRandomOnOpen(boolean adviseRandomOnOpen); + + /** + * If set true, will hint the underlying file system that the file + * access pattern is random, when a sst file is opened. + * Default: true + * + * @return true if hinting random access is on. + */ + boolean adviseRandomOnOpen(); + + /** + * Use adaptive mutex, which spins in the user space before resorting + * to kernel. This could reduce context switch when the mutex is not + * heavily contended. However, if the mutex is hot, we could end up + * wasting spin time. + * Default: false + * + * @param useAdaptiveMutex true if adaptive mutex is used. + * @return the instance of the current Object. + */ + Object setUseAdaptiveMutex(boolean useAdaptiveMutex); + + /** + * Use adaptive mutex, which spins in the user space before resorting + * to kernel. This could reduce context switch when the mutex is not + * heavily contended. However, if the mutex is hot, we could end up + * wasting spin time. + * Default: false + * + * @return true if adaptive mutex is used. + */ + boolean useAdaptiveMutex(); + + /** + * Allows OS to incrementally sync files to disk while they are being + * written, asynchronously, in the background. + * Issue one request for every bytes_per_sync written. 0 turns it off. + * Default: 0 + * + * @param bytesPerSync size in bytes + * @return the instance of the current Object. + */ + Object setBytesPerSync(long bytesPerSync); + + /** + * Allows OS to incrementally sync files to disk while they are being + * written, asynchronously, in the background. + * Issue one request for every bytes_per_sync written. 0 turns it off. + * Default: 0 + * + * @return size in bytes + */ + long bytesPerSync(); +} diff --git a/java/org/rocksdb/Options.java b/java/org/rocksdb/Options.java index a3ea38f7c..16db5e166 100644 --- a/java/org/rocksdb/Options.java +++ b/java/org/rocksdb/Options.java @@ -12,20 +12,11 @@ package org.rocksdb; * If {@link #dispose()} function is not called, then it will be GC'd automatically * and native resources will be released as part of the process. */ -public class Options extends RocksObject { +public class Options extends RocksObject + implements DBOptionsInterface, ColumnFamilyOptionsInterface { static { RocksDB.loadLibrary(); } - static final long DEFAULT_CACHE_SIZE = 8 << 20; - static final int DEFAULT_NUM_SHARD_BITS = -1; - - /** - * Builtin RocksDB comparators - */ - public enum BuiltinComparator { - BYTEWISE_COMPARATOR, REVERSE_BYTEWISE_COMPARATOR; - } - /** * Construct options for opening a RocksDB. * @@ -34,38 +25,18 @@ public class Options extends RocksObject { */ public Options() { super(); - cacheSize_ = DEFAULT_CACHE_SIZE; - numShardBits_ = DEFAULT_NUM_SHARD_BITS; newOptions(); env_ = RocksEnv.getDefault(); } - /** - * If this value is set to true, then the database will be created - * if it is missing during {@code RocksDB.open()}. - * Default: false - * - * @param flag a flag indicating whether to create a database the - * specified database in {@link org.rocksdb.RocksDB#open(Options, String)} operation - * is missing. - * @return the instance of the current Options - * @see org.rocksdb.RocksDB#open(Options, String) - */ + @Override public Options setCreateIfMissing(boolean flag) { assert(isInitialized()); setCreateIfMissing(nativeHandle_, flag); return this; } - /** - *

      If true, missing column families will be automatically created

      - * - *

      Default: false

      - * - * @param flag - * @return true if missing column families shall be created automatically - * on open. - */ + @Override public Options setCreateMissingColumnFamilies(boolean flag) { assert(isInitialized()); setCreateMissingColumnFamilies(nativeHandle_, flag); @@ -83,71 +54,105 @@ public class Options extends RocksObject { env_ = env; return this; } - private native void setEnv(long optHandle, long envHandle); public RocksEnv getEnv() { return env_; } - private native long getEnvHandle(long handle); /** - * Return true if the create_if_missing flag is set to true. - * If true, the database will be created if it is missing. + * Set appropriate parameters for bulk loading. + * The reason that this is a function that returns "this" instead of a + * constructor is to enable chaining of multiple similar calls in the future. * - * @return true if the createIfMissing option is set to true. - * @see #setCreateIfMissing(boolean) + * All data will be in level 0 without any automatic compaction. + * It's recommended to manually call CompactRange(NULL, NULL) before reading + * from the database, because otherwise the read can be very slow. + * + * @return the instance of the current Options. */ + public Options prepareForBulkLoad() { + prepareForBulkLoad(nativeHandle_); + return this; + } + + @Override public boolean createIfMissing() { assert(isInitialized()); 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) - */ + @Override public boolean createMissingColumnFamilies() { assert(isInitialized()); return createMissingColumnFamilies(nativeHandle_); } - /** - * Set {@link org.rocksdb.Options.BuiltinComparator} to be used with RocksDB. - * - * Note: Comparator can be set once upon database creation. - * - * Default: BytewiseComparator. - * @param builtinComparator a {@link org.rocksdb.Options.BuiltinComparator} type. - */ - public void setBuiltinComparator(BuiltinComparator builtinComparator) { + @Override + public Options optimizeForPointLookup( + long blockCacheSizeMb) { + optimizeForPointLookup(nativeHandle_, + blockCacheSizeMb); + return this; + } + + @Override + public Options optimizeLevelStyleCompaction() { + optimizeLevelStyleCompaction(nativeHandle_, + DEFAULT_COMPACTION_MEMTABLE_MEMORY_BUDGET); + return this; + } + + @Override + public Options optimizeLevelStyleCompaction( + long memtableMemoryBudget) { + optimizeLevelStyleCompaction(nativeHandle_, + memtableMemoryBudget); + return this; + } + + @Override + public Options optimizeUniversalStyleCompaction() { + optimizeUniversalStyleCompaction(nativeHandle_, + DEFAULT_COMPACTION_MEMTABLE_MEMORY_BUDGET); + return this; + } + + @Override + public Options optimizeUniversalStyleCompaction( + long memtableMemoryBudget) { + optimizeUniversalStyleCompaction(nativeHandle_, + memtableMemoryBudget); + return this; + } + + @Override + public Options setComparator(BuiltinComparator builtinComparator) { assert(isInitialized()); - setBuiltinComparator(nativeHandle_, builtinComparator.ordinal()); + setComparatorHandle(nativeHandle_, builtinComparator.ordinal()); + return this; } - private native void setBuiltinComparator(long handle, int builtinComparator); + @Override + public Options setComparator(AbstractComparator comparator) { + assert (isInitialized()); + setComparatorHandle(nativeHandle_, comparator.nativeHandle_); + comparator_ = comparator; + return this; + } - /** - * Amount of data to build up in memory (backed by an unsorted log - * on disk) before converting to a sorted on-disk file. - * - * Larger values increase performance, especially during bulk loads. - * Up to {@code max_write_buffer_number} write buffers may be held in memory - * at the same time, so you may wish to adjust this parameter - * to control memory usage. - * - * Also, a larger write buffer will result in a longer recovery time - * the next time the database is opened. - * - * Default: 4MB - * @param writeBufferSize the size of write buffer. - * @return the instance of the current Options. - * @see org.rocksdb.RocksDB#open(Options, String) - * @throws RocksDBException - */ + @Override + public Options setMergeOperatorName(String name) { + setMergeOperatorName(nativeHandle_, name); + return this; + } + + @Override + public Options setMergeOperator(MergeOperator mergeOperator) { + setMergeOperator(nativeHandle_, mergeOperator.newMergeOperatorHandle()); + return this; + } + + @Override public Options setWriteBufferSize(long writeBufferSize) throws RocksDBException { assert(isInitialized()); @@ -155,372 +160,156 @@ public class Options extends RocksObject { return this; } - /** - * Return size of write buffer size. - * - * @return size of write buffer. - * @see #setWriteBufferSize(long) - */ + @Override public long writeBufferSize() { assert(isInitialized()); return writeBufferSize(nativeHandle_); } - /** - * The maximum number of write buffers that are built up in memory. - * The default is 2, so that when 1 write buffer is being flushed to - * storage, new writes can continue to the other write buffer. - * Default: 2 - * - * @param maxWriteBufferNumber maximum number of write buffers. - * @return the instance of the current Options. - * @see org.rocksdb.RocksDB#open(Options, String) - */ + @Override public Options setMaxWriteBufferNumber(int maxWriteBufferNumber) { assert(isInitialized()); setMaxWriteBufferNumber(nativeHandle_, maxWriteBufferNumber); return this; } - /** - * Returns maximum number of write buffers. - * - * @return maximum number of write buffers. - * @see #setMaxWriteBufferNumber(int) - */ + @Override public int maxWriteBufferNumber() { assert(isInitialized()); return maxWriteBufferNumber(nativeHandle_); } - /** - * Use the specified comparator for key ordering. - * - * Comparator should not be disposed before options instances using this comparator is - * disposed. If dispose() function is not called, then comparator object will be - * GC'd automatically. - * - * Comparator instance can be re-used in multiple options instances. - * - * @param comparator java instance. - * @return the instance of the current Options. - * @see RocksDB.open() - */ - public Options setComparator(AbstractComparator comparator) { - assert (isInitialized()); - setComparatorHandle(nativeHandle_, comparator.nativeHandle_); - comparator_ = comparator; - return this; - } - private native void setComparatorHandle(long optHandle, long comparatorHandle); - - /** - * If true, an error will be thrown during RocksDB.open() if the - * database already exists. - * - * @return if true, an error is raised when the specified database - * already exists before open. - */ + @Override public boolean errorIfExists() { assert(isInitialized()); return errorIfExists(nativeHandle_); } - private native boolean errorIfExists(long handle); - /** - * If true, an error will be thrown during RocksDB.open() if the - * database already exists. - * Default: false - * - * @param errorIfExists if true, an exception will be thrown - * during {@code RocksDB.open()} if the database already exists. - * @return the reference to the current option. - * @see org.rocksdb.RocksDB#open(Options, String) - */ + @Override public Options setErrorIfExists(boolean errorIfExists) { assert(isInitialized()); setErrorIfExists(nativeHandle_, errorIfExists); return this; } - private native void setErrorIfExists(long handle, boolean errorIfExists); - /** - * If true, the implementation will do aggressive checking of the - * data it is processing and will stop early if it detects any - * errors. This may have unforeseen ramifications: for example, a - * corruption of one DB entry may cause a large number of entries to - * become unreadable or for the entire DB to become unopenable. - * If any of the writes to the database fails (Put, Delete, Merge, Write), - * the database will switch to read-only mode and fail all other - * Write operations. - * - * @return a boolean indicating whether paranoid-check is on. - */ + @Override public boolean paranoidChecks() { assert(isInitialized()); return paranoidChecks(nativeHandle_); } - private native boolean paranoidChecks(long handle); - /** - * If true, the implementation will do aggressive checking of the - * data it is processing and will stop early if it detects any - * errors. This may have unforeseen ramifications: for example, a - * corruption of one DB entry may cause a large number of entries to - * become unreadable or for the entire DB to become unopenable. - * If any of the writes to the database fails (Put, Delete, Merge, Write), - * the database will switch to read-only mode and fail all other - * Write operations. - * Default: true - * - * @param paranoidChecks a flag to indicate whether paranoid-check - * is on. - * @return the reference to the current option. - */ + @Override public Options setParanoidChecks(boolean paranoidChecks) { assert(isInitialized()); setParanoidChecks(nativeHandle_, paranoidChecks); return this; } - private native void setParanoidChecks( - long handle, boolean paranoidChecks); - /** - * Number of open files that can be used by the DB. You may need to - * increase this if your database has a large working set. Value -1 means - * files opened are always kept open. You can estimate number of files based - * on {@code target_file_size_base} and {@code target_file_size_multiplier} - * for level-based compaction. For universal-style compaction, you can usually - * set it to -1. - * - * @return the maximum number of open files. - */ + @Override public int maxOpenFiles() { assert(isInitialized()); return maxOpenFiles(nativeHandle_); } - private native int maxOpenFiles(long handle); - /** - * Number of open files that can be used by the DB. You may need to - * increase this if your database has a large working set. Value -1 means - * files opened are always kept open. You can estimate number of files based - * on {@code target_file_size_base} and {@code target_file_size_multiplier} - * for level-based compaction. For universal-style compaction, you can usually - * set it to -1. - * Default: 5000 - * - * @param maxOpenFiles the maximum number of open files. - * @return the reference to the current option. - */ + @Override + public Options setMaxTotalWalSize(long maxTotalWalSize) { + assert(isInitialized()); + setMaxTotalWalSize(nativeHandle_, maxTotalWalSize); + return this; + } + + @Override + public long maxTotalWalSize() { + assert(isInitialized()); + return maxTotalWalSize(nativeHandle_); + } + + @Override public Options setMaxOpenFiles(int maxOpenFiles) { assert(isInitialized()); setMaxOpenFiles(nativeHandle_, maxOpenFiles); return this; } - private native void setMaxOpenFiles(long handle, int maxOpenFiles); - /** - * If true, then the contents of data files are not synced - * to stable storage. Their contents remain in the OS buffers till the - * OS decides to flush them. This option is good for bulk-loading - * of data. Once the bulk-loading is complete, please issue a - * sync to the OS to flush all dirty buffers to stable storage. - * - * @return if true, then data-sync is disabled. - */ + @Override public boolean disableDataSync() { assert(isInitialized()); return disableDataSync(nativeHandle_); } - private native boolean disableDataSync(long handle); - /** - * If true, then the contents of data files are not synced - * to stable storage. Their contents remain in the OS buffers till the - * OS decides to flush them. This option is good for bulk-loading - * of data. Once the bulk-loading is complete, please issue a - * sync to the OS to flush all dirty buffers to stable storage. - * Default: false - * - * @param disableDataSync a boolean flag to specify whether to - * disable data sync. - * @return the reference to the current option. - */ + @Override public Options setDisableDataSync(boolean disableDataSync) { assert(isInitialized()); setDisableDataSync(nativeHandle_, disableDataSync); return this; } - private native void setDisableDataSync(long handle, boolean disableDataSync); - /** - * If true, then every store to stable storage will issue a fsync. - * If false, then every store to stable storage will issue a fdatasync. - * This parameter should be set to true while storing data to - * filesystem like ext3 that can lose files after a reboot. - * - * @return boolean value indicating if fsync is used. - */ + @Override public boolean useFsync() { assert(isInitialized()); return useFsync(nativeHandle_); } - private native boolean useFsync(long handle); - /** - * If true, then every store to stable storage will issue a fsync. - * If false, then every store to stable storage will issue a fdatasync. - * This parameter should be set to true while storing data to - * filesystem like ext3 that can lose files after a reboot. - * Default: false - * - * @param useFsync a boolean flag to specify whether to use fsync - * @return the reference to the current option. - */ + @Override public Options setUseFsync(boolean useFsync) { assert(isInitialized()); setUseFsync(nativeHandle_, useFsync); return this; } - private native void setUseFsync(long handle, boolean useFsync); - /** - * Returns the directory of info log. - * - * If it is empty, the log files will be in the same dir as data. - * If it is non empty, the log files will be in the specified dir, - * and the db data dir's absolute path will be used as the log file - * name's prefix. - * - * @return the path to the info log directory - */ + @Override public String dbLogDir() { assert(isInitialized()); return dbLogDir(nativeHandle_); } - private native String dbLogDir(long handle); - /** - * This specifies the info LOG dir. - * If it is empty, the log files will be in the same dir as data. - * If it is non empty, the log files will be in the specified dir, - * and the db data dir's absolute path will be used as the log file - * name's prefix. - * - * @param dbLogDir the path to the info log directory - * @return the reference to the current option. - */ + @Override public Options setDbLogDir(String dbLogDir) { assert(isInitialized()); setDbLogDir(nativeHandle_, dbLogDir); return this; } - private native void setDbLogDir(long handle, String dbLogDir); - /** - * Returns the path to the write-ahead-logs (WAL) directory. - * - * If it is empty, the log files will be in the same dir as data, - * dbname is used as the data dir by default - * If it is non empty, the log files will be in kept the specified dir. - * When destroying the db, - * all log files in wal_dir and the dir itself is deleted - * - * @return the path to the write-ahead-logs (WAL) directory. - */ + @Override public String walDir() { assert(isInitialized()); return walDir(nativeHandle_); } - private native String walDir(long handle); - /** - * This specifies the absolute dir path for write-ahead logs (WAL). - * If it is empty, the log files will be in the same dir as data, - * dbname is used as the data dir by default - * If it is non empty, the log files will be in kept the specified dir. - * When destroying the db, - * all log files in wal_dir and the dir itself is deleted - * - * @param walDir the path to the write-ahead-log directory. - * @return the reference to the current option. - */ + @Override public Options setWalDir(String walDir) { assert(isInitialized()); setWalDir(nativeHandle_, walDir); return this; } - private native void setWalDir(long handle, String walDir); - /** - * The periodicity when obsolete files get deleted. The default - * value is 6 hours. The files that get out of scope by compaction - * process will still get automatically delete on every compaction, - * regardless of this setting - * - * @return the time interval in micros when obsolete files will be deleted. - */ + @Override public long deleteObsoleteFilesPeriodMicros() { assert(isInitialized()); return deleteObsoleteFilesPeriodMicros(nativeHandle_); } - private native long deleteObsoleteFilesPeriodMicros(long handle); - /** - * The periodicity when obsolete files get deleted. The default - * value is 6 hours. The files that get out of scope by compaction - * process will still get automatically delete on every compaction, - * regardless of this setting - * - * @param micros the time interval in micros - * @return the reference to the current option. - */ + @Override public Options setDeleteObsoleteFilesPeriodMicros(long micros) { assert(isInitialized()); setDeleteObsoleteFilesPeriodMicros(nativeHandle_, micros); return this; } - private native void setDeleteObsoleteFilesPeriodMicros( - long handle, long micros); - /** - * Returns the maximum number of concurrent background compaction jobs, - * submitted to the default LOW priority thread pool. - * When increasing this number, we may also want to consider increasing - * number of threads in LOW priority thread pool. - * Default: 1 - * - * @return the maximum number of concurrent background compaction jobs. - * @see org.rocksdb.RocksEnv#setBackgroundThreads(int) - * @see org.rocksdb.RocksEnv#setBackgroundThreads(int, int) - */ + @Override public int maxBackgroundCompactions() { assert(isInitialized()); return maxBackgroundCompactions(nativeHandle_); } - /** - * Creates statistics object which collects metrics about database operations. - Statistics objects should not be shared between DB instances as - it does not use any locks to prevent concurrent updates. - * - * @return the instance of the current Options. - * @see org.rocksdb.RocksDB#open(Options, String) - */ + @Override public Options createStatistics() { assert(isInitialized()); createStatistics(nativeHandle_); return this; } - /** - * Returns statistics object. Calls {@link #createStatistics()} if - * C++ returns {@code nullptr} for statistics. - * - * @return the instance of the statistics object. - * @see #createStatistics() - */ + @Override public Statistics statisticsPtr() { assert(isInitialized()); @@ -533,646 +322,263 @@ public class Options extends RocksObject { return new Statistics(statsPtr); } - /** - * Specifies the maximum number of concurrent background compaction jobs, - * submitted to the default LOW priority thread pool. - * If you're increasing this, also consider increasing number of threads in - * LOW priority thread pool. For more information, see - * Default: 1 - * - * @param maxBackgroundCompactions the maximum number of background - * compaction jobs. - * @return the reference to the current option. - * - * @see org.rocksdb.RocksEnv#setBackgroundThreads(int) - * @see org.rocksdb.RocksEnv#setBackgroundThreads(int, int) - * @see #maxBackgroundFlushes() - */ + @Override public Options setMaxBackgroundCompactions(int maxBackgroundCompactions) { assert(isInitialized()); setMaxBackgroundCompactions(nativeHandle_, maxBackgroundCompactions); return this; } - /** - * Returns the maximum number of concurrent background flush jobs. - * If you're increasing this, also consider increasing number of threads in - * HIGH priority thread pool. For more information, see - * Default: 1 - * - * @return the maximum number of concurrent background flush jobs. - * @see org.rocksdb.RocksEnv#setBackgroundThreads(int) - * @see org.rocksdb.RocksEnv#setBackgroundThreads(int, int) - */ + @Override public int maxBackgroundFlushes() { assert(isInitialized()); return maxBackgroundFlushes(nativeHandle_); } - private native int maxBackgroundFlushes(long handle); - /** - * Specifies the maximum number of concurrent background flush jobs. - * If you're increasing this, also consider increasing number of threads in - * HIGH priority thread pool. For more information, see - * Default: 1 - * - * @param maxBackgroundFlushes number of max concurrent flush jobs - * @return the reference to the current option. - * - * @see org.rocksdb.RocksEnv#setBackgroundThreads(int) - * @see org.rocksdb.RocksEnv#setBackgroundThreads(int, int) - * @see #maxBackgroundCompactions() - */ + @Override public Options setMaxBackgroundFlushes(int maxBackgroundFlushes) { assert(isInitialized()); setMaxBackgroundFlushes(nativeHandle_, maxBackgroundFlushes); return this; } - private native void setMaxBackgroundFlushes( - long handle, int maxBackgroundFlushes); - /** - * Returns the maximum size of a info log file. If the current log file - * is larger than this size, a new info log file will be created. - * If 0, all logs will be written to one log file. - * - * @return the maximum size of the info log file. - */ + @Override public long maxLogFileSize() { assert(isInitialized()); return maxLogFileSize(nativeHandle_); } - private native long maxLogFileSize(long handle); - /** - * Specifies the maximum size of a info log file. If the current log file - * is larger than `max_log_file_size`, a new info log file will - * be created. - * If 0, all logs will be written to one log file. - * - * @param maxLogFileSize the maximum size of a info log file. - * @return the reference to the current option. - * @throws RocksDBException - */ + @Override public Options setMaxLogFileSize(long maxLogFileSize) throws RocksDBException { assert(isInitialized()); setMaxLogFileSize(nativeHandle_, maxLogFileSize); return this; } - private native void setMaxLogFileSize(long handle, long maxLogFileSize) - throws RocksDBException; - /** - * Returns the time interval for the info log file to roll (in seconds). - * If specified with non-zero value, log file will be rolled - * if it has been active longer than `log_file_time_to_roll`. - * Default: 0 (disabled) - * - * @return the time interval in seconds. - */ + @Override public long logFileTimeToRoll() { assert(isInitialized()); return logFileTimeToRoll(nativeHandle_); } - private native long logFileTimeToRoll(long handle); - /** - * Specifies the time interval for the info log file to roll (in seconds). - * If specified with non-zero value, log file will be rolled - * if it has been active longer than `log_file_time_to_roll`. - * Default: 0 (disabled) - * - * @param logFileTimeToRoll the time interval in seconds. - * @return the reference to the current option. - * @throws RocksDBException - */ + @Override public Options setLogFileTimeToRoll(long logFileTimeToRoll) throws RocksDBException{ assert(isInitialized()); setLogFileTimeToRoll(nativeHandle_, logFileTimeToRoll); return this; } - private native void setLogFileTimeToRoll( - long handle, long logFileTimeToRoll) throws RocksDBException; - /** - * Returns the maximum number of info log files to be kept. - * Default: 1000 - * - * @return the maximum number of info log files to be kept. - */ + @Override public long keepLogFileNum() { assert(isInitialized()); return keepLogFileNum(nativeHandle_); } - private native long keepLogFileNum(long handle); - /** - * Specifies the maximum number of info log files to be kept. - * Default: 1000 - * - * @param keepLogFileNum the maximum number of info log files to be kept. - * @return the reference to the current option. - * @throws RocksDBException - */ + @Override public Options setKeepLogFileNum(long keepLogFileNum) throws RocksDBException{ assert(isInitialized()); setKeepLogFileNum(nativeHandle_, keepLogFileNum); return this; } - private native void setKeepLogFileNum(long handle, long keepLogFileNum) - throws RocksDBException; - /** - * Manifest file is rolled over on reaching this limit. - * The older manifest file be deleted. - * The default value is MAX_INT so that roll-over does not take place. - * - * @return the size limit of a manifest file. - */ + @Override public long maxManifestFileSize() { assert(isInitialized()); return maxManifestFileSize(nativeHandle_); } - private native long maxManifestFileSize(long handle); - /** - * Manifest file is rolled over on reaching this limit. - * The older manifest file be deleted. - * The default value is MAX_INT so that roll-over does not take place. - * - * @param maxManifestFileSize the size limit of a manifest file. - * @return the reference to the current option. - */ + @Override public Options setMaxManifestFileSize(long maxManifestFileSize) { assert(isInitialized()); setMaxManifestFileSize(nativeHandle_, maxManifestFileSize); return this; } - private native void setMaxManifestFileSize( - long handle, long maxManifestFileSize); - /** - * Number of shards used for table cache. - * - * @return the number of shards used for table cache. - */ + @Override public int tableCacheNumshardbits() { assert(isInitialized()); return tableCacheNumshardbits(nativeHandle_); } - private native int tableCacheNumshardbits(long handle); - /** - * Number of shards used for table cache. - * - * @param tableCacheNumshardbits the number of chards - * @return the reference to the current option. - */ + @Override public Options setTableCacheNumshardbits(int tableCacheNumshardbits) { assert(isInitialized()); setTableCacheNumshardbits(nativeHandle_, tableCacheNumshardbits); return this; } - private native void setTableCacheNumshardbits( - long handle, int tableCacheNumshardbits); - /** - * During data eviction of table's LRU cache, it would be inefficient - * to strictly follow LRU because this piece of memory will not really - * be released unless its refcount falls to zero. Instead, make two - * passes: the first pass will release items with refcount = 1, - * and if not enough space releases after scanning the number of - * elements specified by this parameter, we will remove items in LRU - * order. - * - * @return scan count limit - */ + @Override public int tableCacheRemoveScanCountLimit() { assert(isInitialized()); return tableCacheRemoveScanCountLimit(nativeHandle_); } - private native int tableCacheRemoveScanCountLimit(long handle); - /** - * During data eviction of table's LRU cache, it would be inefficient - * to strictly follow LRU because this piece of memory will not really - * be released unless its refcount falls to zero. Instead, make two - * passes: the first pass will release items with refcount = 1, - * and if not enough space releases after scanning the number of - * elements specified by this parameter, we will remove items in LRU - * order. - * - * @param limit scan count limit - * @return the reference to the current option. - */ + @Override public Options setTableCacheRemoveScanCountLimit(int limit) { assert(isInitialized()); setTableCacheRemoveScanCountLimit(nativeHandle_, limit); return this; } - private native void setTableCacheRemoveScanCountLimit( - long handle, int limit); - /** - * WalTtlSeconds() and walSizeLimitMB() affect how archived logs - * will be deleted. - *
        - *
      1. If both set to 0, logs will be deleted asap and will not get into - * the archive.
      2. - *
      3. If WAL_ttl_seconds is 0 and WAL_size_limit_MB is not 0, - * WAL files will be checked every 10 min and if total size is greater - * then WAL_size_limit_MB, they will be deleted starting with the - * earliest until size_limit is met. All empty files will be deleted.
      4. - *
      5. If WAL_ttl_seconds is not 0 and WAL_size_limit_MB is 0, then - * WAL files will be checked every WAL_ttl_secondsi / 2 and those that - * are older than WAL_ttl_seconds will be deleted.
      6. - *
      7. If both are not 0, WAL files will be checked every 10 min and both - * checks will be performed with ttl being first.
      8. - *
      - * - * @return the wal-ttl seconds - * @see #walSizeLimitMB() - */ + @Override public long walTtlSeconds() { assert(isInitialized()); return walTtlSeconds(nativeHandle_); } - private native long walTtlSeconds(long handle); - /** - * {@link #walTtlSeconds()} and {@link #walSizeLimitMB()} affect how archived logs - * will be deleted. - *
        - *
      1. If both set to 0, logs will be deleted asap and will not get into - * the archive.
      2. - *
      3. If WAL_ttl_seconds is 0 and WAL_size_limit_MB is not 0, - * WAL files will be checked every 10 min and if total size is greater - * then WAL_size_limit_MB, they will be deleted starting with the - * earliest until size_limit is met. All empty files will be deleted.
      4. - *
      5. If WAL_ttl_seconds is not 0 and WAL_size_limit_MB is 0, then - * WAL files will be checked every WAL_ttl_secondsi / 2 and those that - * are older than WAL_ttl_seconds will be deleted.
      6. - *
      7. If both are not 0, WAL files will be checked every 10 min and both - * checks will be performed with ttl being first.
      8. - * - * @param walTtlSeconds the ttl seconds - * @return the reference to the current option. - * @see #setWalSizeLimitMB(long) - */ + @Override public Options setWalTtlSeconds(long walTtlSeconds) { assert(isInitialized()); setWalTtlSeconds(nativeHandle_, walTtlSeconds); return this; } - private native void setWalTtlSeconds(long handle, long walTtlSeconds); - /** - * {@link #walTtlSeconds()} and {@code #walSizeLimitMB()} affect how archived logs - * will be deleted. - *
          - *
        1. If both set to 0, logs will be deleted asap and will not get into - * the archive.
        2. - *
        3. If WAL_ttl_seconds is 0 and WAL_size_limit_MB is not 0, - * WAL files will be checked every 10 min and if total size is greater - * then WAL_size_limit_MB, they will be deleted starting with the - * earliest until size_limit is met. All empty files will be deleted.
        4. - *
        5. If WAL_ttl_seconds is not 0 and WAL_size_limit_MB is 0, then - * WAL files will be checked every WAL_ttl_seconds i / 2 and those that - * are older than WAL_ttl_seconds will be deleted.
        6. - *
        7. If both are not 0, WAL files will be checked every 10 min and both - * checks will be performed with ttl being first.
        8. - *
        - * @return size limit in mega-bytes. - * @see #walSizeLimitMB() - */ + @Override public long walSizeLimitMB() { assert(isInitialized()); return walSizeLimitMB(nativeHandle_); } - private native long walSizeLimitMB(long handle); - /** - * WalTtlSeconds() and walSizeLimitMB() affect how archived logs - * will be deleted. - *
          - *
        1. If both set to 0, logs will be deleted asap and will not get into - * the archive.
        2. - *
        3. If WAL_ttl_seconds is 0 and WAL_size_limit_MB is not 0, - * WAL files will be checked every 10 min and if total size is greater - * then WAL_size_limit_MB, they will be deleted starting with the - * earliest until size_limit is met. All empty files will be deleted.
        4. - *
        5. If WAL_ttl_seconds is not 0 and WAL_size_limit_MB is 0, then - * WAL files will be checked every WAL_ttl_secondsi / 2 and those that - * are older than WAL_ttl_seconds will be deleted.
        6. - *
        7. If both are not 0, WAL files will be checked every 10 min and both - * checks will be performed with ttl being first.
        8. - * - * @param sizeLimitMB size limit in mega-bytes. - * @return the reference to the current option. - * @see #setWalSizeLimitMB(long) - */ + @Override public Options setWalSizeLimitMB(long sizeLimitMB) { assert(isInitialized()); setWalSizeLimitMB(nativeHandle_, sizeLimitMB); return this; } - private native void setWalSizeLimitMB(long handle, long sizeLimitMB); - /** - * Number of bytes to preallocate (via fallocate) the manifest - * files. Default is 4mb, which is reasonable to reduce random IO - * as well as prevent overallocation for mounts that preallocate - * large amounts of data (such as xfs's allocsize option). - * - * @return size in bytes. - */ + @Override public long manifestPreallocationSize() { assert(isInitialized()); return manifestPreallocationSize(nativeHandle_); } - private native long manifestPreallocationSize(long handle); - /** - * Number of bytes to preallocate (via fallocate) the manifest - * files. Default is 4mb, which is reasonable to reduce random IO - * as well as prevent overallocation for mounts that preallocate - * large amounts of data (such as xfs's allocsize option). - * - * @param size the size in byte - * @return the reference to the current option. - * @throws RocksDBException - */ + @Override public Options setManifestPreallocationSize(long size) throws RocksDBException { assert(isInitialized()); setManifestPreallocationSize(nativeHandle_, size); return this; } - private native void setManifestPreallocationSize( - long handle, long size) throws RocksDBException; - /** - * Data being read from file storage may be buffered in the OS - * Default: true - * - * @return if true, then OS buffering is allowed. - */ + @Override public boolean allowOsBuffer() { assert(isInitialized()); return allowOsBuffer(nativeHandle_); } - private native boolean allowOsBuffer(long handle); - /** - * Data being read from file storage may be buffered in the OS - * Default: true - * - * @param allowOsBuffer if true, then OS buffering is allowed. - * @return the reference to the current option. - */ + @Override public Options setAllowOsBuffer(boolean allowOsBuffer) { assert(isInitialized()); setAllowOsBuffer(nativeHandle_, allowOsBuffer); return this; } - private native void setAllowOsBuffer( - long handle, boolean allowOsBuffer); - /** - * Allow the OS to mmap file for reading sst tables. - * Default: false - * - * @return true if mmap reads are allowed. - */ + @Override public boolean allowMmapReads() { assert(isInitialized()); return allowMmapReads(nativeHandle_); } - private native boolean allowMmapReads(long handle); - /** - * Allow the OS to mmap file for reading sst tables. - * Default: false - * - * @param allowMmapReads true if mmap reads are allowed. - * @return the reference to the current option. - */ + @Override public Options setAllowMmapReads(boolean allowMmapReads) { assert(isInitialized()); setAllowMmapReads(nativeHandle_, allowMmapReads); return this; } - private native void setAllowMmapReads( - long handle, boolean allowMmapReads); - /** - * Allow the OS to mmap file for writing. Default: false - * - * @return true if mmap writes are allowed. - */ + @Override public boolean allowMmapWrites() { assert(isInitialized()); return allowMmapWrites(nativeHandle_); } - private native boolean allowMmapWrites(long handle); - /** - * Allow the OS to mmap file for writing. Default: false - * - * @param allowMmapWrites true if mmap writes are allowd. - * @return the reference to the current option. - */ + @Override public Options setAllowMmapWrites(boolean allowMmapWrites) { assert(isInitialized()); setAllowMmapWrites(nativeHandle_, allowMmapWrites); return this; } - private native void setAllowMmapWrites( - long handle, boolean allowMmapWrites); - /** - * Disable child process inherit open files. Default: true - * - * @return true if child process inheriting open files is disabled. - */ + @Override public boolean isFdCloseOnExec() { assert(isInitialized()); return isFdCloseOnExec(nativeHandle_); } - private native boolean isFdCloseOnExec(long handle); - /** - * Disable child process inherit open files. Default: true - * - * @param isFdCloseOnExec true if child process inheriting open - * files is disabled. - * @return the reference to the current option. - */ + @Override public Options setIsFdCloseOnExec(boolean isFdCloseOnExec) { assert(isInitialized()); setIsFdCloseOnExec(nativeHandle_, isFdCloseOnExec); return this; } - private native void setIsFdCloseOnExec( - long handle, boolean isFdCloseOnExec); - /** - * Skip log corruption error on recovery (If client is ok with - * losing most recent changes) - * Default: false - * - * @return true if log corruption errors are skipped during recovery. - */ + @Override public boolean skipLogErrorOnRecovery() { assert(isInitialized()); return skipLogErrorOnRecovery(nativeHandle_); } - private native boolean skipLogErrorOnRecovery(long handle); - /** - * Skip log corruption error on recovery (If client is ok with - * losing most recent changes) - * Default: false - * - * @param skip true if log corruption errors are skipped during recovery. - * @return the reference to the current option. - */ + @Override public Options setSkipLogErrorOnRecovery(boolean skip) { assert(isInitialized()); setSkipLogErrorOnRecovery(nativeHandle_, skip); return this; } - private native void setSkipLogErrorOnRecovery( - long handle, boolean skip); - /** - * If not zero, dump rocksdb.stats to LOG every stats_dump_period_sec - * Default: 3600 (1 hour) - * - * @return time interval in seconds. - */ + @Override public int statsDumpPeriodSec() { assert(isInitialized()); return statsDumpPeriodSec(nativeHandle_); } - private native int statsDumpPeriodSec(long handle); - /** - * if not zero, dump rocksdb.stats to LOG every stats_dump_period_sec - * Default: 3600 (1 hour) - * - * @param statsDumpPeriodSec time interval in seconds. - * @return the reference to the current option. - */ + @Override public Options setStatsDumpPeriodSec(int statsDumpPeriodSec) { assert(isInitialized()); setStatsDumpPeriodSec(nativeHandle_, statsDumpPeriodSec); return this; } - private native void setStatsDumpPeriodSec( - long handle, int statsDumpPeriodSec); - /** - * If set true, will hint the underlying file system that the file - * access pattern is random, when a sst file is opened. - * Default: true - * - * @return true if hinting random access is on. - */ + @Override public boolean adviseRandomOnOpen() { return adviseRandomOnOpen(nativeHandle_); } - private native boolean adviseRandomOnOpen(long handle); - /** - * If set true, will hint the underlying file system that the file - * access pattern is random, when a sst file is opened. - * Default: true - * - * @param adviseRandomOnOpen true if hinting random access is on. - * @return the reference to the current option. - */ + @Override public Options setAdviseRandomOnOpen(boolean adviseRandomOnOpen) { assert(isInitialized()); setAdviseRandomOnOpen(nativeHandle_, adviseRandomOnOpen); return this; } - private native void setAdviseRandomOnOpen( - long handle, boolean adviseRandomOnOpen); - /** - * Use adaptive mutex, which spins in the user space before resorting - * to kernel. This could reduce context switch when the mutex is not - * heavily contended. However, if the mutex is hot, we could end up - * wasting spin time. - * Default: false - * - * @return true if adaptive mutex is used. - */ + @Override public boolean useAdaptiveMutex() { assert(isInitialized()); return useAdaptiveMutex(nativeHandle_); } - private native boolean useAdaptiveMutex(long handle); - /** - * Use adaptive mutex, which spins in the user space before resorting - * to kernel. This could reduce context switch when the mutex is not - * heavily contended. However, if the mutex is hot, we could end up - * wasting spin time. - * Default: false - * - * @param useAdaptiveMutex true if adaptive mutex is used. - * @return the reference to the current option. - */ + @Override public Options setUseAdaptiveMutex(boolean useAdaptiveMutex) { assert(isInitialized()); setUseAdaptiveMutex(nativeHandle_, useAdaptiveMutex); return this; } - private native void setUseAdaptiveMutex( - long handle, boolean useAdaptiveMutex); - /** - * Allows OS to incrementally sync files to disk while they are being - * written, asynchronously, in the background. - * Issue one request for every bytes_per_sync written. 0 turns it off. - * Default: 0 - * - * @return size in bytes - */ + @Override public long bytesPerSync() { return bytesPerSync(nativeHandle_); } - private native long bytesPerSync(long handle); - /** - * Allows OS to incrementally sync files to disk while they are being - * written, asynchronously, in the background. - * Issue one request for every bytes_per_sync written. 0 turns it off. - * Default: 0 - * - * @param bytesPerSync size in bytes - * @return the reference to the current option. - */ + @Override public Options setBytesPerSync(long bytesPerSync) { assert(isInitialized()); setBytesPerSync(nativeHandle_, bytesPerSync); return this; } - private native void setBytesPerSync( - long handle, long bytesPerSync); - /** - * Set the config for mem-table. - * - * @param config the mem-table config. - * @return the instance of the current Options. - * @throws RocksDBException - */ + @Override public Options setMemTableConfig(MemTableConfig config) throws RocksDBException { memTableConfig_ = config; @@ -1180,1119 +586,392 @@ public class Options extends RocksObject { return this; } - /** - * Use to control write rate of flush and compaction. Flush has higher - * priority than compaction. Rate limiting is disabled if nullptr. - * Default: nullptr - * - * @param config rate limiter config. - * @return the instance of the current Options. - * @throws RocksDBException - */ + @Override public Options setRateLimiterConfig(RateLimiterConfig config) { rateLimiterConfig_ = config; setRateLimiter(nativeHandle_, config.newRateLimiterHandle()); return this; } - /** - * Returns the name of the current mem table representation. - * Memtable format can be set using setTableFormatConfig. - * - * @return the name of the currently-used memtable factory. - * @see #setTableFormatConfig(TableFormatConfig) - */ + @Override public String memTableFactoryName() { assert(isInitialized()); return memTableFactoryName(nativeHandle_); } - /** - * Set the config for table format. - * - * @param config the table format config. - * @return the reference of the current Options. - */ + @Override public Options setTableFormatConfig(TableFormatConfig config) { tableFormatConfig_ = config; setTableFactory(nativeHandle_, config.newTableFactoryHandle()); return this; } - /** - * @return the name of the currently used table factory. - */ + @Override public String tableFactoryName() { assert(isInitialized()); return tableFactoryName(nativeHandle_); } - /** - * This prefix-extractor uses the first n bytes of a key as its prefix. - * - * In some hash-based memtable representation such as HashLinkedList - * and HashSkipList, prefixes are used to partition the keys into - * several buckets. Prefix extractor is used to specify how to - * extract the prefix given a key. - * - * @param n use the first n bytes of a key as its prefix. - */ + @Override public Options useFixedLengthPrefixExtractor(int n) { assert(isInitialized()); useFixedLengthPrefixExtractor(nativeHandle_, n); return this; } -/////////////////////////////////////////////////////////////////////// - /** - * Number of keys between restart points for delta encoding of keys. - * This parameter can be changed dynamically. Most clients should - * leave this parameter alone. - * Default: 16 - * - * @return the number of keys between restart points. - */ - public int blockRestartInterval() { - return blockRestartInterval(nativeHandle_); - } - private native int blockRestartInterval(long handle); - - /** - * Number of keys between restart points for delta encoding of keys. - * This parameter can be changed dynamically. Most clients should - * leave this parameter alone. - * Default: 16 - * - * @param blockRestartInterval the number of keys between restart points. - * @return the reference to the current option. - */ - public Options setBlockRestartInterval(int blockRestartInterval) { - setBlockRestartInterval(nativeHandle_, blockRestartInterval); - return this; - } - private native void setBlockRestartInterval( - long handle, int blockRestartInterval); - - /** - * Compress blocks using the specified compression algorithm. This - parameter can be changed dynamically. - * - * Default: SNAPPY_COMPRESSION, which gives lightweight but fast compression. - * - * @return Compression type. - */ + @Override public CompressionType compressionType() { return CompressionType.values()[compressionType(nativeHandle_)]; } - private native byte compressionType(long handle); - /** - * Compress blocks using the specified compression algorithm. This - parameter can be changed dynamically. - * - * Default: SNAPPY_COMPRESSION, which gives lightweight but fast compression. - * - * @param compressionType Compression Type. - * @return the reference to the current option. - */ + @Override public Options setCompressionType(CompressionType compressionType) { setCompressionType(nativeHandle_, compressionType.getValue()); return this; } - private native void setCompressionType(long handle, byte compressionType); - /** - * Compaction style for DB. - * - * @return Compaction style. - */ + @Override public CompactionStyle compactionStyle() { return CompactionStyle.values()[compactionStyle(nativeHandle_)]; } - private native byte compactionStyle(long handle); - /** - * Set compaction style for DB. - * - * Default: LEVEL. - * - * @param compactionStyle Compaction style. - * @return the reference to the current option. - */ + @Override public Options setCompactionStyle(CompactionStyle compactionStyle) { setCompactionStyle(nativeHandle_, compactionStyle.getValue()); return this; } - private native void setCompactionStyle(long handle, byte compactionStyle); - /** - * If level-styled compaction is used, then this number determines - * the total number of levels. - * - * @return the number of levels. - */ + @Override public int numLevels() { return numLevels(nativeHandle_); } - private native int numLevels(long handle); - /** - * Set the number of levels for this database - * If level-styled compaction is used, then this number determines - * the total number of levels. - * - * @param numLevels the number of levels. - * @return the reference to the current option. - */ + @Override public Options setNumLevels(int numLevels) { setNumLevels(nativeHandle_, numLevels); return this; } - private native void setNumLevels( - long handle, int numLevels); - /** - * The number of files in level 0 to trigger compaction from level-0 to - * level-1. A value < 0 means that level-0 compaction will not be - * triggered by number of files at all. - * Default: 4 - * - * @return the number of files in level 0 to trigger compaction. - */ + @Override public int levelZeroFileNumCompactionTrigger() { return levelZeroFileNumCompactionTrigger(nativeHandle_); } - private native int levelZeroFileNumCompactionTrigger(long handle); - /** - * Number of files to trigger level-0 compaction. A value <0 means that - * level-0 compaction will not be triggered by number of files at all. - * Default: 4 - * - * @param numFiles the number of files in level-0 to trigger compaction. - * @return the reference to the current option. - */ + @Override public Options setLevelZeroFileNumCompactionTrigger( int numFiles) { setLevelZeroFileNumCompactionTrigger( nativeHandle_, numFiles); return this; } - private native void setLevelZeroFileNumCompactionTrigger( - long handle, int numFiles); - /** - * Soft limit on the number of level-0 files. We start slowing down writes - * at this point. A value < 0 means that no writing slow down will be - * triggered by number of files in level-0. - * - * @return the soft limit on the number of level-0 files. - */ + @Override public int levelZeroSlowdownWritesTrigger() { return levelZeroSlowdownWritesTrigger(nativeHandle_); } - private native int levelZeroSlowdownWritesTrigger(long handle); - /** - * Soft limit on number of level-0 files. We start slowing down writes at this - * point. A value <0 means that no writing slow down will be triggered by - * number of files in level-0. - * - * @param numFiles soft limit on number of level-0 files. - * @return the reference to the current option. - */ + @Override public Options setLevelZeroSlowdownWritesTrigger( int numFiles) { setLevelZeroSlowdownWritesTrigger(nativeHandle_, numFiles); return this; } - private native void setLevelZeroSlowdownWritesTrigger( - long handle, int numFiles); - /** - * Maximum number of level-0 files. We stop writes at this point. - * - * @return the hard limit of the number of level-0 file. - */ + @Override public int levelZeroStopWritesTrigger() { return levelZeroStopWritesTrigger(nativeHandle_); } - private native int levelZeroStopWritesTrigger(long handle); - /** - * Maximum number of level-0 files. We stop writes at this point. - * - * @param numFiles the hard limit of the number of level-0 files. - * @return the reference to the current option. - */ + @Override public Options setLevelZeroStopWritesTrigger(int numFiles) { setLevelZeroStopWritesTrigger(nativeHandle_, numFiles); return this; } - private native void setLevelZeroStopWritesTrigger( - long handle, int numFiles); - /** - * The highest level to which a new compacted memtable is pushed if it - * does not create overlap. We try to push to level 2 to avoid the - * relatively expensive level 0=>1 compactions and to avoid some - * expensive manifest file operations. We do not push all the way to - * the largest level since that can generate a lot of wasted disk - * space if the same key space is being repeatedly overwritten. - * - * @return the highest level where a new compacted memtable will be pushed. - */ + @Override public int maxMemCompactionLevel() { return maxMemCompactionLevel(nativeHandle_); } - private native int maxMemCompactionLevel(long handle); - /** - * The highest level to which a new compacted memtable is pushed if it - * does not create overlap. We try to push to level 2 to avoid the - * relatively expensive level 0=>1 compactions and to avoid some - * expensive manifest file operations. We do not push all the way to - * the largest level since that can generate a lot of wasted disk - * space if the same key space is being repeatedly overwritten. - * - * @param maxMemCompactionLevel the highest level to which a new compacted - * mem-table will be pushed. - * @return the reference to the current option. - */ + @Override public Options setMaxMemCompactionLevel(int maxMemCompactionLevel) { setMaxMemCompactionLevel(nativeHandle_, maxMemCompactionLevel); return this; } - private native void setMaxMemCompactionLevel( - long handle, int maxMemCompactionLevel); - /** - * The target file size for compaction. - * This targetFileSizeBase determines a level-1 file size. - * Target file size for level L can be calculated by - * targetFileSizeBase * (targetFileSizeMultiplier ^ (L-1)) - * For example, if targetFileSizeBase is 2MB and - * target_file_size_multiplier is 10, then each file on level-1 will - * be 2MB, and each file on level 2 will be 20MB, - * and each file on level-3 will be 200MB. - * by default targetFileSizeBase is 2MB. - * - * @return the target size of a level-0 file. - * - * @see #targetFileSizeMultiplier() - */ - public int targetFileSizeBase() { + @Override + public long targetFileSizeBase() { return targetFileSizeBase(nativeHandle_); } - private native int targetFileSizeBase(long handle); - /** - * The target file size for compaction. - * This targetFileSizeBase determines a level-1 file size. - * Target file size for level L can be calculated by - * targetFileSizeBase * (targetFileSizeMultiplier ^ (L-1)) - * For example, if targetFileSizeBase is 2MB and - * target_file_size_multiplier is 10, then each file on level-1 will - * be 2MB, and each file on level 2 will be 20MB, - * and each file on level-3 will be 200MB. - * by default targetFileSizeBase is 2MB. - * - * @param targetFileSizeBase the target size of a level-0 file. - * @return the reference to the current option. - * - * @see #setTargetFileSizeMultiplier(int) - */ - public Options setTargetFileSizeBase(int targetFileSizeBase) { + @Override + public Options setTargetFileSizeBase(long targetFileSizeBase) { setTargetFileSizeBase(nativeHandle_, targetFileSizeBase); return this; } - private native void setTargetFileSizeBase( - long handle, int targetFileSizeBase); - /** - * targetFileSizeMultiplier defines the size ratio between a - * level-(L+1) file and level-L file. - * By default targetFileSizeMultiplier is 1, meaning - * files in different levels have the same target. - * - * @return the size ratio between a level-(L+1) file and level-L file. - */ + @Override public int targetFileSizeMultiplier() { return targetFileSizeMultiplier(nativeHandle_); } - private native int targetFileSizeMultiplier(long handle); - /** - * targetFileSizeMultiplier defines the size ratio between a - * level-L file and level-(L+1) file. - * By default target_file_size_multiplier is 1, meaning - * files in different levels have the same target. - * - * @param multiplier the size ratio between a level-(L+1) file - * and level-L file. - * @return the reference to the current option. - */ + @Override public Options setTargetFileSizeMultiplier(int multiplier) { setTargetFileSizeMultiplier(nativeHandle_, multiplier); return this; } - private native void setTargetFileSizeMultiplier( - long handle, int multiplier); - /** - * The upper-bound of the total size of level-1 files in bytes. - * Maximum number of bytes for level L can be calculated as - * (maxBytesForLevelBase) * (maxBytesForLevelMultiplier ^ (L-1)) - * For example, if maxBytesForLevelBase is 20MB, and if - * max_bytes_for_level_multiplier is 10, total data size for level-1 - * will be 20MB, total file size for level-2 will be 200MB, - * and total file size for level-3 will be 2GB. - * by default 'maxBytesForLevelBase' is 10MB. - * - * @return the upper-bound of the total size of leve-1 files in bytes. - * @see #maxBytesForLevelMultiplier() - */ + @Override public long maxBytesForLevelBase() { return maxBytesForLevelBase(nativeHandle_); } - private native long maxBytesForLevelBase(long handle); - /** - * The upper-bound of the total size of level-1 files in bytes. - * Maximum number of bytes for level L can be calculated as - * (maxBytesForLevelBase) * (maxBytesForLevelMultiplier ^ (L-1)) - * For example, if maxBytesForLevelBase is 20MB, and if - * max_bytes_for_level_multiplier is 10, total data size for level-1 - * will be 20MB, total file size for level-2 will be 200MB, - * and total file size for level-3 will be 2GB. - * by default 'maxBytesForLevelBase' is 10MB. - * - * @return the reference to the current option. - * @see #setMaxBytesForLevelMultiplier(int) - */ + @Override public Options setMaxBytesForLevelBase(long maxBytesForLevelBase) { setMaxBytesForLevelBase(nativeHandle_, maxBytesForLevelBase); return this; } - private native void setMaxBytesForLevelBase( - long handle, long maxBytesForLevelBase); - /** - * The ratio between the total size of level-(L+1) files and the total - * size of level-L files for all L. - * DEFAULT: 10 - * - * @return the ratio between the total size of level-(L+1) files and - * the total size of level-L files for all L. - * @see #maxBytesForLevelBase() - */ + @Override public int maxBytesForLevelMultiplier() { return maxBytesForLevelMultiplier(nativeHandle_); } - private native int maxBytesForLevelMultiplier(long handle); - /** - * The ratio between the total size of level-(L+1) files and the total - * size of level-L files for all L. - * DEFAULT: 10 - * - * @param multiplier the ratio between the total size of level-(L+1) - * files and the total size of level-L files for all L. - * @return the reference to the current option. - * @see #setMaxBytesForLevelBase(long) - */ + @Override public Options setMaxBytesForLevelMultiplier(int multiplier) { setMaxBytesForLevelMultiplier(nativeHandle_, multiplier); return this; } - private native void setMaxBytesForLevelMultiplier( - long handle, int multiplier); - /** - * Maximum number of bytes in all compacted files. We avoid expanding - * the lower level file set of a compaction if it would make the - * total compaction cover more than - * (expanded_compaction_factor * targetFileSizeLevel()) many bytes. - * - * @return the maximum number of bytes in all compacted files. - * @see #sourceCompactionFactor() - */ + @Override public int expandedCompactionFactor() { return expandedCompactionFactor(nativeHandle_); } - private native int expandedCompactionFactor(long handle); - /** - * Maximum number of bytes in all compacted files. We avoid expanding - * the lower level file set of a compaction if it would make the - * total compaction cover more than - * (expanded_compaction_factor * targetFileSizeLevel()) many bytes. - * - * @param expandedCompactionFactor the maximum number of bytes in all - * compacted files. - * @return the reference to the current option. - * @see #setSourceCompactionFactor(int) - */ + @Override public Options setExpandedCompactionFactor(int expandedCompactionFactor) { setExpandedCompactionFactor(nativeHandle_, expandedCompactionFactor); return this; } - private native void setExpandedCompactionFactor( - long handle, int expandedCompactionFactor); - /** - * Maximum number of bytes in all source files to be compacted in a - * single compaction run. We avoid picking too many files in the - * source level so that we do not exceed the total source bytes - * for compaction to exceed - * (source_compaction_factor * targetFileSizeLevel()) many bytes. - * Default:1, i.e. pick maxfilesize amount of data as the source of - * a compaction. - * - * @return the maximum number of bytes in all source files to be compactedo. - * @see #expandedCompactionFactor() - */ + @Override public int sourceCompactionFactor() { return sourceCompactionFactor(nativeHandle_); } - private native int sourceCompactionFactor(long handle); - /** - * Maximum number of bytes in all source files to be compacted in a - * single compaction run. We avoid picking too many files in the - * source level so that we do not exceed the total source bytes - * for compaction to exceed - * (source_compaction_factor * targetFileSizeLevel()) many bytes. - * Default:1, i.e. pick maxfilesize amount of data as the source of - * a compaction. - * - * @param sourceCompactionFactor the maximum number of bytes in all - * source files to be compacted in a single compaction run. - * @return the reference to the current option. - * @see #setExpandedCompactionFactor(int) - */ + @Override public Options setSourceCompactionFactor(int sourceCompactionFactor) { setSourceCompactionFactor(nativeHandle_, sourceCompactionFactor); return this; } - private native void setSourceCompactionFactor( - long handle, int sourceCompactionFactor); - /** - * Control maximum bytes of overlaps in grandparent (i.e., level+2) before we - * stop building a single file in a level->level+1 compaction. - * - * @return maximum bytes of overlaps in "grandparent" level. - */ + @Override public int maxGrandparentOverlapFactor() { return maxGrandparentOverlapFactor(nativeHandle_); } - private native int maxGrandparentOverlapFactor(long handle); - /** - * Control maximum bytes of overlaps in grandparent (i.e., level+2) before we - * stop building a single file in a level->level+1 compaction. - * - * @param maxGrandparentOverlapFactor maximum bytes of overlaps in - * "grandparent" level. - * @return the reference to the current option. - */ + @Override public Options setMaxGrandparentOverlapFactor( int maxGrandparentOverlapFactor) { setMaxGrandparentOverlapFactor(nativeHandle_, maxGrandparentOverlapFactor); return this; } - private native void setMaxGrandparentOverlapFactor( - long handle, int maxGrandparentOverlapFactor); - /** - * Puts are delayed 0-1 ms when any level has a compaction score that exceeds - * soft_rate_limit. This is ignored when == 0.0. - * CONSTRAINT: soft_rate_limit <= hard_rate_limit. If this constraint does not - * hold, RocksDB will set soft_rate_limit = hard_rate_limit - * Default: 0 (disabled) - * - * @return soft-rate-limit for put delay. - */ + @Override public double softRateLimit() { return softRateLimit(nativeHandle_); } - private native double softRateLimit(long handle); - /** - * Puts are delayed 0-1 ms when any level has a compaction score that exceeds - * soft_rate_limit. This is ignored when == 0.0. - * CONSTRAINT: soft_rate_limit <= hard_rate_limit. If this constraint does not - * hold, RocksDB will set soft_rate_limit = hard_rate_limit - * Default: 0 (disabled) - * - * @param softRateLimit the soft-rate-limit of a compaction score - * for put delay. - * @return the reference to the current option. - */ + @Override public Options setSoftRateLimit(double softRateLimit) { setSoftRateLimit(nativeHandle_, softRateLimit); return this; } - private native void setSoftRateLimit( - long handle, double softRateLimit); - /** - * Puts are delayed 1ms at a time when any level has a compaction score that - * exceeds hard_rate_limit. This is ignored when <= 1.0. - * Default: 0 (disabled) - * - * @return the hard-rate-limit of a compaction score for put delay. - */ + @Override public double hardRateLimit() { return hardRateLimit(nativeHandle_); } - private native double hardRateLimit(long handle); - /** - * Puts are delayed 1ms at a time when any level has a compaction score that - * exceeds hard_rate_limit. This is ignored when <= 1.0. - * Default: 0 (disabled) - * - * @param hardRateLimit the hard-rate-limit of a compaction score for put - * delay. - * @return the reference to the current option. - */ + @Override public Options setHardRateLimit(double hardRateLimit) { setHardRateLimit(nativeHandle_, hardRateLimit); return this; } - private native void setHardRateLimit( - long handle, double hardRateLimit); - /** - * The maximum time interval a put will be stalled when hard_rate_limit - * is enforced. If 0, then there is no limit. - * Default: 1000 - * - * @return the maximum time interval a put will be stalled when - * hard_rate_limit is enforced. - */ + @Override public int rateLimitDelayMaxMilliseconds() { return rateLimitDelayMaxMilliseconds(nativeHandle_); } - private native int rateLimitDelayMaxMilliseconds(long handle); - /** - * The maximum time interval a put will be stalled when hard_rate_limit - * is enforced. If 0, then there is no limit. - * Default: 1000 - * - * @param rateLimitDelayMaxMilliseconds the maximum time interval a put - * will be stalled. - * @return the reference to the current option. - */ + @Override public Options setRateLimitDelayMaxMilliseconds( int rateLimitDelayMaxMilliseconds) { setRateLimitDelayMaxMilliseconds( nativeHandle_, rateLimitDelayMaxMilliseconds); return this; } - private native void setRateLimitDelayMaxMilliseconds( - long handle, int rateLimitDelayMaxMilliseconds); - /** - * The size of one block in arena memory allocation. - * If <= 0, a proper value is automatically calculated (usually 1/10 of - * writer_buffer_size). - * - * There are two additonal restriction of the The specified size: - * (1) size should be in the range of [4096, 2 << 30] and - * (2) be the multiple of the CPU word (which helps with the memory - * alignment). - * - * We'll automatically check and adjust the size number to make sure it - * conforms to the restrictions. - * Default: 0 - * - * @return the size of an arena block - */ + @Override public long arenaBlockSize() { return arenaBlockSize(nativeHandle_); } - private native long arenaBlockSize(long handle); - /** - * The size of one block in arena memory allocation. - * If <= 0, a proper value is automatically calculated (usually 1/10 of - * writer_buffer_size). - * - * There are two additonal restriction of the The specified size: - * (1) size should be in the range of [4096, 2 << 30] and - * (2) be the multiple of the CPU word (which helps with the memory - * alignment). - * - * We'll automatically check and adjust the size number to make sure it - * conforms to the restrictions. - * Default: 0 - * - * @param arenaBlockSize the size of an arena block - * @return the reference to the current option. - * @throws RocksDBException - */ + @Override public Options setArenaBlockSize(long arenaBlockSize) throws RocksDBException { setArenaBlockSize(nativeHandle_, arenaBlockSize); return this; } - private native void setArenaBlockSize( - long handle, long arenaBlockSize) throws RocksDBException; - /** - * Disable automatic compactions. Manual compactions can still - * be issued on this column family - * - * @return true if auto-compactions are disabled. - */ + @Override public boolean disableAutoCompactions() { return disableAutoCompactions(nativeHandle_); } - private native boolean disableAutoCompactions(long handle); - /** - * Disable automatic compactions. Manual compactions can still - * be issued on this column family - * - * @param disableAutoCompactions true if auto-compactions are disabled. - * @return the reference to the current option. - */ + @Override public Options setDisableAutoCompactions(boolean disableAutoCompactions) { setDisableAutoCompactions(nativeHandle_, disableAutoCompactions); return this; } - private native void setDisableAutoCompactions( - long handle, boolean disableAutoCompactions); - /** - * Purge duplicate/deleted keys when a memtable is flushed to storage. - * Default: true - * - * @return true if purging keys is disabled. - */ + @Override public boolean purgeRedundantKvsWhileFlush() { return purgeRedundantKvsWhileFlush(nativeHandle_); } - private native boolean purgeRedundantKvsWhileFlush(long handle); - /** - * Purge duplicate/deleted keys when a memtable is flushed to storage. - * Default: true - * - * @param purgeRedundantKvsWhileFlush true if purging keys is disabled. - * @return the reference to the current option. - */ + @Override public Options setPurgeRedundantKvsWhileFlush( boolean purgeRedundantKvsWhileFlush) { setPurgeRedundantKvsWhileFlush( nativeHandle_, purgeRedundantKvsWhileFlush); return this; } - private native void setPurgeRedundantKvsWhileFlush( - long handle, boolean purgeRedundantKvsWhileFlush); - /** - * If true, compaction will verify checksum on every read that happens - * as part of compaction - * Default: true - * - * @return true if compaction verifies checksum on every read. - */ + @Override public boolean verifyChecksumsInCompaction() { return verifyChecksumsInCompaction(nativeHandle_); } - private native boolean verifyChecksumsInCompaction(long handle); - /** - * If true, compaction will verify checksum on every read that happens - * as part of compaction - * Default: true - * - * @param verifyChecksumsInCompaction true if compaction verifies - * checksum on every read. - * @return the reference to the current option. - */ + @Override public Options setVerifyChecksumsInCompaction( boolean verifyChecksumsInCompaction) { setVerifyChecksumsInCompaction( nativeHandle_, verifyChecksumsInCompaction); return this; } - private native void setVerifyChecksumsInCompaction( - long handle, boolean verifyChecksumsInCompaction); - /** - * Use KeyMayExist API to filter deletes when this is true. - * If KeyMayExist returns false, i.e. the key definitely does not exist, then - * the delete is a noop. KeyMayExist only incurs in-memory look up. - * This optimization avoids writing the delete to storage when appropriate. - * Default: false - * - * @return true if filter-deletes behavior is on. - */ + @Override public boolean filterDeletes() { return filterDeletes(nativeHandle_); } - private native boolean filterDeletes(long handle); - /** - * Use KeyMayExist API to filter deletes when this is true. - * If KeyMayExist returns false, i.e. the key definitely does not exist, then - * the delete is a noop. KeyMayExist only incurs in-memory look up. - * This optimization avoids writing the delete to storage when appropriate. - * Default: false - * - * @param filterDeletes true if filter-deletes behavior is on. - * @return the reference to the current option. - */ + @Override public Options setFilterDeletes(boolean filterDeletes) { setFilterDeletes(nativeHandle_, filterDeletes); return this; } - private native void setFilterDeletes( - long handle, boolean filterDeletes); - /** - * An iteration->Next() sequentially skips over keys with the same - * user-key unless this option is set. This number specifies the number - * of keys (with the same userkey) that will be sequentially - * skipped before a reseek is issued. - * Default: 8 - * - * @return the number of keys could be skipped in a iteration. - */ + @Override public long maxSequentialSkipInIterations() { return maxSequentialSkipInIterations(nativeHandle_); } - private native long maxSequentialSkipInIterations(long handle); - /** - * An iteration->Next() sequentially skips over keys with the same - * user-key unless this option is set. This number specifies the number - * of keys (with the same userkey) that will be sequentially - * skipped before a reseek is issued. - * Default: 8 - * - * @param maxSequentialSkipInIterations the number of keys could - * be skipped in a iteration. - * @return the reference to the current option. - */ + @Override public Options setMaxSequentialSkipInIterations(long maxSequentialSkipInIterations) { setMaxSequentialSkipInIterations(nativeHandle_, maxSequentialSkipInIterations); return this; } - private native void setMaxSequentialSkipInIterations( - long handle, long maxSequentialSkipInIterations); - /** - * Allows thread-safe inplace updates. - * If inplace_callback function is not set, - * Put(key, new_value) will update inplace the existing_value iff - * * key exists in current memtable - * * new sizeof(new_value) <= sizeof(existing_value) - * * existing_value for that key is a put i.e. kTypeValue - * If inplace_callback function is set, check doc for inplace_callback. - * Default: false. - * - * @return true if thread-safe inplace updates are allowed. - */ + @Override public boolean inplaceUpdateSupport() { return inplaceUpdateSupport(nativeHandle_); } - private native boolean inplaceUpdateSupport(long handle); - /** - * Allows thread-safe inplace updates. - * If inplace_callback function is not set, - * Put(key, new_value) will update inplace the existing_value iff - * * key exists in current memtable - * * new sizeof(new_value) <= sizeof(existing_value) - * * existing_value for that key is a put i.e. kTypeValue - * If inplace_callback function is set, check doc for inplace_callback. - * Default: false. - * - * @param inplaceUpdateSupport true if thread-safe inplace updates - * are allowed. - * @return the reference to the current option. - */ + @Override public Options setInplaceUpdateSupport(boolean inplaceUpdateSupport) { setInplaceUpdateSupport(nativeHandle_, inplaceUpdateSupport); return this; } - private native void setInplaceUpdateSupport( - long handle, boolean inplaceUpdateSupport); - /** - * Number of locks used for inplace update - * Default: 10000, if inplace_update_support = true, else 0. - * - * @return the number of locks used for inplace update. - */ + @Override public long inplaceUpdateNumLocks() { return inplaceUpdateNumLocks(nativeHandle_); } - private native long inplaceUpdateNumLocks(long handle); - /** - * Number of locks used for inplace update - * Default: 10000, if inplace_update_support = true, else 0. - * - * @param inplaceUpdateNumLocks the number of locks used for - * inplace updates. - * @return the reference to the current option. - * @throws RocksDBException - */ + @Override public Options setInplaceUpdateNumLocks(long inplaceUpdateNumLocks) throws RocksDBException { setInplaceUpdateNumLocks(nativeHandle_, inplaceUpdateNumLocks); return this; } - private native void setInplaceUpdateNumLocks( - long handle, long inplaceUpdateNumLocks) throws RocksDBException; - /** - * Returns the number of bits used in the prefix bloom filter. - * - * This value will be used only when a prefix-extractor is specified. - * - * @return the number of bloom-bits. - * @see #useFixedLengthPrefixExtractor(int) - */ + @Override public int memtablePrefixBloomBits() { return memtablePrefixBloomBits(nativeHandle_); } - private native int memtablePrefixBloomBits(long handle); - /** - * Sets the number of bits used in the prefix bloom filter. - * - * This value will be used only when a prefix-extractor is specified. - * - * @param memtablePrefixBloomBits the number of bits used in the - * prefix bloom filter. - * @return the reference to the current option. - */ + @Override public Options setMemtablePrefixBloomBits(int memtablePrefixBloomBits) { setMemtablePrefixBloomBits(nativeHandle_, memtablePrefixBloomBits); return this; } - private native void setMemtablePrefixBloomBits( - long handle, int memtablePrefixBloomBits); - /** - * The number of hash probes per key used in the mem-table. - * - * @return the number of hash probes per key. - */ + @Override public int memtablePrefixBloomProbes() { return memtablePrefixBloomProbes(nativeHandle_); } - private native int memtablePrefixBloomProbes(long handle); - /** - * The number of hash probes per key used in the mem-table. - * - * @param memtablePrefixBloomProbes the number of hash probes per key. - * @return the reference to the current option. - */ + @Override public Options setMemtablePrefixBloomProbes(int memtablePrefixBloomProbes) { setMemtablePrefixBloomProbes(nativeHandle_, memtablePrefixBloomProbes); return this; } - private native void setMemtablePrefixBloomProbes( - long handle, int memtablePrefixBloomProbes); - /** - * Control locality of bloom filter probes to improve cache miss rate. - * This option only applies to memtable prefix bloom and plaintable - * prefix bloom. It essentially limits the max number of cache lines each - * bloom filter check can touch. - * This optimization is turned off when set to 0. The number should never - * be greater than number of probes. This option can boost performance - * for in-memory workload but should use with care since it can cause - * higher false positive rate. - * Default: 0 - * - * @return the level of locality of bloom-filter probes. - * @see #setMemtablePrefixBloomProbes(int) - */ + @Override public int bloomLocality() { return bloomLocality(nativeHandle_); } - private native int bloomLocality(long handle); - /** - * Control locality of bloom filter probes to improve cache miss rate. - * This option only applies to memtable prefix bloom and plaintable - * prefix bloom. It essentially limits the max number of cache lines each - * bloom filter check can touch. - * This optimization is turned off when set to 0. The number should never - * be greater than number of probes. This option can boost performance - * for in-memory workload but should use with care since it can cause - * higher false positive rate. - * Default: 0 - * - * @param bloomLocality the level of locality of bloom-filter probes. - * @return the reference to the current option. - */ + @Override public Options setBloomLocality(int bloomLocality) { setBloomLocality(nativeHandle_, bloomLocality); return this; } - private native void setBloomLocality( - long handle, int bloomLocality); - /** - * Maximum number of successive merge operations on a key in the memtable. - * - * When a merge operation is added to the memtable and the maximum number of - * successive merges is reached, the value of the key will be calculated and - * inserted into the memtable instead of the merge operation. This will - * ensure that there are never more than max_successive_merges merge - * operations in the memtable. - * - * Default: 0 (disabled) - * - * @return the maximum number of successive merges. - */ + @Override public long maxSuccessiveMerges() { return maxSuccessiveMerges(nativeHandle_); } - private native long maxSuccessiveMerges(long handle); - /** - * Maximum number of successive merge operations on a key in the memtable. - * - * When a merge operation is added to the memtable and the maximum number of - * successive merges is reached, the value of the key will be calculated and - * inserted into the memtable instead of the merge operation. This will - * ensure that there are never more than max_successive_merges merge - * operations in the memtable. - * - * Default: 0 (disabled) - * - * @param maxSuccessiveMerges the maximum number of successive merges. - * @return the reference to the current option. - * @throws RocksDBException - */ + @Override public Options setMaxSuccessiveMerges(long maxSuccessiveMerges) throws RocksDBException { setMaxSuccessiveMerges(nativeHandle_, maxSuccessiveMerges); return this; } - private native void setMaxSuccessiveMerges( - long handle, long maxSuccessiveMerges) throws RocksDBException; - /** - * The minimum number of write buffers that will be merged together - * before writing to storage. If set to 1, then - * all write buffers are fushed to L0 as individual files and this increases - * read amplification because a get request has to check in all of these - * files. Also, an in-memory merge may result in writing lesser - * data to storage if there are duplicate records in each of these - * individual write buffers. Default: 1 - * - * @return the minimum number of write buffers that will be merged together. - */ + @Override public int minWriteBufferNumberToMerge() { return minWriteBufferNumberToMerge(nativeHandle_); } - private native int minWriteBufferNumberToMerge(long handle); - /** - * The minimum number of write buffers that will be merged together - * before writing to storage. If set to 1, then - * all write buffers are fushed to L0 as individual files and this increases - * read amplification because a get request has to check in all of these - * files. Also, an in-memory merge may result in writing lesser - * data to storage if there are duplicate records in each of these - * individual write buffers. Default: 1 - * - * @param minWriteBufferNumberToMerge the minimum number of write buffers - * that will be merged together. - * @return the reference to the current option. - */ + @Override public Options setMinWriteBufferNumberToMerge(int minWriteBufferNumberToMerge) { setMinWriteBufferNumberToMerge(nativeHandle_, minWriteBufferNumberToMerge); return this; } - private native void setMinWriteBufferNumberToMerge( - long handle, int minWriteBufferNumberToMerge); - /** - * The number of partial merge operands to accumulate before partial - * merge will be performed. Partial merge will not be called - * if the list of values to merge is less than min_partial_merge_operands. - * - * If min_partial_merge_operands < 2, then it will be treated as 2. - * - * Default: 2 - * - * @return min partial merge operands - */ + @Override public int minPartialMergeOperands() { return minPartialMergeOperands(nativeHandle_); } - private native int minPartialMergeOperands(long handle); - /** - * The number of partial merge operands to accumulate before partial - * merge will be performed. Partial merge will not be called - * if the list of values to merge is less than min_partial_merge_operands. - * - * If min_partial_merge_operands < 2, then it will be treated as 2. - * - * Default: 2 - * - * @param minPartialMergeOperands - * @return the reference to the current option. - */ + @Override public Options setMinPartialMergeOperands(int minPartialMergeOperands) { setMinPartialMergeOperands(nativeHandle_, minPartialMergeOperands); return this; } - private native void setMinPartialMergeOperands( - long handle, int minPartialMergeOperands); - - /** - * Set the merge operator to be used for merging two merge operands - * of the same key. The merge function is invoked during - * compaction and at lookup time, if multiple key/value pairs belonging - * to the same key are found in the database. - * - * @param name the name of the merge function, as defined by - * the MergeOperators factory (see utilities/MergeOperators.h) - * The merge function is specified by name and must be one of the - * standard merge operators provided by RocksDB. The available - * operators are "put", "uint64add", "stringappend" and "stringappendtest". - * @return the reference to the current option. - */ - public Options setMergeOperatorName(String name) { - setMergeOperatorName(nativeHandle_, name); - return this; - } - private native void setMergeOperatorName( - long handle, String name); - - /** - * Set the merge operator to be used for merging two different key/value - * pairs that share the same key. The merge function is invoked during - * compaction and at lookup time, if multiple key/value pairs belonging - * to the same key are found in the database. - * - * @param a {@link MergeOperator} object - * @return the reference to the current option. - */ - public Options setMergeOperator(MergeOperator mergeOperator) { - setMergeOperator(nativeHandle_, mergeOperator.newMergeOperatorHandle()); - return this; - } - private native void setMergeOperator( - long handle, long mergeOperatorHandle); /** * Release the memory allocated for the current instance @@ -2303,45 +982,223 @@ public class Options extends RocksObject { disposeInternal(nativeHandle_); } - static final int DEFAULT_PLAIN_TABLE_BLOOM_BITS_PER_KEY = 10; - static final double DEFAULT_PLAIN_TABLE_HASH_TABLE_RATIO = 0.75; - static final int DEFAULT_PLAIN_TABLE_INDEX_SPARSENESS = 16; - private native void newOptions(); + private native void newOptions(long dbOptHandle, + long cfOptHandle); private native void disposeInternal(long handle); + private native void setEnv(long optHandle, long envHandle); + private native long getEnvHandle(long handle); + private native void prepareForBulkLoad(long handle); + + // DB native handles private native void setCreateIfMissing(long handle, boolean flag); 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 void setErrorIfExists(long handle, boolean errorIfExists); + private native boolean errorIfExists(long handle); + private native void setParanoidChecks( + long handle, boolean paranoidChecks); + private native boolean paranoidChecks(long handle); + private native void setRateLimiter(long handle, + long rateLimiterHandle); + private native void setMaxOpenFiles(long handle, int maxOpenFiles); + private native int maxOpenFiles(long handle); + private native void setMaxTotalWalSize(long handle, + long maxTotalWalSize); + private native long maxTotalWalSize(long handle); + private native void createStatistics(long optHandle); + private native long statisticsPtr(long optHandle); + private native void setDisableDataSync(long handle, boolean disableDataSync); + private native boolean disableDataSync(long handle); + private native boolean useFsync(long handle); + private native void setUseFsync(long handle, boolean useFsync); + private native void setDbLogDir(long handle, String dbLogDir); + private native String dbLogDir(long handle); + private native void setWalDir(long handle, String walDir); + private native String walDir(long handle); + private native void setDeleteObsoleteFilesPeriodMicros( + long handle, long micros); + private native long deleteObsoleteFilesPeriodMicros(long handle); + private native void setMaxBackgroundCompactions( + long handle, int maxBackgroundCompactions); + private native int maxBackgroundCompactions(long handle); + private native void setMaxBackgroundFlushes( + long handle, int maxBackgroundFlushes); + private native int maxBackgroundFlushes(long handle); + private native void setMaxLogFileSize(long handle, long maxLogFileSize) + throws RocksDBException; + private native long maxLogFileSize(long handle); + private native void setLogFileTimeToRoll( + long handle, long logFileTimeToRoll) throws RocksDBException; + private native long logFileTimeToRoll(long handle); + private native void setKeepLogFileNum(long handle, long keepLogFileNum) + throws RocksDBException; + private native long keepLogFileNum(long handle); + private native void setMaxManifestFileSize( + long handle, long maxManifestFileSize); + private native long maxManifestFileSize(long handle); + private native void setTableCacheNumshardbits( + long handle, int tableCacheNumshardbits); + private native int tableCacheNumshardbits(long handle); + private native void setTableCacheRemoveScanCountLimit( + long handle, int limit); + private native int tableCacheRemoveScanCountLimit(long handle); + private native void setWalTtlSeconds(long handle, long walTtlSeconds); + private native long walTtlSeconds(long handle); + private native void setWalSizeLimitMB(long handle, long sizeLimitMB); + private native long walSizeLimitMB(long handle); + private native void setManifestPreallocationSize( + long handle, long size) throws RocksDBException; + private native long manifestPreallocationSize(long handle); + private native void setAllowOsBuffer( + long handle, boolean allowOsBuffer); + private native boolean allowOsBuffer(long handle); + private native void setAllowMmapReads( + long handle, boolean allowMmapReads); + private native boolean allowMmapReads(long handle); + private native void setAllowMmapWrites( + long handle, boolean allowMmapWrites); + private native boolean allowMmapWrites(long handle); + private native void setIsFdCloseOnExec( + long handle, boolean isFdCloseOnExec); + private native boolean isFdCloseOnExec(long handle); + private native void setSkipLogErrorOnRecovery( + long handle, boolean skip); + private native boolean skipLogErrorOnRecovery(long handle); + private native void setStatsDumpPeriodSec( + long handle, int statsDumpPeriodSec); + private native int statsDumpPeriodSec(long handle); + private native void setAdviseRandomOnOpen( + long handle, boolean adviseRandomOnOpen); + private native boolean adviseRandomOnOpen(long handle); + private native void setUseAdaptiveMutex( + long handle, boolean useAdaptiveMutex); + private native boolean useAdaptiveMutex(long handle); + private native void setBytesPerSync( + long handle, long bytesPerSync); + private native long bytesPerSync(long handle); + // CF native handles + private native void optimizeForPointLookup(long handle, + long blockCacheSizeMb); + private native void optimizeLevelStyleCompaction(long handle, + long memtableMemoryBudget); + private native void optimizeUniversalStyleCompaction(long handle, + long memtableMemoryBudget); + private native void setComparatorHandle(long handle, int builtinComparator); + private native void setComparatorHandle(long optHandle, long comparatorHandle); + private native void setMergeOperatorName( + long handle, String name); + private native void setMergeOperator( + long handle, long mergeOperatorHandle); + private native void setWriteBufferSize(long handle, long writeBufferSize) + throws RocksDBException; private native long writeBufferSize(long handle); private native void setMaxWriteBufferNumber( long handle, int maxWriteBufferNumber); private native int maxWriteBufferNumber(long handle); - private native void setMaxBackgroundCompactions( - long handle, int maxBackgroundCompactions); - private native int maxBackgroundCompactions(long handle); - private native void createStatistics(long optHandle); - private native long statisticsPtr(long optHandle); - + private native void setMinWriteBufferNumberToMerge( + long handle, int minWriteBufferNumberToMerge); + private native int minWriteBufferNumberToMerge(long handle); + private native void setCompressionType(long handle, byte compressionType); + private native byte compressionType(long handle); + private native void useFixedLengthPrefixExtractor( + long handle, int prefixLength); + private native void setNumLevels( + long handle, int numLevels); + private native int numLevels(long handle); + private native void setLevelZeroFileNumCompactionTrigger( + long handle, int numFiles); + private native int levelZeroFileNumCompactionTrigger(long handle); + private native void setLevelZeroSlowdownWritesTrigger( + long handle, int numFiles); + private native int levelZeroSlowdownWritesTrigger(long handle); + private native void setLevelZeroStopWritesTrigger( + long handle, int numFiles); + private native int levelZeroStopWritesTrigger(long handle); + private native void setMaxMemCompactionLevel( + long handle, int maxMemCompactionLevel); + private native int maxMemCompactionLevel(long handle); + private native void setTargetFileSizeBase( + long handle, long targetFileSizeBase); + private native long targetFileSizeBase(long handle); + private native void setTargetFileSizeMultiplier( + long handle, int multiplier); + private native int targetFileSizeMultiplier(long handle); + private native void setMaxBytesForLevelBase( + long handle, long maxBytesForLevelBase); + private native long maxBytesForLevelBase(long handle); + private native void setMaxBytesForLevelMultiplier( + long handle, int multiplier); + private native int maxBytesForLevelMultiplier(long handle); + private native void setExpandedCompactionFactor( + long handle, int expandedCompactionFactor); + private native int expandedCompactionFactor(long handle); + private native void setSourceCompactionFactor( + long handle, int sourceCompactionFactor); + private native int sourceCompactionFactor(long handle); + private native void setMaxGrandparentOverlapFactor( + long handle, int maxGrandparentOverlapFactor); + private native int maxGrandparentOverlapFactor(long handle); + private native void setSoftRateLimit( + long handle, double softRateLimit); + private native double softRateLimit(long handle); + private native void setHardRateLimit( + long handle, double hardRateLimit); + private native double hardRateLimit(long handle); + private native void setRateLimitDelayMaxMilliseconds( + long handle, int rateLimitDelayMaxMilliseconds); + private native int rateLimitDelayMaxMilliseconds(long handle); + private native void setArenaBlockSize( + long handle, long arenaBlockSize) throws RocksDBException; + private native long arenaBlockSize(long handle); + private native void setDisableAutoCompactions( + long handle, boolean disableAutoCompactions); + private native boolean disableAutoCompactions(long handle); + private native void setCompactionStyle(long handle, byte compactionStyle); + private native byte compactionStyle(long handle); + private native void setPurgeRedundantKvsWhileFlush( + long handle, boolean purgeRedundantKvsWhileFlush); + private native boolean purgeRedundantKvsWhileFlush(long handle); + private native void setVerifyChecksumsInCompaction( + long handle, boolean verifyChecksumsInCompaction); + private native boolean verifyChecksumsInCompaction(long handle); + private native void setFilterDeletes( + long handle, boolean filterDeletes); + private native boolean filterDeletes(long handle); + private native void setMaxSequentialSkipInIterations( + long handle, long maxSequentialSkipInIterations); + private native long maxSequentialSkipInIterations(long handle); private native void setMemTableFactory(long handle, long factoryHandle); - private native void setRateLimiter(long handle, - long rateLimiterHandle); private native String memTableFactoryName(long handle); - private native void setTableFactory(long handle, long factoryHandle); private native String tableFactoryName(long handle); - - private native void useFixedLengthPrefixExtractor( - long handle, int prefixLength); - - long cacheSize_; - int numShardBits_; - AbstractComparator comparator_; + private native void setInplaceUpdateSupport( + long handle, boolean inplaceUpdateSupport); + private native boolean inplaceUpdateSupport(long handle); + private native void setInplaceUpdateNumLocks( + long handle, long inplaceUpdateNumLocks) throws RocksDBException; + private native long inplaceUpdateNumLocks(long handle); + private native void setMemtablePrefixBloomBits( + long handle, int memtablePrefixBloomBits); + private native int memtablePrefixBloomBits(long handle); + private native void setMemtablePrefixBloomProbes( + long handle, int memtablePrefixBloomProbes); + private native int memtablePrefixBloomProbes(long handle); + private native void setBloomLocality( + long handle, int bloomLocality); + private native int bloomLocality(long handle); + private native void setMaxSuccessiveMerges( + long handle, long maxSuccessiveMerges) throws RocksDBException; + private native long maxSuccessiveMerges(long handle); + private native void setMinPartialMergeOperands( + long handle, int minPartialMergeOperands); + private native int minPartialMergeOperands(long handle); + // instance variables RocksEnv env_; MemTableConfig memTableConfig_; TableFormatConfig tableFormatConfig_; RateLimiterConfig rateLimiterConfig_; + AbstractComparator comparator_; } diff --git a/java/org/rocksdb/RateLimiterConfig.java b/java/org/rocksdb/RateLimiterConfig.java index 1b309e6c9..06f3990d0 100644 --- a/java/org/rocksdb/RateLimiterConfig.java +++ b/java/org/rocksdb/RateLimiterConfig.java @@ -11,11 +11,11 @@ package org.rocksdb; public abstract class RateLimiterConfig { /** * This function should only be called by - * {@link org.rocksdb.Options#setRateLimiter(long, long)}, which will + * {@link org.rocksdb.DBOptions#setRateLimiter(long, long)}, which will * create a c++ shared-pointer to the c++ {@code RateLimiter} that is associated * with a Java RateLimiterConfig. * - * @see org.rocksdb.Options#setRateLimiter(long, long) + * @see org.rocksdb.DBOptions#setRateLimiter(long, long) */ abstract protected long newRateLimiterHandle(); } diff --git a/java/org/rocksdb/test/OptionsTest.java b/java/org/rocksdb/test/OptionsTest.java index 222d87b8d..ef88e3503 100644 --- a/java/org/rocksdb/test/OptionsTest.java +++ b/java/org/rocksdb/test/OptionsTest.java @@ -44,6 +44,13 @@ public class OptionsTest { assert(opt.paranoidChecks() == boolValue); } + { + // MaxTotalWalSize test + long longValue = rand.nextLong(); + opt.setMaxTotalWalSize(longValue); + assert(opt.maxTotalWalSize() == longValue); + } + { // MaxOpenFiles test int intValue = rand.nextInt(); opt.setMaxOpenFiles(intValue); @@ -264,9 +271,9 @@ public class OptionsTest { } { // TargetFileSizeBase test - int intValue = rand.nextInt(); - opt.setTargetFileSizeBase(intValue); - assert(opt.targetFileSizeBase() == intValue); + long longValue = rand.nextLong(); + opt.setTargetFileSizeBase(longValue); + assert(opt.targetFileSizeBase() == longValue); } { // TargetFileSizeMultiplier test diff --git a/java/rocksjni/options.cc b/java/rocksjni/options.cc index 7d0834bb9..6cde26327 100644 --- a/java/rocksjni/options.cc +++ b/java/rocksjni/options.cc @@ -12,6 +12,8 @@ #include #include "include/org_rocksdb_Options.h" +#include "include/org_rocksdb_DBOptions.h" +#include "include/org_rocksdb_ColumnFamilyOptions.h" #include "include/org_rocksdb_WriteOptions.h" #include "include/org_rocksdb_ReadOptions.h" #include "include/org_rocksdb_ComparatorOptions.h" @@ -34,11 +36,25 @@ * Method: newOptions * Signature: ()V */ -void Java_org_rocksdb_Options_newOptions(JNIEnv* env, jobject jobj) { +void Java_org_rocksdb_Options_newOptions__(JNIEnv* env, jobject jobj) { rocksdb::Options* op = new rocksdb::Options(); rocksdb::OptionsJni::setHandle(env, jobj, op); } +/* + * Class: org_rocksdb_Options + * Method: newOptions + * Signature: (JJ)V + */ +void Java_org_rocksdb_Options_newOptions__JJ(JNIEnv* env, jobject jobj, + jlong jdboptions, jlong jcfoptions) { + auto dbOpt = reinterpret_cast(jdboptions); + auto cfOpt = reinterpret_cast( + jcfoptions); + rocksdb::Options* op = new rocksdb::Options(*dbOpt, *cfOpt); + rocksdb::OptionsJni::setHandle(env, jobj, op); +} + /* * Class: org_rocksdb_Options * Method: disposeInternal @@ -93,10 +109,10 @@ jboolean Java_org_rocksdb_Options_createMissingColumnFamilies( /* * Class: org_rocksdb_Options - * Method: useReverseBytewiseComparator + * Method: setComparatorHandle * Signature: (JI)V */ -void Java_org_rocksdb_Options_setBuiltinComparator( +void Java_org_rocksdb_Options_setComparatorHandle__JI( JNIEnv* env, jobject jobj, jlong jhandle, jint builtinComparator) { switch (builtinComparator) { case 1: @@ -110,6 +126,41 @@ void Java_org_rocksdb_Options_setBuiltinComparator( } } +/* + * Class: org_rocksdb_Options + * Method: setComparatorHandle + * Signature: (JJ)V + */ +void Java_org_rocksdb_Options_setComparatorHandle__JJ( + JNIEnv* env, jobject jobj, jlong jopt_handle, jlong jcomparator_handle) { + reinterpret_cast(jopt_handle)->comparator = + reinterpret_cast(jcomparator_handle); +} + +/* + * Class: org_rocksdb_Options + * Method: setMergeOperatorName + * Signature: (JJjava/lang/String)V + */ +void Java_org_rocksdb_Options_setMergeOperatorName( + JNIEnv* env, jobject jobj, jlong jhandle, jstring name) { + const char* op_name = env->GetStringUTFChars(name, 0); + reinterpret_cast(jhandle)->merge_operator = + rocksdb::MergeOperators::CreateFromStringId(op_name); +} + +/* + * Class: org_rocksdb_Options + * Method: setMergeOperator + * Signature: (JJjava/lang/String)V + */ +void Java_org_rocksdb_Options_setMergeOperator( + JNIEnv* env, jobject jobj, jlong jhandle, jlong mergeOperatorHandle) { + reinterpret_cast(jhandle)->merge_operator = + *(reinterpret_cast*> + (mergeOperatorHandle)); +} + /* * Class: org_rocksdb_Options * Method: setWriteBufferSize @@ -169,17 +220,6 @@ jlong Java_org_rocksdb_Options_statisticsPtr( return reinterpret_cast(st); } -/* - * Class: org_rocksdb_Options - * Method: setComparatorHandle - * Signature: (JJ)V - */ -void Java_org_rocksdb_Options_setComparatorHandle( - JNIEnv* env, jobject jobj, jlong jopt_handle, jlong jcomparator_handle) { - reinterpret_cast(jopt_handle)->comparator = - reinterpret_cast(jcomparator_handle); -} - /* * Class: org_rocksdb_Options * Method: maxWriteBufferNumber @@ -232,6 +272,29 @@ void Java_org_rocksdb_Options_setParanoidChecks( static_cast(paranoid_checks); } +/* + * Class: org_rocksdb_Options + * Method: setMaxTotalWalSize + * Signature: (JJ)V + */ +void Java_org_rocksdb_Options_setMaxTotalWalSize( + JNIEnv* env, jobject jobj, jlong jhandle, + jlong jmax_total_wal_size) { + reinterpret_cast(jhandle)->max_total_wal_size = + static_cast(jmax_total_wal_size); +} + +/* + * Class: org_rocksdb_Options + * Method: maxTotalWalSize + * Signature: (J)J + */ +jlong Java_org_rocksdb_Options_maxTotalWalSize( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast(jhandle)-> + max_total_wal_size; +} + /* * Class: org_rocksdb_Options * Method: maxOpenFiles @@ -1073,9 +1136,9 @@ void Java_org_rocksdb_Options_setMaxMemCompactionLevel( /* * Class: org_rocksdb_Options * Method: targetFileSizeBase - * Signature: (J)I + * Signature: (J)J */ -jint Java_org_rocksdb_Options_targetFileSizeBase( +jlong Java_org_rocksdb_Options_targetFileSizeBase( JNIEnv* env, jobject jobj, jlong jhandle) { return reinterpret_cast(jhandle)->target_file_size_base; } @@ -1083,13 +1146,13 @@ jint Java_org_rocksdb_Options_targetFileSizeBase( /* * Class: org_rocksdb_Options * Method: setTargetFileSizeBase - * Signature: (JI)V + * Signature: (JJ)V */ void Java_org_rocksdb_Options_setTargetFileSizeBase( JNIEnv* env, jobject jobj, jlong jhandle, - jint jtarget_file_size_base) { + jlong jtarget_file_size_base) { reinterpret_cast(jhandle)->target_file_size_base = - static_cast(jtarget_file_size_base); + static_cast(jtarget_file_size_base); } /* @@ -1619,185 +1682,1927 @@ void Java_org_rocksdb_Options_setMinPartialMergeOperands( static_cast(jmin_partial_merge_operands); } +/* + * Method: optimizeForPointLookup + * Signature: (JJ)V + */ +void Java_org_rocksdb_Options_optimizeForPointLookup( + JNIEnv* env, jobject jobj, jlong jhandle, + jlong block_cache_size_mb) { + reinterpret_cast(jhandle)-> + OptimizeForPointLookup(block_cache_size_mb); +} + +/* + * Method: optimizeLevelStyleCompaction + * Signature: (JJ)V + */ +void Java_org_rocksdb_Options_optimizeLevelStyleCompaction( + JNIEnv* env, jobject jobj, jlong jhandle, + jlong memtable_memory_budget) { + reinterpret_cast(jhandle)-> + OptimizeLevelStyleCompaction(memtable_memory_budget); +} + /* * Class: org_rocksdb_Options - * Method: setMergeOperatorName - * Signature: (JJjava/lang/String)V + * Method: optimizeUniversalStyleCompaction + * Signature: (JJ)V */ -void Java_org_rocksdb_Options_setMergeOperatorName( - JNIEnv* env, jobject jobj, jlong jhandle, jstring name) { - const char* op_name = env->GetStringUTFChars(name, 0); - reinterpret_cast(jhandle)->merge_operator = - rocksdb::MergeOperators::CreateFromStringId(op_name); +void Java_org_rocksdb_Options_optimizeUniversalStyleCompaction( + JNIEnv* env, jobject jobj, jlong jhandle, + jlong memtable_memory_budget) { + reinterpret_cast(jhandle)-> + OptimizeUniversalStyleCompaction(memtable_memory_budget); } /* * Class: org_rocksdb_Options - * Method: setMergeOperator - * Signature: (JJjava/lang/String)V + * Method: prepareForBulkLoad + * Signature: (J)V */ -void Java_org_rocksdb_Options_setMergeOperator( - JNIEnv* env, jobject jobj, jlong jhandle, jlong mergeOperatorHandle) { - reinterpret_cast(jhandle)->merge_operator = - *(reinterpret_cast*> - (mergeOperatorHandle)); +void Java_org_rocksdb_Options_prepareForBulkLoad( + JNIEnv* env, jobject jobj, jlong jhandle) { + reinterpret_cast(jhandle)-> + PrepareForBulkLoad(); } ////////////////////////////////////////////////////////////////////////////// -// WriteOptions +// rocksdb::ColumnFamilyOptions /* - * Class: org_rocksdb_WriteOptions - * Method: newWriteOptions + * Class: org_rocksdb_ColumnFamilyOptions + * Method: newColumnFamilyOptions * Signature: ()V */ -void Java_org_rocksdb_WriteOptions_newWriteOptions( - JNIEnv* env, jobject jwrite_options) { - rocksdb::WriteOptions* op = new rocksdb::WriteOptions(); - rocksdb::WriteOptionsJni::setHandle(env, jwrite_options, op); +void Java_org_rocksdb_ColumnFamilyOptions_newColumnFamilyOptions( + JNIEnv* env, jobject jobj) { + //rocksdb::ColumnFamilyOptions* op = new rocksdb::ColumnFamilyOptions(); + //rocksdb::ColumnFamilyOptionsJni::setHandle(env, jobj, op); } /* - * Class: org_rocksdb_WriteOptions + * Class: org_rocksdb_ColumnFamilyOptions * Method: disposeInternal - * Signature: ()V + * Signature: (J)V */ -void Java_org_rocksdb_WriteOptions_disposeInternal( - JNIEnv* env, jobject jwrite_options, jlong jhandle) { - auto write_options = reinterpret_cast(jhandle); - delete write_options; - - rocksdb::WriteOptionsJni::setHandle(env, jwrite_options, nullptr); +void Java_org_rocksdb_ColumnFamilyOptions_disposeInternal( + JNIEnv* env, jobject jobj, jlong handle) { + delete reinterpret_cast(handle); } /* - * Class: org_rocksdb_WriteOptions - * Method: setSync - * Signature: (JZ)V + * Class: org_rocksdb_ColumnFamilyOptions + * Method: optimizeForPointLookup + * Signature: (JJ)V */ -void Java_org_rocksdb_WriteOptions_setSync( - JNIEnv* env, jobject jwrite_options, jlong jhandle, jboolean jflag) { - reinterpret_cast(jhandle)->sync = jflag; +void Java_org_rocksdb_ColumnFamilyOptions_optimizeForPointLookup( + JNIEnv* env, jobject jobj, jlong jhandle, + jlong block_cache_size_mb) { + reinterpret_cast(jhandle)-> + OptimizeForPointLookup(block_cache_size_mb); } /* - * Class: org_rocksdb_WriteOptions - * Method: sync - * Signature: (J)Z + * Class: org_rocksdb_ColumnFamilyOptions + * Method: optimizeLevelStyleCompaction + * Signature: (JJ)V */ -jboolean Java_org_rocksdb_WriteOptions_sync( - JNIEnv* env, jobject jwrite_options, jlong jhandle) { - return reinterpret_cast(jhandle)->sync; +void Java_org_rocksdb_ColumnFamilyOptions_optimizeLevelStyleCompaction( + JNIEnv* env, jobject jobj, jlong jhandle, + jlong memtable_memory_budget) { + reinterpret_cast(jhandle)-> + OptimizeLevelStyleCompaction(memtable_memory_budget); } /* - * Class: org_rocksdb_WriteOptions - * Method: setDisableWAL - * Signature: (JZ)V + * Class: org_rocksdb_ColumnFamilyOptions + * Method: optimizeUniversalStyleCompaction + * Signature: (JJ)V */ -void Java_org_rocksdb_WriteOptions_setDisableWAL( - JNIEnv* env, jobject jwrite_options, jlong jhandle, jboolean jflag) { - reinterpret_cast(jhandle)->disableWAL = jflag; +void Java_org_rocksdb_ColumnFamilyOptions_optimizeUniversalStyleCompaction( + JNIEnv* env, jobject jobj, jlong jhandle, + jlong memtable_memory_budget) { + reinterpret_cast(jhandle)-> + OptimizeUniversalStyleCompaction(memtable_memory_budget); } /* - * Class: org_rocksdb_WriteOptions - * Method: disableWAL - * Signature: (J)Z + * Class: org_rocksdb_ColumnFamilyOptions + * Method: setComparatorHandle + * Signature: (JI)V */ -jboolean Java_org_rocksdb_WriteOptions_disableWAL( - JNIEnv* env, jobject jwrite_options, jlong jhandle) { - return reinterpret_cast(jhandle)->disableWAL; +void Java_org_rocksdb_ColumnFamilyOptions_setComparatorHandle__JI( + JNIEnv* env, jobject jobj, jlong jhandle, jint builtinComparator) { + switch (builtinComparator) { + case 1: + reinterpret_cast(jhandle)->comparator = + rocksdb::ReverseBytewiseComparator(); + break; + default: + reinterpret_cast(jhandle)->comparator = + rocksdb::BytewiseComparator(); + break; + } } -///////////////////////////////////////////////////////////////////// -// rocksdb::ReadOptions - /* - * Class: org_rocksdb_ReadOptions - * Method: newReadOptions - * Signature: ()V + * Class: org_rocksdb_ColumnFamilyOptions + * Method: setComparatorHandle + * Signature: (JJ)V */ -void Java_org_rocksdb_ReadOptions_newReadOptions( - JNIEnv* env, jobject jobj) { - auto read_opt = new rocksdb::ReadOptions(); - rocksdb::ReadOptionsJni::setHandle(env, jobj, read_opt); +void Java_org_rocksdb_ColumnFamilyOptions_setComparatorHandle__JI( + JNIEnv* env, jobject jobj, jlong jopt_handle, jlong jcomparator_handle) { + reinterpret_cast(jopt_handle)->comparator = + reinterpret_cast(jcomparator_handle); } /* - * Class: org_rocksdb_ReadOptions - * Method: disposeInternal - * Signature: (J)V + * Class: org_rocksdb_ColumnFamilyOptions + * Method: setMergeOperatorName + * Signature: (JJjava/lang/String)V */ -void Java_org_rocksdb_ReadOptions_disposeInternal( - JNIEnv* env, jobject jobj, jlong jhandle) { - delete reinterpret_cast(jhandle); - rocksdb::ReadOptionsJni::setHandle(env, jobj, nullptr); +void Java_org_rocksdb_ColumnFamilyOptions_setMergeOperatorName( + JNIEnv* env, jobject jobj, jlong jhandle, jstring name) { + const char* op_name = env->GetStringUTFChars(name, 0); + reinterpret_cast(jhandle)->merge_operator = + rocksdb::MergeOperators::CreateFromStringId(op_name); } /* - * Class: org_rocksdb_ReadOptions - * Method: verifyChecksums - * Signature: (J)Z + * Class: org_rocksdb_ColumnFamilyOptions + * Method: setMergeOperator + * Signature: (JJjava/lang/String)V */ -jboolean Java_org_rocksdb_ReadOptions_verifyChecksums( - JNIEnv* env, jobject jobj, jlong jhandle) { - return reinterpret_cast( - jhandle)->verify_checksums; +void Java_org_rocksdb_ColumnFamilyOptions_setMergeOperator( + JNIEnv* env, jobject jobj, jlong jhandle, jlong mergeOperatorHandle) { + reinterpret_cast(jhandle)->merge_operator = + *(reinterpret_cast*> + (mergeOperatorHandle)); } /* - * Class: org_rocksdb_ReadOptions - * Method: setVerifyChecksums - * Signature: (JZ)V + * Class: org_rocksdb_ColumnFamilyOptions + * Method: setWriteBufferSize + * Signature: (JJ)I */ -void Java_org_rocksdb_ReadOptions_setVerifyChecksums( - JNIEnv* env, jobject jobj, jlong jhandle, - jboolean jverify_checksums) { - reinterpret_cast(jhandle)->verify_checksums = - static_cast(jverify_checksums); +void Java_org_rocksdb_ColumnFamilyOptions_setWriteBufferSize( + JNIEnv* env, jobject jobj, jlong jhandle, jlong jwrite_buffer_size) { + rocksdb::Status s = rocksdb::check_if_jlong_fits_size_t(jwrite_buffer_size); + if (s.ok()) { + reinterpret_cast(jhandle)-> + write_buffer_size = jwrite_buffer_size; + } else { + rocksdb::RocksDBExceptionJni::ThrowNew(env, s); + } } /* - * Class: org_rocksdb_ReadOptions - * Method: fillCache - * Signature: (J)Z + * Class: org_rocksdb_ColumnFamilyOptions + * Method: writeBufferSize + * Signature: (J)J */ -jboolean Java_org_rocksdb_ReadOptions_fillCache( +jlong Java_org_rocksdb_ColumnFamilyOptions_writeBufferSize( JNIEnv* env, jobject jobj, jlong jhandle) { - return reinterpret_cast(jhandle)->fill_cache; + return reinterpret_cast(jhandle)-> + write_buffer_size; } /* - * Class: org_rocksdb_ReadOptions - * Method: setFillCache - * Signature: (JZ)V + * Class: org_rocksdb_ColumnFamilyOptions + * Method: setMaxWriteBufferNumber + * Signature: (JI)V */ -void Java_org_rocksdb_ReadOptions_setFillCache( - JNIEnv* env, jobject jobj, jlong jhandle, jboolean jfill_cache) { - reinterpret_cast(jhandle)->fill_cache = - static_cast(jfill_cache); +void Java_org_rocksdb_ColumnFamilyOptions_setMaxWriteBufferNumber( + JNIEnv* env, jobject jobj, jlong jhandle, jint jmax_write_buffer_number) { + reinterpret_cast(jhandle)-> + max_write_buffer_number = jmax_write_buffer_number; } /* - * Class: org_rocksdb_ReadOptions - * Method: tailing - * Signature: (J)Z + * Class: org_rocksdb_ColumnFamilyOptions + * Method: maxWriteBufferNumber + * Signature: (J)I */ -jboolean Java_org_rocksdb_ReadOptions_tailing( +jint Java_org_rocksdb_ColumnFamilyOptions_maxWriteBufferNumber( JNIEnv* env, jobject jobj, jlong jhandle) { - return reinterpret_cast(jhandle)->tailing; + return reinterpret_cast(jhandle)-> + max_write_buffer_number; } /* - * Class: org_rocksdb_ReadOptions - * Method: setTailing - * Signature: (JZ)V + * Method: setMemTableFactory + * Signature: (JJ)V */ -void Java_org_rocksdb_ReadOptions_setTailing( - JNIEnv* env, jobject jobj, jlong jhandle, jboolean jtailing) { - reinterpret_cast(jhandle)->tailing = - static_cast(jtailing); +void Java_org_rocksdb_ColumnFamilyOptions_setMemTableFactory( + JNIEnv* env, jobject jobj, jlong jhandle, jlong jfactory_handle) { + reinterpret_cast(jhandle)-> + memtable_factory.reset( + reinterpret_cast(jfactory_handle)); +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: memTableFactoryName + * Signature: (J)Ljava/lang/String + */ +jstring Java_org_rocksdb_ColumnFamilyOptions_memTableFactoryName( + JNIEnv* env, jobject jobj, jlong jhandle) { + auto opt = reinterpret_cast(jhandle); + rocksdb::MemTableRepFactory* tf = opt->memtable_factory.get(); + + // Should never be nullptr. + // Default memtable factory is SkipListFactory + assert(tf); + + // temporarly fix for the historical typo + if (strcmp(tf->Name(), "HashLinkListRepFactory") == 0) { + return env->NewStringUTF("HashLinkedListRepFactory"); + } + + return env->NewStringUTF(tf->Name()); +} + +/* + * Method: useFixedLengthPrefixExtractor + * Signature: (JI)V + */ +void Java_org_rocksdb_ColumnFamilyOptions_useFixedLengthPrefixExtractor( + JNIEnv* env, jobject jobj, jlong jhandle, jint jprefix_length) { + reinterpret_cast(jhandle)-> + prefix_extractor.reset(rocksdb::NewFixedPrefixTransform( + static_cast(jprefix_length))); +} + +/* + * Method: setTableFactory + * Signature: (JJ)V + */ +void Java_org_rocksdb_ColumnFamilyOptions_setTableFactory( + JNIEnv* env, jobject jobj, jlong jhandle, jlong jfactory_handle) { + reinterpret_cast(jhandle)-> + table_factory.reset(reinterpret_cast( + jfactory_handle)); +} + +/* + * Method: tableFactoryName + * Signature: (J)Ljava/lang/String + */ +jstring Java_org_rocksdb_ColumnFamilyOptions_tableFactoryName( + JNIEnv* env, jobject jobj, jlong jhandle) { + auto opt = reinterpret_cast(jhandle); + rocksdb::TableFactory* tf = opt->table_factory.get(); + + // Should never be nullptr. + // Default memtable factory is SkipListFactory + assert(tf); + + return env->NewStringUTF(tf->Name()); +} + + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: minWriteBufferNumberToMerge + * Signature: (J)I + */ +jint Java_org_rocksdb_ColumnFamilyOptions_minWriteBufferNumberToMerge( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast( + jhandle)->min_write_buffer_number_to_merge; +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: setMinWriteBufferNumberToMerge + * Signature: (JI)V + */ +void Java_org_rocksdb_ColumnFamilyOptions_setMinWriteBufferNumberToMerge( + JNIEnv* env, jobject jobj, jlong jhandle, + jint jmin_write_buffer_number_to_merge) { + reinterpret_cast( + jhandle)->min_write_buffer_number_to_merge = + static_cast(jmin_write_buffer_number_to_merge); +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: setCompressionType + * Signature: (JB)V + */ +void Java_org_rocksdb_ColumnFamilyOptions_setCompressionType( + JNIEnv* env, jobject jobj, jlong jhandle, jbyte compression) { + reinterpret_cast(jhandle)-> + compression = static_cast(compression); +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: compressionType + * Signature: (J)B + */ +jbyte Java_org_rocksdb_ColumnFamilyOptions_compressionType( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast(jhandle)-> + compression; +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: setCompactionStyle + * Signature: (JB)V + */ +void Java_org_rocksdb_ColumnFamilyOptions_setCompactionStyle( + JNIEnv* env, jobject jobj, jlong jhandle, jbyte compaction_style) { + reinterpret_cast(jhandle)->compaction_style = + static_cast(compaction_style); +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: compactionStyle + * Signature: (J)B + */ +jbyte Java_org_rocksdb_ColumnFamilyOptions_compactionStyle( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast + (jhandle)->compaction_style; +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: numLevels + * Signature: (J)I + */ +jint Java_org_rocksdb_ColumnFamilyOptions_numLevels( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast(jhandle)->num_levels; +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: setNumLevels + * Signature: (JI)V + */ +void Java_org_rocksdb_ColumnFamilyOptions_setNumLevels( + JNIEnv* env, jobject jobj, jlong jhandle, jint jnum_levels) { + reinterpret_cast(jhandle)->num_levels = + static_cast(jnum_levels); +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: levelZeroFileNumCompactionTrigger + * Signature: (J)I + */ +jint Java_org_rocksdb_ColumnFamilyOptions_levelZeroFileNumCompactionTrigger( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast( + jhandle)->level0_file_num_compaction_trigger; +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: setLevelZeroFileNumCompactionTrigger + * Signature: (JI)V + */ +void Java_org_rocksdb_ColumnFamilyOptions_setLevelZeroFileNumCompactionTrigger( + JNIEnv* env, jobject jobj, jlong jhandle, + jint jlevel0_file_num_compaction_trigger) { + reinterpret_cast( + jhandle)->level0_file_num_compaction_trigger = + static_cast(jlevel0_file_num_compaction_trigger); +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: levelZeroSlowdownWritesTrigger + * Signature: (J)I + */ +jint Java_org_rocksdb_ColumnFamilyOptions_levelZeroSlowdownWritesTrigger( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast( + jhandle)->level0_slowdown_writes_trigger; +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: setLevelSlowdownWritesTrigger + * Signature: (JI)V + */ +void Java_org_rocksdb_ColumnFamilyOptions_setLevelZeroSlowdownWritesTrigger( + JNIEnv* env, jobject jobj, jlong jhandle, + jint jlevel0_slowdown_writes_trigger) { + reinterpret_cast( + jhandle)->level0_slowdown_writes_trigger = + static_cast(jlevel0_slowdown_writes_trigger); +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: levelZeroStopWritesTrigger + * Signature: (J)I + */ +jint Java_org_rocksdb_ColumnFamilyOptions_levelZeroStopWritesTrigger( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast( + jhandle)->level0_stop_writes_trigger; +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: setLevelStopWritesTrigger + * Signature: (JI)V + */ +void Java_org_rocksdb_ColumnFamilyOptions_setLevelZeroStopWritesTrigger( + JNIEnv* env, jobject jobj, jlong jhandle, + jint jlevel0_stop_writes_trigger) { + reinterpret_cast(jhandle)-> + level0_stop_writes_trigger = static_cast( + jlevel0_stop_writes_trigger); +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: maxMemCompactionLevel + * Signature: (J)I + */ +jint Java_org_rocksdb_ColumnFamilyOptions_maxMemCompactionLevel( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast( + jhandle)->max_mem_compaction_level; +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: setMaxMemCompactionLevel + * Signature: (JI)V + */ +void Java_org_rocksdb_ColumnFamilyOptions_setMaxMemCompactionLevel( + JNIEnv* env, jobject jobj, jlong jhandle, + jint jmax_mem_compaction_level) { + reinterpret_cast(jhandle)-> + max_mem_compaction_level = static_cast(jmax_mem_compaction_level); +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: targetFileSizeBase + * Signature: (J)J + */ +jlong Java_org_rocksdb_ColumnFamilyOptions_targetFileSizeBase( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast(jhandle)-> + target_file_size_base; +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: setTargetFileSizeBase + * Signature: (JJ)V + */ +void Java_org_rocksdb_ColumnFamilyOptions_setTargetFileSizeBase( + JNIEnv* env, jobject jobj, jlong jhandle, + jlong jtarget_file_size_base) { + reinterpret_cast(jhandle)-> + target_file_size_base = static_cast(jtarget_file_size_base); +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: targetFileSizeMultiplier + * Signature: (J)I + */ +jint Java_org_rocksdb_ColumnFamilyOptions_targetFileSizeMultiplier( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast( + jhandle)->target_file_size_multiplier; +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: setTargetFileSizeMultiplier + * Signature: (JI)V + */ +void Java_org_rocksdb_ColumnFamilyOptions_setTargetFileSizeMultiplier( + JNIEnv* env, jobject jobj, jlong jhandle, + jint jtarget_file_size_multiplier) { + reinterpret_cast( + jhandle)->target_file_size_multiplier = + static_cast(jtarget_file_size_multiplier); +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: maxBytesForLevelBase + * Signature: (J)J + */ +jlong Java_org_rocksdb_ColumnFamilyOptions_maxBytesForLevelBase( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast( + jhandle)->max_bytes_for_level_base; +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: setMaxBytesForLevelBase + * Signature: (JJ)V + */ +void Java_org_rocksdb_ColumnFamilyOptions_setMaxBytesForLevelBase( + JNIEnv* env, jobject jobj, jlong jhandle, + jlong jmax_bytes_for_level_base) { + reinterpret_cast( + jhandle)->max_bytes_for_level_base = + static_cast(jmax_bytes_for_level_base); +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: maxBytesForLevelMultiplier + * Signature: (J)I + */ +jint Java_org_rocksdb_ColumnFamilyOptions_maxBytesForLevelMultiplier( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast( + jhandle)->max_bytes_for_level_multiplier; +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: setMaxBytesForLevelMultiplier + * Signature: (JI)V + */ +void Java_org_rocksdb_ColumnFamilyOptions_setMaxBytesForLevelMultiplier( + JNIEnv* env, jobject jobj, jlong jhandle, + jint jmax_bytes_for_level_multiplier) { + reinterpret_cast( + jhandle)->max_bytes_for_level_multiplier = + static_cast(jmax_bytes_for_level_multiplier); +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: expandedCompactionFactor + * Signature: (J)I + */ +jint Java_org_rocksdb_ColumnFamilyOptions_expandedCompactionFactor( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast( + jhandle)->expanded_compaction_factor; +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: setExpandedCompactionFactor + * Signature: (JI)V + */ +void Java_org_rocksdb_ColumnFamilyOptions_setExpandedCompactionFactor( + JNIEnv* env, jobject jobj, jlong jhandle, + jint jexpanded_compaction_factor) { + reinterpret_cast( + jhandle)->expanded_compaction_factor = + static_cast(jexpanded_compaction_factor); +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: sourceCompactionFactor + * Signature: (J)I + */ +jint Java_org_rocksdb_ColumnFamilyOptions_sourceCompactionFactor( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast( + jhandle)->source_compaction_factor; +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: setSourceCompactionFactor + * Signature: (JI)V + */ +void Java_org_rocksdb_ColumnFamilyOptions_setSourceCompactionFactor( + JNIEnv* env, jobject jobj, jlong jhandle, + jint jsource_compaction_factor) { + reinterpret_cast( + jhandle)->source_compaction_factor = + static_cast(jsource_compaction_factor); +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: maxGrandparentOverlapFactor + * Signature: (J)I + */ +jint Java_org_rocksdb_ColumnFamilyOptions_maxGrandparentOverlapFactor( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast( + jhandle)->max_grandparent_overlap_factor; +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: setMaxGrandparentOverlapFactor + * Signature: (JI)V + */ +void Java_org_rocksdb_ColumnFamilyOptions_setMaxGrandparentOverlapFactor( + JNIEnv* env, jobject jobj, jlong jhandle, + jint jmax_grandparent_overlap_factor) { + reinterpret_cast( + jhandle)->max_grandparent_overlap_factor = + static_cast(jmax_grandparent_overlap_factor); +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: softRateLimit + * Signature: (J)D + */ +jdouble Java_org_rocksdb_ColumnFamilyOptions_softRateLimit( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast(jhandle)-> + soft_rate_limit; +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: setSoftRateLimit + * Signature: (JD)V + */ +void Java_org_rocksdb_ColumnFamilyOptions_setSoftRateLimit( + JNIEnv* env, jobject jobj, jlong jhandle, jdouble jsoft_rate_limit) { + reinterpret_cast(jhandle)->soft_rate_limit = + static_cast(jsoft_rate_limit); +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: hardRateLimit + * Signature: (J)D + */ +jdouble Java_org_rocksdb_ColumnFamilyOptions_hardRateLimit( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast(jhandle)-> + hard_rate_limit; +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: setHardRateLimit + * Signature: (JD)V + */ +void Java_org_rocksdb_ColumnFamilyOptions_setHardRateLimit( + JNIEnv* env, jobject jobj, jlong jhandle, jdouble jhard_rate_limit) { + reinterpret_cast(jhandle)->hard_rate_limit = + static_cast(jhard_rate_limit); +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: rateLimitDelayMaxMilliseconds + * Signature: (J)I + */ +jint Java_org_rocksdb_ColumnFamilyOptions_rateLimitDelayMaxMilliseconds( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast( + jhandle)->rate_limit_delay_max_milliseconds; +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: setRateLimitDelayMaxMilliseconds + * Signature: (JI)V + */ +void Java_org_rocksdb_ColumnFamilyOptions_setRateLimitDelayMaxMilliseconds( + JNIEnv* env, jobject jobj, jlong jhandle, + jint jrate_limit_delay_max_milliseconds) { + reinterpret_cast( + jhandle)->rate_limit_delay_max_milliseconds = + static_cast(jrate_limit_delay_max_milliseconds); +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: arenaBlockSize + * Signature: (J)J + */ +jlong Java_org_rocksdb_ColumnFamilyOptions_arenaBlockSize( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast(jhandle)-> + arena_block_size; +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: setArenaBlockSize + * Signature: (JJ)V + */ +void Java_org_rocksdb_ColumnFamilyOptions_setArenaBlockSize( + JNIEnv* env, jobject jobj, jlong jhandle, jlong jarena_block_size) { + rocksdb::Status s = rocksdb::check_if_jlong_fits_size_t(jarena_block_size); + if (s.ok()) { + reinterpret_cast(jhandle)-> + arena_block_size = jarena_block_size; + } else { + rocksdb::RocksDBExceptionJni::ThrowNew(env, s); + } +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: disableAutoCompactions + * Signature: (J)Z + */ +jboolean Java_org_rocksdb_ColumnFamilyOptions_disableAutoCompactions( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast( + jhandle)->disable_auto_compactions; +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: setDisableAutoCompactions + * Signature: (JZ)V + */ +void Java_org_rocksdb_ColumnFamilyOptions_setDisableAutoCompactions( + JNIEnv* env, jobject jobj, jlong jhandle, + jboolean jdisable_auto_compactions) { + reinterpret_cast( + jhandle)->disable_auto_compactions = + static_cast(jdisable_auto_compactions); +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: purgeRedundantKvsWhileFlush + * Signature: (J)Z + */ +jboolean Java_org_rocksdb_ColumnFamilyOptions_purgeRedundantKvsWhileFlush( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast( + jhandle)->purge_redundant_kvs_while_flush; +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: setPurgeRedundantKvsWhileFlush + * Signature: (JZ)V + */ +void Java_org_rocksdb_ColumnFamilyOptions_setPurgeRedundantKvsWhileFlush( + JNIEnv* env, jobject jobj, jlong jhandle, + jboolean jpurge_redundant_kvs_while_flush) { + reinterpret_cast( + jhandle)->purge_redundant_kvs_while_flush = + static_cast(jpurge_redundant_kvs_while_flush); +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: verifyChecksumsInCompaction + * Signature: (J)Z + */ +jboolean Java_org_rocksdb_ColumnFamilyOptions_verifyChecksumsInCompaction( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast( + jhandle)->verify_checksums_in_compaction; +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: setVerifyChecksumsInCompaction + * Signature: (JZ)V + */ +void Java_org_rocksdb_ColumnFamilyOptions_setVerifyChecksumsInCompaction( + JNIEnv* env, jobject jobj, jlong jhandle, + jboolean jverify_checksums_in_compaction) { + reinterpret_cast( + jhandle)->verify_checksums_in_compaction = + static_cast(jverify_checksums_in_compaction); +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: filterDeletes + * Signature: (J)Z + */ +jboolean Java_org_rocksdb_ColumnFamilyOptions_filterDeletes( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast(jhandle)-> + filter_deletes; +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: setFilterDeletes + * Signature: (JZ)V + */ +void Java_org_rocksdb_ColumnFamilyOptions_setFilterDeletes( + JNIEnv* env, jobject jobj, jlong jhandle, jboolean jfilter_deletes) { + reinterpret_cast(jhandle)->filter_deletes = + static_cast(jfilter_deletes); +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: maxSequentialSkipInIterations + * Signature: (J)J + */ +jlong Java_org_rocksdb_ColumnFamilyOptions_maxSequentialSkipInIterations( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast( + jhandle)->max_sequential_skip_in_iterations; +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: setMaxSequentialSkipInIterations + * Signature: (JJ)V + */ +void Java_org_rocksdb_ColumnFamilyOptions_setMaxSequentialSkipInIterations( + JNIEnv* env, jobject jobj, jlong jhandle, + jlong jmax_sequential_skip_in_iterations) { + reinterpret_cast( + jhandle)->max_sequential_skip_in_iterations = + static_cast(jmax_sequential_skip_in_iterations); +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: inplaceUpdateSupport + * Signature: (J)Z + */ +jboolean Java_org_rocksdb_ColumnFamilyOptions_inplaceUpdateSupport( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast( + jhandle)->inplace_update_support; +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: setInplaceUpdateSupport + * Signature: (JZ)V + */ +void Java_org_rocksdb_ColumnFamilyOptions_setInplaceUpdateSupport( + JNIEnv* env, jobject jobj, jlong jhandle, + jboolean jinplace_update_support) { + reinterpret_cast( + jhandle)->inplace_update_support = + static_cast(jinplace_update_support); +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: inplaceUpdateNumLocks + * Signature: (J)J + */ +jlong Java_org_rocksdb_ColumnFamilyOptions_inplaceUpdateNumLocks( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast( + jhandle)->inplace_update_num_locks; +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: setInplaceUpdateNumLocks + * Signature: (JJ)V + */ +void Java_org_rocksdb_ColumnFamilyOptions_setInplaceUpdateNumLocks( + JNIEnv* env, jobject jobj, jlong jhandle, + jlong jinplace_update_num_locks) { + rocksdb::Status s = rocksdb::check_if_jlong_fits_size_t( + jinplace_update_num_locks); + if (s.ok()) { + reinterpret_cast(jhandle)-> + inplace_update_num_locks = jinplace_update_num_locks; + } else { + rocksdb::RocksDBExceptionJni::ThrowNew(env, s); + } +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: memtablePrefixBloomBits + * Signature: (J)I + */ +jint Java_org_rocksdb_ColumnFamilyOptions_memtablePrefixBloomBits( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast( + jhandle)->memtable_prefix_bloom_bits; +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: setMemtablePrefixBloomBits + * Signature: (JI)V + */ +void Java_org_rocksdb_ColumnFamilyOptions_setMemtablePrefixBloomBits( + JNIEnv* env, jobject jobj, jlong jhandle, + jint jmemtable_prefix_bloom_bits) { + reinterpret_cast( + jhandle)->memtable_prefix_bloom_bits = + static_cast(jmemtable_prefix_bloom_bits); +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: memtablePrefixBloomProbes + * Signature: (J)I + */ +jint Java_org_rocksdb_ColumnFamilyOptions_memtablePrefixBloomProbes( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast( + jhandle)->memtable_prefix_bloom_probes; +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: setMemtablePrefixBloomProbes + * Signature: (JI)V + */ +void Java_org_rocksdb_ColumnFamilyOptions_setMemtablePrefixBloomProbes( + JNIEnv* env, jobject jobj, jlong jhandle, + jint jmemtable_prefix_bloom_probes) { + reinterpret_cast( + jhandle)->memtable_prefix_bloom_probes = + static_cast(jmemtable_prefix_bloom_probes); +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: bloomLocality + * Signature: (J)I + */ +jint Java_org_rocksdb_ColumnFamilyOptions_bloomLocality( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast(jhandle)-> + bloom_locality; +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: setBloomLocality + * Signature: (JI)V + */ +void Java_org_rocksdb_ColumnFamilyOptions_setBloomLocality( + JNIEnv* env, jobject jobj, jlong jhandle, jint jbloom_locality) { + reinterpret_cast(jhandle)->bloom_locality = + static_cast(jbloom_locality); +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: maxSuccessiveMerges + * Signature: (J)J + */ +jlong Java_org_rocksdb_ColumnFamilyOptions_maxSuccessiveMerges( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast(jhandle)-> + max_successive_merges; +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: setMaxSuccessiveMerges + * Signature: (JJ)V + */ +void Java_org_rocksdb_ColumnFamilyOptions_setMaxSuccessiveMerges( + JNIEnv* env, jobject jobj, jlong jhandle, + jlong jmax_successive_merges) { + rocksdb::Status s = rocksdb::check_if_jlong_fits_size_t( + jmax_successive_merges); + if (s.ok()) { + reinterpret_cast(jhandle)-> + max_successive_merges = jmax_successive_merges; + } else { + rocksdb::RocksDBExceptionJni::ThrowNew(env, s); + } +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: minPartialMergeOperands + * Signature: (J)I + */ +jint Java_org_rocksdb_ColumnFamilyOptions_minPartialMergeOperands( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast( + jhandle)->min_partial_merge_operands; +} + +/* + * Class: org_rocksdb_ColumnFamilyOptions + * Method: setMinPartialMergeOperands + * Signature: (JI)V + */ +void Java_org_rocksdb_ColumnFamilyOptions_setMinPartialMergeOperands( + JNIEnv* env, jobject jobj, jlong jhandle, + jint jmin_partial_merge_operands) { + reinterpret_cast( + jhandle)->min_partial_merge_operands = + static_cast(jmin_partial_merge_operands); +} + +///////////////////////////////////////////////////////////////////// +// rocksdb::DBOptions + +/* + * Class: org_rocksdb_DBOptions + * Method: newDBOptions + * Signature: ()V + */ +void Java_org_rocksdb_DBOptions_newDBOptions(JNIEnv* env, + jobject jobj) { + //rocksdb::DBOptions* dbop = new rocksdb::DBOptions(); + //rocksdb::DBOptionsJni::setHandle(env, jobj, dbop); +} + +/* + * Class: org_rocksdb_DBOptions + * Method: disposeInternal + * Signature: (J)V + */ +void Java_org_rocksdb_DBOptions_disposeInternal( + JNIEnv* env, jobject jobj, jlong handle) { + delete reinterpret_cast(handle); +} + +/* + * Class: org_rocksdb_DBOptions + * Method: setCreateIfMissing + * Signature: (JZ)V + */ +void Java_org_rocksdb_DBOptions_setCreateIfMissing( + JNIEnv* env, jobject jobj, jlong jhandle, jboolean flag) { + reinterpret_cast(jhandle)-> + create_if_missing = flag; +} + +/* + * Class: org_rocksdb_DBOptions + * Method: createIfMissing + * Signature: (J)Z + */ +jboolean Java_org_rocksdb_DBOptions_createIfMissing( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast(jhandle)->create_if_missing; +} + +/* + * Class: org_rocksdb_DBOptions + * Method: setCreateMissingColumnFamilies + * Signature: (JZ)V + */ +void Java_org_rocksdb_DBOptions_setCreateMissingColumnFamilies( + JNIEnv* env, jobject jobj, jlong jhandle, jboolean flag) { + reinterpret_cast + (jhandle)->create_missing_column_families = flag; +} + +/* + * Class: org_rocksdb_DBOptions + * Method: createMissingColumnFamilies + * Signature: (J)Z + */ +jboolean Java_org_rocksdb_DBOptions_createMissingColumnFamilies( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast + (jhandle)->create_missing_column_families; +} + +/* + * Class: org_rocksdb_DBOptions + * Method: setErrorIfExists + * Signature: (JZ)V + */ +void Java_org_rocksdb_DBOptions_setErrorIfExists( + JNIEnv* env, jobject jobj, jlong jhandle, jboolean error_if_exists) { + reinterpret_cast(jhandle)->error_if_exists = + static_cast(error_if_exists); +} + +/* + * Class: org_rocksdb_DBOptions + * Method: errorIfExists + * Signature: (J)Z + */ +jboolean Java_org_rocksdb_DBOptions_errorIfExists( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast(jhandle)->error_if_exists; +} + +/* + * Class: org_rocksdb_DBOptions + * Method: setParanoidChecks + * Signature: (JZ)V + */ +void Java_org_rocksdb_DBOptions_setParanoidChecks( + JNIEnv* env, jobject jobj, jlong jhandle, jboolean paranoid_checks) { + reinterpret_cast(jhandle)->paranoid_checks = + static_cast(paranoid_checks); +} + +/* + * Class: org_rocksdb_DBOptions + * Method: paranoidChecks + * Signature: (J)Z + */ +jboolean Java_org_rocksdb_DBOptions_paranoidChecks( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast(jhandle)->paranoid_checks; +} + +/* + * Class: org_rocksdb_DBOptions + * Method: setRateLimiter + * Signature: (JJ)V + */ +void Java_org_rocksdb_DBOptions_setRateLimiter( + JNIEnv* env, jobject jobj, jlong jhandle, jlong jrate_limiter_handle) { + reinterpret_cast(jhandle)->rate_limiter.reset( + reinterpret_cast(jrate_limiter_handle)); +} + +/* + * Class: org_rocksdb_DBOptions + * Method: setMaxTotalWalSize + * Signature: (JJ)V + */ +void Java_org_rocksdb_DBOptions_setMaxTotalWalSize( + JNIEnv* env, jobject jobj, jlong jhandle, + jlong jmax_total_wal_size) { + reinterpret_cast(jhandle)->max_total_wal_size = + static_cast(jmax_total_wal_size); +} + +/* + * Class: org_rocksdb_DBOptions + * Method: maxTotalWalSize + * Signature: (J)J + */ +jlong Java_org_rocksdb_DBOptions_maxTotalWalSize( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast(jhandle)-> + max_total_wal_size; +} + +/* + * Class: org_rocksdb_DBOptions + * Method: setMaxOpenFiles + * Signature: (JI)V + */ +void Java_org_rocksdb_DBOptions_setMaxOpenFiles( + JNIEnv* env, jobject jobj, jlong jhandle, jint max_open_files) { + reinterpret_cast(jhandle)->max_open_files = + static_cast(max_open_files); +} + +/* + * Class: org_rocksdb_DBOptions + * Method: maxOpenFiles + * Signature: (J)I + */ +jint Java_org_rocksdb_DBOptions_maxOpenFiles( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast(jhandle)->max_open_files; +} + +/* + * Class: org_rocksdb_DBOptions + * Method: createStatistics + * Signature: (J)V + */ +void Java_org_rocksdb_DBOptions_createStatistics( + JNIEnv* env, jobject jobj, jlong jOptHandle) { + reinterpret_cast(jOptHandle)->statistics = + rocksdb::CreateDBStatistics(); +} + +/* + * Class: org_rocksdb_DBOptions + * Method: statisticsPtr + * Signature: (J)J + */ +jlong Java_org_rocksdb_DBOptions_statisticsPtr( + JNIEnv* env, jobject jobj, jlong jOptHandle) { + auto st = reinterpret_cast(jOptHandle)-> + statistics.get(); + return reinterpret_cast(st); +} + +/* + * Class: org_rocksdb_DBOptions + * Method: setDisableDataSync + * Signature: (JZ)V + */ +void Java_org_rocksdb_DBOptions_setDisableDataSync( + JNIEnv* env, jobject jobj, jlong jhandle, jboolean disableDataSync) { + reinterpret_cast(jhandle)->disableDataSync = + static_cast(disableDataSync); +} + +/* + * Class: org_rocksdb_DBOptions + * Method: disableDataSync + * Signature: (J)Z + */ +jboolean Java_org_rocksdb_DBOptions_disableDataSync( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast(jhandle)->disableDataSync; +} + +/* + * Class: org_rocksdb_DBOptions + * Method: setUseFsync + * Signature: (JZ)V + */ +void Java_org_rocksdb_DBOptions_setUseFsync( + JNIEnv* env, jobject jobj, jlong jhandle, jboolean use_fsync) { + reinterpret_cast(jhandle)->use_fsync = + static_cast(use_fsync); +} + +/* + * Class: org_rocksdb_DBOptions + * Method: useFsync + * Signature: (J)Z + */ +jboolean Java_org_rocksdb_DBOptions_useFsync( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast(jhandle)->use_fsync; +} + +/* + * Class: org_rocksdb_DBOptions + * Method: setDbLogDir + * Signature: (JLjava/lang/String)V + */ +void Java_org_rocksdb_DBOptions_setDbLogDir( + JNIEnv* env, jobject jobj, jlong jhandle, jstring jdb_log_dir) { + const char* log_dir = env->GetStringUTFChars(jdb_log_dir, 0); + reinterpret_cast(jhandle)->db_log_dir.assign(log_dir); + env->ReleaseStringUTFChars(jdb_log_dir, log_dir); +} + +/* + * Class: org_rocksdb_DBOptions + * Method: dbLogDir + * Signature: (J)Ljava/lang/String + */ +jstring Java_org_rocksdb_DBOptions_dbLogDir( + JNIEnv* env, jobject jobj, jlong jhandle) { + return env->NewStringUTF( + reinterpret_cast(jhandle)->db_log_dir.c_str()); +} + +/* + * Class: org_rocksdb_DBOptions + * Method: setWalDir + * Signature: (JLjava/lang/String)V + */ +void Java_org_rocksdb_DBOptions_setWalDir( + JNIEnv* env, jobject jobj, jlong jhandle, jstring jwal_dir) { + const char* wal_dir = env->GetStringUTFChars(jwal_dir, 0); + reinterpret_cast(jhandle)->wal_dir.assign(wal_dir); + env->ReleaseStringUTFChars(jwal_dir, wal_dir); +} + +/* + * Class: org_rocksdb_DBOptions + * Method: walDir + * Signature: (J)Ljava/lang/String + */ +jstring Java_org_rocksdb_DBOptions_walDir( + JNIEnv* env, jobject jobj, jlong jhandle) { + return env->NewStringUTF( + reinterpret_cast(jhandle)->wal_dir.c_str()); +} + +/* + * Class: org_rocksdb_DBOptions + * Method: setDeleteObsoleteFilesPeriodMicros + * Signature: (JJ)V + */ +void Java_org_rocksdb_DBOptions_setDeleteObsoleteFilesPeriodMicros( + JNIEnv* env, jobject jobj, jlong jhandle, jlong micros) { + reinterpret_cast(jhandle) + ->delete_obsolete_files_period_micros = + static_cast(micros); +} + +/* + * Class: org_rocksdb_DBOptions + * Method: deleteObsoleteFilesPeriodMicros + * Signature: (J)J + */ +jlong Java_org_rocksdb_DBOptions_deleteObsoleteFilesPeriodMicros( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast(jhandle) + ->delete_obsolete_files_period_micros; +} + +/* + * Class: org_rocksdb_DBOptions + * Method: setMaxBackgroundCompactions + * Signature: (JI)V + */ +void Java_org_rocksdb_DBOptions_setMaxBackgroundCompactions( + JNIEnv* env, jobject jobj, jlong jhandle, jint max) { + reinterpret_cast(jhandle) + ->max_background_compactions = static_cast(max); +} + +/* + * Class: org_rocksdb_DBOptions + * Method: maxBackgroundCompactions + * Signature: (J)I + */ +jint Java_org_rocksdb_DBOptions_maxBackgroundCompactions( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast( + jhandle)->max_background_compactions; +} + +/* + * Class: org_rocksdb_DBOptions + * Method: setMaxBackgroundFlushes + * Signature: (JI)V + */ +void Java_org_rocksdb_DBOptions_setMaxBackgroundFlushes( + JNIEnv* env, jobject jobj, jlong jhandle, jint max_background_flushes) { + reinterpret_cast(jhandle)->max_background_flushes = + static_cast(max_background_flushes); +} + +/* + * Class: org_rocksdb_DBOptions + * Method: maxBackgroundFlushes + * Signature: (J)I + */ +jint Java_org_rocksdb_DBOptions_maxBackgroundFlushes( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast(jhandle)-> + max_background_flushes; +} + +/* + * Class: org_rocksdb_DBOptions + * Method: setMaxLogFileSize + * Signature: (JJ)V + */ +void Java_org_rocksdb_DBOptions_setMaxLogFileSize( + JNIEnv* env, jobject jobj, jlong jhandle, jlong max_log_file_size) { + rocksdb::Status s = rocksdb::check_if_jlong_fits_size_t(max_log_file_size); + if (s.ok()) { + reinterpret_cast(jhandle)->max_log_file_size = + max_log_file_size; + } else { + rocksdb::RocksDBExceptionJni::ThrowNew(env, s); + } +} + +/* + * Class: org_rocksdb_DBOptions + * Method: maxLogFileSize + * Signature: (J)J + */ +jlong Java_org_rocksdb_DBOptions_maxLogFileSize( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast(jhandle)->max_log_file_size; +} + +/* + * Class: org_rocksdb_DBOptions + * Method: setLogFileTimeToRoll + * Signature: (JJ)V + */ +void Java_org_rocksdb_DBOptions_setLogFileTimeToRoll( + JNIEnv* env, jobject jobj, jlong jhandle, jlong log_file_time_to_roll) { + rocksdb::Status s = rocksdb::check_if_jlong_fits_size_t( + log_file_time_to_roll); + if (s.ok()) { + reinterpret_cast(jhandle)->log_file_time_to_roll = + log_file_time_to_roll; + } else { + rocksdb::RocksDBExceptionJni::ThrowNew(env, s); + } +} + +/* + * Class: org_rocksdb_DBOptions + * Method: logFileTimeToRoll + * Signature: (J)J + */ +jlong Java_org_rocksdb_DBOptions_logFileTimeToRoll( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast(jhandle)->log_file_time_to_roll; +} + +/* + * Class: org_rocksdb_DBOptions + * Method: setKeepLogFileNum + * Signature: (JJ)V + */ +void Java_org_rocksdb_DBOptions_setKeepLogFileNum( + JNIEnv* env, jobject jobj, jlong jhandle, jlong keep_log_file_num) { + rocksdb::Status s = rocksdb::check_if_jlong_fits_size_t(keep_log_file_num); + if (s.ok()) { + reinterpret_cast(jhandle)->keep_log_file_num = + keep_log_file_num; + } else { + rocksdb::RocksDBExceptionJni::ThrowNew(env, s); + } +} + +/* + * Class: org_rocksdb_DBOptions + * Method: keepLogFileNum + * Signature: (J)J + */ +jlong Java_org_rocksdb_DBOptions_keepLogFileNum( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast(jhandle)->keep_log_file_num; +} + +/* + * Class: org_rocksdb_DBOptions + * Method: setMaxManifestFileSize + * Signature: (JJ)V + */ +void Java_org_rocksdb_DBOptions_setMaxManifestFileSize( + JNIEnv* env, jobject jobj, jlong jhandle, jlong max_manifest_file_size) { + reinterpret_cast(jhandle)->max_manifest_file_size = + static_cast(max_manifest_file_size); +} + +/* + * Class: org_rocksdb_DBOptions + * Method: maxManifestFileSize + * Signature: (J)J + */ +jlong Java_org_rocksdb_DBOptions_maxManifestFileSize( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast(jhandle)-> + max_manifest_file_size; +} + +/* + * Class: org_rocksdb_DBOptions + * Method: setTableCacheNumshardbits + * Signature: (JI)V + */ +void Java_org_rocksdb_DBOptions_setTableCacheNumshardbits( + JNIEnv* env, jobject jobj, jlong jhandle, jint table_cache_numshardbits) { + reinterpret_cast(jhandle)->table_cache_numshardbits = + static_cast(table_cache_numshardbits); +} + +/* + * Class: org_rocksdb_DBOptions + * Method: tableCacheNumshardbits + * Signature: (J)I + */ +jint Java_org_rocksdb_DBOptions_tableCacheNumshardbits( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast(jhandle)-> + table_cache_numshardbits; +} + +/* + * Class: org_rocksdb_DBOptions + * Method: setTableCacheRemoveScanCountLimit + * Signature: (JI)V + */ +void Java_org_rocksdb_DBOptions_setTableCacheRemoveScanCountLimit( + JNIEnv* env, jobject jobj, jlong jhandle, jint limit) { + reinterpret_cast( + jhandle)->table_cache_remove_scan_count_limit = static_cast(limit); +} + +/* + * Class: org_rocksdb_DBOptions + * Method: tableCacheRemoveScanCountLimit + * Signature: (J)I + */ +jint Java_org_rocksdb_DBOptions_tableCacheRemoveScanCountLimit( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast( + jhandle)->table_cache_remove_scan_count_limit; +} + +/* + * Class: org_rocksdb_DBOptions + * Method: setWalTtlSeconds + * Signature: (JJ)V + */ +void Java_org_rocksdb_DBOptions_setWalTtlSeconds( + JNIEnv* env, jobject jobj, jlong jhandle, jlong WAL_ttl_seconds) { + reinterpret_cast(jhandle)->WAL_ttl_seconds = + static_cast(WAL_ttl_seconds); +} + +/* + * Class: org_rocksdb_DBOptions + * Method: walTtlSeconds + * Signature: (J)J + */ +jlong Java_org_rocksdb_DBOptions_walTtlSeconds( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast(jhandle)->WAL_ttl_seconds; +} + +/* + * Class: org_rocksdb_DBOptions + * Method: setWalSizeLimitMB + * Signature: (JJ)V + */ +void Java_org_rocksdb_DBOptions_setWalSizeLimitMB( + JNIEnv* env, jobject jobj, jlong jhandle, jlong WAL_size_limit_MB) { + reinterpret_cast(jhandle)->WAL_size_limit_MB = + static_cast(WAL_size_limit_MB); +} + +/* + * Class: org_rocksdb_DBOptions + * Method: walTtlSeconds + * Signature: (J)J + */ +jlong Java_org_rocksdb_DBOptions_walSizeLimitMB( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast(jhandle)->WAL_size_limit_MB; +} + +/* + * Class: org_rocksdb_DBOptions + * Method: setManifestPreallocationSize + * Signature: (JJ)V + */ +void Java_org_rocksdb_DBOptions_setManifestPreallocationSize( + JNIEnv* env, jobject jobj, jlong jhandle, jlong preallocation_size) { + rocksdb::Status s = rocksdb::check_if_jlong_fits_size_t(preallocation_size); + if (s.ok()) { + reinterpret_cast(jhandle)-> + manifest_preallocation_size = preallocation_size; + } else { + rocksdb::RocksDBExceptionJni::ThrowNew(env, s); + } +} + +/* + * Class: org_rocksdb_DBOptions + * Method: manifestPreallocationSize + * Signature: (J)J + */ +jlong Java_org_rocksdb_DBOptions_manifestPreallocationSize( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast(jhandle) + ->manifest_preallocation_size; +} + +/* + * Class: org_rocksdb_DBOptions + * Method: setAllowOsBuffer + * Signature: (JZ)V + */ +void Java_org_rocksdb_DBOptions_setAllowOsBuffer( + JNIEnv* env, jobject jobj, jlong jhandle, jboolean allow_os_buffer) { + reinterpret_cast(jhandle)->allow_os_buffer = + static_cast(allow_os_buffer); +} + +/* + * Class: org_rocksdb_DBOptions + * Method: allowOsBuffer + * Signature: (J)Z + */ +jboolean Java_org_rocksdb_DBOptions_allowOsBuffer( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast(jhandle)->allow_os_buffer; +} + +/* + * Class: org_rocksdb_DBOptions + * Method: setAllowMmapReads + * Signature: (JZ)V + */ +void Java_org_rocksdb_DBOptions_setAllowMmapReads( + JNIEnv* env, jobject jobj, jlong jhandle, jboolean allow_mmap_reads) { + reinterpret_cast(jhandle)->allow_mmap_reads = + static_cast(allow_mmap_reads); +} + +/* + * Class: org_rocksdb_DBOptions + * Method: allowMmapReads + * Signature: (J)Z + */ +jboolean Java_org_rocksdb_DBOptions_allowMmapReads( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast(jhandle)->allow_mmap_reads; +} + +/* + * Class: org_rocksdb_DBOptions + * Method: setAllowMmapWrites + * Signature: (JZ)V + */ +void Java_org_rocksdb_DBOptions_setAllowMmapWrites( + JNIEnv* env, jobject jobj, jlong jhandle, jboolean allow_mmap_writes) { + reinterpret_cast(jhandle)->allow_mmap_writes = + static_cast(allow_mmap_writes); +} + +/* + * Class: org_rocksdb_DBOptions + * Method: allowMmapWrites + * Signature: (J)Z + */ +jboolean Java_org_rocksdb_DBOptions_allowMmapWrites( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast(jhandle)->allow_mmap_writes; +} + +/* + * Class: org_rocksdb_DBOptions + * Method: setIsFdCloseOnExec + * Signature: (JZ)V + */ +void Java_org_rocksdb_DBOptions_setIsFdCloseOnExec( + JNIEnv* env, jobject jobj, jlong jhandle, jboolean is_fd_close_on_exec) { + reinterpret_cast(jhandle)->is_fd_close_on_exec = + static_cast(is_fd_close_on_exec); +} + +/* + * Class: org_rocksdb_DBOptions + * Method: isFdCloseOnExec + * Signature: (J)Z + */ +jboolean Java_org_rocksdb_DBOptions_isFdCloseOnExec( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast(jhandle)->is_fd_close_on_exec; +} + +/* + * Class: org_rocksdb_DBOptions + * Method: setSkipLogErrorOnRecovery + * Signature: (JZ)V + */ +void Java_org_rocksdb_DBOptions_setSkipLogErrorOnRecovery( + JNIEnv* env, jobject jobj, jlong jhandle, jboolean skip) { + reinterpret_cast(jhandle)->skip_log_error_on_recovery = + static_cast(skip); +} + +/* + * Class: org_rocksdb_DBOptions + * Method: skipLogErrorOnRecovery + * Signature: (J)Z + */ +jboolean Java_org_rocksdb_DBOptions_skipLogErrorOnRecovery( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast(jhandle) + ->skip_log_error_on_recovery; +} + +/* + * Class: org_rocksdb_DBOptions + * Method: setStatsDumpPeriodSec + * Signature: (JI)V + */ +void Java_org_rocksdb_DBOptions_setStatsDumpPeriodSec( + JNIEnv* env, jobject jobj, jlong jhandle, jint stats_dump_period_sec) { + reinterpret_cast(jhandle)->stats_dump_period_sec = + static_cast(stats_dump_period_sec); +} + +/* + * Class: org_rocksdb_DBOptions + * Method: statsDumpPeriodSec + * Signature: (J)I + */ +jint Java_org_rocksdb_DBOptions_statsDumpPeriodSec( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast(jhandle)->stats_dump_period_sec; +} + +/* + * Class: org_rocksdb_DBOptions + * Method: setAdviseRandomOnOpen + * Signature: (JZ)V + */ +void Java_org_rocksdb_DBOptions_setAdviseRandomOnOpen( + JNIEnv* env, jobject jobj, jlong jhandle, jboolean advise_random_on_open) { + reinterpret_cast(jhandle)->advise_random_on_open = + static_cast(advise_random_on_open); +} + +/* + * Class: org_rocksdb_DBOptions + * Method: adviseRandomOnOpen + * Signature: (J)Z + */ +jboolean Java_org_rocksdb_DBOptions_adviseRandomOnOpen( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast(jhandle)->advise_random_on_open; +} + +/* + * Class: org_rocksdb_DBOptions + * Method: setUseAdaptiveMutex + * Signature: (JZ)V + */ +void Java_org_rocksdb_DBOptions_setUseAdaptiveMutex( + JNIEnv* env, jobject jobj, jlong jhandle, jboolean use_adaptive_mutex) { + reinterpret_cast(jhandle)->use_adaptive_mutex = + static_cast(use_adaptive_mutex); +} + +/* + * Class: org_rocksdb_DBOptions + * Method: useAdaptiveMutex + * Signature: (J)Z + */ +jboolean Java_org_rocksdb_DBOptions_useAdaptiveMutex( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast(jhandle)->use_adaptive_mutex; +} + +/* + * Class: org_rocksdb_DBOptions + * Method: setBytesPerSync + * Signature: (JJ)V + */ +void Java_org_rocksdb_DBOptions_setBytesPerSync( + JNIEnv* env, jobject jobj, jlong jhandle, jlong bytes_per_sync) { + reinterpret_cast(jhandle)->bytes_per_sync = + static_cast(bytes_per_sync); +} + +/* + * Class: org_rocksdb_DBOptions + * Method: bytesPerSync + * Signature: (J)J + */ +jlong Java_org_rocksdb_DBOptions_bytesPerSync( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast(jhandle)->bytes_per_sync; +} + +////////////////////////////////////////////////////////////////////////////// +// rocksdb::WriteOptions + +/* + * Class: org_rocksdb_WriteOptions + * Method: newWriteOptions + * Signature: ()V + */ +void Java_org_rocksdb_WriteOptions_newWriteOptions( + JNIEnv* env, jobject jwrite_options) { + rocksdb::WriteOptions* op = new rocksdb::WriteOptions(); + rocksdb::WriteOptionsJni::setHandle(env, jwrite_options, op); +} + +/* + * Class: org_rocksdb_WriteOptions + * Method: disposeInternal + * Signature: ()V + */ +void Java_org_rocksdb_WriteOptions_disposeInternal( + JNIEnv* env, jobject jwrite_options, jlong jhandle) { + auto write_options = reinterpret_cast(jhandle); + delete write_options; + + rocksdb::WriteOptionsJni::setHandle(env, jwrite_options, nullptr); +} + +/* + * Class: org_rocksdb_WriteOptions + * Method: setSync + * Signature: (JZ)V + */ +void Java_org_rocksdb_WriteOptions_setSync( + JNIEnv* env, jobject jwrite_options, jlong jhandle, jboolean jflag) { + reinterpret_cast(jhandle)->sync = jflag; +} + +/* + * Class: org_rocksdb_WriteOptions + * Method: sync + * Signature: (J)Z + */ +jboolean Java_org_rocksdb_WriteOptions_sync( + JNIEnv* env, jobject jwrite_options, jlong jhandle) { + return reinterpret_cast(jhandle)->sync; +} + +/* + * Class: org_rocksdb_WriteOptions + * Method: setDisableWAL + * Signature: (JZ)V + */ +void Java_org_rocksdb_WriteOptions_setDisableWAL( + JNIEnv* env, jobject jwrite_options, jlong jhandle, jboolean jflag) { + reinterpret_cast(jhandle)->disableWAL = jflag; +} + +/* + * Class: org_rocksdb_WriteOptions + * Method: disableWAL + * Signature: (J)Z + */ +jboolean Java_org_rocksdb_WriteOptions_disableWAL( + JNIEnv* env, jobject jwrite_options, jlong jhandle) { + return reinterpret_cast(jhandle)->disableWAL; +} + +///////////////////////////////////////////////////////////////////// +// rocksdb::ReadOptions + +/* + * Class: org_rocksdb_ReadOptions + * Method: newReadOptions + * Signature: ()V + */ +void Java_org_rocksdb_ReadOptions_newReadOptions( + JNIEnv* env, jobject jobj) { + auto read_opt = new rocksdb::ReadOptions(); + rocksdb::ReadOptionsJni::setHandle(env, jobj, read_opt); +} + +/* + * Class: org_rocksdb_ReadOptions + * Method: disposeInternal + * Signature: (J)V + */ +void Java_org_rocksdb_ReadOptions_disposeInternal( + JNIEnv* env, jobject jobj, jlong jhandle) { + delete reinterpret_cast(jhandle); + rocksdb::ReadOptionsJni::setHandle(env, jobj, nullptr); +} + +/* + * Class: org_rocksdb_ReadOptions + * Method: setVerifyChecksums + * Signature: (JZ)V + */ +void Java_org_rocksdb_ReadOptions_setVerifyChecksums( + JNIEnv* env, jobject jobj, jlong jhandle, + jboolean jverify_checksums) { + reinterpret_cast(jhandle)->verify_checksums = + static_cast(jverify_checksums); +} + +/* + * Class: org_rocksdb_ReadOptions + * Method: verifyChecksums + * Signature: (J)Z + */ +jboolean Java_org_rocksdb_ReadOptions_verifyChecksums( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast( + jhandle)->verify_checksums; +} + +/* + * Class: org_rocksdb_ReadOptions + * Method: setFillCache + * Signature: (JZ)V + */ +void Java_org_rocksdb_ReadOptions_setFillCache( + JNIEnv* env, jobject jobj, jlong jhandle, jboolean jfill_cache) { + reinterpret_cast(jhandle)->fill_cache = + static_cast(jfill_cache); +} + +/* + * Class: org_rocksdb_ReadOptions + * Method: fillCache + * Signature: (J)Z + */ +jboolean Java_org_rocksdb_ReadOptions_fillCache( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast(jhandle)->fill_cache; +} + +/* + * Class: org_rocksdb_ReadOptions + * Method: setTailing + * Signature: (JZ)V + */ +void Java_org_rocksdb_ReadOptions_setTailing( + JNIEnv* env, jobject jobj, jlong jhandle, jboolean jtailing) { + reinterpret_cast(jhandle)->tailing = + static_cast(jtailing); +} + +/* + * Class: org_rocksdb_ReadOptions + * Method: tailing + * Signature: (J)Z + */ +jboolean Java_org_rocksdb_ReadOptions_tailing( + JNIEnv* env, jobject jobj, jlong jhandle) { + return reinterpret_cast(jhandle)->tailing; } /*