HDDS-308. SCM should identify a container with pending deletes using container reports. Contributed by Lokesh Jain.
This commit is contained in:
parent
3ac07b720b
commit
a8dae0047c
@ -215,7 +215,8 @@ public ContainerReportsProto getContainerReport() throws IOException {
|
|||||||
.setReadBytes(containerData.getReadBytes())
|
.setReadBytes(containerData.getReadBytes())
|
||||||
.setWriteBytes(containerData.getWriteBytes())
|
.setWriteBytes(containerData.getWriteBytes())
|
||||||
.setUsed(containerData.getBytesUsed())
|
.setUsed(containerData.getBytesUsed())
|
||||||
.setState(getState(containerData));
|
.setState(getState(containerData))
|
||||||
|
.setDeleteTransactionId(containerData.getDeleteTransactionId());
|
||||||
|
|
||||||
crBuilder.addReports(ciBuilder.build());
|
crBuilder.addReports(ciBuilder.build());
|
||||||
}
|
}
|
||||||
|
@ -372,9 +372,7 @@ public void deleteBlocks(List<BlockID> blockIDs) throws IOException {
|
|||||||
}
|
}
|
||||||
|
|
||||||
try {
|
try {
|
||||||
Map<Long, Long> deleteTransactionsMap =
|
deletedBlockLog.addTransactions(containerBlocks);
|
||||||
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"
|
||||||
|
@ -42,9 +42,10 @@ public interface DeletedBlockLog extends Closeable {
|
|||||||
* Once DatanodeDeletedBlockTransactions is full, the scan behavior will
|
* Once DatanodeDeletedBlockTransactions is full, the scan behavior will
|
||||||
* stop.
|
* stop.
|
||||||
* @param transactions a list of TXs will be set into.
|
* @param transactions a list of TXs will be set into.
|
||||||
|
* @return Mapping from containerId to latest transactionId for the container.
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
void getTransactions(DatanodeDeletedBlockTransactions transactions)
|
Map<Long, Long> getTransactions(DatanodeDeletedBlockTransactions transactions)
|
||||||
throws IOException;
|
throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -101,10 +102,9 @@ void addTransaction(long containerID, List<Long> blocks)
|
|||||||
* 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
|
||||||
*/
|
*/
|
||||||
Map<Long, Long> addTransactions(Map<Long, List<Long>> containerBlocksMap)
|
void addTransactions(Map<Long, List<Long>> containerBlocksMap)
|
||||||
throws IOException;
|
throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -26,6 +26,7 @@
|
|||||||
.StorageContainerDatanodeProtocolProtos.ContainerBlocksDeletionACKProto
|
.StorageContainerDatanodeProtocolProtos.ContainerBlocksDeletionACKProto
|
||||||
.DeleteBlockTransactionResult;
|
.DeleteBlockTransactionResult;
|
||||||
import org.apache.hadoop.hdds.scm.container.Mapping;
|
import org.apache.hadoop.hdds.scm.container.Mapping;
|
||||||
|
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
||||||
import org.apache.hadoop.hdfs.DFSUtil;
|
import org.apache.hadoop.hdfs.DFSUtil;
|
||||||
import org.apache.hadoop.hdds.protocol.proto
|
import org.apache.hadoop.hdds.protocol.proto
|
||||||
.StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction;
|
.StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction;
|
||||||
@ -45,13 +46,14 @@
|
|||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.concurrent.ConcurrentHashMap;
|
|
||||||
import java.util.UUID;
|
import java.util.UUID;
|
||||||
|
import java.util.concurrent.ConcurrentHashMap;
|
||||||
import java.util.concurrent.atomic.AtomicInteger;
|
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;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
import static java.lang.Math.min;
|
||||||
import static org.apache.hadoop.hdds.scm.ScmConfigKeys
|
import static org.apache.hadoop.hdds.scm.ScmConfigKeys
|
||||||
.OZONE_SCM_BLOCK_DELETION_MAX_RETRY;
|
.OZONE_SCM_BLOCK_DELETION_MAX_RETRY;
|
||||||
import static org.apache.hadoop.hdds.scm.ScmConfigKeys
|
import static org.apache.hadoop.hdds.scm.ScmConfigKeys
|
||||||
@ -239,21 +241,26 @@ public void commitTransactions(
|
|||||||
// set of dns which have successfully committed transaction txId.
|
// set of dns which have successfully committed transaction txId.
|
||||||
dnsWithCommittedTxn = transactionToDNsCommitMap.get(txID);
|
dnsWithCommittedTxn = transactionToDNsCommitMap.get(txID);
|
||||||
Long containerId = transactionResult.getContainerID();
|
Long containerId = transactionResult.getContainerID();
|
||||||
if (dnsWithCommittedTxn == null || containerId == null) {
|
if (dnsWithCommittedTxn == null) {
|
||||||
LOG.warn("Transaction txId={} commit by dnId={} failed."
|
LOG.warn("Transaction txId={} commit by dnId={} for containerID={} "
|
||||||
+ " Corresponding entry not found.", txID, dnID);
|
+ "failed. Corresponding entry not found.", txID, dnID,
|
||||||
|
containerId);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
dnsWithCommittedTxn.add(dnID);
|
dnsWithCommittedTxn.add(dnID);
|
||||||
Collection<DatanodeDetails> containerDnsDetails =
|
Pipeline pipeline =
|
||||||
containerManager.getContainerWithPipeline(containerId)
|
containerManager.getContainerWithPipeline(containerId)
|
||||||
.getPipeline().getDatanodes().values();
|
.getPipeline();
|
||||||
|
Collection<DatanodeDetails> containerDnsDetails =
|
||||||
|
pipeline.getDatanodes().values();
|
||||||
// The delete entry can be safely removed from the log if all the
|
// The delete entry can be safely removed from the log if all the
|
||||||
// corresponding nodes commit the txn.
|
// corresponding nodes commit the txn. It is required to check that
|
||||||
if (dnsWithCommittedTxn.size() >= containerDnsDetails.size()) {
|
// the nodes returned in the pipeline match the replication factor.
|
||||||
|
if (min(containerDnsDetails.size(), dnsWithCommittedTxn.size())
|
||||||
|
>= pipeline.getFactor().getNumber()) {
|
||||||
List<UUID> containerDns = containerDnsDetails.stream()
|
List<UUID> containerDns = containerDnsDetails.stream()
|
||||||
.map(dnDetails -> dnDetails.getUuid())
|
.map(DatanodeDetails::getUuid)
|
||||||
.collect(Collectors.toList());
|
.collect(Collectors.toList());
|
||||||
if (dnsWithCommittedTxn.containsAll(containerDns)) {
|
if (dnsWithCommittedTxn.containsAll(containerDns)) {
|
||||||
transactionToDNsCommitMap.remove(txID);
|
transactionToDNsCommitMap.remove(txID);
|
||||||
@ -338,15 +345,13 @@ public int getNumOfValidTransactions() throws IOException {
|
|||||||
* {@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 Map<Long, Long> addTransactions(
|
public void addTransactions(
|
||||||
Map<Long, List<Long>> containerBlocksMap)
|
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;
|
||||||
@ -356,13 +361,11 @@ public Map<Long, Long> addTransactions(
|
|||||||
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();
|
||||||
}
|
}
|
||||||
@ -376,10 +379,11 @@ public void close() throws IOException {
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void getTransactions(DatanodeDeletedBlockTransactions transactions)
|
public Map<Long, Long> getTransactions(
|
||||||
throws IOException {
|
DatanodeDeletedBlockTransactions transactions) throws IOException {
|
||||||
lock.lock();
|
lock.lock();
|
||||||
try {
|
try {
|
||||||
|
Map<Long, Long> deleteTransactionMap = new HashMap<>();
|
||||||
deletedStore.iterate(null, (key, value) -> {
|
deletedStore.iterate(null, (key, value) -> {
|
||||||
if (!Arrays.equals(LATEST_TXID, key)) {
|
if (!Arrays.equals(LATEST_TXID, key)) {
|
||||||
DeletedBlocksTransaction block = DeletedBlocksTransaction
|
DeletedBlocksTransaction block = DeletedBlocksTransaction
|
||||||
@ -388,6 +392,7 @@ public void getTransactions(DatanodeDeletedBlockTransactions transactions)
|
|||||||
if (block.getCount() > -1 && block.getCount() <= maxRetry) {
|
if (block.getCount() > -1 && block.getCount() <= maxRetry) {
|
||||||
if (transactions.addTransaction(block,
|
if (transactions.addTransaction(block,
|
||||||
transactionToDNsCommitMap.get(block.getTxID()))) {
|
transactionToDNsCommitMap.get(block.getTxID()))) {
|
||||||
|
deleteTransactionMap.put(block.getContainerID(), block.getTxID());
|
||||||
transactionToDNsCommitMap
|
transactionToDNsCommitMap
|
||||||
.putIfAbsent(block.getTxID(), new ConcurrentHashSet<>());
|
.putIfAbsent(block.getTxID(), new ConcurrentHashSet<>());
|
||||||
}
|
}
|
||||||
@ -396,6 +401,7 @@ public void getTransactions(DatanodeDeletedBlockTransactions transactions)
|
|||||||
}
|
}
|
||||||
return true;
|
return true;
|
||||||
});
|
});
|
||||||
|
return deleteTransactionMap;
|
||||||
} finally {
|
} finally {
|
||||||
lock.unlock();
|
lock.unlock();
|
||||||
}
|
}
|
||||||
|
@ -0,0 +1,38 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
* <p>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p>
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.hadoop.hdds.scm.block;
|
||||||
|
|
||||||
|
import org.apache.hadoop.hdds.server.events.EventHandler;
|
||||||
|
import org.apache.hadoop.hdds.server.events.EventPublisher;
|
||||||
|
|
||||||
|
public class PendingDeleteHandler implements
|
||||||
|
EventHandler<PendingDeleteStatusList> {
|
||||||
|
|
||||||
|
private SCMBlockDeletingService scmBlockDeletingService;
|
||||||
|
|
||||||
|
public PendingDeleteHandler(
|
||||||
|
SCMBlockDeletingService scmBlockDeletingService) {
|
||||||
|
this.scmBlockDeletingService = scmBlockDeletingService;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void onMessage(PendingDeleteStatusList pendingDeleteStatusList,
|
||||||
|
EventPublisher publisher) {
|
||||||
|
scmBlockDeletingService.handlePendingDeletes(pendingDeleteStatusList);
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,79 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
* <p>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p>
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.hadoop.hdds.scm.block;
|
||||||
|
|
||||||
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||||
|
|
||||||
|
import java.util.LinkedList;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
public class PendingDeleteStatusList {
|
||||||
|
|
||||||
|
private List<PendingDeleteStatus> pendingDeleteStatuses;
|
||||||
|
private DatanodeDetails datanodeDetails;
|
||||||
|
|
||||||
|
public PendingDeleteStatusList(DatanodeDetails datanodeDetails) {
|
||||||
|
this.datanodeDetails = datanodeDetails;
|
||||||
|
pendingDeleteStatuses = new LinkedList<>();
|
||||||
|
}
|
||||||
|
|
||||||
|
public void addPendingDeleteStatus(long dnDeleteTransactionId,
|
||||||
|
long scmDeleteTransactionId, long containerId) {
|
||||||
|
pendingDeleteStatuses.add(
|
||||||
|
new PendingDeleteStatus(dnDeleteTransactionId, scmDeleteTransactionId,
|
||||||
|
containerId));
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class PendingDeleteStatus {
|
||||||
|
private long dnDeleteTransactionId;
|
||||||
|
private long scmDeleteTransactionId;
|
||||||
|
private long containerId;
|
||||||
|
|
||||||
|
public PendingDeleteStatus(long dnDeleteTransactionId,
|
||||||
|
long scmDeleteTransactionId, long containerId) {
|
||||||
|
this.dnDeleteTransactionId = dnDeleteTransactionId;
|
||||||
|
this.scmDeleteTransactionId = scmDeleteTransactionId;
|
||||||
|
this.containerId = containerId;
|
||||||
|
}
|
||||||
|
|
||||||
|
public long getDnDeleteTransactionId() {
|
||||||
|
return dnDeleteTransactionId;
|
||||||
|
}
|
||||||
|
|
||||||
|
public long getScmDeleteTransactionId() {
|
||||||
|
return scmDeleteTransactionId;
|
||||||
|
}
|
||||||
|
|
||||||
|
public long getContainerId() {
|
||||||
|
return containerId;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
public List<PendingDeleteStatus> getPendingDeleteStatuses() {
|
||||||
|
return pendingDeleteStatuses;
|
||||||
|
}
|
||||||
|
|
||||||
|
public int getNumPendingDeletes() {
|
||||||
|
return pendingDeleteStatuses.size();
|
||||||
|
}
|
||||||
|
|
||||||
|
public DatanodeDetails getDatanodeDetails() {
|
||||||
|
return datanodeDetails;
|
||||||
|
}
|
||||||
|
}
|
@ -39,6 +39,7 @@
|
|||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
import java.util.UUID;
|
import java.util.UUID;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
|
|
||||||
@ -56,7 +57,7 @@
|
|||||||
*/
|
*/
|
||||||
public class SCMBlockDeletingService extends BackgroundService {
|
public class SCMBlockDeletingService extends BackgroundService {
|
||||||
|
|
||||||
static final Logger LOG =
|
public static final Logger LOG =
|
||||||
LoggerFactory.getLogger(SCMBlockDeletingService.class);
|
LoggerFactory.getLogger(SCMBlockDeletingService.class);
|
||||||
|
|
||||||
// ThreadPoolSize=2, 1 for scheduler and the other for the scanner.
|
// ThreadPoolSize=2, 1 for scheduler and the other for the scanner.
|
||||||
@ -106,6 +107,19 @@ public BackgroundTaskQueue getTasks() {
|
|||||||
return queue;
|
return queue;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void handlePendingDeletes(PendingDeleteStatusList deletionStatusList) {
|
||||||
|
DatanodeDetails dnDetails = deletionStatusList.getDatanodeDetails();
|
||||||
|
for (PendingDeleteStatusList.PendingDeleteStatus deletionStatus : deletionStatusList
|
||||||
|
.getPendingDeleteStatuses()) {
|
||||||
|
LOG.info(
|
||||||
|
"Block deletion txnID mismatch in datanode {} for containerID {}."
|
||||||
|
+ " Datanode delete txnID: {}, SCM txnID: {}",
|
||||||
|
dnDetails.getUuid(), deletionStatus.getContainerId(),
|
||||||
|
deletionStatus.getDnDeleteTransactionId(),
|
||||||
|
deletionStatus.getScmDeleteTransactionId());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
private class DeletedBlockTransactionScanner
|
private class DeletedBlockTransactionScanner
|
||||||
implements BackgroundTask<EmptyTaskResult> {
|
implements BackgroundTask<EmptyTaskResult> {
|
||||||
|
|
||||||
@ -123,11 +137,12 @@ public EmptyTaskResult call() throws Exception {
|
|||||||
LOG.debug("Running DeletedBlockTransactionScanner");
|
LOG.debug("Running DeletedBlockTransactionScanner");
|
||||||
DatanodeDeletedBlockTransactions transactions = null;
|
DatanodeDeletedBlockTransactions transactions = null;
|
||||||
List<DatanodeDetails> datanodes = nodeManager.getNodes(NodeState.HEALTHY);
|
List<DatanodeDetails> datanodes = nodeManager.getNodes(NodeState.HEALTHY);
|
||||||
|
Map<Long, Long> transactionMap = null;
|
||||||
if (datanodes != null) {
|
if (datanodes != null) {
|
||||||
transactions = new DatanodeDeletedBlockTransactions(mappingService,
|
transactions = new DatanodeDeletedBlockTransactions(mappingService,
|
||||||
blockDeleteLimitSize, datanodes.size());
|
blockDeleteLimitSize, datanodes.size());
|
||||||
try {
|
try {
|
||||||
deletedBlockLog.getTransactions(transactions);
|
transactionMap = deletedBlockLog.getTransactions(transactions);
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
// We may tolerant a number of failures for sometime
|
// We may tolerant a number of failures for sometime
|
||||||
// but if it continues to fail, at some point we need to raise
|
// but if it continues to fail, at some point we need to raise
|
||||||
@ -159,6 +174,7 @@ public EmptyTaskResult call() throws Exception {
|
|||||||
transactions.getTransactionIDList(dnId)));
|
transactions.getTransactionIDList(dnId)));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
mappingService.updateDeleteTransactionId(transactionMap);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (dnTxCount > 0) {
|
if (dnTxCount > 0) {
|
||||||
|
@ -23,11 +23,13 @@
|
|||||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.SCMContainerInfo;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.SCMContainerInfo;
|
||||||
|
import org.apache.hadoop.hdds.scm.block.PendingDeleteStatusList;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
||||||
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
||||||
import org.apache.hadoop.hdds.scm.container.closer.ContainerCloser;
|
import org.apache.hadoop.hdds.scm.container.closer.ContainerCloser;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
||||||
|
import org.apache.hadoop.hdds.scm.events.SCMEvents;
|
||||||
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
|
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
|
||||||
import org.apache.hadoop.hdds.scm.node.NodeManager;
|
import org.apache.hadoop.hdds.scm.node.NodeManager;
|
||||||
import org.apache.hadoop.hdds.scm.pipelines.PipelineSelector;
|
import org.apache.hadoop.hdds.scm.pipelines.PipelineSelector;
|
||||||
@ -43,6 +45,7 @@
|
|||||||
import org.apache.hadoop.ozone.lease.Lease;
|
import org.apache.hadoop.ozone.lease.Lease;
|
||||||
import org.apache.hadoop.ozone.lease.LeaseException;
|
import org.apache.hadoop.ozone.lease.LeaseException;
|
||||||
import org.apache.hadoop.ozone.lease.LeaseManager;
|
import org.apache.hadoop.ozone.lease.LeaseManager;
|
||||||
|
import org.apache.hadoop.utils.BatchOperation;
|
||||||
import org.apache.hadoop.utils.MetadataStore;
|
import org.apache.hadoop.utils.MetadataStore;
|
||||||
import org.apache.hadoop.utils.MetadataStoreBuilder;
|
import org.apache.hadoop.utils.MetadataStoreBuilder;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
@ -86,6 +89,7 @@ public class ContainerMapping implements Mapping {
|
|||||||
private final LeaseManager<ContainerInfo> containerLeaseManager;
|
private final LeaseManager<ContainerInfo> containerLeaseManager;
|
||||||
private final float containerCloseThreshold;
|
private final float containerCloseThreshold;
|
||||||
private final ContainerCloser closer;
|
private final ContainerCloser closer;
|
||||||
|
private final EventPublisher eventPublisher;
|
||||||
private final long size;
|
private final long size;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -128,6 +132,7 @@ public ContainerMapping(
|
|||||||
OZONE_SCM_CONTAINER_SIZE_DEFAULT) * 1024 * 1024 * 1024;
|
OZONE_SCM_CONTAINER_SIZE_DEFAULT) * 1024 * 1024 * 1024;
|
||||||
this.containerStateManager =
|
this.containerStateManager =
|
||||||
new ContainerStateManager(conf, this);
|
new ContainerStateManager(conf, this);
|
||||||
|
LOG.trace("Container State Manager created.");
|
||||||
|
|
||||||
this.pipelineSelector = new PipelineSelector(nodeManager,
|
this.pipelineSelector = new PipelineSelector(nodeManager,
|
||||||
containerStateManager, conf, eventPublisher);
|
containerStateManager, conf, eventPublisher);
|
||||||
@ -135,7 +140,7 @@ public ContainerMapping(
|
|||||||
this.containerCloseThreshold = conf.getFloat(
|
this.containerCloseThreshold = conf.getFloat(
|
||||||
ScmConfigKeys.OZONE_SCM_CONTAINER_CLOSE_THRESHOLD,
|
ScmConfigKeys.OZONE_SCM_CONTAINER_CLOSE_THRESHOLD,
|
||||||
ScmConfigKeys.OZONE_SCM_CONTAINER_CLOSE_THRESHOLD_DEFAULT);
|
ScmConfigKeys.OZONE_SCM_CONTAINER_CLOSE_THRESHOLD_DEFAULT);
|
||||||
LOG.trace("Container State Manager created.");
|
this.eventPublisher = eventPublisher;
|
||||||
|
|
||||||
long containerCreationLeaseTimeout = conf.getTimeDuration(
|
long containerCreationLeaseTimeout = conf.getTimeDuration(
|
||||||
ScmConfigKeys.OZONE_SCM_CONTAINER_CREATION_LEASE_TIMEOUT,
|
ScmConfigKeys.OZONE_SCM_CONTAINER_CREATION_LEASE_TIMEOUT,
|
||||||
@ -398,8 +403,13 @@ public HddsProtos.LifeCycleState updateContainerState(
|
|||||||
*/
|
*/
|
||||||
public void updateDeleteTransactionId(Map<Long, Long> deleteTransactionMap)
|
public void updateDeleteTransactionId(Map<Long, Long> deleteTransactionMap)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
|
if (deleteTransactionMap == null) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
lock.lock();
|
lock.lock();
|
||||||
try {
|
try {
|
||||||
|
BatchOperation batch = new BatchOperation();
|
||||||
for (Map.Entry<Long, Long> entry : deleteTransactionMap.entrySet()) {
|
for (Map.Entry<Long, Long> entry : deleteTransactionMap.entrySet()) {
|
||||||
long containerID = entry.getKey();
|
long containerID = entry.getKey();
|
||||||
byte[] dbKey = Longs.toByteArray(containerID);
|
byte[] dbKey = Longs.toByteArray(containerID);
|
||||||
@ -413,10 +423,11 @@ public void updateDeleteTransactionId(Map<Long, Long> deleteTransactionMap)
|
|||||||
ContainerInfo containerInfo = ContainerInfo.fromProtobuf(
|
ContainerInfo containerInfo = ContainerInfo.fromProtobuf(
|
||||||
HddsProtos.SCMContainerInfo.parseFrom(containerBytes));
|
HddsProtos.SCMContainerInfo.parseFrom(containerBytes));
|
||||||
containerInfo.updateDeleteTransactionId(entry.getValue());
|
containerInfo.updateDeleteTransactionId(entry.getValue());
|
||||||
containerStore.put(dbKey, containerInfo.getProtobuf().toByteArray());
|
batch.put(dbKey, containerInfo.getProtobuf().toByteArray());
|
||||||
containerStateManager
|
|
||||||
.updateDeleteTransactionId(containerID, entry.getValue());
|
|
||||||
}
|
}
|
||||||
|
containerStore.writeBatch(batch);
|
||||||
|
containerStateManager
|
||||||
|
.updateDeleteTransactionId(deleteTransactionMap);
|
||||||
} finally {
|
} finally {
|
||||||
lock.unlock();
|
lock.unlock();
|
||||||
}
|
}
|
||||||
@ -484,7 +495,8 @@ public void processContainerReports(DatanodeDetails datanodeDetails,
|
|||||||
throws IOException {
|
throws IOException {
|
||||||
List<StorageContainerDatanodeProtocolProtos.ContainerInfo>
|
List<StorageContainerDatanodeProtocolProtos.ContainerInfo>
|
||||||
containerInfos = reports.getReportsList();
|
containerInfos = reports.getReportsList();
|
||||||
|
PendingDeleteStatusList pendingDeleteStatusList =
|
||||||
|
new PendingDeleteStatusList(datanodeDetails);
|
||||||
for (StorageContainerDatanodeProtocolProtos.ContainerInfo datanodeState :
|
for (StorageContainerDatanodeProtocolProtos.ContainerInfo datanodeState :
|
||||||
containerInfos) {
|
containerInfos) {
|
||||||
byte[] dbKey = Longs.toByteArray(datanodeState.getContainerID());
|
byte[] dbKey = Longs.toByteArray(datanodeState.getContainerID());
|
||||||
@ -498,6 +510,14 @@ public void processContainerReports(DatanodeDetails datanodeDetails,
|
|||||||
HddsProtos.SCMContainerInfo newState =
|
HddsProtos.SCMContainerInfo newState =
|
||||||
reconcileState(datanodeState, knownState, datanodeDetails);
|
reconcileState(datanodeState, knownState, datanodeDetails);
|
||||||
|
|
||||||
|
if (knownState.getDeleteTransactionId() > datanodeState
|
||||||
|
.getDeleteTransactionId()) {
|
||||||
|
pendingDeleteStatusList
|
||||||
|
.addPendingDeleteStatus(datanodeState.getDeleteTransactionId(),
|
||||||
|
knownState.getDeleteTransactionId(),
|
||||||
|
knownState.getContainerID());
|
||||||
|
}
|
||||||
|
|
||||||
// FIX ME: This can be optimized, we write twice to memory, where a
|
// FIX ME: This can be optimized, we write twice to memory, where a
|
||||||
// single write would work well.
|
// single write would work well.
|
||||||
//
|
//
|
||||||
@ -529,6 +549,11 @@ public void processContainerReports(DatanodeDetails datanodeDetails,
|
|||||||
lock.unlock();
|
lock.unlock();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
if (pendingDeleteStatusList.getNumPendingDeletes() > 0) {
|
||||||
|
eventPublisher.fireEvent(SCMEvents.PENDING_DELETE_STATUS,
|
||||||
|
pendingDeleteStatusList);
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -47,6 +47,7 @@
|
|||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
import java.util.NavigableSet;
|
import java.util.NavigableSet;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.concurrent.ConcurrentHashMap;
|
import java.util.concurrent.ConcurrentHashMap;
|
||||||
@ -360,13 +361,14 @@ public ContainerInfo updateContainerInfo(ContainerInfo info)
|
|||||||
/**
|
/**
|
||||||
* Update deleteTransactionId for a container.
|
* Update deleteTransactionId for a container.
|
||||||
*
|
*
|
||||||
* @param containerID ContainerID of the container whose delete
|
* @param deleteTransactionMap maps containerId to its new
|
||||||
* transactionId needs to be updated.
|
* deleteTransactionID
|
||||||
* @param transactionId latest transactionId to be updated for the container
|
|
||||||
*/
|
*/
|
||||||
public void updateDeleteTransactionId(Long containerID, long transactionId) {
|
public void updateDeleteTransactionId(Map<Long, Long> deleteTransactionMap) {
|
||||||
containers.getContainerMap().get(ContainerID.valueof(containerID))
|
for (Map.Entry<Long, Long> entry : deleteTransactionMap.entrySet()) {
|
||||||
.updateDeleteTransactionId(transactionId);
|
containers.getContainerMap().get(ContainerID.valueof(entry.getKey()))
|
||||||
|
.updateDeleteTransactionId(entry.getValue());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -20,6 +20,7 @@
|
|||||||
package org.apache.hadoop.hdds.scm.events;
|
package org.apache.hadoop.hdds.scm.events;
|
||||||
|
|
||||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||||
|
import org.apache.hadoop.hdds.scm.block.PendingDeleteStatusList;
|
||||||
import org.apache.hadoop.hdds.scm.command.CommandStatusReportHandler
|
import org.apache.hadoop.hdds.scm.command.CommandStatusReportHandler
|
||||||
.CloseContainerStatus;
|
.CloseContainerStatus;
|
||||||
import org.apache.hadoop.hdds.scm.command.CommandStatusReportHandler
|
import org.apache.hadoop.hdds.scm.command.CommandStatusReportHandler
|
||||||
@ -146,6 +147,14 @@ public final class SCMEvents {
|
|||||||
new TypedEvent(DeleteBlockCommandStatus.class,
|
new TypedEvent(DeleteBlockCommandStatus.class,
|
||||||
"DeleteBlockCommandStatus");
|
"DeleteBlockCommandStatus");
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This event will be triggered while processing container reports from DN
|
||||||
|
* when deleteTransactionID of container in report mismatches with the
|
||||||
|
* deleteTransactionID on SCM.
|
||||||
|
*/
|
||||||
|
public static final Event<PendingDeleteStatusList> PENDING_DELETE_STATUS =
|
||||||
|
new TypedEvent(PendingDeleteStatusList.class, "PendingDeleteStatus");
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This is the command for ReplicationManager to handle under/over
|
* This is the command for ReplicationManager to handle under/over
|
||||||
* replication. Sent by the ContainerReportHandler after processing the
|
* replication. Sent by the ContainerReportHandler after processing the
|
||||||
|
@ -34,6 +34,7 @@
|
|||||||
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
||||||
import org.apache.hadoop.hdds.scm.block.BlockManager;
|
import org.apache.hadoop.hdds.scm.block.BlockManager;
|
||||||
import org.apache.hadoop.hdds.scm.block.BlockManagerImpl;
|
import org.apache.hadoop.hdds.scm.block.BlockManagerImpl;
|
||||||
|
import org.apache.hadoop.hdds.scm.block.PendingDeleteHandler;
|
||||||
import org.apache.hadoop.hdds.scm.command.CommandStatusReportHandler;
|
import org.apache.hadoop.hdds.scm.command.CommandStatusReportHandler;
|
||||||
import org.apache.hadoop.hdds.scm.container.CloseContainerEventHandler;
|
import org.apache.hadoop.hdds.scm.container.CloseContainerEventHandler;
|
||||||
import org.apache.hadoop.hdds.scm.container.ContainerActionsHandler;
|
import org.apache.hadoop.hdds.scm.container.ContainerActionsHandler;
|
||||||
@ -219,6 +220,8 @@ private StorageContainerManager(OzoneConfiguration conf) throws IOException {
|
|||||||
StaleNodeHandler staleNodeHandler = new StaleNodeHandler(node2ContainerMap);
|
StaleNodeHandler staleNodeHandler = new StaleNodeHandler(node2ContainerMap);
|
||||||
DeadNodeHandler deadNodeHandler = new DeadNodeHandler(node2ContainerMap);
|
DeadNodeHandler deadNodeHandler = new DeadNodeHandler(node2ContainerMap);
|
||||||
ContainerActionsHandler actionsHandler = new ContainerActionsHandler();
|
ContainerActionsHandler actionsHandler = new ContainerActionsHandler();
|
||||||
|
PendingDeleteHandler pendingDeleteHandler =
|
||||||
|
new PendingDeleteHandler(scmBlockManager.getSCMBlockDeletingService());
|
||||||
|
|
||||||
ContainerReportHandler containerReportHandler =
|
ContainerReportHandler containerReportHandler =
|
||||||
new ContainerReportHandler(scmContainerManager, node2ContainerMap,
|
new ContainerReportHandler(scmContainerManager, node2ContainerMap,
|
||||||
@ -235,6 +238,8 @@ private StorageContainerManager(OzoneConfiguration conf) throws IOException {
|
|||||||
eventQueue.addHandler(SCMEvents.DEAD_NODE, deadNodeHandler);
|
eventQueue.addHandler(SCMEvents.DEAD_NODE, deadNodeHandler);
|
||||||
eventQueue.addHandler(SCMEvents.CMD_STATUS_REPORT, cmdStatusReportHandler);
|
eventQueue.addHandler(SCMEvents.CMD_STATUS_REPORT, cmdStatusReportHandler);
|
||||||
eventQueue.addHandler(SCMEvents.START_REPLICATION, replicationStatus);
|
eventQueue.addHandler(SCMEvents.START_REPLICATION, replicationStatus);
|
||||||
|
eventQueue
|
||||||
|
.addHandler(SCMEvents.PENDING_DELETE_STATUS, pendingDeleteHandler);
|
||||||
|
|
||||||
long watcherTimeout =
|
long watcherTimeout =
|
||||||
conf.getTimeDuration(ScmConfigKeys.HDDS_SCM_WATCHER_TIMEOUT,
|
conf.getTimeDuration(ScmConfigKeys.HDDS_SCM_WATCHER_TIMEOUT,
|
||||||
|
@ -106,31 +106,6 @@ public void testAllocateBlock() throws Exception {
|
|||||||
Assert.assertNotNull(block);
|
Assert.assertNotNull(block);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testDeleteBlock() throws Exception {
|
|
||||||
AllocatedBlock block = blockManager.allocateBlock(DEFAULT_BLOCK_SIZE,
|
|
||||||
type, factor, containerOwner);
|
|
||||||
Assert.assertNotNull(block);
|
|
||||||
long transactionId =
|
|
||||||
mapping.getContainer(block.getBlockID().getContainerID())
|
|
||||||
.getDeleteTransactionId();
|
|
||||||
Assert.assertEquals(0, transactionId);
|
|
||||||
blockManager.deleteBlocks(Collections.singletonList(
|
|
||||||
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
|
||||||
public void testAllocateOversizedBlock() throws IOException {
|
public void testAllocateOversizedBlock() throws IOException {
|
||||||
long size = 6 * GB;
|
long size = 6 * GB;
|
||||||
|
@ -23,6 +23,11 @@
|
|||||||
import org.apache.hadoop.hdds.client.ReplicationType;
|
import org.apache.hadoop.hdds.client.ReplicationType;
|
||||||
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||||
|
import org.apache.hadoop.hdds.protocol.proto
|
||||||
|
.StorageContainerDatanodeProtocolProtos.ContainerInfo;
|
||||||
|
import org.apache.hadoop.hdds.protocol.proto
|
||||||
|
.StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
|
||||||
|
import org.apache.hadoop.hdds.scm.block.SCMBlockDeletingService;
|
||||||
import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
|
import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
|
||||||
import org.apache.hadoop.hdfs.DFSUtil;
|
import org.apache.hadoop.hdfs.DFSUtil;
|
||||||
import org.apache.hadoop.ozone.MiniOzoneCluster;
|
import org.apache.hadoop.ozone.MiniOzoneCluster;
|
||||||
@ -43,6 +48,7 @@
|
|||||||
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
|
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
|
||||||
import org.apache.hadoop.ozone.ozShell.TestOzoneShell;
|
import org.apache.hadoop.ozone.ozShell.TestOzoneShell;
|
||||||
import org.apache.hadoop.test.GenericTestUtils;
|
import org.apache.hadoop.test.GenericTestUtils;
|
||||||
|
import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
|
||||||
import org.apache.hadoop.utils.MetadataStore;
|
import org.apache.hadoop.utils.MetadataStore;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.BeforeClass;
|
import org.junit.BeforeClass;
|
||||||
@ -101,7 +107,7 @@ public void testBlockDeletion()
|
|||||||
String volumeName = UUID.randomUUID().toString();
|
String volumeName = UUID.randomUUID().toString();
|
||||||
String bucketName = UUID.randomUUID().toString();
|
String bucketName = UUID.randomUUID().toString();
|
||||||
|
|
||||||
String value = RandomStringUtils.random(1000000);
|
String value = RandomStringUtils.random(10000000);
|
||||||
store.createVolume(volumeName);
|
store.createVolume(volumeName);
|
||||||
OzoneVolume volume = store.getVolume(volumeName);
|
OzoneVolume volume = store.getVolume(volumeName);
|
||||||
volume.createBucket(bucketName);
|
volume.createBucket(bucketName);
|
||||||
@ -111,7 +117,9 @@ public void testBlockDeletion()
|
|||||||
|
|
||||||
OzoneOutputStream out = bucket.createKey(keyName, value.getBytes().length,
|
OzoneOutputStream out = bucket.createKey(keyName, value.getBytes().length,
|
||||||
ReplicationType.STAND_ALONE, ReplicationFactor.ONE);
|
ReplicationType.STAND_ALONE, ReplicationFactor.ONE);
|
||||||
out.write(value.getBytes());
|
for (int i = 0; i < 100; i++) {
|
||||||
|
out.write(value.getBytes());
|
||||||
|
}
|
||||||
out.close();
|
out.close();
|
||||||
|
|
||||||
OmKeyArgs keyArgs = new OmKeyArgs.Builder().setVolumeName(volumeName)
|
OmKeyArgs keyArgs = new OmKeyArgs.Builder().setVolumeName(volumeName)
|
||||||
@ -144,6 +152,49 @@ public void testBlockDeletion()
|
|||||||
Assert.assertTrue(!containerIdsWithDeletedBlocks.isEmpty());
|
Assert.assertTrue(!containerIdsWithDeletedBlocks.isEmpty());
|
||||||
// Containers in the DN and SCM should have same delete transactionIds
|
// Containers in the DN and SCM should have same delete transactionIds
|
||||||
matchContainerTransactionIds();
|
matchContainerTransactionIds();
|
||||||
|
// Containers in the DN and SCM should have same delete transactionIds
|
||||||
|
// after DN restart. The assertion is just to verify that the state of
|
||||||
|
// containerInfos in dn and scm is consistent after dn restart.
|
||||||
|
cluster.restartHddsDatanode(0);
|
||||||
|
matchContainerTransactionIds();
|
||||||
|
|
||||||
|
// verify PENDING_DELETE_STATUS event is fired
|
||||||
|
verifyBlockDeletionEvent();
|
||||||
|
}
|
||||||
|
|
||||||
|
private void verifyBlockDeletionEvent()
|
||||||
|
throws IOException, InterruptedException {
|
||||||
|
LogCapturer logCapturer =
|
||||||
|
LogCapturer.captureLogs(SCMBlockDeletingService.LOG);
|
||||||
|
// Create dummy container reports with deleteTransactionId set as 0
|
||||||
|
ContainerReportsProto containerReport = dnContainerSet.getContainerReport();
|
||||||
|
ContainerReportsProto.Builder dummyReportsBuilder =
|
||||||
|
ContainerReportsProto.newBuilder();
|
||||||
|
for (ContainerInfo containerInfo : containerReport.getReportsList()) {
|
||||||
|
dummyReportsBuilder.addReports(
|
||||||
|
ContainerInfo.newBuilder(containerInfo).setDeleteTransactionId(0)
|
||||||
|
.build());
|
||||||
|
}
|
||||||
|
ContainerReportsProto dummyReport = dummyReportsBuilder.build();
|
||||||
|
|
||||||
|
logCapturer.clearOutput();
|
||||||
|
scm.getScmContainerManager().processContainerReports(
|
||||||
|
cluster.getHddsDatanodes().get(0).getDatanodeDetails(), dummyReport);
|
||||||
|
// wait for event to be handled by event handler
|
||||||
|
Thread.sleep(1000);
|
||||||
|
String output = logCapturer.getOutput();
|
||||||
|
for (ContainerInfo containerInfo : dummyReport.getReportsList()) {
|
||||||
|
long containerId = containerInfo.getContainerID();
|
||||||
|
// Event should be triggered only for containers which have deleted blocks
|
||||||
|
if (containerIdsWithDeletedBlocks.contains(containerId)) {
|
||||||
|
Assert.assertTrue(output.contains(
|
||||||
|
"for containerID " + containerId + ". Datanode delete txnID"));
|
||||||
|
} else {
|
||||||
|
Assert.assertTrue(!output.contains(
|
||||||
|
"for containerID " + containerId + ". Datanode delete txnID"));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
logCapturer.clearOutput();
|
||||||
}
|
}
|
||||||
|
|
||||||
private void matchContainerTransactionIds() throws IOException {
|
private void matchContainerTransactionIds() throws IOException {
|
||||||
|
Loading…
Reference in New Issue
Block a user