Browse Source
Reviewers: Bill Bejeck <bill@confluent.io>, John Roesler <john@confluent.io>, Guozhang Wang <guozhang@confluent.io>pull/6135/merge
Matthias J. Sax
6 years ago
committed by
GitHub
12 changed files with 2761 additions and 252 deletions
File diff suppressed because it is too large
Load Diff
@ -0,0 +1,62 @@
@@ -0,0 +1,62 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
package org.apache.kafka.streams.state.internals; |
||||
|
||||
import org.apache.kafka.common.utils.Bytes; |
||||
import org.apache.kafka.streams.KeyValue; |
||||
import org.apache.kafka.streams.state.KeyValueIterator; |
||||
import org.rocksdb.RocksIterator; |
||||
|
||||
import java.util.Comparator; |
||||
import java.util.Set; |
||||
|
||||
class RocksDBRangeIterator extends RocksDbIterator { |
||||
// RocksDB's JNI interface does not expose getters/setters that allow the
|
||||
// comparator to be pluggable, and the default is lexicographic, so it's
|
||||
// safe to just force lexicographic comparator here for now.
|
||||
private final Comparator<byte[]> comparator = Bytes.BYTES_LEXICO_COMPARATOR; |
||||
private final byte[] rawToKey; |
||||
|
||||
RocksDBRangeIterator(final String storeName, |
||||
final RocksIterator iter, |
||||
final Set<KeyValueIterator<Bytes, byte[]>> openIterators, |
||||
final Bytes from, |
||||
final Bytes to) { |
||||
super(storeName, iter, openIterators); |
||||
iter.seek(from.get()); |
||||
rawToKey = to.get(); |
||||
if (rawToKey == null) { |
||||
throw new NullPointerException("RocksDBRangeIterator: RawToKey is null for key " + to); |
||||
} |
||||
} |
||||
|
||||
@Override |
||||
public KeyValue<Bytes, byte[]> makeNext() { |
||||
final KeyValue<Bytes, byte[]> next = super.makeNext(); |
||||
|
||||
if (next == null) { |
||||
return allDone(); |
||||
} else { |
||||
if (comparator.compare(next.key.get(), rawToKey) <= 0) { |
||||
return next; |
||||
} else { |
||||
return allDone(); |
||||
} |
||||
} |
||||
} |
||||
} |
||||
|
@ -0,0 +1,394 @@
@@ -0,0 +1,394 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
package org.apache.kafka.streams.state.internals; |
||||
|
||||
import org.apache.kafka.common.utils.AbstractIterator; |
||||
import org.apache.kafka.common.utils.Bytes; |
||||
import org.apache.kafka.streams.KeyValue; |
||||
import org.apache.kafka.streams.errors.InvalidStateStoreException; |
||||
import org.apache.kafka.streams.errors.ProcessorStateException; |
||||
import org.apache.kafka.streams.state.KeyValueIterator; |
||||
import org.rocksdb.ColumnFamilyDescriptor; |
||||
import org.rocksdb.ColumnFamilyHandle; |
||||
import org.rocksdb.ColumnFamilyOptions; |
||||
import org.rocksdb.DBOptions; |
||||
import org.rocksdb.RocksDB; |
||||
import org.rocksdb.RocksDBException; |
||||
import org.rocksdb.RocksIterator; |
||||
import org.rocksdb.WriteBatch; |
||||
import org.slf4j.Logger; |
||||
import org.slf4j.LoggerFactory; |
||||
|
||||
import java.nio.charset.StandardCharsets; |
||||
import java.util.ArrayList; |
||||
import java.util.Collection; |
||||
import java.util.Comparator; |
||||
import java.util.List; |
||||
import java.util.NoSuchElementException; |
||||
import java.util.Objects; |
||||
|
||||
import static java.util.Arrays.asList; |
||||
import static org.apache.kafka.streams.state.internals.StoreProxyUtils.getValueWithUnknownTimestamp; |
||||
|
||||
/** |
||||
* A persistent key-(value-timestamp) store based on RocksDB. |
||||
*/ |
||||
public class RocksDBTimestampedStore extends RocksDBStore { |
||||
private static final Logger log = LoggerFactory.getLogger(RocksDBTimestampedStore.class); |
||||
|
||||
RocksDBTimestampedStore(final String name) { |
||||
super(name); |
||||
} |
||||
|
||||
@Override |
||||
void openRocksDB(final DBOptions dbOptions, |
||||
final ColumnFamilyOptions columnFamilyOptions) { |
||||
final List<ColumnFamilyDescriptor> columnFamilyDescriptors = asList( |
||||
new ColumnFamilyDescriptor(RocksDB.DEFAULT_COLUMN_FAMILY, columnFamilyOptions), |
||||
new ColumnFamilyDescriptor("keyValueWithTimestamp".getBytes(StandardCharsets.UTF_8), columnFamilyOptions)); |
||||
final List<ColumnFamilyHandle> columnFamilies = new ArrayList<>(columnFamilyDescriptors.size()); |
||||
|
||||
try { |
||||
db = RocksDB.open(dbOptions, dbDir.getAbsolutePath(), columnFamilyDescriptors, columnFamilies); |
||||
|
||||
final ColumnFamilyHandle noTimestampColumnFamily = columnFamilies.get(0); |
||||
|
||||
final RocksIterator noTimestampsIter = db.newIterator(noTimestampColumnFamily); |
||||
noTimestampsIter.seekToFirst(); |
||||
if (noTimestampsIter.isValid()) { |
||||
log.info("Opening store {} in upgrade mode", name); |
||||
dbAccessor = new DualColumnFamilyAccessor(noTimestampColumnFamily, columnFamilies.get(1)); |
||||
} else { |
||||
log.info("Opening store {} in regular mode", name); |
||||
dbAccessor = new SingleColumnFamilyAccessor(columnFamilies.get(1)); |
||||
} |
||||
noTimestampsIter.close(); |
||||
} catch (final RocksDBException e) { |
||||
if ("Column family not found: : keyValueWithTimestamp".equals(e.getMessage())) { |
||||
try { |
||||
db = RocksDB.open(dbOptions, dbDir.getAbsolutePath(), columnFamilyDescriptors.subList(0, 1), columnFamilies); |
||||
columnFamilies.add(db.createColumnFamily(columnFamilyDescriptors.get(1))); |
||||
} catch (final RocksDBException fatal) { |
||||
throw new ProcessorStateException("Error opening store " + name + " at location " + dbDir.toString(), fatal); |
||||
} |
||||
log.info("Opening store {} in upgrade mode", name); |
||||
dbAccessor = new DualColumnFamilyAccessor(columnFamilies.get(0), columnFamilies.get(1)); |
||||
} else { |
||||
throw new ProcessorStateException("Error opening store " + name + " at location " + dbDir.toString(), e); |
||||
} |
||||
} |
||||
} |
||||
|
||||
|
||||
|
||||
private class DualColumnFamilyAccessor implements RocksDBAccessor { |
||||
private final ColumnFamilyHandle oldColumnFamily; |
||||
private final ColumnFamilyHandle newColumnFamily; |
||||
|
||||
private DualColumnFamilyAccessor(final ColumnFamilyHandle oldColumnFamily, |
||||
final ColumnFamilyHandle newColumnFamily) { |
||||
this.oldColumnFamily = oldColumnFamily; |
||||
this.newColumnFamily = newColumnFamily; |
||||
} |
||||
|
||||
@Override |
||||
public void put(final byte[] key, |
||||
final byte[] valueWithTimestamp) { |
||||
if (valueWithTimestamp == null) { |
||||
try { |
||||
db.delete(oldColumnFamily, wOptions, key); |
||||
} catch (final RocksDBException e) { |
||||
// String format is happening in wrapping stores. So formatted message is thrown from wrapping stores.
|
||||
throw new ProcessorStateException("Error while removing key from store " + name, e); |
||||
} |
||||
try { |
||||
db.delete(newColumnFamily, wOptions, key); |
||||
} catch (final RocksDBException e) { |
||||
// String format is happening in wrapping stores. So formatted message is thrown from wrapping stores.
|
||||
throw new ProcessorStateException("Error while removing key from store " + name, e); |
||||
} |
||||
} else { |
||||
try { |
||||
db.delete(oldColumnFamily, wOptions, key); |
||||
} catch (final RocksDBException e) { |
||||
// String format is happening in wrapping stores. So formatted message is thrown from wrapping stores.
|
||||
throw new ProcessorStateException("Error while removing key from store " + name, e); |
||||
} |
||||
try { |
||||
db.put(newColumnFamily, wOptions, key, valueWithTimestamp); |
||||
} catch (final RocksDBException e) { |
||||
// String format is happening in wrapping stores. So formatted message is thrown from wrapping stores.
|
||||
throw new ProcessorStateException("Error while putting key/value into store " + name, e); |
||||
} |
||||
} |
||||
} |
||||
|
||||
@Override |
||||
public void prepareBatch(final List<KeyValue<Bytes, byte[]>> entries, |
||||
final WriteBatch batch) throws RocksDBException { |
||||
for (final KeyValue<Bytes, byte[]> entry : entries) { |
||||
Objects.requireNonNull(entry.key, "key cannot be null"); |
||||
if (entry.value == null) { |
||||
batch.delete(oldColumnFamily, entry.key.get()); |
||||
batch.delete(newColumnFamily, entry.key.get()); |
||||
} else { |
||||
batch.delete(oldColumnFamily, entry.key.get()); |
||||
batch.put(newColumnFamily, entry.key.get(), entry.value); |
||||
} |
||||
} |
||||
} |
||||
|
||||
@Override |
||||
public byte[] get(final byte[] key) throws RocksDBException { |
||||
final byte[] valueWithTimestamp = db.get(newColumnFamily, key); |
||||
if (valueWithTimestamp != null) { |
||||
return valueWithTimestamp; |
||||
} |
||||
|
||||
final byte[] plainValue = db.get(oldColumnFamily, key); |
||||
if (plainValue != null) { |
||||
final byte[] valueWithUnknownTimestamp = getValueWithUnknownTimestamp(plainValue); |
||||
// this does only work, because the changelog topic contains correct data already
|
||||
// for other format changes, we cannot take this short cut and can only migrate data
|
||||
// from old to new store on put()
|
||||
put(key, valueWithUnknownTimestamp); |
||||
return valueWithUnknownTimestamp; |
||||
} |
||||
|
||||
return null; |
||||
} |
||||
|
||||
@Override |
||||
public byte[] getOnly(final byte[] key) throws RocksDBException { |
||||
final byte[] valueWithTimestamp = db.get(newColumnFamily, key); |
||||
if (valueWithTimestamp != null) { |
||||
return valueWithTimestamp; |
||||
} |
||||
|
||||
final byte[] plainValue = db.get(oldColumnFamily, key); |
||||
if (plainValue != null) { |
||||
return getValueWithUnknownTimestamp(plainValue); |
||||
} |
||||
|
||||
return null; |
||||
} |
||||
|
||||
@Override |
||||
public KeyValueIterator<Bytes, byte[]> range(final Bytes from, |
||||
final Bytes to) { |
||||
return new RocksDBDualCFRangeIterator( |
||||
name, |
||||
db.newIterator(newColumnFamily), |
||||
db.newIterator(oldColumnFamily), |
||||
from, |
||||
to); |
||||
} |
||||
|
||||
@Override |
||||
public KeyValueIterator<Bytes, byte[]> all() { |
||||
final RocksIterator innerIterWithTimestamp = db.newIterator(newColumnFamily); |
||||
innerIterWithTimestamp.seekToFirst(); |
||||
final RocksIterator innerIterNoTimestamp = db.newIterator(oldColumnFamily); |
||||
innerIterNoTimestamp.seekToFirst(); |
||||
return new RocksDBDualCFIterator(name, innerIterWithTimestamp, innerIterNoTimestamp); |
||||
} |
||||
|
||||
@Override |
||||
public long approximateNumEntries() throws RocksDBException { |
||||
return db.getLongProperty(oldColumnFamily, "rocksdb.estimate-num-keys") |
||||
+ db.getLongProperty(newColumnFamily, "rocksdb.estimate-num-keys"); |
||||
} |
||||
|
||||
@Override |
||||
public void flush() throws RocksDBException { |
||||
db.flush(fOptions, oldColumnFamily); |
||||
db.flush(fOptions, newColumnFamily); |
||||
} |
||||
|
||||
@Override |
||||
public void prepareBatchForRestore(final Collection<KeyValue<byte[], byte[]>> records, |
||||
final WriteBatch batch) throws RocksDBException { |
||||
for (final KeyValue<byte[], byte[]> record : records) { |
||||
if (record.value == null) { |
||||
batch.delete(oldColumnFamily, record.key); |
||||
batch.delete(newColumnFamily, record.key); |
||||
} else { |
||||
batch.delete(oldColumnFamily, record.key); |
||||
batch.put(newColumnFamily, record.key, record.value); |
||||
} |
||||
} |
||||
} |
||||
|
||||
@Override |
||||
public void close() { |
||||
oldColumnFamily.close(); |
||||
newColumnFamily.close(); |
||||
} |
||||
|
||||
@Override |
||||
public void toggleDbForBulkLoading() { |
||||
try { |
||||
db.compactRange(oldColumnFamily, true, 1, 0); |
||||
} catch (final RocksDBException e) { |
||||
throw new ProcessorStateException("Error while range compacting during restoring store " + name, e); |
||||
} |
||||
try { |
||||
db.compactRange(newColumnFamily, true, 1, 0); |
||||
} catch (final RocksDBException e) { |
||||
throw new ProcessorStateException("Error while range compacting during restoring store " + name, e); |
||||
} |
||||
} |
||||
} |
||||
|
||||
private class RocksDBDualCFIterator extends AbstractIterator<KeyValue<Bytes, byte[]>> |
||||
implements KeyValueIterator<Bytes, byte[]> { |
||||
|
||||
// RocksDB's JNI interface does not expose getters/setters that allow the
|
||||
// comparator to be pluggable, and the default is lexicographic, so it's
|
||||
// safe to just force lexicographic comparator here for now.
|
||||
private final Comparator<byte[]> comparator = Bytes.BYTES_LEXICO_COMPARATOR; |
||||
|
||||
private final String storeName; |
||||
private final RocksIterator iterWithTimestamp; |
||||
private final RocksIterator iterNoTimestamp; |
||||
|
||||
private volatile boolean open = true; |
||||
|
||||
private byte[] nextWithTimestamp; |
||||
private byte[] nextNoTimestamp; |
||||
private KeyValue<Bytes, byte[]> next; |
||||
|
||||
RocksDBDualCFIterator(final String storeName, |
||||
final RocksIterator iterWithTimestamp, |
||||
final RocksIterator iterNoTimestamp) { |
||||
this.iterWithTimestamp = iterWithTimestamp; |
||||
this.iterNoTimestamp = iterNoTimestamp; |
||||
this.storeName = storeName; |
||||
} |
||||
|
||||
@Override |
||||
public synchronized boolean hasNext() { |
||||
if (!open) { |
||||
throw new InvalidStateStoreException(String.format("RocksDB iterator for store %s has closed", storeName)); |
||||
} |
||||
return super.hasNext(); |
||||
} |
||||
|
||||
@Override |
||||
public synchronized KeyValue<Bytes, byte[]> next() { |
||||
return super.next(); |
||||
} |
||||
|
||||
@Override |
||||
public KeyValue<Bytes, byte[]> makeNext() { |
||||
if (nextNoTimestamp == null && iterNoTimestamp.isValid()) { |
||||
nextNoTimestamp = iterNoTimestamp.key(); |
||||
} |
||||
|
||||
if (nextWithTimestamp == null && iterWithTimestamp.isValid()) { |
||||
nextWithTimestamp = iterWithTimestamp.key(); |
||||
} |
||||
|
||||
if (nextNoTimestamp == null && !iterNoTimestamp.isValid()) { |
||||
if (nextWithTimestamp == null && !iterWithTimestamp.isValid()) { |
||||
return allDone(); |
||||
} else { |
||||
next = KeyValue.pair(new Bytes(nextWithTimestamp), iterWithTimestamp.value()); |
||||
nextWithTimestamp = null; |
||||
iterWithTimestamp.next(); |
||||
} |
||||
} else { |
||||
if (nextWithTimestamp == null) { |
||||
next = KeyValue.pair(new Bytes(nextNoTimestamp), getValueWithUnknownTimestamp(iterNoTimestamp.value())); |
||||
nextNoTimestamp = null; |
||||
iterNoTimestamp.next(); |
||||
} else { |
||||
if (comparator.compare(nextNoTimestamp, nextWithTimestamp) <= 0) { |
||||
next = KeyValue.pair(new Bytes(nextNoTimestamp), getValueWithUnknownTimestamp(iterNoTimestamp.value())); |
||||
nextNoTimestamp = null; |
||||
iterNoTimestamp.next(); |
||||
} else { |
||||
next = KeyValue.pair(new Bytes(nextWithTimestamp), iterWithTimestamp.value()); |
||||
nextWithTimestamp = null; |
||||
iterWithTimestamp.next(); |
||||
} |
||||
} |
||||
} |
||||
|
||||
return next; |
||||
} |
||||
|
||||
@Override |
||||
public void remove() { |
||||
throw new UnsupportedOperationException("RocksDB iterator does not support remove()"); |
||||
} |
||||
|
||||
@Override |
||||
public synchronized void close() { |
||||
openIterators.remove(this); |
||||
iterNoTimestamp.close(); |
||||
iterWithTimestamp.close(); |
||||
open = false; |
||||
} |
||||
|
||||
@Override |
||||
public Bytes peekNextKey() { |
||||
if (!hasNext()) { |
||||
throw new NoSuchElementException(); |
||||
} |
||||
return next.key; |
||||
} |
||||
} |
||||
|
||||
private class RocksDBDualCFRangeIterator extends RocksDBDualCFIterator { |
||||
// RocksDB's JNI interface does not expose getters/setters that allow the
|
||||
// comparator to be pluggable, and the default is lexicographic, so it's
|
||||
// safe to just force lexicographic comparator here for now.
|
||||
private final Comparator<byte[]> comparator = Bytes.BYTES_LEXICO_COMPARATOR; |
||||
private final byte[] upperBoundKey; |
||||
|
||||
RocksDBDualCFRangeIterator(final String storeName, |
||||
final RocksIterator iterWithTimestamp, |
||||
final RocksIterator iterNoTimestamp, |
||||
final Bytes from, |
||||
final Bytes to) { |
||||
super(storeName, iterWithTimestamp, iterNoTimestamp); |
||||
iterWithTimestamp.seek(from.get()); |
||||
iterNoTimestamp.seek(from.get()); |
||||
upperBoundKey = to.get(); |
||||
if (upperBoundKey == null) { |
||||
throw new NullPointerException("RocksDBDualCFRangeIterator: upperBoundKey is null for key " + to); |
||||
} |
||||
} |
||||
|
||||
@Override |
||||
public KeyValue<Bytes, byte[]> makeNext() { |
||||
final KeyValue<Bytes, byte[]> next = super.makeNext(); |
||||
|
||||
if (next == null) { |
||||
return allDone(); |
||||
} else { |
||||
if (comparator.compare(next.key.get(), upperBoundKey) <= 0) { |
||||
return next; |
||||
} else { |
||||
return allDone(); |
||||
} |
||||
} |
||||
} |
||||
} |
||||
|
||||
} |
@ -0,0 +1,89 @@
@@ -0,0 +1,89 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
package org.apache.kafka.streams.state.internals; |
||||
|
||||
import org.apache.kafka.common.utils.AbstractIterator; |
||||
import org.apache.kafka.common.utils.Bytes; |
||||
import org.apache.kafka.streams.KeyValue; |
||||
import org.apache.kafka.streams.errors.InvalidStateStoreException; |
||||
import org.apache.kafka.streams.state.KeyValueIterator; |
||||
import org.rocksdb.RocksIterator; |
||||
|
||||
import java.util.NoSuchElementException; |
||||
import java.util.Set; |
||||
|
||||
class RocksDbIterator extends AbstractIterator<KeyValue<Bytes, byte[]>> implements KeyValueIterator<Bytes, byte[]> { |
||||
|
||||
private final String storeName; |
||||
private final RocksIterator iter; |
||||
private final Set<KeyValueIterator<Bytes, byte[]>> openIterators; |
||||
|
||||
private volatile boolean open = true; |
||||
|
||||
private KeyValue<Bytes, byte[]> next; |
||||
|
||||
RocksDbIterator(final String storeName, |
||||
final RocksIterator iter, |
||||
final Set<KeyValueIterator<Bytes, byte[]>> openIterators) { |
||||
this.storeName = storeName; |
||||
this.iter = iter; |
||||
this.openIterators = openIterators; |
||||
} |
||||
|
||||
@Override |
||||
public synchronized boolean hasNext() { |
||||
if (!open) { |
||||
throw new InvalidStateStoreException(String.format("RocksDB iterator for store %s has closed", storeName)); |
||||
} |
||||
return super.hasNext(); |
||||
} |
||||
|
||||
@Override |
||||
public KeyValue<Bytes, byte[]> makeNext() { |
||||
if (!iter.isValid()) { |
||||
return allDone(); |
||||
} else { |
||||
next = getKeyValue(); |
||||
iter.next(); |
||||
return next; |
||||
} |
||||
} |
||||
|
||||
private KeyValue<Bytes, byte[]> getKeyValue() { |
||||
return new KeyValue<>(new Bytes(iter.key()), iter.value()); |
||||
} |
||||
|
||||
@Override |
||||
public void remove() { |
||||
throw new UnsupportedOperationException("RocksDB iterator does not support remove()"); |
||||
} |
||||
|
||||
@Override |
||||
public synchronized void close() { |
||||
openIterators.remove(this); |
||||
iter.close(); |
||||
open = false; |
||||
} |
||||
|
||||
@Override |
||||
public Bytes peekNextKey() { |
||||
if (!hasNext()) { |
||||
throw new NoSuchElementException(); |
||||
} |
||||
return next.key; |
||||
} |
||||
} |
@ -0,0 +1,36 @@
@@ -0,0 +1,36 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
package org.apache.kafka.streams.state.internals; |
||||
|
||||
import java.nio.ByteBuffer; |
||||
|
||||
import static org.apache.kafka.clients.consumer.ConsumerRecord.NO_TIMESTAMP; |
||||
|
||||
class StoreProxyUtils { |
||||
|
||||
static byte[] getValueWithUnknownTimestamp(final byte[] rawValue) { |
||||
if (rawValue == null) { |
||||
return null; |
||||
} |
||||
return ByteBuffer |
||||
.allocate(8 + rawValue.length) |
||||
.putLong(NO_TIMESTAMP) |
||||
.put(rawValue) |
||||
.array(); |
||||
} |
||||
|
||||
} |
@ -0,0 +1,263 @@
@@ -0,0 +1,263 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
package org.apache.kafka.streams.state.internals; |
||||
|
||||
import org.easymock.EasyMockRunner; |
||||
import org.easymock.Mock; |
||||
import org.junit.Test; |
||||
import org.junit.runner.RunWith; |
||||
import org.rocksdb.AccessHint; |
||||
import org.rocksdb.BuiltinComparator; |
||||
import org.rocksdb.ColumnFamilyOptions; |
||||
import org.rocksdb.CompactionPriority; |
||||
import org.rocksdb.CompactionStyle; |
||||
import org.rocksdb.ComparatorOptions; |
||||
import org.rocksdb.CompressionType; |
||||
import org.rocksdb.DBOptions; |
||||
import org.rocksdb.Env; |
||||
import org.rocksdb.InfoLogLevel; |
||||
import org.rocksdb.LRUCache; |
||||
import org.rocksdb.Logger; |
||||
import org.rocksdb.Options; |
||||
import org.rocksdb.PlainTableConfig; |
||||
import org.rocksdb.RateLimiter; |
||||
import org.rocksdb.RocksDB; |
||||
import org.rocksdb.SstFileManager; |
||||
import org.rocksdb.StringAppendOperator; |
||||
import org.rocksdb.VectorMemTableConfig; |
||||
import org.rocksdb.WALRecoveryMode; |
||||
import org.rocksdb.util.BytewiseComparator; |
||||
|
||||
import java.lang.reflect.InvocationTargetException; |
||||
import java.lang.reflect.Method; |
||||
import java.util.ArrayList; |
||||
import java.util.LinkedList; |
||||
import java.util.List; |
||||
|
||||
import static org.easymock.EasyMock.replay; |
||||
import static org.easymock.EasyMock.reset; |
||||
import static org.easymock.EasyMock.verify; |
||||
import static org.hamcrest.Matchers.instanceOf; |
||||
import static org.hamcrest.Matchers.matchesPattern; |
||||
import static org.junit.Assert.assertThat; |
||||
import static org.junit.Assert.fail; |
||||
|
||||
/** |
||||
* The purpose of this test is, to catch interface changes if we upgrade {@link RocksDB}. |
||||
* Using reflections, we make sure the {@link RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapter} maps all |
||||
* methods from {@link DBOptions} and {@link ColumnFamilyOptions} to/from {@link Options} correctly. |
||||
*/ |
||||
@RunWith(EasyMockRunner.class) |
||||
public class RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapterTest { |
||||
private final List<String> ignoreMethods = new LinkedList<String>() { |
||||
{ |
||||
add("isOwningHandle"); |
||||
add("dispose"); |
||||
add("wait"); |
||||
add("equals"); |
||||
add("getClass"); |
||||
add("hashCode"); |
||||
add("notify"); |
||||
add("notifyAll"); |
||||
add("toString"); |
||||
} |
||||
}; |
||||
|
||||
@Mock |
||||
private DBOptions dbOptions; |
||||
@Mock |
||||
private ColumnFamilyOptions columnFamilyOptions; |
||||
|
||||
@Test |
||||
public void shouldOverwriteAllOptionsMethods() throws Exception { |
||||
for (final Method method : Options.class.getMethods()) { |
||||
if (!ignoreMethods.contains(method.getName())) { |
||||
RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapter.class |
||||
.getDeclaredMethod(method.getName(), method.getParameterTypes()); |
||||
} |
||||
} |
||||
} |
||||
|
||||
@Test |
||||
public void shouldForwardAllDbOptionsCalls() throws Exception { |
||||
for (final Method method : Options.class.getMethods()) { |
||||
if (!ignoreMethods.contains(method.getName())) { |
||||
try { |
||||
DBOptions.class.getMethod(method.getName(), method.getParameterTypes()); |
||||
verifyDBOptionsMethodCall(method); |
||||
} catch (final NoSuchMethodException expectedAndSwallow) { } |
||||
} |
||||
} |
||||
} |
||||
|
||||
private void verifyDBOptionsMethodCall(final Method method) throws Exception { |
||||
final RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapter optionsFacadeDbOptions |
||||
= new RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapter(dbOptions, new ColumnFamilyOptions()); |
||||
|
||||
final Object[] parameters = getDBOptionsParameters(method.getParameterTypes()); |
||||
|
||||
try { |
||||
reset(dbOptions); |
||||
replay(dbOptions); |
||||
method.invoke(optionsFacadeDbOptions, parameters); |
||||
verify(); |
||||
fail("Should have called DBOptions." + method.getName() + "()"); |
||||
} catch (final InvocationTargetException undeclaredMockMethodCall) { |
||||
assertThat(undeclaredMockMethodCall.getCause(), instanceOf(AssertionError.class)); |
||||
assertThat(undeclaredMockMethodCall.getCause().getMessage().trim(), |
||||
matchesPattern("Unexpected method call DBOptions\\." + method.getName() + "((.*\n*)*):")); |
||||
} |
||||
} |
||||
|
||||
private Object[] getDBOptionsParameters(final Class<?>[] parameterTypes) throws Exception { |
||||
final Object[] parameters = new Object[parameterTypes.length]; |
||||
|
||||
for (int i = 0; i < parameterTypes.length; ++i) { |
||||
switch (parameterTypes[i].getName()) { |
||||
case "boolean": |
||||
parameters[i] = true; |
||||
break; |
||||
case "int": |
||||
parameters[i] = 0; |
||||
break; |
||||
case "long": |
||||
parameters[i] = 0L; |
||||
break; |
||||
case "java.util.Collection": |
||||
parameters[i] = new ArrayList<>(); |
||||
break; |
||||
case "org.rocksdb.AccessHint": |
||||
parameters[i] = AccessHint.NONE; |
||||
break; |
||||
case "org.rocksdb.Cache": |
||||
parameters[i] = new LRUCache(1L); |
||||
break; |
||||
case "org.rocksdb.Env": |
||||
parameters[i] = Env.getDefault(); |
||||
break; |
||||
case "org.rocksdb.InfoLogLevel": |
||||
parameters[i] = InfoLogLevel.FATAL_LEVEL; |
||||
break; |
||||
case "org.rocksdb.Logger": |
||||
parameters[i] = new Logger(new Options()) { |
||||
@Override |
||||
protected void log(final InfoLogLevel infoLogLevel, final String logMsg) {} |
||||
}; |
||||
break; |
||||
case "org.rocksdb.RateLimiter": |
||||
parameters[i] = new RateLimiter(1L); |
||||
break; |
||||
case "org.rocksdb.SstFileManager": |
||||
parameters[i] = new SstFileManager(Env.getDefault()); |
||||
break; |
||||
case "org.rocksdb.WALRecoveryMode": |
||||
parameters[i] = WALRecoveryMode.AbsoluteConsistency; |
||||
break; |
||||
default: |
||||
parameters[i] = parameterTypes[i].newInstance(); |
||||
} |
||||
} |
||||
|
||||
return parameters; |
||||
} |
||||
|
||||
@Test |
||||
public void shouldForwardAllColumnFamilyCalls() throws Exception { |
||||
for (final Method method : Options.class.getMethods()) { |
||||
if (!ignoreMethods.contains(method.getName())) { |
||||
try { |
||||
ColumnFamilyOptions.class.getMethod(method.getName(), method.getParameterTypes()); |
||||
verifyColumnFamilyOptionsMethodCall(method); |
||||
} catch (final NoSuchMethodException expectedAndSwallow) { } |
||||
} |
||||
} |
||||
} |
||||
|
||||
private void verifyColumnFamilyOptionsMethodCall(final Method method) throws Exception { |
||||
final RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapter optionsFacadeColumnFamilyOptions |
||||
= new RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapter(new DBOptions(), columnFamilyOptions); |
||||
|
||||
final Object[] parameters = getColumnFamilyOptionsParameters(method.getParameterTypes()); |
||||
|
||||
try { |
||||
reset(columnFamilyOptions); |
||||
replay(columnFamilyOptions); |
||||
method.invoke(optionsFacadeColumnFamilyOptions, parameters); |
||||
verify(); |
||||
fail("Should have called ColumnFamilyOptions." + method.getName() + "()"); |
||||
} catch (final InvocationTargetException undeclaredMockMethodCall) { |
||||
assertThat(undeclaredMockMethodCall.getCause(), instanceOf(AssertionError.class)); |
||||
assertThat(undeclaredMockMethodCall.getCause().getMessage().trim(), |
||||
matchesPattern("Unexpected method call ColumnFamilyOptions\\." + method.getName() + "(.*)")); |
||||
} |
||||
} |
||||
|
||||
private Object[] getColumnFamilyOptionsParameters(final Class<?>[] parameterTypes) throws Exception { |
||||
final Object[] parameters = new Object[parameterTypes.length]; |
||||
|
||||
for (int i = 0; i < parameterTypes.length; ++i) { |
||||
switch (parameterTypes[i].getName()) { |
||||
case "boolean": |
||||
parameters[i] = true; |
||||
break; |
||||
case "double": |
||||
parameters[i] = 0.0d; |
||||
break; |
||||
case "int": |
||||
parameters[i] = 0; |
||||
break; |
||||
case "long": |
||||
parameters[i] = 0L; |
||||
break; |
||||
case "[I": |
||||
parameters[i] = new int[0]; |
||||
break; |
||||
case "java.util.List": |
||||
parameters[i] = new ArrayList<>(); |
||||
break; |
||||
case "org.rocksdb.AbstractComparator": |
||||
parameters[i] = new BytewiseComparator(new ComparatorOptions()); |
||||
break; |
||||
case "org.rocksdb.BuiltinComparator": |
||||
parameters[i] = BuiltinComparator.BYTEWISE_COMPARATOR; |
||||
break; |
||||
case "org.rocksdb.CompactionPriority": |
||||
parameters[i] = CompactionPriority.ByCompensatedSize; |
||||
break; |
||||
case "org.rocksdb.CompactionStyle": |
||||
parameters[i] = CompactionStyle.UNIVERSAL; |
||||
break; |
||||
case "org.rocksdb.CompressionType": |
||||
parameters[i] = CompressionType.NO_COMPRESSION; |
||||
break; |
||||
case "org.rocksdb.MemTableConfig": |
||||
parameters[i] = new VectorMemTableConfig(); |
||||
break; |
||||
case "org.rocksdb.MergeOperator": |
||||
parameters[i] = new StringAppendOperator(); |
||||
break; |
||||
case "org.rocksdb.TableFormatConfig": |
||||
parameters[i] = new PlainTableConfig(); |
||||
break; |
||||
default: |
||||
parameters[i] = parameterTypes[i].newInstance(); |
||||
} |
||||
} |
||||
|
||||
return parameters; |
||||
} |
||||
} |
@ -0,0 +1,293 @@
@@ -0,0 +1,293 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
package org.apache.kafka.streams.state.internals; |
||||
|
||||
import org.apache.kafka.common.utils.Bytes; |
||||
import org.apache.kafka.streams.KeyValue; |
||||
import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender; |
||||
import org.apache.kafka.streams.state.KeyValueIterator; |
||||
import org.hamcrest.core.IsNull; |
||||
import org.junit.Test; |
||||
import org.rocksdb.ColumnFamilyDescriptor; |
||||
import org.rocksdb.ColumnFamilyHandle; |
||||
import org.rocksdb.ColumnFamilyOptions; |
||||
import org.rocksdb.DBOptions; |
||||
import org.rocksdb.RocksDB; |
||||
|
||||
import java.io.File; |
||||
import java.nio.charset.StandardCharsets; |
||||
import java.util.ArrayList; |
||||
import java.util.List; |
||||
|
||||
import static java.util.Arrays.asList; |
||||
import static org.hamcrest.CoreMatchers.hasItem; |
||||
import static org.hamcrest.CoreMatchers.is; |
||||
import static org.junit.Assert.assertArrayEquals; |
||||
import static org.junit.Assert.assertFalse; |
||||
import static org.junit.Assert.assertThat; |
||||
|
||||
public class RocksDBTimestampedStoreTest extends RocksDBStoreTest { |
||||
|
||||
RocksDBStore getRocksDBStore() { |
||||
return new RocksDBTimestampedStore(DB_NAME); |
||||
} |
||||
|
||||
@Test |
||||
public void shouldMigrateDataFromDefaultToTimestampColumnFamily() throws Exception { |
||||
prepareOldStore(); |
||||
|
||||
LogCaptureAppender.setClassLoggerToDebug(RocksDBTimestampedStore.class); |
||||
|
||||
final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(); |
||||
rocksDBStore.init(context, rocksDBStore); |
||||
assertThat(appender.getMessages(), hasItem("Opening store " + DB_NAME + " in upgrade mode")); |
||||
LogCaptureAppender.unregister(appender); |
||||
|
||||
// approx: 7 entries on old CF, 0 in new CF
|
||||
assertThat(rocksDBStore.approximateNumEntries(), is(7L)); |
||||
|
||||
// get()
|
||||
|
||||
// should be no-op on both CF
|
||||
assertThat(rocksDBStore.get(new Bytes("unknown".getBytes())), new IsNull<>()); |
||||
// approx: 7 entries on old CF, 0 in new CF
|
||||
assertThat(rocksDBStore.approximateNumEntries(), is(7L)); |
||||
|
||||
// should migrate key1 from old to new CF
|
||||
// must return timestamp plus value, ie, it's not 1 byte but 9 bytes
|
||||
assertThat(rocksDBStore.get(new Bytes("key1".getBytes())).length, is(8 + 1)); |
||||
// one delete on old CF, one put on new CF
|
||||
// approx: 6 entries on old CF, 1 in new CF
|
||||
assertThat(rocksDBStore.approximateNumEntries(), is(7L)); |
||||
|
||||
// put()
|
||||
|
||||
// should migrate key2 from old to new CF with new value
|
||||
rocksDBStore.put(new Bytes("key2".getBytes()), "timestamp+22".getBytes()); |
||||
// one delete on old CF, one put on new CF
|
||||
// approx: 5 entries on old CF, 2 in new CF
|
||||
assertThat(rocksDBStore.approximateNumEntries(), is(7L)); |
||||
|
||||
// should delete key3 from old and new CF
|
||||
rocksDBStore.put(new Bytes("key3".getBytes()), null); |
||||
// count is off by one, due to two delete operations (even if one does not delete anything)
|
||||
// approx: 4 entries on old CF, 1 in new CF
|
||||
assertThat(rocksDBStore.approximateNumEntries(), is(5L)); |
||||
|
||||
// should add new key8 to new CF
|
||||
rocksDBStore.put(new Bytes("key8".getBytes()), "timestamp+88888888".getBytes()); |
||||
// one delete on old CF, one put on new CF
|
||||
// approx: 3 entries on old CF, 2 in new CF
|
||||
assertThat(rocksDBStore.approximateNumEntries(), is(5L)); |
||||
|
||||
// putIfAbsent()
|
||||
|
||||
// should migrate key4 from old to new CF with old value
|
||||
assertThat(rocksDBStore.putIfAbsent(new Bytes("key4".getBytes()), "timestamp+4444".getBytes()).length, is(8 + 4)); |
||||
// one delete on old CF, one put on new CF
|
||||
// approx: 2 entries on old CF, 3 in new CF
|
||||
assertThat(rocksDBStore.approximateNumEntries(), is(5L)); |
||||
|
||||
// should add new key11 to new CF
|
||||
assertThat(rocksDBStore.putIfAbsent(new Bytes("key11".getBytes()), "timestamp+11111111111".getBytes()), new IsNull<>()); |
||||
// one delete on old CF, one put on new CF
|
||||
// approx: 1 entries on old CF, 4 in new CF
|
||||
assertThat(rocksDBStore.approximateNumEntries(), is(5L)); |
||||
|
||||
// should not delete key5 but migrate to new CF
|
||||
assertThat(rocksDBStore.putIfAbsent(new Bytes("key5".getBytes()), null).length, is(8 + 5)); |
||||
// one delete on old CF, one put on new CF
|
||||
// approx: 0 entries on old CF, 5 in new CF
|
||||
assertThat(rocksDBStore.approximateNumEntries(), is(5L)); |
||||
|
||||
// should be no-op on both CF
|
||||
assertThat(rocksDBStore.putIfAbsent(new Bytes("key12".getBytes()), null), new IsNull<>()); |
||||
// two delete operation, however, only one is counted because old CF count was zero before already
|
||||
// approx: 0 entries on old CF, 4 in new CF
|
||||
assertThat(rocksDBStore.approximateNumEntries(), is(4L)); |
||||
|
||||
// delete()
|
||||
|
||||
// should delete key6 from old and new CF
|
||||
assertThat(rocksDBStore.delete(new Bytes("key6".getBytes())).length, is(8 + 6)); |
||||
// two delete operation, however, only one is counted because old CF count was zero before already
|
||||
// approx: 0 entries on old CF, 3 in new CF
|
||||
assertThat(rocksDBStore.approximateNumEntries(), is(3L)); |
||||
|
||||
|
||||
iteratorsShouldNotMigrateData(); |
||||
assertThat(rocksDBStore.approximateNumEntries(), is(3L)); |
||||
|
||||
rocksDBStore.close(); |
||||
|
||||
verifyOldAndNewColumnFamily(); |
||||
} |
||||
|
||||
private void iteratorsShouldNotMigrateData() { |
||||
// iterating should not migrate any data, but return all key over both CF (plus surrogate timestamps for old CF)
|
||||
final KeyValueIterator<Bytes, byte[]> itAll = rocksDBStore.all(); |
||||
{ |
||||
final KeyValue<Bytes, byte[]> keyValue = itAll.next(); |
||||
assertArrayEquals("key1".getBytes(), keyValue.key.get()); |
||||
// unknown timestamp == -1 plus value == 1
|
||||
assertArrayEquals(new byte[]{-1, -1, -1, -1, -1, -1, -1, -1, '1'}, keyValue.value); |
||||
} |
||||
{ |
||||
final KeyValue<Bytes, byte[]> keyValue = itAll.next(); |
||||
assertArrayEquals("key11".getBytes(), keyValue.key.get()); |
||||
assertArrayEquals(new byte[]{'t', 'i', 'm', 'e', 's', 't', 'a', 'm', 'p', '+', '1', '1', '1', '1', '1', '1', '1', '1', '1', '1', '1'}, keyValue.value); |
||||
} |
||||
{ |
||||
final KeyValue<Bytes, byte[]> keyValue = itAll.next(); |
||||
assertArrayEquals("key2".getBytes(), keyValue.key.get()); |
||||
assertArrayEquals(new byte[]{'t', 'i', 'm', 'e', 's', 't', 'a', 'm', 'p', '+', '2', '2'}, keyValue.value); |
||||
} |
||||
{ |
||||
final KeyValue<Bytes, byte[]> keyValue = itAll.next(); |
||||
assertArrayEquals("key4".getBytes(), keyValue.key.get()); |
||||
// unknown timestamp == -1 plus value == 4444
|
||||
assertArrayEquals(new byte[]{-1, -1, -1, -1, -1, -1, -1, -1, '4', '4', '4', '4'}, keyValue.value); |
||||
} |
||||
{ |
||||
final KeyValue<Bytes, byte[]> keyValue = itAll.next(); |
||||
assertArrayEquals("key5".getBytes(), keyValue.key.get()); |
||||
// unknown timestamp == -1 plus value == 55555
|
||||
assertArrayEquals(new byte[]{-1, -1, -1, -1, -1, -1, -1, -1, '5', '5', '5', '5', '5'}, keyValue.value); |
||||
} |
||||
{ |
||||
final KeyValue<Bytes, byte[]> keyValue = itAll.next(); |
||||
assertArrayEquals("key7".getBytes(), keyValue.key.get()); |
||||
// unknown timestamp == -1 plus value == 7777777
|
||||
assertArrayEquals(new byte[]{-1, -1, -1, -1, -1, -1, -1, -1, '7', '7', '7', '7', '7', '7', '7'}, keyValue.value); |
||||
} |
||||
{ |
||||
final KeyValue<Bytes, byte[]> keyValue = itAll.next(); |
||||
assertArrayEquals("key8".getBytes(), keyValue.key.get()); |
||||
assertArrayEquals(new byte[]{'t', 'i', 'm', 'e', 's', 't', 'a', 'm', 'p', '+', '8', '8', '8', '8', '8', '8', '8', '8'}, keyValue.value); |
||||
} |
||||
assertFalse(itAll.hasNext()); |
||||
itAll.close(); |
||||
|
||||
final KeyValueIterator<Bytes, byte[]> it = |
||||
rocksDBStore.range(new Bytes("key2".getBytes()), new Bytes("key5".getBytes())); |
||||
{ |
||||
final KeyValue<Bytes, byte[]> keyValue = it.next(); |
||||
assertArrayEquals("key2".getBytes(), keyValue.key.get()); |
||||
assertArrayEquals(new byte[]{'t', 'i', 'm', 'e', 's', 't', 'a', 'm', 'p', '+', '2', '2'}, keyValue.value); |
||||
} |
||||
{ |
||||
final KeyValue<Bytes, byte[]> keyValue = it.next(); |
||||
assertArrayEquals("key4".getBytes(), keyValue.key.get()); |
||||
// unknown timestamp == -1 plus value == 4444
|
||||
assertArrayEquals(new byte[]{-1, -1, -1, -1, -1, -1, -1, -1, '4', '4', '4', '4'}, keyValue.value); |
||||
} |
||||
{ |
||||
final KeyValue<Bytes, byte[]> keyValue = it.next(); |
||||
assertArrayEquals("key5".getBytes(), keyValue.key.get()); |
||||
// unknown timestamp == -1 plus value == 55555
|
||||
assertArrayEquals(new byte[]{-1, -1, -1, -1, -1, -1, -1, -1, '5', '5', '5', '5', '5'}, keyValue.value); |
||||
} |
||||
assertFalse(it.hasNext()); |
||||
it.close(); |
||||
} |
||||
|
||||
private void verifyOldAndNewColumnFamily() throws Exception { |
||||
final DBOptions dbOptions = new DBOptions(); |
||||
final ColumnFamilyOptions columnFamilyOptions = new ColumnFamilyOptions(); |
||||
|
||||
final List<ColumnFamilyDescriptor> columnFamilyDescriptors = asList( |
||||
new ColumnFamilyDescriptor(RocksDB.DEFAULT_COLUMN_FAMILY, columnFamilyOptions), |
||||
new ColumnFamilyDescriptor("keyValueWithTimestamp".getBytes(StandardCharsets.UTF_8), columnFamilyOptions)); |
||||
final List<ColumnFamilyHandle> columnFamilies = new ArrayList<>(columnFamilyDescriptors.size()); |
||||
|
||||
RocksDB db = RocksDB.open( |
||||
dbOptions, |
||||
new File(new File(context.stateDir(), "rocksdb"), DB_NAME).getAbsolutePath(), |
||||
columnFamilyDescriptors, |
||||
columnFamilies); |
||||
|
||||
ColumnFamilyHandle noTimestampColumnFamily = columnFamilies.get(0); |
||||
final ColumnFamilyHandle withTimestampColumnFamily = columnFamilies.get(1); |
||||
|
||||
assertThat(db.get(noTimestampColumnFamily, "unknown".getBytes()), new IsNull<>()); |
||||
assertThat(db.get(noTimestampColumnFamily, "key1".getBytes()), new IsNull<>()); |
||||
assertThat(db.get(noTimestampColumnFamily, "key2".getBytes()), new IsNull<>()); |
||||
assertThat(db.get(noTimestampColumnFamily, "key3".getBytes()), new IsNull<>()); |
||||
assertThat(db.get(noTimestampColumnFamily, "key4".getBytes()), new IsNull<>()); |
||||
assertThat(db.get(noTimestampColumnFamily, "key5".getBytes()), new IsNull<>()); |
||||
assertThat(db.get(noTimestampColumnFamily, "key6".getBytes()), new IsNull<>()); |
||||
assertThat(db.get(noTimestampColumnFamily, "key7".getBytes()).length, is(7)); |
||||
assertThat(db.get(noTimestampColumnFamily, "key8".getBytes()), new IsNull<>()); |
||||
assertThat(db.get(noTimestampColumnFamily, "key11".getBytes()), new IsNull<>()); |
||||
assertThat(db.get(noTimestampColumnFamily, "key12".getBytes()), new IsNull<>()); |
||||
|
||||
assertThat(db.get(withTimestampColumnFamily, "unknown".getBytes()), new IsNull<>()); |
||||
assertThat(db.get(withTimestampColumnFamily, "key1".getBytes()).length, is(8 + 1)); |
||||
assertThat(db.get(withTimestampColumnFamily, "key2".getBytes()).length, is(12)); |
||||
assertThat(db.get(withTimestampColumnFamily, "key3".getBytes()), new IsNull<>()); |
||||
assertThat(db.get(withTimestampColumnFamily, "key4".getBytes()).length, is(8 + 4)); |
||||
assertThat(db.get(withTimestampColumnFamily, "key5".getBytes()).length, is(8 + 5)); |
||||
assertThat(db.get(withTimestampColumnFamily, "key6".getBytes()), new IsNull<>()); |
||||
assertThat(db.get(withTimestampColumnFamily, "key7".getBytes()), new IsNull<>()); |
||||
assertThat(db.get(withTimestampColumnFamily, "key8".getBytes()).length, is(18)); |
||||
assertThat(db.get(withTimestampColumnFamily, "key11".getBytes()).length, is(21)); |
||||
assertThat(db.get(withTimestampColumnFamily, "key12".getBytes()), new IsNull<>()); |
||||
|
||||
db.close(); |
||||
|
||||
// check that still in upgrade mode
|
||||
LogCaptureAppender appender = LogCaptureAppender.createAndRegister(); |
||||
rocksDBStore.init(context, rocksDBStore); |
||||
assertThat(appender.getMessages(), hasItem("Opening store " + DB_NAME + " in upgrade mode")); |
||||
LogCaptureAppender.unregister(appender); |
||||
rocksDBStore.close(); |
||||
|
||||
// clear old CF
|
||||
columnFamilies.clear(); |
||||
db = RocksDB.open( |
||||
dbOptions, |
||||
new File(new File(context.stateDir(), "rocksdb"), DB_NAME).getAbsolutePath(), |
||||
columnFamilyDescriptors, |
||||
columnFamilies); |
||||
|
||||
noTimestampColumnFamily = columnFamilies.get(0); |
||||
db.delete(noTimestampColumnFamily, "key7".getBytes()); |
||||
db.close(); |
||||
|
||||
// check that still in regular mode
|
||||
appender = LogCaptureAppender.createAndRegister(); |
||||
rocksDBStore.init(context, rocksDBStore); |
||||
assertThat(appender.getMessages(), hasItem("Opening store " + DB_NAME + " in regular mode")); |
||||
LogCaptureAppender.unregister(appender); |
||||
} |
||||
|
||||
private void prepareOldStore() { |
||||
final RocksDBStore keyValueStore = new RocksDBStore(DB_NAME); |
||||
keyValueStore.init(context, keyValueStore); |
||||
|
||||
keyValueStore.put(new Bytes("key1".getBytes()), "1".getBytes()); |
||||
keyValueStore.put(new Bytes("key2".getBytes()), "22".getBytes()); |
||||
keyValueStore.put(new Bytes("key3".getBytes()), "333".getBytes()); |
||||
keyValueStore.put(new Bytes("key4".getBytes()), "4444".getBytes()); |
||||
keyValueStore.put(new Bytes("key5".getBytes()), "55555".getBytes()); |
||||
keyValueStore.put(new Bytes("key6".getBytes()), "666666".getBytes()); |
||||
keyValueStore.put(new Bytes("key7".getBytes()), "7777777".getBytes()); |
||||
|
||||
keyValueStore.close(); |
||||
} |
||||
|
||||
} |
Loading…
Reference in new issue