HDDS-1449. JVM Exit in datanode while committing a key. Contributed by Mukul Kumar Singh. (#825)

This commit is contained in:
Mukul Kumar Singh 2019-05-22 17:18:40 +05:30 committed by bshashikant
parent 67f9a7b165
commit 2fc6f8599a
19 changed files with 688 additions and 575 deletions

View File

@ -48,7 +48,7 @@
import org.apache.hadoop.ozone.protocol.commands.SCMCommand; import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
import org.apache.hadoop.util.Time; import org.apache.hadoop.util.Time;
import org.apache.hadoop.utils.BatchOperation; import org.apache.hadoop.utils.BatchOperation;
import org.apache.hadoop.utils.MetadataStore; import org.apache.hadoop.ozone.container.common.utils.ContainerCache.ReferenceCountedDB;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -198,18 +198,19 @@ private void deleteKeyValueContainerBlocks(
} }
int newDeletionBlocks = 0; int newDeletionBlocks = 0;
MetadataStore containerDB = BlockUtils.getDB(containerData, conf); try(ReferenceCountedDB containerDB =
BlockUtils.getDB(containerData, conf)) {
for (Long blk : delTX.getLocalIDList()) { for (Long blk : delTX.getLocalIDList()) {
BatchOperation batch = new BatchOperation(); BatchOperation batch = new BatchOperation();
byte[] blkBytes = Longs.toByteArray(blk); byte[] blkBytes = Longs.toByteArray(blk);
byte[] blkInfo = containerDB.get(blkBytes); byte[] blkInfo = containerDB.getStore().get(blkBytes);
if (blkInfo != null) { if (blkInfo != null) {
byte[] deletingKeyBytes = byte[] deletingKeyBytes =
DFSUtil.string2Bytes(OzoneConsts.DELETING_KEY_PREFIX + blk); DFSUtil.string2Bytes(OzoneConsts.DELETING_KEY_PREFIX + blk);
byte[] deletedKeyBytes = byte[] deletedKeyBytes =
DFSUtil.string2Bytes(OzoneConsts.DELETED_KEY_PREFIX + blk); DFSUtil.string2Bytes(OzoneConsts.DELETED_KEY_PREFIX + blk);
if (containerDB.get(deletingKeyBytes) != null if (containerDB.getStore().get(deletingKeyBytes) != null
|| containerDB.get(deletedKeyBytes) != null) { || containerDB.getStore().get(deletedKeyBytes) != null) {
LOG.debug(String.format( LOG.debug(String.format(
"Ignoring delete for block %d in container %d." "Ignoring delete for block %d in container %d."
+ " Entry already added.", blk, containerId)); + " Entry already added.", blk, containerId));
@ -220,7 +221,7 @@ private void deleteKeyValueContainerBlocks(
batch.put(deletingKeyBytes, blkInfo); batch.put(deletingKeyBytes, blkInfo);
batch.delete(blkBytes); batch.delete(blkBytes);
try { try {
containerDB.writeBatch(batch); containerDB.getStore().writeBatch(batch);
newDeletionBlocks++; newDeletionBlocks++;
LOG.debug("Transited Block {} to DELETING state in container {}", LOG.debug("Transited Block {} to DELETING state in container {}",
blk, containerId); blk, containerId);
@ -237,7 +238,7 @@ private void deleteKeyValueContainerBlocks(
} }
} }
containerDB containerDB.getStore()
.put(DFSUtil.string2Bytes(OzoneConsts.DELETE_TRANSACTION_KEY_PREFIX), .put(DFSUtil.string2Bytes(OzoneConsts.DELETE_TRANSACTION_KEY_PREFIX),
Longs.toByteArray(delTX.getTxID())); Longs.toByteArray(delTX.getTxID()));
containerData containerData
@ -245,6 +246,7 @@ private void deleteKeyValueContainerBlocks(
// update pending deletion blocks count in in-memory container status // update pending deletion blocks count in in-memory container status
containerData.incrPendingDeletionBlocks(newDeletionBlocks); containerData.incrPendingDeletionBlocks(newDeletionBlocks);
} }
}
@Override @Override
public SCMCommandProto.Type getCommandType() { public SCMCommandProto.Type getCommandType() {

View File

@ -28,8 +28,11 @@
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import java.io.Closeable;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock; import java.util.concurrent.locks.ReentrantLock;
@ -92,8 +95,8 @@ public void shutdownCache() {
MapIterator iterator = cache.mapIterator(); MapIterator iterator = cache.mapIterator();
while (iterator.hasNext()) { while (iterator.hasNext()) {
iterator.next(); iterator.next();
MetadataStore db = (MetadataStore) iterator.getValue(); ReferenceCountedDB db = (ReferenceCountedDB) iterator.getValue();
closeDB((String)iterator.getKey(), db); db.setEvicted(true);
} }
// reset the cache // reset the cache
cache.clear(); cache.clear();
@ -107,11 +110,11 @@ public void shutdownCache() {
*/ */
@Override @Override
protected boolean removeLRU(LinkEntry entry) { protected boolean removeLRU(LinkEntry entry) {
MetadataStore db = (MetadataStore) entry.getValue(); ReferenceCountedDB db = (ReferenceCountedDB) entry.getValue();
String dbFile = (String)entry.getKey(); String dbFile = (String)entry.getKey();
lock.lock(); lock.lock();
try { try {
closeDB(dbFile, db); db.setEvicted(false);
return true; return true;
} catch (Exception e) { } catch (Exception e) {
LOG.error("Eviction for db:{} failed", dbFile, e); LOG.error("Eviction for db:{} failed", dbFile, e);
@ -128,26 +131,30 @@ protected boolean removeLRU(LinkEntry entry) {
* @param containerDBType - DB type of the container. * @param containerDBType - DB type of the container.
* @param containerDBPath - DB path of the container. * @param containerDBPath - DB path of the container.
* @param conf - Hadoop Configuration. * @param conf - Hadoop Configuration.
* @return MetadataStore. * @return ReferenceCountedDB.
*/ */
public MetadataStore getDB(long containerID, String containerDBType, public ReferenceCountedDB getDB(long containerID, String containerDBType,
String containerDBPath, Configuration conf) String containerDBPath, Configuration conf)
throws IOException { throws IOException {
Preconditions.checkState(containerID >= 0, Preconditions.checkState(containerID >= 0,
"Container ID cannot be negative."); "Container ID cannot be negative.");
lock.lock(); lock.lock();
try { try {
MetadataStore db = (MetadataStore) this.get(containerDBPath); ReferenceCountedDB db = (ReferenceCountedDB) this.get(containerDBPath);
if (db == null) { if (db == null) {
db = MetadataStoreBuilder.newBuilder() MetadataStore metadataStore =
MetadataStoreBuilder.newBuilder()
.setDbFile(new File(containerDBPath)) .setDbFile(new File(containerDBPath))
.setCreateIfMissing(false) .setCreateIfMissing(false)
.setConf(conf) .setConf(conf)
.setDBType(containerDBType) .setDBType(containerDBType)
.build(); .build();
db = new ReferenceCountedDB(metadataStore, containerDBPath);
this.put(containerDBPath, db); this.put(containerDBPath, db);
} }
// increment the reference before returning the object
db.incrementReference();
return db; return db;
} catch (Exception e) { } catch (Exception e) {
LOG.error("Error opening DB. Container:{} ContainerPath:{}", LOG.error("Error opening DB. Container:{} ContainerPath:{}",
@ -161,16 +168,70 @@ public MetadataStore getDB(long containerID, String containerDBType,
/** /**
* Remove a DB handler from cache. * Remove a DB handler from cache.
* *
* @param containerPath - path of the container db file. * @param containerDBPath - path of the container db file.
*/ */
public void removeDB(String containerPath) { public void removeDB(String containerDBPath) {
lock.lock(); lock.lock();
try { try {
MetadataStore db = (MetadataStore)this.get(containerPath); ReferenceCountedDB db = (ReferenceCountedDB)this.get(containerDBPath);
closeDB(containerPath, db); if (db != null) {
this.remove(containerPath); // marking it as evicted will close the db as well.
db.setEvicted(true);
}
this.remove(containerDBPath);
} finally { } finally {
lock.unlock(); lock.unlock();
} }
} }
/**
* Class to implement reference counting over instances handed by Container
* Cache.
*/
public class ReferenceCountedDB implements Closeable {
private final AtomicInteger referenceCount;
private final AtomicBoolean isEvicted;
private final MetadataStore store;
private final String containerDBPath;
public ReferenceCountedDB(MetadataStore store, String containerDBPath) {
this.referenceCount = new AtomicInteger(0);
this.isEvicted = new AtomicBoolean(false);
this.store = store;
this.containerDBPath = containerDBPath;
}
private void incrementReference() {
this.referenceCount.incrementAndGet();
}
private void decrementReference() {
this.referenceCount.decrementAndGet();
cleanup();
}
private void setEvicted(boolean checkNoReferences) {
Preconditions.checkState(!checkNoReferences ||
(referenceCount.get() == 0),
"checkNoReferences:%b, referencount:%d",
checkNoReferences, referenceCount.get());
isEvicted.set(true);
cleanup();
}
private void cleanup() {
if (referenceCount.get() == 0 && isEvicted.get() && store != null) {
closeDB(containerDBPath, store);
}
}
public MetadataStore getStore() {
return store;
}
public void close() {
decrementReference();
}
}
} }

View File

@ -31,11 +31,12 @@
import org.apache.hadoop.utils.MetaStoreIterator; import org.apache.hadoop.utils.MetaStoreIterator;
import org.apache.hadoop.utils.MetadataKeyFilters; import org.apache.hadoop.utils.MetadataKeyFilters;
import org.apache.hadoop.utils.MetadataKeyFilters.KeyPrefixFilter; import org.apache.hadoop.utils.MetadataKeyFilters.KeyPrefixFilter;
import org.apache.hadoop.utils.MetadataStore; import org.apache.hadoop.ozone.container.common.utils.ContainerCache.ReferenceCountedDB;
import org.apache.hadoop.utils.MetadataStore.KeyValue; import org.apache.hadoop.utils.MetadataStore.KeyValue;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import java.io.Closeable;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.util.NoSuchElementException; import java.util.NoSuchElementException;
@ -48,12 +49,14 @@
* {@link MetadataKeyFilters#getNormalKeyFilter()} * {@link MetadataKeyFilters#getNormalKeyFilter()}
*/ */
@InterfaceAudience.Public @InterfaceAudience.Public
public class KeyValueBlockIterator implements BlockIterator<BlockData> { public class KeyValueBlockIterator implements BlockIterator<BlockData>,
Closeable {
private static final Logger LOG = LoggerFactory.getLogger( private static final Logger LOG = LoggerFactory.getLogger(
KeyValueBlockIterator.class); KeyValueBlockIterator.class);
private MetaStoreIterator<KeyValue> blockIterator; private MetaStoreIterator<KeyValue> blockIterator;
private final ReferenceCountedDB db;
private static KeyPrefixFilter defaultBlockFilter = MetadataKeyFilters private static KeyPrefixFilter defaultBlockFilter = MetadataKeyFilters
.getNormalKeyFilter(); .getNormalKeyFilter();
private KeyPrefixFilter blockFilter; private KeyPrefixFilter blockFilter;
@ -91,9 +94,9 @@ public KeyValueBlockIterator(long id, File path, KeyPrefixFilter filter)
containerData; containerData;
keyValueContainerData.setDbFile(KeyValueContainerLocationUtil keyValueContainerData.setDbFile(KeyValueContainerLocationUtil
.getContainerDBFile(metdataPath, containerId)); .getContainerDBFile(metdataPath, containerId));
MetadataStore metadataStore = BlockUtils.getDB(keyValueContainerData, new db = BlockUtils.getDB(keyValueContainerData, new
OzoneConfiguration()); OzoneConfiguration());
blockIterator = metadataStore.iterator(); blockIterator = db.getStore().iterator();
blockFilter = filter; blockFilter = filter;
} }
@ -145,4 +148,8 @@ public void seekToLast() {
nextBlock = null; nextBlock = null;
blockIterator.seekToLast(); blockIterator.seekToLast();
} }
public void close() {
db.close();
}
} }

View File

@ -54,7 +54,6 @@
.KeyValueContainerLocationUtil; .KeyValueContainerLocationUtil;
import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerUtil; import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerUtil;
import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException; import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
import org.apache.hadoop.utils.MetadataStore;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import org.apache.commons.io.FileUtils; import org.apache.commons.io.FileUtils;
@ -74,6 +73,7 @@
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.Result.UNSUPPORTED_REQUEST; .Result.UNSUPPORTED_REQUEST;
import org.apache.hadoop.ozone.container.common.utils.ContainerCache.ReferenceCountedDB;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -349,11 +349,12 @@ private void updateContainerData(Runnable update)
void compactDB() throws StorageContainerException { void compactDB() throws StorageContainerException {
try { try {
MetadataStore db = BlockUtils.getDB(containerData, config); try(ReferenceCountedDB db = BlockUtils.getDB(containerData, config)) {
db.compactDB(); db.getStore().compactDB();
LOG.info("Container {} is closed with bcsId {}.", LOG.info("Container {} is closed with bcsId {}.",
containerData.getContainerID(), containerData.getContainerID(),
containerData.getBlockCommitSequenceId()); containerData.getBlockCommitSequenceId());
}
} catch (StorageContainerException ex) { } catch (StorageContainerException ex) {
throw ex; throw ex;
} catch (IOException ex) { } catch (IOException ex) {

View File

@ -30,12 +30,12 @@
import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils; import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
import org.apache.hadoop.ozone.container.keyvalue.helpers.ChunkUtils; import org.apache.hadoop.ozone.container.keyvalue.helpers.ChunkUtils;
import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerLocationUtil; import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerLocationUtil;
import org.apache.hadoop.utils.MetadataStore;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.util.List; import java.util.List;
import org.apache.hadoop.ozone.container.common.utils.ContainerCache.ReferenceCountedDB;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -236,19 +236,19 @@ private void checkBlockDB() throws IOException {
onDiskContainerData.setDbFile(dbFile); onDiskContainerData.setDbFile(dbFile);
MetadataStore db = BlockUtils try(ReferenceCountedDB db =
.getDB(onDiskContainerData, checkConfig); BlockUtils.getDB(onDiskContainerData, checkConfig)) {
iterateBlockDB(db); iterateBlockDB(db);
} }
}
private void iterateBlockDB(MetadataStore db) private void iterateBlockDB(ReferenceCountedDB db)
throws IOException { throws IOException {
Preconditions.checkState(db != null); Preconditions.checkState(db != null);
// get "normal" keys from the Block DB // get "normal" keys from the Block DB
KeyValueBlockIterator kvIter = new KeyValueBlockIterator(containerID, try(KeyValueBlockIterator kvIter = new KeyValueBlockIterator(containerID,
new File(onDiskContainerData.getContainerPath())); new File(onDiskContainerData.getContainerPath()))) {
// ensure there is a chunk file for each key in the DB // ensure there is a chunk file for each key in the DB
while (kvIter.hasNext()) { while (kvIter.hasNext()) {
@ -262,7 +262,7 @@ private void iterateBlockDB(MetadataStore db)
if (!chunkFile.exists()) { if (!chunkFile.exists()) {
// concurrent mutation in Block DB? lookup the block again. // concurrent mutation in Block DB? lookup the block again.
byte[] bdata = db.get( byte[] bdata = db.getStore().get(
Longs.toByteArray(block.getBlockID().getLocalID())); Longs.toByteArray(block.getBlockID().getLocalID()));
if (bdata == null) { if (bdata == null) {
LOG.trace("concurrency with delete, ignoring deleted block"); LOG.trace("concurrency with delete, ignoring deleted block");
@ -276,6 +276,7 @@ private void iterateBlockDB(MetadataStore db)
} }
} }
} }
}
private void loadContainerData() throws IOException { private void loadContainerData() throws IOException {

View File

@ -37,7 +37,7 @@
import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils; import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData; import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
import org.apache.hadoop.ozone.container.common.utils.ContainerCache; import org.apache.hadoop.ozone.container.common.utils.ContainerCache;
import org.apache.hadoop.utils.MetadataStore; import org.apache.hadoop.ozone.container.common.utils.ContainerCache.ReferenceCountedDB;
import java.io.IOException; import java.io.IOException;
@ -66,7 +66,7 @@ private BlockUtils() {
* @return MetadataStore handle. * @return MetadataStore handle.
* @throws StorageContainerException * @throws StorageContainerException
*/ */
public static MetadataStore getDB(KeyValueContainerData containerData, public static ReferenceCountedDB getDB(KeyValueContainerData containerData,
Configuration conf) throws Configuration conf) throws
StorageContainerException { StorageContainerException {
Preconditions.checkNotNull(containerData); Preconditions.checkNotNull(containerData);

View File

@ -39,6 +39,7 @@
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import org.apache.commons.io.FileUtils; import org.apache.commons.io.FileUtils;
import org.apache.hadoop.ozone.container.common.utils.ContainerCache.ReferenceCountedDB;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -174,11 +175,13 @@ public static void parseKVContainerData(KeyValueContainerData kvContainerData,
} }
kvContainerData.setDbFile(dbFile); kvContainerData.setDbFile(dbFile);
MetadataStore metadata = BlockUtils.getDB(kvContainerData, config); try(ReferenceCountedDB metadata =
BlockUtils.getDB(kvContainerData, config)) {
long bytesUsed = 0; long bytesUsed = 0;
List<Map.Entry<byte[], byte[]>> liveKeys = metadata List<Map.Entry<byte[], byte[]>> liveKeys = metadata.getStore()
.getRangeKVs(null, Integer.MAX_VALUE, .getRangeKVs(null, Integer.MAX_VALUE,
MetadataKeyFilters.getNormalKeyFilter()); MetadataKeyFilters.getNormalKeyFilter());
bytesUsed = liveKeys.parallelStream().mapToLong(e-> { bytesUsed = liveKeys.parallelStream().mapToLong(e-> {
BlockData blockData; BlockData blockData;
try { try {
@ -191,6 +194,7 @@ public static void parseKVContainerData(KeyValueContainerData kvContainerData,
kvContainerData.setBytesUsed(bytesUsed); kvContainerData.setBytesUsed(bytesUsed);
kvContainerData.setKeyCount(liveKeys.size()); kvContainerData.setKeyCount(liveKeys.size());
} }
}
/** /**
* Returns the path where data or chunks live for a given container. * Returns the path where data or chunks live for a given container.

View File

@ -35,7 +35,7 @@
import org.apache.hadoop.ozone.container.common.utils.ContainerCache; import org.apache.hadoop.ozone.container.common.utils.ContainerCache;
import org.apache.hadoop.utils.BatchOperation; import org.apache.hadoop.utils.BatchOperation;
import org.apache.hadoop.utils.MetadataKeyFilters; import org.apache.hadoop.utils.MetadataKeyFilters;
import org.apache.hadoop.utils.MetadataStore; import org.apache.hadoop.ozone.container.common.utils.ContainerCache.ReferenceCountedDB;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -84,16 +84,15 @@ public long putBlock(Container container, BlockData data) throws IOException {
"cannot be negative"); "cannot be negative");
// We are not locking the key manager since LevelDb serializes all actions // We are not locking the key manager since LevelDb serializes all actions
// against a single DB. We rely on DB level locking to avoid conflicts. // against a single DB. We rely on DB level locking to avoid conflicts.
MetadataStore db = BlockUtils.getDB((KeyValueContainerData) container try(ReferenceCountedDB db = BlockUtils.
.getContainerData(), config); getDB((KeyValueContainerData) container.getContainerData(), config)) {
// This is a post condition that acts as a hint to the user. // This is a post condition that acts as a hint to the user.
// Should never fail. // Should never fail.
Preconditions.checkNotNull(db, "DB cannot be null here"); Preconditions.checkNotNull(db, "DB cannot be null here");
long bcsId = data.getBlockCommitSequenceId(); long bcsId = data.getBlockCommitSequenceId();
long containerBCSId = ((KeyValueContainerData) container.getContainerData()) long containerBCSId = ((KeyValueContainerData) container.
.getBlockCommitSequenceId(); getContainerData()).getBlockCommitSequenceId();
// default blockCommitSequenceId for any block is 0. It the putBlock // default blockCommitSequenceId for any block is 0. It the putBlock
// request is not coming via Ratis(for test scenarios), it will be 0. // request is not coming via Ratis(for test scenarios), it will be 0.
@ -117,7 +116,7 @@ public long putBlock(Container container, BlockData data) throws IOException {
data.getProtoBufMessage().toByteArray()); data.getProtoBufMessage().toByteArray());
batch.put(blockCommitSequenceIdKey, batch.put(blockCommitSequenceIdKey,
Longs.toByteArray(bcsId)); Longs.toByteArray(bcsId));
db.writeBatch(batch); db.getStore().writeBatch(batch);
container.updateBlockCommitSequenceId(bcsId); container.updateBlockCommitSequenceId(bcsId);
// Increment keycount here // Increment keycount here
container.getContainerData().incrKeyCount(); container.getContainerData().incrKeyCount();
@ -126,6 +125,7 @@ public long putBlock(Container container, BlockData data) throws IOException {
+ bcsId + " chunk size " + data.getChunks().size()); + bcsId + " chunk size " + data.getChunks().size());
return data.getSize(); return data.getSize();
} }
}
/** /**
* Gets an existing block. * Gets an existing block.
@ -146,7 +146,7 @@ public BlockData getBlock(Container container, BlockID blockID)
KeyValueContainerData containerData = (KeyValueContainerData) container KeyValueContainerData containerData = (KeyValueContainerData) container
.getContainerData(); .getContainerData();
MetadataStore db = BlockUtils.getDB(containerData, config); try(ReferenceCountedDB db = BlockUtils.getDB(containerData, config)) {
// This is a post condition that acts as a hint to the user. // This is a post condition that acts as a hint to the user.
// Should never fail. // Should never fail.
Preconditions.checkNotNull(db, "DB cannot be null here"); Preconditions.checkNotNull(db, "DB cannot be null here");
@ -158,10 +158,10 @@ public BlockData getBlock(Container container, BlockID blockID)
+ container.getContainerData().getContainerID() + " bcsId is " + container.getContainerData().getContainerID() + " bcsId is "
+ containerBCSId + ".", UNKNOWN_BCSID); + containerBCSId + ".", UNKNOWN_BCSID);
} }
byte[] kData = db.get(Longs.toByteArray(blockID.getLocalID())); byte[] kData = db.getStore().get(Longs.toByteArray(blockID.getLocalID()));
if (kData == null) { if (kData == null) {
throw new StorageContainerException("Unable to find the block." + blockID, throw new StorageContainerException("Unable to find the block." +
NO_SUCH_BLOCK); blockID, NO_SUCH_BLOCK);
} }
ContainerProtos.BlockData blockData = ContainerProtos.BlockData blockData =
ContainerProtos.BlockData.parseFrom(kData); ContainerProtos.BlockData.parseFrom(kData);
@ -173,6 +173,7 @@ public BlockData getBlock(Container container, BlockID blockID)
} }
return BlockData.getFromProtoBuf(blockData); return BlockData.getFromProtoBuf(blockData);
} }
}
/** /**
* Returns the length of the committed block. * Returns the length of the committed block.
@ -187,11 +188,11 @@ public long getCommittedBlockLength(Container container, BlockID blockID)
throws IOException { throws IOException {
KeyValueContainerData containerData = (KeyValueContainerData) container KeyValueContainerData containerData = (KeyValueContainerData) container
.getContainerData(); .getContainerData();
MetadataStore db = BlockUtils.getDB(containerData, config); try(ReferenceCountedDB db = BlockUtils.getDB(containerData, config)) {
// This is a post condition that acts as a hint to the user. // This is a post condition that acts as a hint to the user.
// Should never fail. // Should never fail.
Preconditions.checkNotNull(db, "DB cannot be null here"); Preconditions.checkNotNull(db, "DB cannot be null here");
byte[] kData = db.get(Longs.toByteArray(blockID.getLocalID())); byte[] kData = db.getStore().get(Longs.toByteArray(blockID.getLocalID()));
if (kData == null) { if (kData == null) {
throw new StorageContainerException("Unable to find the block.", throw new StorageContainerException("Unable to find the block.",
NO_SUCH_BLOCK); NO_SUCH_BLOCK);
@ -200,6 +201,7 @@ public long getCommittedBlockLength(Container container, BlockID blockID)
ContainerProtos.BlockData.parseFrom(kData); ContainerProtos.BlockData.parseFrom(kData);
return blockData.getSize(); return blockData.getSize();
} }
}
/** /**
* Deletes an existing block. * Deletes an existing block.
@ -218,7 +220,7 @@ public void deleteBlock(Container container, BlockID blockID) throws
KeyValueContainerData cData = (KeyValueContainerData) container KeyValueContainerData cData = (KeyValueContainerData) container
.getContainerData(); .getContainerData();
MetadataStore db = BlockUtils.getDB(cData, config); try(ReferenceCountedDB db = BlockUtils.getDB(cData, config)) {
// This is a post condition that acts as a hint to the user. // This is a post condition that acts as a hint to the user.
// Should never fail. // Should never fail.
Preconditions.checkNotNull(db, "DB cannot be null here"); Preconditions.checkNotNull(db, "DB cannot be null here");
@ -227,16 +229,16 @@ public void deleteBlock(Container container, BlockID blockID) throws
// to delete a Block which might have just gotten inserted after // to delete a Block which might have just gotten inserted after
// the get check. // the get check.
byte[] kKey = Longs.toByteArray(blockID.getLocalID()); byte[] kKey = Longs.toByteArray(blockID.getLocalID());
byte[] kData = db.get(kKey); try {
if (kData == null) { db.getStore().delete(kKey);
} catch (IOException e) {
throw new StorageContainerException("Unable to find the block.", throw new StorageContainerException("Unable to find the block.",
NO_SUCH_BLOCK); NO_SUCH_BLOCK);
} }
db.delete(kKey);
// Decrement blockcount here // Decrement blockcount here
container.getContainerData().decrKeyCount(); container.getContainerData().decrKeyCount();
} }
}
/** /**
* List blocks in a container. * List blocks in a container.
@ -258,11 +260,11 @@ public List<BlockData> listBlock(Container container, long startLocalID, int
List<BlockData> result = null; List<BlockData> result = null;
KeyValueContainerData cData = (KeyValueContainerData) container KeyValueContainerData cData = (KeyValueContainerData) container
.getContainerData(); .getContainerData();
MetadataStore db = BlockUtils.getDB(cData, config); try(ReferenceCountedDB db = BlockUtils.getDB(cData, config)) {
result = new ArrayList<>(); result = new ArrayList<>();
byte[] startKeyInBytes = Longs.toByteArray(startLocalID); byte[] startKeyInBytes = Longs.toByteArray(startLocalID);
List<Map.Entry<byte[], byte[]>> range = List<Map.Entry<byte[], byte[]>> range =
db.getSequentialRangeKVs(startKeyInBytes, count, db.getStore().getSequentialRangeKVs(startKeyInBytes, count,
MetadataKeyFilters.getNormalKeyFilter()); MetadataKeyFilters.getNormalKeyFilter());
for (Map.Entry<byte[], byte[]> entry : range) { for (Map.Entry<byte[], byte[]> entry : range) {
BlockData value = BlockUtils.getBlockData(entry.getValue()); BlockData value = BlockUtils.getBlockData(entry.getValue());
@ -271,6 +273,7 @@ public List<BlockData> listBlock(Container container, long startLocalID, int
} }
return result; return result;
} }
}
/** /**
* Shutdown KeyValueContainerManager. * Shutdown KeyValueContainerManager.

View File

@ -43,7 +43,7 @@
import org.apache.hadoop.utils.BackgroundTaskResult; import org.apache.hadoop.utils.BackgroundTaskResult;
import org.apache.hadoop.utils.BatchOperation; import org.apache.hadoop.utils.BatchOperation;
import org.apache.hadoop.utils.MetadataKeyFilters.KeyPrefixFilter; import org.apache.hadoop.utils.MetadataKeyFilters.KeyPrefixFilter;
import org.apache.hadoop.utils.MetadataStore; import org.apache.hadoop.ozone.container.common.utils.ContainerCache.ReferenceCountedDB;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -185,13 +185,13 @@ public BackgroundTaskResult call() throws Exception {
ContainerBackgroundTaskResult crr = new ContainerBackgroundTaskResult(); ContainerBackgroundTaskResult crr = new ContainerBackgroundTaskResult();
long startTime = Time.monotonicNow(); long startTime = Time.monotonicNow();
// Scan container's db and get list of under deletion blocks // Scan container's db and get list of under deletion blocks
MetadataStore meta = BlockUtils.getDB( try (ReferenceCountedDB meta = BlockUtils.getDB(containerData, conf)) {
(KeyValueContainerData) containerData, conf);
// # of blocks to delete is throttled // # of blocks to delete is throttled
KeyPrefixFilter filter = KeyPrefixFilter filter =
new KeyPrefixFilter().addFilter(OzoneConsts.DELETING_KEY_PREFIX); new KeyPrefixFilter().addFilter(OzoneConsts.DELETING_KEY_PREFIX);
List<Map.Entry<byte[], byte[]>> toDeleteBlocks = List<Map.Entry<byte[], byte[]>> toDeleteBlocks =
meta.getSequentialRangeKVs(null, blockLimitPerTask, filter); meta.getStore().getSequentialRangeKVs(null, blockLimitPerTask,
filter);
if (toDeleteBlocks.isEmpty()) { if (toDeleteBlocks.isEmpty()) {
LOG.debug("No under deletion block found in container : {}", LOG.debug("No under deletion block found in container : {}",
containerData.getContainerID()); containerData.getContainerID());
@ -227,7 +227,8 @@ public BackgroundTaskResult call() throws Exception {
} }
}); });
// Once files are deleted... replace deleting entries with deleted entries // Once files are deleted... replace deleting entries with deleted
// entries
BatchOperation batch = new BatchOperation(); BatchOperation batch = new BatchOperation();
succeedBlocks.forEach(entry -> { succeedBlocks.forEach(entry -> {
String blockId = String blockId =
@ -237,7 +238,7 @@ public BackgroundTaskResult call() throws Exception {
DFSUtil.string2Bytes(blockId)); DFSUtil.string2Bytes(blockId));
batch.delete(DFSUtil.string2Bytes(entry)); batch.delete(DFSUtil.string2Bytes(entry));
}); });
meta.writeBatch(batch); meta.getStore().writeBatch(batch);
// update count of pending deletion blocks in in-memory container status // update count of pending deletion blocks in in-memory container status
containerData.decrPendingDeletionBlocks(succeedBlocks.size()); containerData.decrPendingDeletionBlocks(succeedBlocks.size());
@ -249,6 +250,7 @@ public BackgroundTaskResult call() throws Exception {
crr.addAll(succeedBlocks); crr.addAll(succeedBlocks);
return crr; return crr;
} }
}
@Override @Override
public int getPriority() { public int getPriority() {

View File

@ -38,7 +38,7 @@
import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils; import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerUtil; import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerUtil;
import org.apache.hadoop.utils.MetadataKeyFilters; import org.apache.hadoop.utils.MetadataKeyFilters;
import org.apache.hadoop.utils.MetadataStore; import org.apache.hadoop.ozone.container.common.utils.ContainerCache.ReferenceCountedDB;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -180,28 +180,31 @@ public void verifyContainerData(ContainerData containerData)
KeyValueContainerUtil.parseKVContainerData(kvContainerData, config); KeyValueContainerUtil.parseKVContainerData(kvContainerData, config);
KeyValueContainer kvContainer = new KeyValueContainer( KeyValueContainer kvContainer = new KeyValueContainer(
kvContainerData, config); kvContainerData, config);
MetadataStore containerDB = BlockUtils.getDB(kvContainerData, config); try(ReferenceCountedDB containerDB = BlockUtils.getDB(kvContainerData,
config)) {
MetadataKeyFilters.KeyPrefixFilter filter = MetadataKeyFilters.KeyPrefixFilter filter =
new MetadataKeyFilters.KeyPrefixFilter() new MetadataKeyFilters.KeyPrefixFilter()
.addFilter(OzoneConsts.DELETING_KEY_PREFIX); .addFilter(OzoneConsts.DELETING_KEY_PREFIX);
int numPendingDeletionBlocks = int numPendingDeletionBlocks =
containerDB.getSequentialRangeKVs(null, Integer.MAX_VALUE, filter) containerDB.getStore().getSequentialRangeKVs(null,
Integer.MAX_VALUE, filter)
.size(); .size();
kvContainerData.incrPendingDeletionBlocks(numPendingDeletionBlocks); kvContainerData.incrPendingDeletionBlocks(numPendingDeletionBlocks);
byte[] delTxnId = containerDB.get( byte[] delTxnId = containerDB.getStore().get(
DFSUtil.string2Bytes(OzoneConsts.DELETE_TRANSACTION_KEY_PREFIX)); DFSUtil.string2Bytes(OzoneConsts.DELETE_TRANSACTION_KEY_PREFIX));
if (delTxnId != null) { if (delTxnId != null) {
kvContainerData kvContainerData
.updateDeleteTransactionId(Longs.fromByteArray(delTxnId)); .updateDeleteTransactionId(Longs.fromByteArray(delTxnId));
} }
// sets the BlockCommitSequenceId. // sets the BlockCommitSequenceId.
byte[] bcsId = containerDB.get( byte[] bcsId = containerDB.getStore().get(DFSUtil.string2Bytes(
DFSUtil.string2Bytes(OzoneConsts.BLOCK_COMMIT_SEQUENCE_ID_PREFIX)); OzoneConsts.BLOCK_COMMIT_SEQUENCE_ID_PREFIX));
if (bcsId != null) { if (bcsId != null) {
kvContainerData kvContainerData
.updateBlockCommitSequenceId(Longs.fromByteArray(bcsId)); .updateBlockCommitSequenceId(Longs.fromByteArray(bcsId));
} }
containerSet.addContainer(kvContainer); containerSet.addContainer(kvContainer);
}
} else { } else {
throw new StorageContainerException("Container File is corrupted. " + throw new StorageContainerException("Container File is corrupted. " +
"ContainerType is KeyValueContainer but cast to " + "ContainerType is KeyValueContainer but cast to " +

View File

@ -34,7 +34,7 @@
import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils; import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.utils.MetadataKeyFilters; import org.apache.hadoop.utils.MetadataKeyFilters;
import org.apache.hadoop.utils.MetadataStore; import org.apache.hadoop.ozone.container.common.utils.ContainerCache.ReferenceCountedDB;
import org.junit.After; import org.junit.After;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
@ -109,8 +109,8 @@ public void testKeyValueBlockIteratorWithMixedBlocks() throws Exception {
createContainerWithBlocks(containerID, normalBlocks, deletedBlocks); createContainerWithBlocks(containerID, normalBlocks, deletedBlocks);
String containerPath = new File(containerData.getMetadataPath()) String containerPath = new File(containerData.getMetadataPath())
.getParent(); .getParent();
KeyValueBlockIterator keyValueBlockIterator = new KeyValueBlockIterator( try(KeyValueBlockIterator keyValueBlockIterator = new KeyValueBlockIterator(
containerID, new File(containerPath)); containerID, new File(containerPath))) {
int counter = 0; int counter = 0;
while (keyValueBlockIterator.hasNext()) { while (keyValueBlockIterator.hasNext()) {
@ -135,6 +135,7 @@ public void testKeyValueBlockIteratorWithMixedBlocks() throws Exception {
"for ContainerID " + containerID, ex); "for ContainerID " + containerID, ex);
} }
} }
}
@Test @Test
public void testKeyValueBlockIteratorWithNextBlock() throws Exception { public void testKeyValueBlockIteratorWithNextBlock() throws Exception {
@ -142,8 +143,8 @@ public void testKeyValueBlockIteratorWithNextBlock() throws Exception {
createContainerWithBlocks(containerID, 2, 0); createContainerWithBlocks(containerID, 2, 0);
String containerPath = new File(containerData.getMetadataPath()) String containerPath = new File(containerData.getMetadataPath())
.getParent(); .getParent();
KeyValueBlockIterator keyValueBlockIterator = new KeyValueBlockIterator( try(KeyValueBlockIterator keyValueBlockIterator = new KeyValueBlockIterator(
containerID, new File(containerPath)); containerID, new File(containerPath))) {
long blockID = 0L; long blockID = 0L;
assertEquals(blockID++, keyValueBlockIterator.nextBlock().getLocalID()); assertEquals(blockID++, keyValueBlockIterator.nextBlock().getLocalID());
assertEquals(blockID, keyValueBlockIterator.nextBlock().getLocalID()); assertEquals(blockID, keyValueBlockIterator.nextBlock().getLocalID());
@ -155,6 +156,7 @@ public void testKeyValueBlockIteratorWithNextBlock() throws Exception {
"for ContainerID " + containerID, ex); "for ContainerID " + containerID, ex);
} }
} }
}
@Test @Test
public void testKeyValueBlockIteratorWithHasNext() throws Exception { public void testKeyValueBlockIteratorWithHasNext() throws Exception {
@ -162,8 +164,8 @@ public void testKeyValueBlockIteratorWithHasNext() throws Exception {
createContainerWithBlocks(containerID, 2, 0); createContainerWithBlocks(containerID, 2, 0);
String containerPath = new File(containerData.getMetadataPath()) String containerPath = new File(containerData.getMetadataPath())
.getParent(); .getParent();
KeyValueBlockIterator keyValueBlockIterator = new KeyValueBlockIterator( try(KeyValueBlockIterator keyValueBlockIterator = new KeyValueBlockIterator(
containerID, new File(containerPath)); containerID, new File(containerPath))) {
long blockID = 0L; long blockID = 0L;
// Even calling multiple times hasNext() should not move entry forward. // Even calling multiple times hasNext() should not move entry forward.
@ -196,8 +198,7 @@ public void testKeyValueBlockIteratorWithHasNext() throws Exception {
GenericTestUtils.assertExceptionContains("Block Iterator reached end " + GenericTestUtils.assertExceptionContains("Block Iterator reached end " +
"for ContainerID " + containerID, ex); "for ContainerID " + containerID, ex);
} }
}
} }
@Test @Test
@ -208,9 +209,9 @@ public void testKeyValueBlockIteratorWithFilter() throws Exception {
createContainerWithBlocks(containerId, normalBlocks, deletedBlocks); createContainerWithBlocks(containerId, normalBlocks, deletedBlocks);
String containerPath = new File(containerData.getMetadataPath()) String containerPath = new File(containerData.getMetadataPath())
.getParent(); .getParent();
KeyValueBlockIterator keyValueBlockIterator = new KeyValueBlockIterator( try(KeyValueBlockIterator keyValueBlockIterator = new KeyValueBlockIterator(
containerId, new File(containerPath), MetadataKeyFilters containerId, new File(containerPath), MetadataKeyFilters
.getDeletingKeyFilter()); .getDeletingKeyFilter())) {
int counter = 5; int counter = 5;
while (keyValueBlockIterator.hasNext()) { while (keyValueBlockIterator.hasNext()) {
@ -218,6 +219,7 @@ containerId, new File(containerPath), MetadataKeyFilters
assertEquals(blockData.getLocalID(), counter++); assertEquals(blockData.getLocalID(), counter++);
} }
} }
}
@Test @Test
public void testKeyValueBlockIteratorWithOnlyDeletedBlocks() throws public void testKeyValueBlockIteratorWithOnlyDeletedBlocks() throws
@ -226,12 +228,13 @@ public void testKeyValueBlockIteratorWithOnlyDeletedBlocks() throws
createContainerWithBlocks(containerId, 0, 5); createContainerWithBlocks(containerId, 0, 5);
String containerPath = new File(containerData.getMetadataPath()) String containerPath = new File(containerData.getMetadataPath())
.getParent(); .getParent();
KeyValueBlockIterator keyValueBlockIterator = new KeyValueBlockIterator( try(KeyValueBlockIterator keyValueBlockIterator = new KeyValueBlockIterator(
containerId, new File(containerPath)); containerId, new File(containerPath))) {
//As all blocks are deleted blocks, blocks does not match with normal key //As all blocks are deleted blocks, blocks does not match with normal key
// filter. // filter.
assertFalse(keyValueBlockIterator.hasNext()); assertFalse(keyValueBlockIterator.hasNext());
} }
}
/** /**
* Creates a container with specified number of normal blocks and deleted * Creates a container with specified number of normal blocks and deleted
@ -251,7 +254,8 @@ private void createContainerWithBlocks(long containerId, int
container = new KeyValueContainer(containerData, conf); container = new KeyValueContainer(containerData, conf);
container.create(volumeSet, new RoundRobinVolumeChoosingPolicy(), UUID container.create(volumeSet, new RoundRobinVolumeChoosingPolicy(), UUID
.randomUUID().toString()); .randomUUID().toString());
MetadataStore metadataStore = BlockUtils.getDB(containerData, conf); try(ReferenceCountedDB metadataStore = BlockUtils.getDB(containerData,
conf)) {
List<ContainerProtos.ChunkInfo> chunkList = new ArrayList<>(); List<ContainerProtos.ChunkInfo> chunkList = new ArrayList<>();
ChunkInfo info = new ChunkInfo("chunkfile", 0, 1024); ChunkInfo info = new ChunkInfo("chunkfile", 0, 1024);
@ -261,7 +265,8 @@ private void createContainerWithBlocks(long containerId, int
BlockID blockID = new BlockID(containerId, i); BlockID blockID = new BlockID(containerId, i);
BlockData blockData = new BlockData(blockID); BlockData blockData = new BlockData(blockID);
blockData.setChunks(chunkList); blockData.setChunks(chunkList);
metadataStore.put(Longs.toByteArray(blockID.getLocalID()), blockData metadataStore.getStore().put(Longs.toByteArray(blockID.getLocalID()),
blockData
.getProtoBufMessage().toByteArray()); .getProtoBufMessage().toByteArray());
} }
@ -269,10 +274,11 @@ private void createContainerWithBlocks(long containerId, int
BlockID blockID = new BlockID(containerId, i); BlockID blockID = new BlockID(containerId, i);
BlockData blockData = new BlockData(blockID); BlockData blockData = new BlockData(blockID);
blockData.setChunks(chunkList); blockData.setChunks(chunkList);
metadataStore.put(DFSUtil.string2Bytes(OzoneConsts metadataStore.getStore().put(DFSUtil.string2Bytes(OzoneConsts
.DELETING_KEY_PREFIX + blockID.getLocalID()), blockData .DELETING_KEY_PREFIX + blockID.getLocalID()), blockData
.getProtoBufMessage().toByteArray()); .getProtoBufMessage().toByteArray());
} }
} }
}
} }

View File

@ -36,7 +36,7 @@
import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils; import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.util.DiskChecker; import org.apache.hadoop.util.DiskChecker;
import org.apache.hadoop.utils.MetadataStore; import org.apache.hadoop.ozone.container.common.utils.ContainerCache.ReferenceCountedDB;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Before; import org.junit.Before;
@ -132,8 +132,8 @@ public void testBlockIterator() throws Exception{
private void addBlocks(int count) throws Exception { private void addBlocks(int count) throws Exception {
long containerId = keyValueContainerData.getContainerID(); long containerId = keyValueContainerData.getContainerID();
MetadataStore metadataStore = BlockUtils.getDB(keyValueContainer try(ReferenceCountedDB metadataStore = BlockUtils.getDB(keyValueContainer
.getContainerData(), conf); .getContainerData(), conf)) {
for (int i = 0; i < count; i++) { for (int i = 0; i < count; i++) {
// Creating BlockData // Creating BlockData
BlockID blockID = new BlockID(containerId, i); BlockID blockID = new BlockID(containerId, i);
@ -145,10 +145,11 @@ private void addBlocks(int count) throws Exception {
.getLocalID(), 0), 0, 1024); .getLocalID(), 0), 0, 1024);
chunkList.add(info.getProtoBufMessage()); chunkList.add(info.getProtoBufMessage());
blockData.setChunks(chunkList); blockData.setChunks(chunkList);
metadataStore.put(Longs.toByteArray(blockID.getLocalID()), blockData metadataStore.getStore().put(Longs.toByteArray(blockID.getLocalID()),
blockData
.getProtoBufMessage().toByteArray()); .getProtoBufMessage().toByteArray());
} }
}
} }
@SuppressWarnings("RedundantCast") @SuppressWarnings("RedundantCast")
@ -191,9 +192,12 @@ public void testContainerImportExport() throws Exception {
int numberOfKeysToWrite = 12; int numberOfKeysToWrite = 12;
//write one few keys to check the key count after import //write one few keys to check the key count after import
MetadataStore metadataStore = BlockUtils.getDB(keyValueContainerData, conf); try(ReferenceCountedDB metadataStore =
BlockUtils.getDB(keyValueContainerData, conf)) {
for (int i = 0; i < numberOfKeysToWrite; i++) { for (int i = 0; i < numberOfKeysToWrite; i++) {
metadataStore.put(("test" + i).getBytes(UTF_8), "test".getBytes(UTF_8)); metadataStore.getStore().put(("test" + i).getBytes(UTF_8),
"test".getBytes(UTF_8));
}
} }
BlockUtils.removeDB(keyValueContainerData, conf); BlockUtils.removeDB(keyValueContainerData, conf);

View File

@ -35,7 +35,7 @@
import org.apache.hadoop.ozone.container.common.volume.VolumeSet; import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils; import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.utils.MetadataStore; import org.apache.hadoop.ozone.container.common.utils.ContainerCache.ReferenceCountedDB;
import org.junit.After; import org.junit.After;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
@ -149,7 +149,8 @@ private void createContainerWithBlocks(long containerId, int normalBlocks,
container = new KeyValueContainer(containerData, conf); container = new KeyValueContainer(containerData, conf);
container.create(volumeSet, new RoundRobinVolumeChoosingPolicy(), container.create(volumeSet, new RoundRobinVolumeChoosingPolicy(),
UUID.randomUUID().toString()); UUID.randomUUID().toString());
MetadataStore metadataStore = BlockUtils.getDB(containerData, conf); try (ReferenceCountedDB metadataStore = BlockUtils.getDB(containerData,
conf)) {
chunkManager = new ChunkManagerImpl(true); chunkManager = new ChunkManagerImpl(true);
assertTrue(containerData.getChunksPath() != null); assertTrue(containerData.getChunksPath() != null);
@ -184,14 +185,15 @@ private void createContainerWithBlocks(long containerId, int normalBlocks,
if (i >= normalBlocks) { if (i >= normalBlocks) {
// deleted key // deleted key
metadataStore.put(DFSUtil.string2Bytes( metadataStore.getStore().put(DFSUtil.string2Bytes(
OzoneConsts.DELETING_KEY_PREFIX + blockID.getLocalID()), OzoneConsts.DELETING_KEY_PREFIX + blockID.getLocalID()),
blockData.getProtoBufMessage().toByteArray()); blockData.getProtoBufMessage().toByteArray());
} else { } else {
// normal key // normal key
metadataStore.put(Longs.toByteArray(blockID.getLocalID()), metadataStore.getStore().put(Longs.toByteArray(blockID.getLocalID()),
blockData.getProtoBufMessage().toByteArray()); blockData.getProtoBufMessage().toByteArray());
} }
} }
} }
} }
}

View File

@ -40,7 +40,7 @@
import org.apache.hadoop.ozone.web.utils.OzoneUtils; import org.apache.hadoop.ozone.web.utils.OzoneUtils;
import org.apache.hadoop.utils.MetadataKeyFilters; import org.apache.hadoop.utils.MetadataKeyFilters;
import org.apache.hadoop.utils.MetadataKeyFilters.KeyPrefixFilter; import org.apache.hadoop.utils.MetadataKeyFilters.KeyPrefixFilter;
import org.apache.hadoop.utils.MetadataStore; import org.apache.hadoop.ozone.container.common.utils.ContainerCache.ReferenceCountedDB;
import java.io.IOException; import java.io.IOException;
import java.io.OutputStream; import java.io.OutputStream;
@ -119,16 +119,17 @@ public Map<String, OmKeyInfo> createKeys(int numOfKeys, int keySize)
public List<String> getPendingDeletionBlocks(Long containerID) public List<String> getPendingDeletionBlocks(Long containerID)
throws IOException { throws IOException {
List<String> pendingDeletionBlocks = Lists.newArrayList(); List<String> pendingDeletionBlocks = Lists.newArrayList();
MetadataStore meta = getContainerMetadata(containerID); ReferenceCountedDB meta = getContainerMetadata(containerID);
KeyPrefixFilter filter = KeyPrefixFilter filter =
new KeyPrefixFilter().addFilter(OzoneConsts.DELETING_KEY_PREFIX); new KeyPrefixFilter().addFilter(OzoneConsts.DELETING_KEY_PREFIX);
List<Map.Entry<byte[], byte[]>> kvs = meta List<Map.Entry<byte[], byte[]>> kvs = meta.getStore()
.getRangeKVs(null, Integer.MAX_VALUE, filter); .getRangeKVs(null, Integer.MAX_VALUE, filter);
kvs.forEach(entry -> { kvs.forEach(entry -> {
String key = DFSUtil.bytes2String(entry.getKey()); String key = DFSUtil.bytes2String(entry.getKey());
pendingDeletionBlocks pendingDeletionBlocks
.add(key.replace(OzoneConsts.DELETING_KEY_PREFIX, "")); .add(key.replace(OzoneConsts.DELETING_KEY_PREFIX, ""));
}); });
meta.close();
return pendingDeletionBlocks; return pendingDeletionBlocks;
} }
@ -143,17 +144,18 @@ public List<Long> getAllBlocks(Set<Long> containerIDs)
public List<Long> getAllBlocks(Long containeID) throws IOException { public List<Long> getAllBlocks(Long containeID) throws IOException {
List<Long> allBlocks = Lists.newArrayList(); List<Long> allBlocks = Lists.newArrayList();
MetadataStore meta = getContainerMetadata(containeID); ReferenceCountedDB meta = getContainerMetadata(containeID);
List<Map.Entry<byte[], byte[]>> kvs = List<Map.Entry<byte[], byte[]>> kvs =
meta.getRangeKVs(null, Integer.MAX_VALUE, meta.getStore().getRangeKVs(null, Integer.MAX_VALUE,
MetadataKeyFilters.getNormalKeyFilter()); MetadataKeyFilters.getNormalKeyFilter());
kvs.forEach(entry -> { kvs.forEach(entry -> {
allBlocks.add(Longs.fromByteArray(entry.getKey())); allBlocks.add(Longs.fromByteArray(entry.getKey()));
}); });
meta.close();
return allBlocks; return allBlocks;
} }
private MetadataStore getContainerMetadata(Long containerID) private ReferenceCountedDB getContainerMetadata(Long containerID)
throws IOException { throws IOException {
ContainerWithPipeline containerWithPipeline = cluster ContainerWithPipeline containerWithPipeline = cluster
.getStorageContainerManager().getClientProtocolServer() .getStorageContainerManager().getClientProtocolServer()

View File

@ -953,8 +953,8 @@ public void testGetKeyDetails() throws IOException, OzoneException {
.getContainerData()); .getContainerData());
String containerPath = new File(containerData.getMetadataPath()) String containerPath = new File(containerData.getMetadataPath())
.getParent(); .getParent();
KeyValueBlockIterator keyValueBlockIterator = new KeyValueBlockIterator( try(KeyValueBlockIterator keyValueBlockIterator = new KeyValueBlockIterator(
containerID, new File(containerPath)); containerID, new File(containerPath))) {
while (keyValueBlockIterator.hasNext()) { while (keyValueBlockIterator.hasNext()) {
BlockData blockData = keyValueBlockIterator.nextBlock(); BlockData blockData = keyValueBlockIterator.nextBlock();
if (blockData.getBlockID().getLocalID() == localID) { if (blockData.getBlockID().getLocalID() == localID) {
@ -968,6 +968,7 @@ public void testGetKeyDetails() throws IOException, OzoneException {
} }
} }
} }
}
/** /**
* Tests reading a corrputed chunk file throws checksum exception. * Tests reading a corrputed chunk file throws checksum exception.
@ -1115,8 +1116,8 @@ private void corruptData(Container container, OzoneKey key)
(KeyValueContainerData) container.getContainerData(); (KeyValueContainerData) container.getContainerData();
String containerPath = String containerPath =
new File(containerData.getMetadataPath()).getParent(); new File(containerData.getMetadataPath()).getParent();
KeyValueBlockIterator keyValueBlockIterator = try (KeyValueBlockIterator keyValueBlockIterator =
new KeyValueBlockIterator(containerID, new File(containerPath)); new KeyValueBlockIterator(containerID, new File(containerPath))) {
// Find the block corresponding to the key we put. We use the localID of // Find the block corresponding to the key we put. We use the localID of
// the BlockData to identify out key. // the BlockData to identify out key.
@ -1141,6 +1142,7 @@ private void corruptData(Container container, OzoneKey key)
String newData = new String("corrupted data"); String newData = new String("corrupted data");
FileUtils.writeByteArrayToFile(chunkFile, newData.getBytes()); FileUtils.writeByteArrayToFile(chunkFile, newData.getBytes());
} }
}
@Test @Test
public void testDeleteKey() public void testDeleteKey()

View File

@ -45,7 +45,7 @@
import org.apache.hadoop.test.GenericTestUtils.LogCapturer; import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
import org.apache.hadoop.utils.BackgroundService; import org.apache.hadoop.utils.BackgroundService;
import org.apache.hadoop.utils.MetadataKeyFilters; import org.apache.hadoop.utils.MetadataKeyFilters;
import org.apache.hadoop.utils.MetadataStore; import org.apache.hadoop.ozone.container.common.utils.ContainerCache.ReferenceCountedDB;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
@ -119,7 +119,7 @@ private void createToDeleteBlocks(ContainerSet containerSet,
containerSet.addContainer(container); containerSet.addContainer(container);
data = (KeyValueContainerData) containerSet.getContainer( data = (KeyValueContainerData) containerSet.getContainer(
containerID).getContainerData(); containerID).getContainerData();
MetadataStore metadata = BlockUtils.getDB(data, conf); try(ReferenceCountedDB metadata = BlockUtils.getDB(data, conf)) {
for (int j = 0; j < numOfBlocksPerContainer; j++) { for (int j = 0; j < numOfBlocksPerContainer; j++) {
BlockID blockID = BlockID blockID =
ContainerTestHelper.getTestBlockID(containerID); ContainerTestHelper.getTestBlockID(containerID);
@ -146,11 +146,12 @@ private void createToDeleteBlocks(ContainerSet containerSet,
chunks.add(info); chunks.add(info);
} }
kd.setChunks(chunks); kd.setChunks(chunks);
metadata.put(DFSUtil.string2Bytes(deleteStateName), metadata.getStore().put(DFSUtil.string2Bytes(deleteStateName),
kd.getProtoBufMessage().toByteArray()); kd.getProtoBufMessage().toByteArray());
} }
} }
} }
}
/** /**
* Run service runDeletingTasks and wait for it's been processed. * Run service runDeletingTasks and wait for it's been processed.
@ -166,17 +167,19 @@ private void deleteAndWait(BlockDeletingServiceTestImpl service,
* Get under deletion blocks count from DB, * Get under deletion blocks count from DB,
* note this info is parsed from container.db. * note this info is parsed from container.db.
*/ */
private int getUnderDeletionBlocksCount(MetadataStore meta) private int getUnderDeletionBlocksCount(ReferenceCountedDB meta)
throws IOException { throws IOException {
List<Map.Entry<byte[], byte[]>> underDeletionBlocks = List<Map.Entry<byte[], byte[]>> underDeletionBlocks =
meta.getRangeKVs(null, 100, new MetadataKeyFilters.KeyPrefixFilter() meta.getStore().getRangeKVs(null, 100,
new MetadataKeyFilters.KeyPrefixFilter()
.addFilter(OzoneConsts.DELETING_KEY_PREFIX)); .addFilter(OzoneConsts.DELETING_KEY_PREFIX));
return underDeletionBlocks.size(); return underDeletionBlocks.size();
} }
private int getDeletedBlocksCount(MetadataStore db) throws IOException { private int getDeletedBlocksCount(ReferenceCountedDB db) throws IOException {
List<Map.Entry<byte[], byte[]>> underDeletionBlocks = List<Map.Entry<byte[], byte[]>> underDeletionBlocks =
db.getRangeKVs(null, 100, new MetadataKeyFilters.KeyPrefixFilter() db.getStore().getRangeKVs(null, 100,
new MetadataKeyFilters.KeyPrefixFilter()
.addFilter(OzoneConsts.DELETED_KEY_PREFIX)); .addFilter(OzoneConsts.DELETED_KEY_PREFIX));
return underDeletionBlocks.size(); return underDeletionBlocks.size();
} }
@ -202,8 +205,8 @@ public void testBlockDeletion() throws Exception {
containerSet.listContainer(0L, 1, containerData); containerSet.listContainer(0L, 1, containerData);
Assert.assertEquals(1, containerData.size()); Assert.assertEquals(1, containerData.size());
MetadataStore meta = BlockUtils.getDB( try(ReferenceCountedDB meta = BlockUtils.getDB(
(KeyValueContainerData) containerData.get(0), conf); (KeyValueContainerData) containerData.get(0), conf)) {
Map<Long, Container> containerMap = containerSet.getContainerMapCopy(); Map<Long, Container> containerMap = containerSet.getContainerMapCopy();
// NOTE: this test assumes that all the container is KetValueContainer and // NOTE: this test assumes that all the container is KetValueContainer and
// have DeleteTransactionId in KetValueContainerData. If other // have DeleteTransactionId in KetValueContainerData. If other
@ -233,6 +236,7 @@ public void testBlockDeletion() throws Exception {
deleteAndWait(svc, 3); deleteAndWait(svc, 3);
Assert.assertEquals(0, getUnderDeletionBlocksCount(meta)); Assert.assertEquals(0, getUnderDeletionBlocksCount(meta));
Assert.assertEquals(3, getDeletedBlocksCount(meta)); Assert.assertEquals(3, getDeletedBlocksCount(meta));
}
svc.shutdown(); svc.shutdown();
} }
@ -311,8 +315,8 @@ public void testBlockDeletionTimeout() throws Exception {
// get container meta data // get container meta data
List<ContainerData> containerData = Lists.newArrayList(); List<ContainerData> containerData = Lists.newArrayList();
containerSet.listContainer(0L, 1, containerData); containerSet.listContainer(0L, 1, containerData);
MetadataStore meta = BlockUtils.getDB( try(ReferenceCountedDB meta = BlockUtils.getDB(
(KeyValueContainerData) containerData.get(0), conf); (KeyValueContainerData) containerData.get(0), conf)) {
LogCapturer newLog = LogCapturer.captureLogs(BackgroundService.LOG); LogCapturer newLog = LogCapturer.captureLogs(BackgroundService.LOG);
GenericTestUtils.waitFor(() -> { GenericTestUtils.waitFor(() -> {
@ -330,6 +334,7 @@ public void testBlockDeletionTimeout() throws Exception {
// out warning log. // out warning log.
Assert.assertTrue(!newLog.getOutput().contains( Assert.assertTrue(!newLog.getOutput().contains(
"Background task executes timed out, retrying in next interval")); "Background task executes timed out, retrying in next interval"));
}
svc.shutdown(); svc.shutdown();
} }

View File

@ -47,7 +47,7 @@
import org.apache.hadoop.ozone.container.keyvalue.interfaces.ChunkManager; import org.apache.hadoop.ozone.container.keyvalue.interfaces.ChunkManager;
import org.apache.hadoop.ozone.container.keyvalue.interfaces.BlockManager; import org.apache.hadoop.ozone.container.keyvalue.interfaces.BlockManager;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.utils.MetadataStore; import org.apache.hadoop.ozone.container.common.utils.ContainerCache.ReferenceCountedDB;
import org.junit.After; import org.junit.After;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.Assert; import org.junit.Assert;
@ -202,7 +202,7 @@ public void testCreateContainer() throws Exception {
Path meta = kvData.getDbFile().toPath().getParent(); Path meta = kvData.getDbFile().toPath().getParent();
Assert.assertTrue(meta != null && Files.exists(meta)); Assert.assertTrue(meta != null && Files.exists(meta));
MetadataStore store = null; ReferenceCountedDB store = null;
try { try {
store = BlockUtils.getDB(kvData, conf); store = BlockUtils.getDB(kvData, conf);
Assert.assertNotNull(store); Assert.assertNotNull(store);

View File

@ -50,7 +50,7 @@
import org.apache.hadoop.ozone.protocol.commands.RetriableDatanodeEventWatcher; import org.apache.hadoop.ozone.protocol.commands.RetriableDatanodeEventWatcher;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.test.GenericTestUtils.LogCapturer; import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
import org.apache.hadoop.utils.MetadataStore; import org.apache.hadoop.ozone.container.common.utils.ContainerCache.ReferenceCountedDB;
import org.junit.Assert; import org.junit.Assert;
import org.junit.BeforeClass; import org.junit.BeforeClass;
import org.junit.Test; import org.junit.Test;
@ -300,9 +300,12 @@ private void verifyBlocksCreated(
cluster.getHddsDatanodes().get(0).getDatanodeStateMachine() cluster.getHddsDatanodes().get(0).getDatanodeStateMachine()
.getContainer().getContainerSet(); .getContainer().getContainerSet();
OzoneTestUtils.performOperationOnKeyContainers((blockID) -> { OzoneTestUtils.performOperationOnKeyContainers((blockID) -> {
MetadataStore db = BlockUtils.getDB((KeyValueContainerData) dnContainerSet try(ReferenceCountedDB db =
.getContainer(blockID.getContainerID()).getContainerData(), conf); BlockUtils.getDB((KeyValueContainerData) dnContainerSet
Assert.assertNotNull(db.get(Longs.toByteArray(blockID.getLocalID()))); .getContainer(blockID.getContainerID()).getContainerData(), conf)) {
Assert.assertNotNull(db.getStore().get(
Longs.toByteArray(blockID.getLocalID())));
}
}, omKeyLocationInfoGroups); }, omKeyLocationInfoGroups);
} }
@ -312,13 +315,16 @@ private void verifyBlocksDeleted(
cluster.getHddsDatanodes().get(0).getDatanodeStateMachine() cluster.getHddsDatanodes().get(0).getDatanodeStateMachine()
.getContainer().getContainerSet(); .getContainer().getContainerSet();
OzoneTestUtils.performOperationOnKeyContainers((blockID) -> { OzoneTestUtils.performOperationOnKeyContainers((blockID) -> {
MetadataStore db = BlockUtils.getDB((KeyValueContainerData) dnContainerSet try(ReferenceCountedDB db =
.getContainer(blockID.getContainerID()).getContainerData(), conf); BlockUtils.getDB((KeyValueContainerData) dnContainerSet
Assert.assertNull(db.get(Longs.toByteArray(blockID.getLocalID()))); .getContainer(blockID.getContainerID()).getContainerData(), conf)) {
Assert.assertNull(db.get(DFSUtil.string2Bytes( Assert.assertNull(db.getStore().get(
Longs.toByteArray(blockID.getLocalID())));
Assert.assertNull(db.getStore().get(DFSUtil.string2Bytes(
OzoneConsts.DELETING_KEY_PREFIX + blockID.getLocalID()))); OzoneConsts.DELETING_KEY_PREFIX + blockID.getLocalID())));
Assert.assertNotNull(DFSUtil Assert.assertNotNull(DFSUtil.string2Bytes(
.string2Bytes(OzoneConsts.DELETED_KEY_PREFIX + blockID.getLocalID())); OzoneConsts.DELETED_KEY_PREFIX + blockID.getLocalID()));
}
containerIdsWithDeletedBlocks.add(blockID.getContainerID()); containerIdsWithDeletedBlocks.add(blockID.getContainerID());
}, omKeyLocationInfoGroups); }, omKeyLocationInfoGroups);
} }

View File

@ -41,7 +41,7 @@
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
import org.apache.hadoop.ozone.protocol.commands.CloseContainerCommand; import org.apache.hadoop.ozone.protocol.commands.CloseContainerCommand;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.utils.MetadataStore; import org.apache.hadoop.ozone.container.common.utils.ContainerCache.ReferenceCountedDB;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.Assert; import org.junit.Assert;
import org.junit.BeforeClass; import org.junit.BeforeClass;
@ -226,7 +226,7 @@ public void testCloseContainerViaRatis() throws IOException,
List<DatanodeDetails> datanodes = pipeline.getNodes(); List<DatanodeDetails> datanodes = pipeline.getNodes();
Assert.assertEquals(3, datanodes.size()); Assert.assertEquals(3, datanodes.size());
List<MetadataStore> metadataStores = new ArrayList<>(datanodes.size()); List<ReferenceCountedDB> metadataStores = new ArrayList<>(datanodes.size());
for (DatanodeDetails details : datanodes) { for (DatanodeDetails details : datanodes) {
Assert.assertFalse(isContainerClosed(cluster, containerID, details)); Assert.assertFalse(isContainerClosed(cluster, containerID, details));
//send the order to close the container //send the order to close the container
@ -237,8 +237,10 @@ public void testCloseContainerViaRatis() throws IOException,
Container dnContainer = cluster.getHddsDatanodes().get(index) Container dnContainer = cluster.getHddsDatanodes().get(index)
.getDatanodeStateMachine().getContainer().getContainerSet() .getDatanodeStateMachine().getContainer().getContainerSet()
.getContainer(containerID); .getContainer(containerID);
metadataStores.add(BlockUtils.getDB((KeyValueContainerData) dnContainer try(ReferenceCountedDB store = BlockUtils.getDB(
.getContainerData(), conf)); (KeyValueContainerData) dnContainer.getContainerData(), conf)) {
metadataStores.add(store);
}
} }
// There should be as many rocks db as the number of datanodes in pipeline. // There should be as many rocks db as the number of datanodes in pipeline.