HDFS-12149. Ozone: RocksDB implementation of ozone metadata store. Contributed by Weiwei Yang.

This commit is contained in:
Anu Engineer 2017-07-20 18:22:03 -07:00 committed by Owen O'Malley
parent 6225622dee
commit bbbfc885e7
6 changed files with 433 additions and 8 deletions

View File

@ -192,6 +192,11 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
<groupId>org.fusesource.leveldbjni</groupId>
<artifactId>leveldbjni-all</artifactId>
</dependency>
<dependency>
<groupId>org.rocksdb</groupId>
<artifactId>rocksdbjni</artifactId>
<version>5.5.5</version>
</dependency>
<!-- 'mvn dependency:analyze' fails to detect use of this dependency -->
<dependency>
<groupId>org.bouncycastle</groupId>

View File

@ -126,8 +126,10 @@ public void delete(byte[] key) {
*/
@Override
public void close() throws IOException {
if (db != null){
db.close();
}
}
/**
* Returns true if the DB is empty.
@ -163,6 +165,7 @@ public DBIterator getIterator() {
@Override
public void destroy() throws IOException {
close();
JniDBFactory.factory.destroy(dbFile, dbOptions);
}

View File

@ -21,6 +21,7 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.iq80.leveldb.Options;
import org.rocksdb.BlockBasedTableConfig;
import java.io.File;
import java.io.IOException;
@ -82,8 +83,15 @@ public MetadataStore build() throws IOException {
}
store = new LevelDBStore(dbFile, options);
} else if (OZONE_METADATA_STORE_IMPL_ROCKSDB.equals(impl)) {
// TODO replace with rocksDB impl
store = new LevelDBStore(dbFile, new Options());
org.rocksdb.Options opts = new org.rocksdb.Options();
opts.setCreateIfMissing(createIfMissing);
if (cacheSize > 0) {
BlockBasedTableConfig tableConfig = new BlockBasedTableConfig();
tableConfig.setBlockCacheSize(cacheSize);
opts.setTableFormatConfig(tableConfig);
}
store = new RocksDBStore(dbFile, opts);
} else {
throw new IllegalArgumentException("Invalid argument for "
+ OzoneConfigKeys.OZONE_METADATA_STORE_IMPL

View File

@ -0,0 +1,318 @@
/*
* 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.hadoop.utils;
import com.google.common.base.Preconditions;
import org.apache.commons.io.FileUtils;
import org.apache.commons.lang3.tuple.ImmutablePair;
import org.rocksdb.RocksIterator;
import org.rocksdb.Options;
import org.rocksdb.WriteOptions;
import org.rocksdb.RocksDB;
import org.rocksdb.RocksDBException;
import org.rocksdb.WriteBatch;
import org.rocksdb.DbPath;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.File;
import java.io.IOException;
import java.util.Map;
import java.util.List;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.AbstractMap;
/**
* RocksDB implementation of ozone metadata store.
*/
public class RocksDBStore implements MetadataStore {
private static final Logger LOG =
LoggerFactory.getLogger(RocksDBStore.class);
private RocksDB db = null;
private File dbLocation;
private WriteOptions writeOptions;
private Options dbOptions;
public RocksDBStore(File dbFile, Options options) throws IOException {
Preconditions.checkNotNull(dbFile, "DB file location cannot be null");
RocksDB.loadLibrary();
dbOptions = options;
dbLocation = dbFile;
writeOptions = new WriteOptions();
writeOptions.setSync(true);
writeOptions.setNoSlowdown(true);
try {
db = RocksDB.open(dbOptions, dbLocation.getAbsolutePath());
} catch (RocksDBException e) {
throw new IOException("Failed init RocksDB, db path : "
+ dbFile.getAbsolutePath(), e);
}
if (LOG.isDebugEnabled()) {
LOG.debug("RocksDB successfully opened.");
LOG.debug("[Option] dbLocation= {}", dbLocation.getAbsolutePath());
LOG.debug("[Option] createIfMissing = {}", options.createIfMissing());
LOG.debug("[Option] compactionPriority= {}", options.compactionStyle());
LOG.debug("[Option] compressionType= {}", options.compressionType());
LOG.debug("[Option] maxOpenFiles= {}", options.maxOpenFiles());
LOG.debug("[Option] writeBufferSize= {}", options.writeBufferSize());
}
}
private IOException toIOException(String msg, RocksDBException e) {
String statusCode = e.getStatus() == null ? "N/A" :
e.getStatus().getCodeString();
String errMessage = e.getMessage() == null ? "Unknown error" :
e.getMessage();
String output = msg + "; status : " + statusCode
+ "; message : " + errMessage;
return new IOException(output, e);
}
@Override
public void put(byte[] key, byte[] value) throws IOException {
try {
db.put(writeOptions, key, value);
} catch (RocksDBException e) {
throw toIOException("Failed to put key-value to metadata store", e);
}
}
@Override
public boolean isEmpty() throws IOException {
RocksIterator it = null;
try {
it = db.newIterator();
it.seekToFirst();
return !it.isValid();
} finally {
if (it != null) {
it.close();
}
}
}
@Override
public byte[] get(byte[] key) throws IOException {
try {
return db.get(key);
} catch (RocksDBException e) {
throw toIOException("Failed to get the value for the given key", e);
}
}
@Override
public void delete(byte[] key) throws IOException {
try {
db.delete(key);
} catch (RocksDBException e) {
throw toIOException("Failed to delete the given key", e);
}
}
@Override
public List<Map.Entry<byte[], byte[]>> getRangeKVs(byte[] startKey,
int count, MetadataKeyFilters.MetadataKeyFilter... filters)
throws IOException, IllegalArgumentException {
List<Map.Entry<byte[], byte[]>> result = new ArrayList<>();
long start = System.currentTimeMillis();
if (count < 0) {
throw new IllegalArgumentException(
"Invalid count given " + count + ", count must be greater than 0");
}
RocksIterator it = null;
try {
it = db.newIterator();
if (startKey == null) {
it.seekToFirst();
} else {
if(get(startKey) == null) {
throw new IOException("Invalid start key, not found in current db");
}
it.seek(startKey);
}
while(it.isValid() && result.size() < count) {
byte[] currentKey = it.key();
byte[] currentValue = it.value();
it.prev();
final byte[] prevKey = it.isValid() ? it.key() : null;
it.seek(currentKey);
it.next();
final byte[] nextKey = it.isValid() ? it.key() : null;
if (filters == null || Arrays.asList(filters).stream()
.allMatch(entry -> entry.filterKey(prevKey,
currentKey, nextKey))) {
result.add(new AbstractMap.SimpleImmutableEntry<>(currentKey,
currentValue));
}
}
} finally {
if (it != null) {
it.close();
}
long end = System.currentTimeMillis();
long timeConsumed = end - start;
if (LOG.isDebugEnabled()) {
LOG.debug("Time consumed for getRangeKVs() is {}ms,"
+ " result length is {}.", timeConsumed, result.size());
}
}
return result;
}
@Override
public void writeBatch(BatchOperation operation)
throws IOException {
List<BatchOperation.SingleOperation> operations =
operation.getOperations();
if (!operations.isEmpty()) {
try (WriteBatch writeBatch = new WriteBatch()) {
for (BatchOperation.SingleOperation opt : operations) {
switch (opt.getOpt()) {
case DELETE:
writeBatch.remove(opt.getKey());
break;
case PUT:
writeBatch.put(opt.getKey(), opt.getValue());
break;
default:
throw new IllegalArgumentException("Invalid operation "
+ opt.getOpt());
}
}
db.write(writeOptions, writeBatch);
} catch (RocksDBException e) {
throw toIOException("Batch write operation failed", e);
}
}
}
@Override
public void compactDB() throws IOException {
if (db != null) {
try {
db.compactRange();
} catch (RocksDBException e) {
throw toIOException("Failed to compact db", e);
}
}
}
private void deleteQuietly(File fileOrDir) {
if (fileOrDir != null && fileOrDir.exists()) {
try {
FileUtils.forceDelete(fileOrDir);
} catch (IOException e) {
LOG.warn("Failed to delete dir {}", fileOrDir.getAbsolutePath(), e);
}
}
}
@Override
public void destroy() throws IOException {
// Make sure db is closed.
close();
// There is no destroydb java API available,
// equivalently we can delete all db directories.
deleteQuietly(dbLocation);
deleteQuietly(new File(dbOptions.dbLogDir()));
deleteQuietly(new File(dbOptions.walDir()));
List<DbPath> dbPaths = dbOptions.dbPaths();
if (dbPaths != null) {
dbPaths.forEach(dbPath -> {
deleteQuietly(new File(dbPath.toString()));
});
}
}
@Override
public ImmutablePair<byte[], byte[]> peekAround(int offset,
byte[] from) throws IOException, IllegalArgumentException {
RocksIterator it = null;
try {
it = db.newIterator();
if (from == null) {
it.seekToFirst();
} else {
it.seek(from);
}
if (!it.isValid()) {
throw new IOException("Key not found");
}
switch (offset) {
case 0:
break;
case 1:
it.next();
break;
case -1:
it.prev();
break;
default:
throw new IllegalArgumentException(
"Position can only be -1, 0 " + "or 1, but found " + offset);
}
return it.isValid() ? new ImmutablePair<>(it.key(), it.value()) : null;
} finally {
if (it != null) {
it.close();
}
}
}
@Override
public void iterate(byte[] from, EntryConsumer consumer)
throws IOException {
RocksIterator it = null;
try {
it = db.newIterator();
if (from != null) {
it.seek(from);
} else {
it.seekToFirst();
}
while (it.isValid()) {
if (!consumer.consume(it.key(), it.value())) {
break;
}
it.next();
}
} finally {
if (it != null) {
it.close();
}
}
}
@Override
public void close() throws IOException {
if (db != null) {
db.close();
}
}
}

View File

@ -598,7 +598,7 @@
<property>
<name>ozone.metastore.impl</name>
<value>LevelDB</value>
<value>RocksDB</value>
<description>
Ozone metadata store implementation. Ozone metadata are well distributed
to multiple services such as ksm, scm. They are stored in some local

View File

@ -22,6 +22,7 @@
import org.apache.commons.lang3.tuple.ImmutablePair;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSUtilClient;
import org.apache.hadoop.utils.BatchOperation;
import org.apache.hadoop.utils.MetadataStore;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.utils.MetadataKeyFilters.KeyPrefixFilter;
@ -33,6 +34,8 @@
import org.junit.Test;
import org.junit.Assert;
import org.junit.rules.ExpectedException;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import java.io.File;
import java.io.IOException;
@ -40,15 +43,35 @@
import java.util.Map;
import java.util.ArrayList;
import java.util.UUID;
import java.util.Collection;
import java.util.Arrays;
import java.util.Iterator;
import java.util.concurrent.atomic.AtomicInteger;
import static org.junit.runners.Parameterized.*;
/**
* Test class for ozone metadata store.
*/
@RunWith(Parameterized.class)
public class TestMetadataStore {
private final String storeImpl;
public TestMetadataStore(String metadataImpl) {
this.storeImpl = metadataImpl;
}
@Parameters
public static Collection<Object[]> data() {
return Arrays.asList(new Object[][] {
{OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_LEVELDB},
{OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_ROCKSDB}
});
}
private MetadataStore store;
private File testDir;
private final static int MAX_GETRANGE_LENGTH = 100;
@Rule
@ -56,11 +79,11 @@ public class TestMetadataStore {
@Before
public void init() throws IOException {
testDir = GenericTestUtils.getTestDir(getClass().getSimpleName());
testDir = GenericTestUtils.getTestDir(getClass().getSimpleName()
+ "-" + storeImpl.toLowerCase());
Configuration conf = new OzoneConfiguration();
conf.set(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL,
OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_LEVELDB);
conf.set(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL, storeImpl);
store = MetadataStoreBuilder.newBuilder()
.setConf(conf)
@ -293,4 +316,72 @@ public void testInvalidStartKey() throws IOException {
expectedException.expectMessage("Invalid start key");
store.getRangeKVs(getBytes("unknownKey"), MAX_GETRANGE_LENGTH);
}
@Test
public void testDestroyDB() throws IOException {
// create a new DB to test db destroy
Configuration conf = new OzoneConfiguration();
conf.set(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL, storeImpl);
File dbDir = GenericTestUtils.getTestDir(getClass().getSimpleName()
+ "-" + storeImpl.toLowerCase() + "-toDestroy");
MetadataStore dbStore = MetadataStoreBuilder.newBuilder()
.setConf(conf)
.setCreateIfMissing(true)
.setDbFile(dbDir)
.build();
dbStore.put(getBytes("key1"), getBytes("value1"));
dbStore.put(getBytes("key2"), getBytes("value2"));
Assert.assertFalse(dbStore.isEmpty());
Assert.assertTrue(dbDir.exists());
Assert.assertTrue(dbDir.listFiles().length > 0);
dbStore.destroy();
Assert.assertFalse(dbDir.exists());
}
@Test
public void testBatchWrite() throws IOException {
Configuration conf = new OzoneConfiguration();
conf.set(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL, storeImpl);
File dbDir = GenericTestUtils.getTestDir(getClass().getSimpleName()
+ "-" + storeImpl.toLowerCase() + "-batchWrite");
MetadataStore dbStore = MetadataStoreBuilder.newBuilder()
.setConf(conf)
.setCreateIfMissing(true)
.setDbFile(dbDir)
.build();
List<String> expectedResult = Lists.newArrayList();
for (int i = 0; i<10; i++) {
dbStore.put(getBytes("batch-" + i), getBytes("batch-value-" + i));
expectedResult.add("batch-" + i);
}
BatchOperation batch = new BatchOperation();
batch.delete(getBytes("batch-2"));
batch.delete(getBytes("batch-3"));
batch.delete(getBytes("batch-4"));
batch.put(getBytes("batch-new-2"), getBytes("batch-new-value-2"));
expectedResult.remove("batch-2");
expectedResult.remove("batch-3");
expectedResult.remove("batch-4");
expectedResult.add("batch-new-2");
dbStore.writeBatch(batch);
Iterator<String> it = expectedResult.iterator();
AtomicInteger count = new AtomicInteger(0);
dbStore.iterate(null, (key, value) -> {
count.incrementAndGet();
return it.hasNext() && it.next().equals(getString(key));
});
Assert.assertEquals(8, count.get());
}
}