[RocksJava] Incorporated changes for D32151

This commit is contained in:
fyrz 2015-01-30 21:57:40 +01:00
parent 68cd93b873
commit 391f85fc82
4 changed files with 120 additions and 14 deletions

View File

@ -23,7 +23,6 @@ public class TransactionLogIterator extends RocksObject {
* <strong>REQUIRES</strong>: Valid() to be true.</p>
*/
public void next() {
assert(isValid());
next(nativeHandle_);
}

View File

@ -53,6 +53,13 @@ public class WriteBatch extends AbstractWriteBatch {
iterate(handler.nativeHandle_);
}
/**
* <p>Private WriteBatch constructor which is used to construct
* WriteBatch instances from C++ side. As the reference to this
* object is also managed from C++ side the handle will be disowned.</p>
*
* @param nativeHandle address of native instance.
*/
WriteBatch(long nativeHandle) {
super();
disOwnNativeHandle();

View File

@ -41,6 +41,70 @@ public class TransactionLogIteratorTest {
@Test
public void getBatch() throws RocksDBException {
final int numberOfPuts = 5;
RocksDB db = null;
Options options = null;
ColumnFamilyHandle cfHandle = null;
TransactionLogIterator transactionLogIterator = null;
try {
options = new Options().
setCreateIfMissing(true).
setWalTtlSeconds(1000).
setWalSizeLimitMB(10);
db = RocksDB.open(options, dbFolder.getRoot().getAbsolutePath());
for (int i = 0; i < numberOfPuts; i++){
db.put(String.valueOf(i).getBytes(),
String.valueOf(i).getBytes());
}
db.flush(new FlushOptions().setWaitForFlush(true));
// the latest sequence number is 5 because 5 puts
// were written beforehand
assertThat(db.getLatestSequenceNumber()).
isEqualTo(numberOfPuts);
// insert 5 writes into a cf
cfHandle = db.createColumnFamily(
new ColumnFamilyDescriptor("new_cf".getBytes()));
for (int i = 0; i < numberOfPuts; i++){
db.put(cfHandle, String.valueOf(i).getBytes(),
String.valueOf(i).getBytes());
}
// the latest sequence number is 10 because
// (5 + 5) puts were written beforehand
assertThat(db.getLatestSequenceNumber()).
isEqualTo(numberOfPuts + numberOfPuts);
// Get updates since the beginning
transactionLogIterator = db.getUpdatesSince(0);
assertThat(transactionLogIterator.isValid()).isTrue();
transactionLogIterator.status();
// The first sequence number is 1
TransactionLogIterator.BatchResult batchResult =
transactionLogIterator.getBatch();
assertThat(batchResult.sequenceNumber()).isEqualTo(1);
} finally {
if (transactionLogIterator != null) {
transactionLogIterator.dispose();
}
if (cfHandle != null) {
cfHandle.dispose();
}
if (db != null) {
db.close();
}
if (options != null) {
options.dispose();
}
}
}
@Test
public void transactionLogIteratorStallAtLastRecord() throws RocksDBException {
RocksDB db = null;
Options options = null;
TransactionLogIterator transactionLogIterator = null;
@ -51,22 +115,59 @@ public class TransactionLogIteratorTest {
setWalSizeLimitMB(10);
db = RocksDB.open(options, dbFolder.getRoot().getAbsolutePath());
db.put("key1".getBytes(), "value1".getBytes());
// Get updates since the beginning
transactionLogIterator = db.getUpdatesSince(0);
transactionLogIterator.status();
assertThat(transactionLogIterator.isValid()).isTrue();
transactionLogIterator.next();
assertThat(transactionLogIterator.isValid()).isFalse();
transactionLogIterator.status();
db.put("key2".getBytes(), "value2".getBytes());
transactionLogIterator.next();
transactionLogIterator.status();
assertThat(transactionLogIterator.isValid()).isTrue();
for (int i = 0; i < 250; i++){
db.put(String.valueOf(i).getBytes(),
String.valueOf(i).getBytes());
} finally {
if (transactionLogIterator != null) {
transactionLogIterator.dispose();
}
db.flush(new FlushOptions().setWaitForFlush(true));
if (db != null) {
db.close();
}
if (options != null) {
options.dispose();
}
}
}
assertThat(db.getLatestSequenceNumber()).isEqualTo(250);
@Test
public void transactionLogIteratorCheckAfterRestart() throws RocksDBException {
final int numberOfKeys = 2;
RocksDB db = null;
Options options = null;
TransactionLogIterator transactionLogIterator = null;
try {
options = new Options().
setCreateIfMissing(true).
setWalTtlSeconds(1000).
setWalSizeLimitMB(10);
db = RocksDB.open(options, dbFolder.getRoot().getAbsolutePath());
db.put("key1".getBytes(), "value1".getBytes());
db.put("key2".getBytes(), "value2".getBytes());
db.flush(new FlushOptions().setWaitForFlush(true));
// reopen
db.close();
db = RocksDB.open(options, dbFolder.getRoot().getAbsolutePath());
assertThat(db.getLatestSequenceNumber()).isEqualTo(numberOfKeys);
transactionLogIterator = db.getUpdatesSince(0);
assertThat(transactionLogIterator.isValid()).isTrue();
transactionLogIterator.status();
TransactionLogIterator.BatchResult batchResult =
transactionLogIterator.getBatch();
assertThat(batchResult.sequenceNumber()).isEqualTo(1);
for (int i = 0; i < numberOfKeys; i++) {
transactionLogIterator.status();
assertThat(transactionLogIterator.isValid()).isTrue();
transactionLogIterator.next();
}
} finally {
if (transactionLogIterator != null) {
transactionLogIterator.dispose();

View File

@ -21,8 +21,7 @@
*/
void Java_org_rocksdb_TransactionLogIterator_disposeInternal(
JNIEnv* env, jobject jobj, jlong handle) {
auto* it = reinterpret_cast<rocksdb::TransactionLogIterator*>(handle);
delete it;
delete reinterpret_cast<rocksdb::TransactionLogIterator*>(handle);
}
/*