HDDS-1691 : RDBTable#isExist should use Rocksdb#keyMayExist (#1013)
This commit is contained in:
parent
062eb605ac
commit
0a37ccf379
@ -122,7 +122,11 @@ public boolean isEmpty() throws IOException {
|
||||
@Override
|
||||
public boolean isExist(byte[] key) throws IOException {
|
||||
try {
|
||||
return db.get(handle, key) != null;
|
||||
// RocksDB#keyMayExist
|
||||
// If the key definitely does not exist in the database, then this
|
||||
// method returns false, else true.
|
||||
return db.keyMayExist(handle, key, new StringBuilder())
|
||||
&& db.get(handle, key) != null;
|
||||
} catch (RocksDBException e) {
|
||||
throw toIOException(
|
||||
"Error in accessing DB. ", e);
|
||||
|
@ -32,6 +32,7 @@
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.commons.codec.binary.StringUtils;
|
||||
import org.apache.hadoop.hdfs.DFSUtil;
|
||||
|
||||
import org.apache.commons.lang3.RandomStringUtils;
|
||||
@ -289,4 +290,39 @@ public void testRocksDBCheckpointCleanup() throws Exception {
|
||||
checkpoint.getCheckpointLocation()));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Not strictly a unit test. Just a confirmation of the expected behavior
|
||||
* of RocksDB keyMayExist API.
|
||||
* Expected behavior - On average, keyMayExist latency < key.get() latency
|
||||
* for invalid keys.
|
||||
* @throws Exception if unable to read from RocksDB.
|
||||
*/
|
||||
@Test
|
||||
public void testRocksDBKeyMayExistApi() throws Exception {
|
||||
try (RDBStore newStore =
|
||||
new RDBStore(folder.newFolder(), options, configSet)) {
|
||||
RocksDB db = newStore.getDb();
|
||||
|
||||
//Test with 50 invalid keys.
|
||||
long start = System.nanoTime();
|
||||
for (int i = 0; i < 50; i++) {
|
||||
Assert.assertTrue(db.get(
|
||||
StringUtils.getBytesUtf16("key" + i))== null);
|
||||
}
|
||||
long end = System.nanoTime();
|
||||
long keyGetLatency = end - start;
|
||||
|
||||
start = System.nanoTime();
|
||||
for (int i = 0; i < 50; i++) {
|
||||
Assert.assertFalse(db.keyMayExist(
|
||||
StringUtils.getBytesUtf16("key" + i), new StringBuilder()));
|
||||
}
|
||||
end = System.nanoTime();
|
||||
long keyMayExistLatency = end - start;
|
||||
|
||||
Assert.assertTrue(keyMayExistLatency < keyGetLatency);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
@ -31,6 +31,7 @@
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import org.apache.commons.codec.binary.StringUtils;
|
||||
import org.apache.hadoop.hdds.HddsConfigKeys;
|
||||
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||
import org.apache.hadoop.hdds.protocol.StorageType;
|
||||
@ -49,9 +50,14 @@
|
||||
import org.apache.hadoop.ozone.OzoneTestUtils;
|
||||
import org.apache.hadoop.ozone.client.rest.OzoneException;
|
||||
import org.apache.hadoop.ozone.common.BlockGroup;
|
||||
import org.apache.hadoop.ozone.om.codec.OmKeyInfoCodec;
|
||||
import org.apache.hadoop.ozone.om.exceptions.OMException;
|
||||
import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes;
|
||||
import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
|
||||
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
|
||||
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
|
||||
import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
|
||||
import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
|
||||
import org.apache.hadoop.ozone.om.helpers.ServiceInfo;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ServicePort;
|
||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.VolumeList;
|
||||
@ -72,8 +78,10 @@
|
||||
import org.apache.hadoop.ozone.web.response.ListVolumes;
|
||||
import org.apache.hadoop.ozone.web.response.VolumeInfo;
|
||||
import org.apache.hadoop.ozone.web.utils.OzoneUtils;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.test.GenericTestUtils;
|
||||
import org.apache.hadoop.util.Time;
|
||||
import org.apache.hadoop.utils.db.RDBStore;
|
||||
import org.apache.hadoop.utils.db.Table;
|
||||
import org.apache.hadoop.utils.db.Table.KeyValue;
|
||||
import org.apache.hadoop.utils.db.TableIterator;
|
||||
@ -85,6 +93,8 @@
|
||||
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ADDRESS_KEY;
|
||||
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_NOT_FOUND;
|
||||
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_NOT_FOUND;
|
||||
import static org.apache.hadoop.ozone.security.acl.IAccessAuthorizer.ACLType.ALL;
|
||||
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
@ -92,6 +102,7 @@
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.Timeout;
|
||||
import org.rocksdb.RocksDB;
|
||||
|
||||
/**
|
||||
* Test Ozone Manager operation in distributed handler scenario.
|
||||
@ -1370,4 +1381,56 @@ public void testVersion() {
|
||||
String actualVersion = cluster.getOzoneManager().getSoftwareVersion();
|
||||
Assert.assertEquals(expectedVersion, actualVersion);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test if OM RocksDB keyMayExist API works for keys that are present.
|
||||
* Test added in this module since we need access to custom codec dependent
|
||||
* objects like OMKeyInfo.
|
||||
* @throws Exception if OM or RocksDB operations fail.
|
||||
*/
|
||||
@Test
|
||||
public void testDBKeyMayExist() throws Exception {
|
||||
RDBStore rdbStore = (RDBStore) cluster.getOzoneManager()
|
||||
.getMetadataManager().getStore();
|
||||
RocksDB db = rdbStore.getDb();
|
||||
UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
|
||||
|
||||
OmVolumeArgs volumeArgs = OmVolumeArgs.newBuilder()
|
||||
.setVolume("vol1")
|
||||
.setAdminName("bilbo")
|
||||
.setOwnerName("bilbo")
|
||||
.build();
|
||||
cluster.getOzoneManager().createVolume(volumeArgs);
|
||||
|
||||
OmBucketInfo bucketInfo = OmBucketInfo.newBuilder()
|
||||
.setVolumeName("vol1")
|
||||
.setBucketName("bucket1")
|
||||
.build();
|
||||
cluster.getOzoneManager().createBucket(bucketInfo);
|
||||
|
||||
OmKeyArgs keyArgs = new OmKeyArgs.Builder()
|
||||
.setBucketName("bucket1")
|
||||
.setFactor(HddsProtos.ReplicationFactor.ONE)
|
||||
.setDataSize(0)
|
||||
.setType(HddsProtos.ReplicationType.STAND_ALONE)
|
||||
.setAcls(OzoneUtils.getAclList(ugi.getUserName(), ugi.getGroups(),
|
||||
ALL, ALL))
|
||||
.setVolumeName("vol1")
|
||||
.setKeyName(UUID.randomUUID().toString())
|
||||
.setDataSize(16 * 1024 * 1024 * 10)
|
||||
.build();
|
||||
OpenKeySession keySession = cluster.getOzoneManager().getKeyManager()
|
||||
.openKey(keyArgs);
|
||||
OmKeyInfo keyInfo = keySession.getKeyInfo();
|
||||
OmKeyInfoCodec omKeyInfoCodec = new OmKeyInfoCodec();
|
||||
|
||||
db.put(StringUtils.getBytesUtf16("OMKey1"),
|
||||
omKeyInfoCodec.toPersistedFormat(keyInfo));
|
||||
|
||||
StringBuilder sb = new StringBuilder();
|
||||
Assert.assertTrue(db.keyMayExist(StringUtils.getBytesUtf16("OMKey1"),
|
||||
sb));
|
||||
Assert.assertTrue(sb.length() > 0);
|
||||
}
|
||||
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user