HDDS-72. Add deleteTransactionId field in ContainerInfo. Contributed by Lokesh Jain.

This commit is contained in:
Xiaoyu Yao 2018-06-11 16:02:32 -07:00
parent 7c3dc39083
commit 23bfd9f7e4
25 changed files with 402 additions and 71 deletions

View File

@ -32,6 +32,8 @@ import org.apache.hadoop.util.Time;
import java.io.IOException; import java.io.IOException;
import java.util.Comparator; import java.util.Comparator;
import static java.lang.Math.max;
/** /**
* Class wraps ozone container info. * Class wraps ozone container info.
*/ */
@ -60,6 +62,7 @@ public class ContainerInfo
private long stateEnterTime; private long stateEnterTime;
private String owner; private String owner;
private long containerID; private long containerID;
private long deleteTransactionId;
ContainerInfo( ContainerInfo(
long containerID, long containerID,
HddsProtos.LifeCycleState state, HddsProtos.LifeCycleState state,
@ -68,7 +71,8 @@ public class ContainerInfo
long usedBytes, long usedBytes,
long numberOfKeys, long numberOfKeys,
long stateEnterTime, long stateEnterTime,
String owner) { String owner,
long deleteTransactionId) {
this.containerID = containerID; this.containerID = containerID;
this.pipeline = pipeline; this.pipeline = pipeline;
this.allocatedBytes = allocatedBytes; this.allocatedBytes = allocatedBytes;
@ -78,6 +82,7 @@ public class ContainerInfo
this.state = state; this.state = state;
this.stateEnterTime = stateEnterTime; this.stateEnterTime = stateEnterTime;
this.owner = owner; this.owner = owner;
this.deleteTransactionId = deleteTransactionId;
} }
/** /**
@ -96,6 +101,7 @@ public class ContainerInfo
builder.setStateEnterTime(info.getStateEnterTime()); builder.setStateEnterTime(info.getStateEnterTime());
builder.setOwner(info.getOwner()); builder.setOwner(info.getOwner());
builder.setContainerID(info.getContainerID()); builder.setContainerID(info.getContainerID());
builder.setDeleteTransactionId(info.getDeleteTransactionId());
return builder.build(); return builder.build();
} }
@ -141,6 +147,14 @@ public class ContainerInfo
return numberOfKeys; return numberOfKeys;
} }
public long getDeleteTransactionId() {
return deleteTransactionId;
}
public void updateDeleteTransactionId(long transactionId) {
deleteTransactionId = max(transactionId, deleteTransactionId);
}
public ContainerID containerID() { public ContainerID containerID() {
return new ContainerID(getContainerID()); return new ContainerID(getContainerID());
} }
@ -174,6 +188,7 @@ public class ContainerInfo
builder.setState(state); builder.setState(state);
builder.setStateEnterTime(stateEnterTime); builder.setStateEnterTime(stateEnterTime);
builder.setContainerID(getContainerID()); builder.setContainerID(getContainerID());
builder.setDeleteTransactionId(deleteTransactionId);
if (getOwner() != null) { if (getOwner() != null) {
builder.setOwner(getOwner()); builder.setOwner(getOwner());
@ -292,6 +307,7 @@ public class ContainerInfo
private long stateEnterTime; private long stateEnterTime;
private String owner; private String owner;
private long containerID; private long containerID;
private long deleteTransactionId;
public Builder setContainerID(long id) { public Builder setContainerID(long id) {
Preconditions.checkState(id >= 0); Preconditions.checkState(id >= 0);
@ -334,10 +350,15 @@ public class ContainerInfo
return this; return this;
} }
public Builder setDeleteTransactionId(long deleteTransactionId) {
this.deleteTransactionId = deleteTransactionId;
return this;
}
public ContainerInfo build() { public ContainerInfo build() {
return new return new
ContainerInfo(containerID, state, pipeline, ContainerInfo(containerID, state, pipeline, allocated,
allocated, used, keys, stateEnterTime, owner); used, keys, stateEnterTime, owner, deleteTransactionId);
} }
} }
} }

View File

@ -114,6 +114,8 @@ public final class OzoneConsts {
public static final String OZONE_HANDLER_LOCAL = "local"; public static final String OZONE_HANDLER_LOCAL = "local";
public static final String DELETING_KEY_PREFIX = "#deleting#"; public static final String DELETING_KEY_PREFIX = "#deleting#";
public static final String DELETED_KEY_PREFIX = "#deleted#";
public static final String DELETE_TRANSACTION_KEY_PREFIX = "#delTX#";
public static final String OPEN_KEY_PREFIX = "#open#"; public static final String OPEN_KEY_PREFIX = "#open#";
public static final String OPEN_KEY_ID_DELIMINATOR = "#"; public static final String OPEN_KEY_ID_DELIMINATOR = "#";

View File

@ -22,17 +22,27 @@ import com.google.common.base.Strings;
import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.ozone.OzoneConsts; import org.apache.hadoop.ozone.OzoneConsts;
import java.util.ArrayList;
import java.util.List;
/** /**
* An utility class to filter levelDB keys. * An utility class to filter levelDB keys.
*/ */
public final class MetadataKeyFilters { public final class MetadataKeyFilters {
private static KeyPrefixFilter deletingKeyFilter = private static KeyPrefixFilter deletingKeyFilter =
new MetadataKeyFilters.KeyPrefixFilter(OzoneConsts.DELETING_KEY_PREFIX); new MetadataKeyFilters.KeyPrefixFilter()
.addFilter(OzoneConsts.DELETING_KEY_PREFIX);
private static KeyPrefixFilter deletedKeyFilter =
new MetadataKeyFilters.KeyPrefixFilter()
.addFilter(OzoneConsts.DELETED_KEY_PREFIX);
private static KeyPrefixFilter normalKeyFilter = private static KeyPrefixFilter normalKeyFilter =
new MetadataKeyFilters.KeyPrefixFilter(OzoneConsts.DELETING_KEY_PREFIX, new MetadataKeyFilters.KeyPrefixFilter()
true); .addFilter(OzoneConsts.DELETING_KEY_PREFIX, true)
.addFilter(OzoneConsts.DELETED_KEY_PREFIX, true)
.addFilter(OzoneConsts.DELETE_TRANSACTION_KEY_PREFIX, true);
private MetadataKeyFilters() { private MetadataKeyFilters() {
} }
@ -41,6 +51,10 @@ public final class MetadataKeyFilters {
return deletingKeyFilter; return deletingKeyFilter;
} }
public static KeyPrefixFilter getDeletedKeyFilter() {
return deletedKeyFilter;
}
public static KeyPrefixFilter getNormalKeyFilter() { public static KeyPrefixFilter getNormalKeyFilter() {
return normalKeyFilter; return normalKeyFilter;
} }
@ -73,37 +87,95 @@ public final class MetadataKeyFilters {
*/ */
public static class KeyPrefixFilter implements MetadataKeyFilter { public static class KeyPrefixFilter implements MetadataKeyFilter {
private String keyPrefix = null; private List<String> positivePrefixList = new ArrayList<>();
private List<String> negativePrefixList = new ArrayList<>();
private boolean atleastOnePositiveMatch;
private int keysScanned = 0; private int keysScanned = 0;
private int keysHinted = 0; private int keysHinted = 0;
private Boolean negative;
public KeyPrefixFilter(String keyPrefix) { public KeyPrefixFilter() {}
this(keyPrefix, false);
/**
* KeyPrefixFilter constructor. It is made of positive and negative prefix
* list. PositivePrefixList is the list of prefixes which are accepted
* whereas negativePrefixList contains the list of prefixes which are
* rejected.
*
* @param atleastOnePositiveMatch if positive it requires key to be accepted
* by atleast one positive filter.
*/
public KeyPrefixFilter(boolean atleastOnePositiveMatch) {
this.atleastOnePositiveMatch = atleastOnePositiveMatch;
} }
public KeyPrefixFilter(String keyPrefix, boolean negative) { public KeyPrefixFilter addFilter(String keyPrefix) {
this.keyPrefix = keyPrefix; addFilter(keyPrefix, false);
this.negative = negative; return this;
}
public KeyPrefixFilter addFilter(String keyPrefix, boolean negative) {
Preconditions.checkArgument(!Strings.isNullOrEmpty(keyPrefix),
"KeyPrefix is null or empty: " + keyPrefix);
// keyPrefix which needs to be added should not be prefix of any opposing
// filter already present. If keyPrefix is a negative filter it should not
// be a prefix of any positive filter. Nor should any opposing filter be
// a prefix of keyPrefix.
// For example if b0 is accepted b can not be rejected and
// if b is accepted b0 can not be rejected. If these scenarios need to be
// handled we need to add priorities.
if (negative) {
Preconditions.checkArgument(positivePrefixList.stream().noneMatch(
prefix -> prefix.startsWith(keyPrefix) || keyPrefix
.startsWith(prefix)),
"KeyPrefix: " + keyPrefix + " already accepted.");
this.negativePrefixList.add(keyPrefix);
} else {
Preconditions.checkArgument(negativePrefixList.stream().noneMatch(
prefix -> prefix.startsWith(keyPrefix) || keyPrefix
.startsWith(prefix)),
"KeyPrefix: " + keyPrefix + " already rejected.");
this.positivePrefixList.add(keyPrefix);
}
return this;
} }
@Override @Override
public boolean filterKey(byte[] preKey, byte[] currentKey, public boolean filterKey(byte[] preKey, byte[] currentKey,
byte[] nextKey) { byte[] nextKey) {
keysScanned++; keysScanned++;
boolean accept = false; if (currentKey == null) {
if (Strings.isNullOrEmpty(keyPrefix)) { return false;
accept = true;
} else {
byte [] prefixBytes = DFSUtil.string2Bytes(keyPrefix);
if (currentKey != null && prefixMatch(prefixBytes, currentKey)) {
keysHinted++;
accept = true;
} else {
accept = false;
}
} }
return (negative) ? !accept : accept; boolean accept;
// There are no filters present
if (positivePrefixList.isEmpty() && negativePrefixList.isEmpty()) {
return true;
}
accept = !positivePrefixList.isEmpty() && positivePrefixList.stream()
.anyMatch(prefix -> {
byte[] prefixBytes = DFSUtil.string2Bytes(prefix);
return prefixMatch(prefixBytes, currentKey);
});
if (accept) {
keysHinted++;
return true;
} else if (atleastOnePositiveMatch) {
return false;
}
accept = !negativePrefixList.isEmpty() && negativePrefixList.stream()
.allMatch(prefix -> {
byte[] prefixBytes = DFSUtil.string2Bytes(prefix);
return !prefixMatch(prefixBytes, currentKey);
});
if (accept) {
keysHinted++;
return true;
}
return false;
} }
@Override @Override
@ -116,7 +188,7 @@ public final class MetadataKeyFilters {
return keysHinted; return keysHinted;
} }
private boolean prefixMatch(byte[] prefix, byte[] key) { private static boolean prefixMatch(byte[] prefix, byte[] key) {
Preconditions.checkNotNull(prefix); Preconditions.checkNotNull(prefix);
Preconditions.checkNotNull(key); Preconditions.checkNotNull(key);
if (key.length < prefix.length) { if (key.length < prefix.length) {

View File

@ -146,6 +146,7 @@ message SCMContainerInfo {
required uint64 numberOfKeys = 6; required uint64 numberOfKeys = 6;
optional int64 stateEnterTime = 7; optional int64 stateEnterTime = 7;
required string owner = 8; required string owner = 8;
optional int64 deleteTransactionId = 9;
} }
message GetScmInfoRequestProto { message GetScmInfoRequestProto {

View File

@ -275,7 +275,7 @@ public class TestMetadataStore {
// Filter keys by prefix. // Filter keys by prefix.
// It should returns all "b*" entries. // It should returns all "b*" entries.
MetadataKeyFilter filter1 = new KeyPrefixFilter("b"); MetadataKeyFilter filter1 = new KeyPrefixFilter().addFilter("b");
result = store.getRangeKVs(null, 100, filter1); result = store.getRangeKVs(null, 100, filter1);
Assert.assertEquals(10, result.size()); Assert.assertEquals(10, result.size());
Assert.assertTrue(result.stream().allMatch(entry -> Assert.assertTrue(result.stream().allMatch(entry ->
@ -422,4 +422,63 @@ public class TestMetadataStore {
Assert.assertEquals(8, count.get()); Assert.assertEquals(8, count.get());
} }
@Test
public void testKeyPrefixFilter() throws IOException {
List<Map.Entry<byte[], byte[]>> result = null;
RuntimeException exception = null;
try {
new KeyPrefixFilter().addFilter("b0", true).addFilter("b");
} catch (IllegalArgumentException e) {
exception = e;
}
Assert.assertTrue(
exception.getMessage().contains("KeyPrefix: b already rejected"));
try {
new KeyPrefixFilter().addFilter("b0").addFilter("b", true);
} catch (IllegalArgumentException e) {
exception = e;
}
Assert.assertTrue(
exception.getMessage().contains("KeyPrefix: b already accepted"));
try {
new KeyPrefixFilter().addFilter("b", true).addFilter("b0");
} catch (IllegalArgumentException e) {
exception = e;
}
Assert.assertTrue(
exception.getMessage().contains("KeyPrefix: b0 already rejected"));
try {
new KeyPrefixFilter().addFilter("b").addFilter("b0", true);
} catch (IllegalArgumentException e) {
exception = e;
}
Assert.assertTrue(
exception.getMessage().contains("KeyPrefix: b0 already accepted"));
MetadataKeyFilter filter1 = new KeyPrefixFilter(true)
.addFilter("a0")
.addFilter("a1")
.addFilter("b", true);
result = store.getRangeKVs(null, 100, filter1);
Assert.assertEquals(2, result.size());
Assert.assertTrue(result.stream()
.anyMatch(entry -> new String(entry.getKey()).startsWith("a0"))
&& result.stream()
.anyMatch(entry -> new String(entry.getKey()).startsWith("a1")));
filter1 = new KeyPrefixFilter(true).addFilter("b", true);
result = store.getRangeKVs(null, 100, filter1);
Assert.assertEquals(0, result.size());
filter1 = new KeyPrefixFilter().addFilter("b", true);
result = store.getRangeKVs(null, 100, filter1);
Assert.assertEquals(10, result.size());
Assert.assertTrue(result.stream()
.allMatch(entry -> new String(entry.getKey()).startsWith("a")));
}
} }

View File

@ -33,6 +33,8 @@ import java.util.Map;
import java.util.TreeMap; import java.util.TreeMap;
import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLong;
import static java.lang.Math.max;
/** /**
* This class maintains the information about a container in the ozone world. * This class maintains the information about a container in the ozone world.
* <p> * <p>
@ -57,6 +59,7 @@ public class ContainerData {
* Number of pending deletion blocks in container. * Number of pending deletion blocks in container.
*/ */
private int numPendingDeletionBlocks; private int numPendingDeletionBlocks;
private long deleteTransactionId;
private AtomicLong readBytes; private AtomicLong readBytes;
private AtomicLong writeBytes; private AtomicLong writeBytes;
private AtomicLong readCount; private AtomicLong readCount;
@ -78,6 +81,7 @@ public class ContainerData {
this.containerID = containerID; this.containerID = containerID;
this.state = ContainerLifeCycleState.OPEN; this.state = ContainerLifeCycleState.OPEN;
this.numPendingDeletionBlocks = 0; this.numPendingDeletionBlocks = 0;
this.deleteTransactionId = 0;
this.readCount = new AtomicLong(0L); this.readCount = new AtomicLong(0L);
this.readBytes = new AtomicLong(0L); this.readBytes = new AtomicLong(0L);
this.writeCount = new AtomicLong(0L); this.writeCount = new AtomicLong(0L);
@ -101,6 +105,7 @@ public class ContainerData {
this.containerID = containerID; this.containerID = containerID;
this.state = state; this.state = state;
this.numPendingDeletionBlocks = 0; this.numPendingDeletionBlocks = 0;
this.deleteTransactionId = 0;
this.readCount = new AtomicLong(0L); this.readCount = new AtomicLong(0L);
this.readBytes = new AtomicLong(0L); this.readBytes = new AtomicLong(0L);
this.writeCount = new AtomicLong(0L); this.writeCount = new AtomicLong(0L);
@ -425,6 +430,22 @@ public class ContainerData {
return this.numPendingDeletionBlocks; return this.numPendingDeletionBlocks;
} }
/**
* Sets deleteTransactionId to latest delete transactionId for the container.
*
* @param transactionId latest transactionId of the container.
*/
public void updateDeleteTransactionId(long transactionId) {
deleteTransactionId = max(transactionId, deleteTransactionId);
}
/**
* Return the latest deleteTransactionId of the container.
*/
public long getDeleteTransactionId() {
return deleteTransactionId;
}
/** /**
* Get the number of bytes read from the container. * Get the number of bytes read from the container.
* @return the number of bytes read from the container. * @return the number of bytes read from the container.

View File

@ -20,6 +20,8 @@ package org.apache.hadoop.ozone.container.common.helpers;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerInfo; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerInfo;
import static java.lang.Math.max;
/** /**
* Container Report iterates the closed containers and sends a container report * Container Report iterates the closed containers and sends a container report
* to SCM. * to SCM.
@ -35,6 +37,7 @@ public class ContainerReport {
private long readBytes; private long readBytes;
private long writeBytes; private long writeBytes;
private long containerID; private long containerID;
private long deleteTransactionId;
public long getContainerID() { public long getContainerID() {
return containerID; return containerID;
@ -63,6 +66,7 @@ public class ContainerReport {
this.readBytes = 0L; this.readBytes = 0L;
this.writeCount = 0L; this.writeCount = 0L;
this.writeBytes = 0L; this.writeBytes = 0L;
this.deleteTransactionId = 0;
} }
/** /**
@ -96,6 +100,9 @@ public class ContainerReport {
if (info.hasWriteBytes()) { if (info.hasWriteBytes()) {
report.setWriteBytes(info.getWriteBytes()); report.setWriteBytes(info.getWriteBytes());
} }
if (info.hasDeleteTransactionId()) {
report.updateDeleteTransactionId(info.getDeleteTransactionId());
}
report.setContainerID(info.getContainerID()); report.setContainerID(info.getContainerID());
return report; return report;
@ -186,6 +193,10 @@ public class ContainerReport {
this.bytesUsed = bytesUsed; this.bytesUsed = bytesUsed;
} }
public void updateDeleteTransactionId(long transactionId) {
this.deleteTransactionId = max(transactionId, deleteTransactionId);
}
/** /**
* Gets a containerInfo protobuf message from ContainerReports. * Gets a containerInfo protobuf message from ContainerReports.
* *
@ -202,6 +213,7 @@ public class ContainerReport {
.setWriteBytes(this.getWriteBytes()) .setWriteBytes(this.getWriteBytes())
.setFinalhash(this.getFinalhash()) .setFinalhash(this.getFinalhash())
.setContainerID(this.getContainerID()) .setContainerID(this.getContainerID())
.setDeleteTransactionId(this.deleteTransactionId)
.build(); .build();
} }
} }

View File

@ -20,11 +20,13 @@ package org.apache.hadoop.ozone.container.common.impl;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.primitives.Longs;
import org.apache.commons.io.FileUtils; import org.apache.commons.io.FileUtils;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.hdds.scm.container.common.helpers import org.apache.hadoop.hdds.scm.container.common.helpers
.StorageContainerException; .StorageContainerException;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.server.datanode.StorageLocation; import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
@ -246,12 +248,18 @@ public class ContainerManagerImpl implements ContainerManager {
} }
containerData = ContainerData.getFromProtBuf(containerDataProto, conf); containerData = ContainerData.getFromProtBuf(containerDataProto, conf);
// Initialize pending deletion blocks count in in-memory // Initialize pending deletion blocks and deleted blocks count in
// container status. // in-memory containerData.
MetadataStore metadata = KeyUtils.getDB(containerData, conf); MetadataStore metadata = KeyUtils.getDB(containerData, conf);
List<Map.Entry<byte[], byte[]>> underDeletionBlocks = metadata List<Map.Entry<byte[], byte[]>> underDeletionBlocks = metadata
.getSequentialRangeKVs(null, Integer.MAX_VALUE, .getSequentialRangeKVs(null, Integer.MAX_VALUE,
MetadataKeyFilters.getDeletingKeyFilter()); MetadataKeyFilters.getDeletingKeyFilter());
byte[] transactionID = metadata.get(DFSUtil.string2Bytes(
OzoneConsts.DELETE_TRANSACTION_KEY_PREFIX + containerID));
if (transactionID != null) {
containerData
.updateDeleteTransactionId(Longs.fromByteArray(transactionID));
}
containerData.incrPendingDeletionBlocks(underDeletionBlocks.size()); containerData.incrPendingDeletionBlocks(underDeletionBlocks.size());
List<Map.Entry<byte[], byte[]>> liveKeys = metadata List<Map.Entry<byte[], byte[]>> liveKeys = metadata
@ -908,7 +916,8 @@ public class ContainerManagerImpl implements ContainerManager {
.setWriteCount(container.getWriteCount()) .setWriteCount(container.getWriteCount())
.setReadBytes(container.getReadBytes()) .setReadBytes(container.getReadBytes())
.setWriteBytes(container.getWriteBytes()) .setWriteBytes(container.getWriteBytes())
.setState(getState(containerId)); .setState(getState(containerId))
.setDeleteTransactionId(container.getDeleteTransactionId());
crBuilder.addReports(ciBuilder.build()); crBuilder.addReports(ciBuilder.build());
} }

View File

@ -175,8 +175,8 @@ public class BlockDeletingService extends BackgroundService{
// Scan container's db and get list of under deletion blocks // Scan container's db and get list of under deletion blocks
MetadataStore meta = KeyUtils.getDB(containerData, conf); MetadataStore meta = KeyUtils.getDB(containerData, conf);
// # of blocks to delete is throttled // # of blocks to delete is throttled
KeyPrefixFilter filter = new KeyPrefixFilter( KeyPrefixFilter filter =
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.getSequentialRangeKVs(null, blockLimitPerTask, filter);
if (toDeleteBlocks.isEmpty()) { if (toDeleteBlocks.isEmpty()) {
@ -214,10 +214,16 @@ public class BlockDeletingService extends BackgroundService{
} }
}); });
// Once files are deleted ... clean up DB // Once files are deleted... replace deleting entries with deleted entries
BatchOperation batch = new BatchOperation(); BatchOperation batch = new BatchOperation();
succeedBlocks.forEach(entry -> succeedBlocks.forEach(entry -> {
batch.delete(DFSUtil.string2Bytes(entry))); String blockId =
entry.substring(OzoneConsts.DELETING_KEY_PREFIX.length());
String deletedEntry = OzoneConsts.DELETED_KEY_PREFIX + blockId;
batch.put(DFSUtil.string2Bytes(deletedEntry),
DFSUtil.string2Bytes(blockId));
batch.delete(DFSUtil.string2Bytes(entry));
});
meta.writeBatch(batch); meta.writeBatch(batch);
// update count of pending deletion blocks in in-memory container status // update count of pending deletion blocks in in-memory container status
containerManager.decrPendingDeletionBlocks(succeedBlocks.size(), containerManager.decrPendingDeletionBlocks(succeedBlocks.size(),

View File

@ -186,6 +186,9 @@ public class DeleteBlocksCommandHandler implements CommandHandler {
LOG.debug("Block {} not found or already under deletion in" LOG.debug("Block {} not found or already under deletion in"
+ " container {}, skip deleting it.", blk, containerId); + " container {}, skip deleting it.", blk, containerId);
} }
containerDB.put(DFSUtil.string2Bytes(
OzoneConsts.DELETE_TRANSACTION_KEY_PREFIX + delTX.getContainerID()),
Longs.toByteArray(delTX.getTxID()));
} }
// update pending deletion blocks count in in-memory container status // update pending deletion blocks count in in-memory container status

View File

@ -160,6 +160,7 @@ message ContainerInfo {
optional int64 writeBytes = 8; optional int64 writeBytes = 8;
optional string finalhash = 9; optional string finalhash = 9;
optional hadoop.hdds.LifeCycleState state = 10; optional hadoop.hdds.LifeCycleState state = 10;
optional int64 deleteTransactionId = 11;
} }
/* /*

View File

@ -361,13 +361,10 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
} }
} }
// We update SCM DB first, so if this step fails, we end up here,
// nothing gets into the delLog so no blocks will be accidentally
// removed. If we write the log first, once log is written, the
// async deleting service will start to scan and might be picking
// up some blocks to do real deletions, that might cause data loss.
try { try {
deletedBlockLog.addTransactions(containerBlocks); Map<Long, Long> deleteTransactionsMap =
deletedBlockLog.addTransactions(containerBlocks);
containerManager.updateDeleteTransactionId(deleteTransactionsMap);
} catch (IOException e) { } catch (IOException e) {
throw new IOException( throw new IOException(
"Skip writing the deleted blocks info to" "Skip writing the deleted blocks info to"

View File

@ -108,9 +108,10 @@ public interface DeletedBlockLog extends Closeable {
* number of containers) together (on success) or non (on failure). * number of containers) together (on success) or non (on failure).
* *
* @param containerBlocksMap a map of containerBlocks. * @param containerBlocksMap a map of containerBlocks.
* @return Mapping from containerId to latest transactionId for the container.
* @throws IOException * @throws IOException
*/ */
void addTransactions(Map<Long, List<Long>> containerBlocksMap) Map<Long, Long> addTransactions(Map<Long, List<Long>> containerBlocksMap)
throws IOException; throws IOException;
/** /**

View File

@ -36,6 +36,7 @@ import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
@ -306,12 +307,15 @@ public class DeletedBlockLogImpl implements DeletedBlockLog {
* {@inheritDoc} * {@inheritDoc}
* *
* @param containerBlocksMap a map of containerBlocks. * @param containerBlocksMap a map of containerBlocks.
* @return Mapping from containerId to latest transactionId for the container.
* @throws IOException * @throws IOException
*/ */
@Override @Override
public void addTransactions(Map<Long, List<Long>> containerBlocksMap) public Map<Long, Long> addTransactions(
Map<Long, List<Long>> containerBlocksMap)
throws IOException { throws IOException {
BatchOperation batch = new BatchOperation(); BatchOperation batch = new BatchOperation();
Map<Long, Long> deleteTransactionsMap = new HashMap<>();
lock.lock(); lock.lock();
try { try {
long currentLatestID = lastTxID; long currentLatestID = lastTxID;
@ -321,11 +325,13 @@ public class DeletedBlockLogImpl implements DeletedBlockLog {
byte[] key = Longs.toByteArray(currentLatestID); byte[] key = Longs.toByteArray(currentLatestID);
DeletedBlocksTransaction tx = constructNewTransaction(currentLatestID, DeletedBlocksTransaction tx = constructNewTransaction(currentLatestID,
entry.getKey(), entry.getValue()); entry.getKey(), entry.getValue());
deleteTransactionsMap.put(entry.getKey(), currentLatestID);
batch.put(key, tx.toByteArray()); batch.put(key, tx.toByteArray());
} }
lastTxID = currentLatestID; lastTxID = currentLatestID;
batch.put(LATEST_TXID, Longs.toByteArray(lastTxID)); batch.put(LATEST_TXID, Longs.toByteArray(lastTxID));
deletedStore.writeBatch(batch); deletedStore.writeBatch(batch);
return deleteTransactionsMap;
} finally { } finally {
lock.unlock(); lock.unlock();
} }

View File

@ -341,6 +341,39 @@ public class ContainerMapping implements Mapping {
} }
} }
/**
* Update deleteTransactionId according to deleteTransactionMap.
*
* @param deleteTransactionMap Maps the containerId to latest delete
* transaction id for the container.
* @throws IOException
*/
public void updateDeleteTransactionId(Map<Long, Long> deleteTransactionMap)
throws IOException {
lock.lock();
try {
for (Map.Entry<Long, Long> entry : deleteTransactionMap.entrySet()) {
long containerID = entry.getKey();
byte[] dbKey = Longs.toByteArray(containerID);
byte[] containerBytes = containerStore.get(dbKey);
if (containerBytes == null) {
throw new SCMException(
"Failed to increment number of deleted blocks for container "
+ containerID + ", reason : " + "container doesn't exist.",
SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER);
}
ContainerInfo containerInfo = ContainerInfo.fromProtobuf(
HddsProtos.SCMContainerInfo.parseFrom(containerBytes));
containerInfo.updateDeleteTransactionId(entry.getValue());
containerStore.put(dbKey, containerInfo.getProtobuf().toByteArray());
containerStateManager
.updateDeleteTransactionId(containerID, entry.getValue());
}
} finally {
lock.unlock();
}
}
/** /**
* Returns the container State Manager. * Returns the container State Manager.
* *
@ -441,6 +474,7 @@ public class ContainerMapping implements Mapping {
builder.setState(knownState.getState()); builder.setState(knownState.getState());
builder.setStateEnterTime(knownState.getStateEnterTime()); builder.setStateEnterTime(knownState.getStateEnterTime());
builder.setContainerID(knownState.getContainerID()); builder.setContainerID(knownState.getContainerID());
builder.setDeleteTransactionId(knownState.getDeleteTransactionId());
if (knownState.getOwner() != null) { if (knownState.getOwner() != null) {
builder.setOwner(knownState.getOwner()); builder.setOwner(knownState.getOwner());
} }
@ -571,6 +605,7 @@ public class ContainerMapping implements Mapping {
.setPipeline(oldInfo.getPipeline()) .setPipeline(oldInfo.getPipeline())
.setState(oldInfo.getState()) .setState(oldInfo.getState())
.setUsedBytes(oldInfo.getUsedBytes()) .setUsedBytes(oldInfo.getUsedBytes())
.setDeleteTransactionId(oldInfo.getDeleteTransactionId())
.build(); .build();
containerStore.put(dbKey, newInfo.getProtobuf().toByteArray()); containerStore.put(dbKey, newInfo.getProtobuf().toByteArray());
} else { } else {

View File

@ -304,6 +304,7 @@ public class ContainerStateManager implements Closeable {
.setStateEnterTime(Time.monotonicNow()) .setStateEnterTime(Time.monotonicNow())
.setOwner(owner) .setOwner(owner)
.setContainerID(containerCount.incrementAndGet()) .setContainerID(containerCount.incrementAndGet())
.setDeleteTransactionId(0)
.build(); .build();
Preconditions.checkNotNull(containerInfo); Preconditions.checkNotNull(containerInfo);
containers.addContainer(containerInfo); containers.addContainer(containerInfo);
@ -351,6 +352,17 @@ public class ContainerStateManager implements Closeable {
return containers.getContainerInfo(info); return containers.getContainerInfo(info);
} }
/**
* Update deleteTransactionId for a container.
*
* @param containerID ContainerID of the container whose delete
* transactionId needs to be updated.
* @param transactionId latest transactionId to be updated for the container
*/
public void updateDeleteTransactionId(Long containerID, long transactionId) {
containers.getContainerMap().get(ContainerID.valueof(containerID))
.updateDeleteTransactionId(transactionId);
}
/** /**
* Return a container matching the attributes specified. * Return a container matching the attributes specified.

View File

@ -26,6 +26,7 @@ import org.apache.hadoop.hdds.scm.node.NodeManager;
import java.io.Closeable; import java.io.Closeable;
import java.io.IOException; import java.io.IOException;
import java.util.List; import java.util.List;
import java.util.Map;
/** /**
* Mapping class contains the mapping from a name to a pipeline mapping. This is * Mapping class contains the mapping from a name to a pipeline mapping. This is
@ -104,6 +105,16 @@ public interface Mapping extends Closeable {
ContainerReportsProto reports) ContainerReportsProto reports)
throws IOException; throws IOException;
/**
* Update deleteTransactionId according to deleteTransactionMap.
*
* @param deleteTransactionMap Maps the containerId to latest delete
* transaction id for the container.
* @throws IOException
*/
void updateDeleteTransactionId(Map<Long, Long> deleteTransactionMap)
throws IOException;
/** /**
* Returns the nodeManager. * Returns the nodeManager.
* @return NodeManager * @return NodeManager

View File

@ -109,8 +109,24 @@ public class TestBlockManager {
AllocatedBlock block = blockManager.allocateBlock(DEFAULT_BLOCK_SIZE, AllocatedBlock block = blockManager.allocateBlock(DEFAULT_BLOCK_SIZE,
type, factor, containerOwner); type, factor, containerOwner);
Assert.assertNotNull(block); Assert.assertNotNull(block);
long transactionId =
mapping.getContainer(block.getBlockID().getContainerID())
.getDeleteTransactionId();
Assert.assertEquals(0, transactionId);
blockManager.deleteBlocks(Collections.singletonList( blockManager.deleteBlocks(Collections.singletonList(
block.getBlockID())); block.getBlockID()));
Assert.assertEquals(++transactionId,
mapping.getContainer(block.getBlockID().getContainerID())
.getDeleteTransactionId());
block = blockManager.allocateBlock(DEFAULT_BLOCK_SIZE,
type, factor, containerOwner);
Assert.assertNotNull(block);
blockManager.deleteBlocks(Collections.singletonList(
block.getBlockID()));
Assert.assertEquals(++transactionId,
mapping.getContainer(block.getBlockID().getContainerID())
.getDeleteTransactionId());
} }
@Test @Test

View File

@ -203,7 +203,8 @@ public class TestContainerMapping {
.setWriteCount(100000000L) .setWriteCount(100000000L)
.setReadBytes(2000000000L) .setReadBytes(2000000000L)
.setWriteBytes(2000000000L) .setWriteBytes(2000000000L)
.setContainerID(info.getContainerID()); .setContainerID(info.getContainerID())
.setDeleteTransactionId(0);
reports.add(ciBuilder.build()); reports.add(ciBuilder.build());
@ -237,7 +238,8 @@ public class TestContainerMapping {
.setWriteCount(500000000L) .setWriteCount(500000000L)
.setReadBytes(5368705120L) .setReadBytes(5368705120L)
.setWriteBytes(5368705120L) .setWriteBytes(5368705120L)
.setContainerID(info.getContainerID()); .setContainerID(info.getContainerID())
.setDeleteTransactionId(0);
reports.add(ciBuilder.build()); reports.add(ciBuilder.build());

View File

@ -212,7 +212,8 @@ public class TestContainerCloser {
.setReadCount(100000000L) .setReadCount(100000000L)
.setWriteCount(100000000L) .setWriteCount(100000000L)
.setReadBytes(2000000000L) .setReadBytes(2000000000L)
.setWriteBytes(2000000000L); .setWriteBytes(2000000000L)
.setDeleteTransactionId(0);
reports.addReports(ciBuilder); reports.addReports(ciBuilder);
mapping.processContainerReports(TestUtils.getDatanodeDetails(), mapping.processContainerReports(TestUtils.getDatanodeDetails(),
reports.build()); reports.build());

View File

@ -38,6 +38,7 @@ import org.apache.hadoop.ozone.web.handlers.UserArgs;
import org.apache.hadoop.ozone.web.handlers.VolumeArgs; import org.apache.hadoop.ozone.web.handlers.VolumeArgs;
import org.apache.hadoop.ozone.web.interfaces.StorageHandler; import org.apache.hadoop.ozone.web.interfaces.StorageHandler;
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.KeyPrefixFilter; import org.apache.hadoop.utils.MetadataKeyFilters.KeyPrefixFilter;
import org.apache.hadoop.utils.MetadataKeyFilters.MetadataKeyFilter; import org.apache.hadoop.utils.MetadataKeyFilters.MetadataKeyFilter;
import org.apache.hadoop.utils.MetadataStore; import org.apache.hadoop.utils.MetadataStore;
@ -120,7 +121,7 @@ public class TestStorageContainerManagerHelper {
List<String> pendingDeletionBlocks = Lists.newArrayList(); List<String> pendingDeletionBlocks = Lists.newArrayList();
MetadataStore meta = getContainerMetadata(containerID); MetadataStore meta = getContainerMetadata(containerID);
KeyPrefixFilter filter = KeyPrefixFilter filter =
new KeyPrefixFilter(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
.getRangeKVs(null, Integer.MAX_VALUE, filter); .getRangeKVs(null, Integer.MAX_VALUE, filter);
kvs.forEach(entry -> { kvs.forEach(entry -> {
@ -147,7 +148,8 @@ public class TestStorageContainerManagerHelper {
(preKey, currentKey, nextKey) -> !DFSUtil.bytes2String(currentKey) (preKey, currentKey, nextKey) -> !DFSUtil.bytes2String(currentKey)
.startsWith(OzoneConsts.DELETING_KEY_PREFIX); .startsWith(OzoneConsts.DELETING_KEY_PREFIX);
List<Map.Entry<byte[], byte[]>> kvs = List<Map.Entry<byte[], byte[]>> kvs =
meta.getRangeKVs(null, Integer.MAX_VALUE, filter); meta.getRangeKVs(null, Integer.MAX_VALUE,
MetadataKeyFilters.getNormalKeyFilter());
kvs.forEach(entry -> { kvs.forEach(entry -> {
allBlocks.add(Longs.fromByteArray(entry.getKey())); allBlocks.add(Longs.fromByteArray(entry.getKey()));
}); });

View File

@ -66,8 +66,6 @@ import static org.apache.hadoop.ozone.OzoneConfigKeys
.OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER; .OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER;
import static org.apache.hadoop.ozone.OzoneConfigKeys import static org.apache.hadoop.ozone.OzoneConfigKeys
.OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL; .OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL;
import static org.apache.hadoop.ozone.container
.ContainerTestHelper.createSingleNodePipeline;
/** /**
* Tests to test block deleting service. * Tests to test block deleting service.
@ -183,8 +181,15 @@ public class TestBlockDeletingService {
private int getUnderDeletionBlocksCount(MetadataStore meta) private int getUnderDeletionBlocksCount(MetadataStore 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.getRangeKVs(null, 100, new MetadataKeyFilters.KeyPrefixFilter()
OzoneConsts.DELETING_KEY_PREFIX)); .addFilter(OzoneConsts.DELETING_KEY_PREFIX));
return underDeletionBlocks.size();
}
private int getDeletedBlocksCount(MetadataStore db) throws IOException {
List<Map.Entry<byte[], byte[]>> underDeletionBlocks =
db.getRangeKVs(null, 100, new MetadataKeyFilters.KeyPrefixFilter()
.addFilter(OzoneConsts.DELETED_KEY_PREFIX));
return underDeletionBlocks.size(); return underDeletionBlocks.size();
} }
@ -205,20 +210,34 @@ public class TestBlockDeletingService {
List<ContainerData> containerData = Lists.newArrayList(); List<ContainerData> containerData = Lists.newArrayList();
containerManager.listContainer(0L, 1, containerData); containerManager.listContainer(0L, 1, containerData);
Assert.assertEquals(1, containerData.size()); Assert.assertEquals(1, containerData.size());
MetadataStore meta = KeyUtils.getDB(containerData.get(0), conf);
// Ensure there is 100 blocks under deletion MetadataStore meta = KeyUtils.getDB(containerData.get(0), conf);
Map<Long, ContainerData> containerMap =
((ContainerManagerImpl) containerManager).getContainerMap();
long transactionId =
containerMap.get(containerData.get(0).getContainerID())
.getDeleteTransactionId();
// Number of deleted blocks in container should be equal to 0 before
// block delete
Assert.assertEquals(0, transactionId);
// Ensure there are 3 blocks under deletion and 0 deleted blocks
Assert.assertEquals(3, getUnderDeletionBlocksCount(meta)); Assert.assertEquals(3, getUnderDeletionBlocksCount(meta));
Assert.assertEquals(0, getDeletedBlocksCount(meta));
// An interval will delete 1 * 2 blocks // An interval will delete 1 * 2 blocks
deleteAndWait(svc, 1); deleteAndWait(svc, 1);
Assert.assertEquals(1, getUnderDeletionBlocksCount(meta)); Assert.assertEquals(1, getUnderDeletionBlocksCount(meta));
Assert.assertEquals(2, getDeletedBlocksCount(meta));
deleteAndWait(svc, 2); deleteAndWait(svc, 2);
Assert.assertEquals(0, getUnderDeletionBlocksCount(meta)); Assert.assertEquals(0, getUnderDeletionBlocksCount(meta));
Assert.assertEquals(3, getDeletedBlocksCount(meta));
deleteAndWait(svc, 3); deleteAndWait(svc, 3);
Assert.assertEquals(0, getUnderDeletionBlocksCount(meta)); Assert.assertEquals(0, getUnderDeletionBlocksCount(meta));
Assert.assertEquals(3, getDeletedBlocksCount(meta));
svc.shutdown(); svc.shutdown();
shutdownContainerMangaer(containerManager); shutdownContainerMangaer(containerManager);

View File

@ -638,7 +638,8 @@ public class TestKeySpaceManager {
MetadataStore store = cluster.getKeySpaceManager(). MetadataStore store = cluster.getKeySpaceManager().
getMetadataManager().getStore(); getMetadataManager().getStore();
List<Map.Entry<byte[], byte[]>> list = store.getRangeKVs(null, 10, List<Map.Entry<byte[], byte[]>> list = store.getRangeKVs(null, 10,
new MetadataKeyFilters.KeyPrefixFilter(DELETING_KEY_PREFIX)); new MetadataKeyFilters.KeyPrefixFilter()
.addFilter(DELETING_KEY_PREFIX));
Assert.assertEquals(1, list.size()); Assert.assertEquals(1, list.size());
// Delete the key again to test deleting non-existing key. // Delete the key again to test deleting non-existing key.

View File

@ -352,8 +352,8 @@ public class KSMMetadataManagerImpl implements KSMMetadataManager {
ResultCodes.FAILED_BUCKET_NOT_FOUND); ResultCodes.FAILED_BUCKET_NOT_FOUND);
} }
MetadataKeyFilter filter = new KeyPrefixFilter( MetadataKeyFilter filter = new KeyPrefixFilter()
getKeyWithDBPrefix(volumeName, bucketName, keyPrefix)); .addFilter(getKeyWithDBPrefix(volumeName, bucketName, keyPrefix));
List<Map.Entry<byte[], byte[]>> rangeResult; List<Map.Entry<byte[], byte[]>> rangeResult;
if (!Strings.isNullOrEmpty(startKey)) { if (!Strings.isNullOrEmpty(startKey)) {
@ -449,7 +449,8 @@ public class KSMMetadataManagerImpl implements KSMMetadataManager {
private VolumeList getAllVolumes() throws IOException { private VolumeList getAllVolumes() throws IOException {
// Scan all users in database // Scan all users in database
KeyPrefixFilter filter = new KeyPrefixFilter(OzoneConsts.KSM_USER_PREFIX); KeyPrefixFilter filter =
new KeyPrefixFilter().addFilter(OzoneConsts.KSM_USER_PREFIX);
// We are not expecting a huge number of users per cluster, // We are not expecting a huge number of users per cluster,
// it should be fine to scan all users in db and return us a // it should be fine to scan all users in db and return us a
// list of volume names in string per user. // list of volume names in string per user.
@ -497,7 +498,7 @@ public class KSMMetadataManagerImpl implements KSMMetadataManager {
List<BlockGroup> keyBlocksList = Lists.newArrayList(); List<BlockGroup> keyBlocksList = Lists.newArrayList();
long now = Time.now(); long now = Time.now();
final MetadataKeyFilter openKeyFilter = final MetadataKeyFilter openKeyFilter =
new KeyPrefixFilter(OPEN_KEY_PREFIX); new KeyPrefixFilter().addFilter(OPEN_KEY_PREFIX);
List<Map.Entry<byte[], byte[]>> rangeResult = List<Map.Entry<byte[], byte[]>> rangeResult =
store.getSequentialRangeKVs(null, Integer.MAX_VALUE, store.getSequentialRangeKVs(null, Integer.MAX_VALUE,
openKeyFilter); openKeyFilter);

View File

@ -64,9 +64,14 @@ public class BenchMarkContainerStateMap {
.setPipeline(pipeline) .setPipeline(pipeline)
// This is bytes allocated for blocks inside container, not the // This is bytes allocated for blocks inside container, not the
// container size // container size
.setAllocatedBytes(0).setUsedBytes(0).setNumberOfKeys(0) .setAllocatedBytes(0)
.setStateEnterTime(Time.monotonicNow()).setOwner("OZONE") .setUsedBytes(0)
.setContainerID(x).build(); .setNumberOfKeys(0)
.setStateEnterTime(Time.monotonicNow())
.setOwner("OZONE")
.setContainerID(x)
.setDeleteTransactionId(0)
.build();
stateMap.addContainer(containerInfo); stateMap.addContainer(containerInfo);
currentCount++; currentCount++;
} catch (SCMException e) { } catch (SCMException e) {
@ -80,9 +85,14 @@ public class BenchMarkContainerStateMap {
.setPipeline(pipeline) .setPipeline(pipeline)
// This is bytes allocated for blocks inside container, not the // This is bytes allocated for blocks inside container, not the
// container size // container size
.setAllocatedBytes(0).setUsedBytes(0).setNumberOfKeys(0) .setAllocatedBytes(0)
.setStateEnterTime(Time.monotonicNow()).setOwner("OZONE") .setUsedBytes(0)
.setContainerID(y).build(); .setNumberOfKeys(0)
.setStateEnterTime(Time.monotonicNow())
.setOwner("OZONE")
.setContainerID(y)
.setDeleteTransactionId(0)
.build();
stateMap.addContainer(containerInfo); stateMap.addContainer(containerInfo);
currentCount++; currentCount++;
} catch (SCMException e) { } catch (SCMException e) {
@ -95,9 +105,14 @@ public class BenchMarkContainerStateMap {
.setPipeline(pipeline) .setPipeline(pipeline)
// This is bytes allocated for blocks inside container, not the // This is bytes allocated for blocks inside container, not the
// container size // container size
.setAllocatedBytes(0).setUsedBytes(0).setNumberOfKeys(0) .setAllocatedBytes(0)
.setStateEnterTime(Time.monotonicNow()).setOwner("OZONE") .setUsedBytes(0)
.setContainerID(currentCount++).build(); .setNumberOfKeys(0)
.setStateEnterTime(Time.monotonicNow())
.setOwner("OZONE")
.setContainerID(currentCount++)
.setDeleteTransactionId(0)
.build();
stateMap.addContainer(containerInfo); stateMap.addContainer(containerInfo);
} catch (SCMException e) { } catch (SCMException e) {
e.printStackTrace(); e.printStackTrace();
@ -155,9 +170,14 @@ public class BenchMarkContainerStateMap {
.setPipeline(pipeline) .setPipeline(pipeline)
// This is bytes allocated for blocks inside container, not the // This is bytes allocated for blocks inside container, not the
// container size // container size
.setAllocatedBytes(0).setUsedBytes(0).setNumberOfKeys(0) .setAllocatedBytes(0)
.setStateEnterTime(Time.monotonicNow()).setOwner("OZONE") .setUsedBytes(0)
.setContainerID(cid).build(); .setNumberOfKeys(0)
.setStateEnterTime(Time.monotonicNow())
.setOwner("OZONE")
.setContainerID(cid)
.setDeleteTransactionId(0)
.build();
state.stateMap.addContainer(containerInfo); state.stateMap.addContainer(containerInfo);
} }