diff --git a/dev-support/annotations/src/main/java/org/apache/ozone/annotations/ReplicateAnnotationProcessor.java b/dev-support/annotations/src/main/java/org/apache/ozone/annotations/ReplicateAnnotationProcessor.java
new file mode 100644
index 000000000000..6e5f482a6af6
--- /dev/null
+++ b/dev-support/annotations/src/main/java/org/apache/ozone/annotations/ReplicateAnnotationProcessor.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *
http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *
Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.ozone.annotations;
+
+import javax.annotation.processing.AbstractProcessor;
+import javax.annotation.processing.RoundEnvironment;
+import javax.annotation.processing.SupportedAnnotationTypes;
+import javax.lang.model.element.Element;
+import javax.lang.model.element.ExecutableElement;
+import javax.lang.model.element.TypeElement;
+import javax.lang.model.type.TypeMirror;
+import javax.tools.Diagnostic.Kind;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.TimeoutException;
+
+/**
+ * Annotation Processor that verifies if the methods that are marked with
+ * Replicate annotation have proper method signature which throws
+ * TimeoutException.
+ */
+@SupportedAnnotationTypes("org.apache.hadoop.hdds.scm.metadata.Replicate")
+public class ReplicateAnnotationProcessor extends AbstractProcessor {
+
+ private static final String ANNOTATION_SIMPLE_NAME = "Replicate";
+
+ @Override
+ public boolean process(Set extends TypeElement> annotations,
+ RoundEnvironment roundEnv) {
+ for (TypeElement annotation : annotations) {
+ if (!annotation.getSimpleName().contentEquals(ANNOTATION_SIMPLE_NAME)) {
+ continue;
+ }
+ roundEnv.getElementsAnnotatedWith(annotation)
+ .forEach(this::checkMethodSignature);
+ }
+ return false;
+ }
+
+ /**
+ * Checks if the method signature is correct.
+ *
+ * @param element the method element to be checked.
+ */
+ private void checkMethodSignature(Element element) {
+ if (!(element instanceof ExecutableElement)) {
+ processingEnv.getMessager().printMessage(Kind.ERROR,
+ "Replicate annotation should be on method.");
+ return;
+ }
+ final ExecutableElement executableElement = (ExecutableElement) element;
+ final List extends TypeMirror> exceptions =
+ executableElement.getThrownTypes();
+
+ if (exceptions.stream().map(TypeMirror::toString)
+ .noneMatch(TimeoutException.class.getName()::equals)) {
+ processingEnv.getMessager().printMessage(Kind.ERROR,
+ "Method with Replicate annotation should declare " +
+ "TimeoutException in its signature.");
+ }
+ }
+}
diff --git a/dev-support/annotations/src/main/resources/META-INF/services/javax.annotation.processing.Processor b/dev-support/annotations/src/main/resources/META-INF/services/javax.annotation.processing.Processor
index 35b0c1e2f11e..a0c3b81d2354 100644
--- a/dev-support/annotations/src/main/resources/META-INF/services/javax.annotation.processing.Processor
+++ b/dev-support/annotations/src/main/resources/META-INF/services/javax.annotation.processing.Processor
@@ -13,4 +13,5 @@
# See the License for the specific language governing permissions and
# limitations under the License.
-org.apache.ozone.annotations.RequestFeatureValidatorProcessor
\ No newline at end of file
+org.apache.ozone.annotations.RequestFeatureValidatorProcessor
+org.apache.ozone.annotations.ReplicateAnnotationProcessor
\ No newline at end of file
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/StorageContainerDatanodeProtocol.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/StorageContainerDatanodeProtocol.java
index 3af038677cd2..e356b7130e97 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/StorageContainerDatanodeProtocol.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/StorageContainerDatanodeProtocol.java
@@ -38,6 +38,7 @@
.StorageContainerDatanodeProtocolProtos.SCMVersionResponseProto;
import java.io.IOException;
+import java.util.concurrent.TimeoutException;
import org.apache.hadoop.hdds.scm.ScmConfig;
import org.apache.hadoop.security.KerberosInfo;
@@ -71,7 +72,7 @@ SCMVersionResponseProto getVersion(SCMVersionRequestProto versionRequest)
* @throws IOException
*/
SCMHeartbeatResponseProto sendHeartbeat(SCMHeartbeatRequestProto heartbeat)
- throws IOException;
+ throws IOException, TimeoutException;
/**
* Register Datanode.
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerDatanodeProtocolServerSideTranslatorPB.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerDatanodeProtocolServerSideTranslatorPB.java
index d465ddfb7f79..33efc1abe9b7 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerDatanodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerDatanodeProtocolServerSideTranslatorPB.java
@@ -20,6 +20,7 @@
import static org.apache.hadoop.ozone.container.upgrade.UpgradeUtils.toLayoutVersionProto;
import java.io.IOException;
+import java.util.concurrent.TimeoutException;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.NodeReportProto;
@@ -120,7 +121,7 @@ public SCMDatanodeResponse processMessage(SCMDatanodeRequest request)
default:
throw new ServiceException("Unknown command type: " + cmdType);
}
- } catch (IOException e) {
+ } catch (IOException | TimeoutException e) {
throw new ServiceException(e);
}
}
diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocol/ScmBlockLocationProtocol.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocol/ScmBlockLocationProtocol.java
index 0f84eaabc49c..f9515987bf00 100644
--- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocol/ScmBlockLocationProtocol.java
+++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocol/ScmBlockLocationProtocol.java
@@ -33,6 +33,7 @@
import java.io.Closeable;
import java.io.IOException;
import java.util.List;
+import java.util.concurrent.TimeoutException;
/**
* ScmBlockLocationProtocol is used by an HDFS node to find the set of nodes
@@ -63,7 +64,7 @@ public interface ScmBlockLocationProtocol extends Closeable {
@Deprecated
default List allocateBlock(long size, int numBlocks,
ReplicationType type, ReplicationFactor factor, String owner,
- ExcludeList excludeList) throws IOException {
+ ExcludeList excludeList) throws IOException, TimeoutException {
return allocateBlock(size, numBlocks, ReplicationConfig
.fromProtoTypeAndFactor(type, factor), owner, excludeList);
}
diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/authority/CertificateStore.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/authority/CertificateStore.java
index 872b29c281f5..fef65fb367c6 100644
--- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/authority/CertificateStore.java
+++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/authority/CertificateStore.java
@@ -35,6 +35,7 @@
import java.util.List;
import java.util.Optional;
import java.util.UUID;
+import java.util.concurrent.TimeoutException;
/**
* This interface allows the DefaultCA to be portable and use different DB
@@ -61,7 +62,8 @@ public interface CertificateStore {
*/
@Replicate
void storeValidCertificate(BigInteger serialID,
- X509Certificate certificate, NodeType role) throws IOException;
+ X509Certificate certificate, NodeType role)
+ throws IOException, TimeoutException;
void storeValidScmCertificate(BigInteger serialID,
X509Certificate certificate) throws IOException;
@@ -95,7 +97,7 @@ Optional revokeCertificates(List serialIDs,
CRLReason reason,
Date revocationTime,
CRLApprover approver)
- throws IOException;
+ throws IOException, TimeoutException;
/**
* Deletes an expired certificate from the store. Please note: We don't
diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/authority/DefaultCAServer.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/authority/DefaultCAServer.java
index 83be3aaf3bae..635af8fe6ada 100644
--- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/authority/DefaultCAServer.java
+++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/authority/DefaultCAServer.java
@@ -60,6 +60,7 @@
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.Future;
+import java.util.concurrent.TimeoutException;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import java.util.function.Consumer;
@@ -257,7 +258,8 @@ public Future requestCertificate(
default:
return null; // cannot happen, keeping checkstyle happy.
}
- } catch (CertificateException | IOException | OperatorCreationException e) {
+ } catch (CertificateException | IOException | OperatorCreationException |
+ TimeoutException e) {
LOG.error("Unable to issue a certificate.", e);
xcertHolder.completeExceptionally(
new SCMSecurityException(e, UNABLE_TO_ISSUE_CERTIFICATE));
@@ -268,7 +270,7 @@ public Future requestCertificate(
private X509CertificateHolder signAndStoreCertificate(LocalDate beginDate,
LocalDate endDate, PKCS10CertificationRequest csr, NodeType role)
throws IOException,
- OperatorCreationException, CertificateException {
+ OperatorCreationException, CertificateException, TimeoutException {
lock.lock();
X509CertificateHolder xcert;
@@ -313,7 +315,7 @@ public Future> revokeCertificates(
store.revokeCertificates(certificates,
getCACertificate(), reason, revocationTime, crlApprover)
);
- } catch (IOException ex) {
+ } catch (IOException | TimeoutException ex) {
LOG.error("Revoking the certificate failed.", ex.getCause());
revoked.completeExceptionally(new SCMSecurityException(ex));
}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManager.java
index 8acbb61efad0..094b085af507 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManager.java
@@ -20,6 +20,7 @@
import java.io.Closeable;
import java.io.IOException;
import java.util.List;
+import java.util.concurrent.TimeoutException;
import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
@@ -43,7 +44,7 @@ public interface BlockManager extends Closeable {
*/
AllocatedBlock allocateBlock(long size, ReplicationConfig replicationConfig,
String owner,
- ExcludeList excludeList) throws IOException;
+ ExcludeList excludeList) throws IOException, TimeoutException;
/**
* Deletes a list of blocks in an atomic operation. Internally, SCM
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java
index 1673f30e7b3a..c4e5d1a0d1ae 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java
@@ -25,6 +25,7 @@
import java.util.Map;
import java.util.Objects;
import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
import org.apache.hadoop.hdds.client.BlockID;
import org.apache.hadoop.hdds.client.ContainerBlockID;
@@ -159,7 +160,7 @@ public void stop() throws IOException {
public AllocatedBlock allocateBlock(final long size,
ReplicationConfig replicationConfig,
String owner, ExcludeList excludeList)
- throws IOException {
+ throws IOException, TimeoutException {
if (LOG.isTraceEnabled()) {
LOG.trace("Size : {} , replicationConfig: {}", size, replicationConfig);
}
@@ -194,7 +195,7 @@ public AllocatedBlock allocateBlock(final long size,
* @return AllocatedBlock
*/
private AllocatedBlock newBlock(ContainerInfo containerInfo)
- throws NotLeaderException {
+ throws NotLeaderException, TimeoutException {
try {
final Pipeline pipeline = pipelineManager
.getPipeline(containerInfo.getPipelineID());
@@ -255,7 +256,7 @@ public void deleteBlocks(List keyBlocksInfoList)
try {
deletedBlockLog.addTransactions(containerBlocks);
- } catch (IOException e) {
+ } catch (IOException | TimeoutException e) {
throw new IOException("Skip writing the deleted blocks info to"
+ " the delLog because addTransaction fails. " + keyBlocksInfoList
.size() + "Keys skipped", e);
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLog.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLog.java
index 4b5ec0a2af08..fdfe9290596f 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLog.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLog.java
@@ -29,6 +29,7 @@
import java.util.List;
import java.util.Map;
import java.util.UUID;
+import java.util.concurrent.TimeoutException;
/**
* The DeletedBlockLog is a persisted log in SCM to keep tracking
@@ -47,7 +48,7 @@ public interface DeletedBlockLog extends Closeable {
* @throws IOException
*/
DatanodeDeletedBlockTransactions getTransactions(int blockDeletionLimit)
- throws IOException;
+ throws IOException, TimeoutException;
/**
* Return all failed transactions in the log. A transaction is considered
@@ -69,7 +70,7 @@ List getFailedTransactions()
* @param txIDs - transaction ID.
*/
void incrementCount(List txIDs)
- throws IOException;
+ throws IOException, TimeoutException;
/**
* Commits a transaction means to delete all footprints of a transaction
@@ -95,7 +96,7 @@ void commitTransactions(List transactionResults,
* @throws IOException
*/
void addTransactions(Map> containerBlocksMap)
- throws IOException;
+ throws IOException, TimeoutException;
/**
* Returns the total number of valid transactions. A transaction is
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLogImpl.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLogImpl.java
index 78a87b882010..199f4b5c96be 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLogImpl.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLogImpl.java
@@ -25,6 +25,7 @@
import java.util.LinkedHashSet;
import java.util.ArrayList;
import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
@@ -152,7 +153,8 @@ public List getFailedTransactions()
* @throws IOException
*/
@Override
- public void incrementCount(List txIDs) throws IOException {
+ public void incrementCount(List txIDs)
+ throws IOException, TimeoutException {
lock.lock();
try {
ArrayList txIDsToUpdate = new ArrayList<>();
@@ -264,7 +266,7 @@ public void commitTransactions(
try {
deletedBlockLogStateManager.removeTransactionsFromDB(txIDsToBeDeleted);
metrics.incrBlockDeletionTransactionCompleted(txIDsToBeDeleted.size());
- } catch (IOException e) {
+ } catch (IOException | TimeoutException e) {
LOG.warn("Could not commit delete block transactions: "
+ txIDsToBeDeleted, e);
}
@@ -342,7 +344,7 @@ public void onFlush() {
*/
@Override
public void addTransactions(Map> containerBlocksMap)
- throws IOException {
+ throws IOException, TimeoutException {
lock.lock();
try {
ArrayList txsToBeAdded = new ArrayList<>();
@@ -388,7 +390,7 @@ private void getTransaction(DeletedBlocksTransaction tx,
@Override
public DatanodeDeletedBlockTransactions getTransactions(
- int blockDeletionLimit) throws IOException {
+ int blockDeletionLimit) throws IOException, TimeoutException {
lock.lock();
try {
DatanodeDeletedBlockTransactions transactions =
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLogStateManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLogStateManager.java
index eb85dd73ef1b..37187a53e50b 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLogStateManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLogStateManager.java
@@ -25,6 +25,7 @@
import java.io.IOException;
import java.util.ArrayList;
+import java.util.concurrent.TimeoutException;
/**
* DeletedBlockLogStateManager interface to
@@ -33,15 +34,15 @@
public interface DeletedBlockLogStateManager {
@Replicate
void addTransactionsToDB(ArrayList txs)
- throws IOException;
+ throws IOException, TimeoutException;
@Replicate
void removeTransactionsFromDB(ArrayList txIDs)
- throws IOException;
+ throws IOException, TimeoutException;
@Replicate
void increaseRetryCountOfTransactionInDB(ArrayList txIDs)
- throws IOException;
+ throws IOException, TimeoutException;
TableIterator> getReadOnlyIterator();
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/AbstractContainerReportHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/AbstractContainerReportHandler.java
index dd9f71aef8f3..eb3cf7ddbdd2 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/AbstractContainerReportHandler.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/AbstractContainerReportHandler.java
@@ -42,6 +42,7 @@
import java.util.List;
import java.util.Set;
import java.util.UUID;
+import java.util.concurrent.TimeoutException;
import java.util.function.Supplier;
/**
@@ -79,10 +80,11 @@ public class AbstractContainerReportHandler {
* @param publisher EventPublisher instance
* @throws IOException
* @throws InvalidStateTransitionException
+ * @throws TimeoutException
*/
protected void processContainerReplica(final DatanodeDetails datanodeDetails,
final ContainerReplicaProto replicaProto, final EventPublisher publisher)
- throws IOException, InvalidStateTransitionException {
+ throws IOException, InvalidStateTransitionException, TimeoutException {
ContainerInfo container = getContainerManager().getContainer(
ContainerID.valueOf(replicaProto.getContainerID()));
processContainerReplica(
@@ -99,11 +101,12 @@ protected void processContainerReplica(final DatanodeDetails datanodeDetails,
* @param publisher EventPublisher instance
*
* @throws IOException In case of any Exception while processing the report
+ * @throws TimeoutException In case of timeout while updating container state
*/
protected void processContainerReplica(final DatanodeDetails datanodeDetails,
final ContainerInfo containerInfo,
final ContainerReplicaProto replicaProto, final EventPublisher publisher)
- throws IOException, InvalidStateTransitionException {
+ throws IOException, InvalidStateTransitionException, TimeoutException {
final ContainerID containerId = containerInfo.containerID();
if (logger.isDebugEnabled()) {
@@ -235,12 +238,13 @@ private List getOtherReplicas(ContainerID containerId,
* @param replica ContainerReplica
* @boolean true - replica should be ignored in the next process
* @throws IOException In case of Exception
+ * @throws TimeoutException In case of timeout while updating container state
*/
private boolean updateContainerState(final DatanodeDetails datanode,
final ContainerInfo container,
final ContainerReplicaProto replica,
final EventPublisher publisher)
- throws IOException, InvalidStateTransitionException {
+ throws IOException, InvalidStateTransitionException, TimeoutException {
final ContainerID containerId = container.containerID();
boolean ignored = false;
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/CloseContainerEventHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/CloseContainerEventHandler.java
index 1aab691f1e9e..2ff9b79ce6c4 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/CloseContainerEventHandler.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/CloseContainerEventHandler.java
@@ -18,6 +18,7 @@
import java.io.IOException;
import java.util.List;
+import java.util.concurrent.TimeoutException;
import java.util.stream.Collectors;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
@@ -106,7 +107,8 @@ public void onMessage(ContainerID containerID, EventPublisher publisher) {
} catch (NotLeaderException nle) {
LOG.warn("Skip sending close container command,"
+ " since current SCM is not leader.", nle);
- } catch (IOException | InvalidStateTransitionException ex) {
+ } catch (IOException | InvalidStateTransitionException |
+ TimeoutException ex) {
LOG.error("Failed to close the container {}.", containerID, ex);
}
}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerManager.java
index a903be614f1f..247f1c3b859a 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerManager.java
@@ -22,6 +22,7 @@
import java.util.List;
import java.util.Map;
import java.util.Set;
+import java.util.concurrent.TimeoutException;
import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
@@ -103,7 +104,8 @@ default List getContainers() {
* @throws IOException
*/
ContainerInfo allocateContainer(ReplicationConfig replicationConfig,
- String owner) throws IOException;
+ String owner)
+ throws IOException, TimeoutException;
/**
* Update container state.
@@ -114,7 +116,7 @@ ContainerInfo allocateContainer(ReplicationConfig replicationConfig,
*/
void updateContainerState(ContainerID containerID,
LifeCycleEvent event)
- throws IOException, InvalidStateTransitionException;
+ throws IOException, InvalidStateTransitionException, TimeoutException;
/**
* Returns the latest list of replicas for given containerId.
@@ -187,7 +189,7 @@ ContainerInfo getMatchingContainer(long size, String owner,
* @throws IOException
*/
void deleteContainer(ContainerID containerID)
- throws IOException;
+ throws IOException, TimeoutException;
/**
* Returns the list of containersIDs.
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerManagerImpl.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerManagerImpl.java
index f3e802545020..6a76f02c4ca4 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerManagerImpl.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerManagerImpl.java
@@ -28,6 +28,7 @@
import java.util.Optional;
import java.util.Random;
import java.util.Set;
+import java.util.concurrent.TimeoutException;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import java.util.stream.Collectors;
@@ -202,7 +203,7 @@ public int getContainerStateCount(final LifeCycleState state) {
@Override
public ContainerInfo allocateContainer(
final ReplicationConfig replicationConfig, final String owner)
- throws IOException {
+ throws IOException, TimeoutException {
// Acquire pipeline manager lock, to avoid any updates to pipeline
// while allocate container happens. This is to avoid scenario like
// mentioned in HDDS-5655.
@@ -255,7 +256,7 @@ public ContainerInfo allocateContainer(
}
private ContainerInfo createContainer(Pipeline pipeline, String owner)
- throws IOException {
+ throws IOException, TimeoutException {
final ContainerInfo containerInfo = allocateContainer(pipeline, owner);
if (LOG.isTraceEnabled()) {
LOG.trace("New container allocated: {}", containerInfo);
@@ -265,7 +266,7 @@ private ContainerInfo createContainer(Pipeline pipeline, String owner)
private ContainerInfo allocateContainer(final Pipeline pipeline,
final String owner)
- throws IOException {
+ throws IOException, TimeoutException {
final long uniqueId = sequenceIdGen.getNextId(CONTAINER_ID);
Preconditions.checkState(uniqueId > 0,
"Cannot allocate container, negative container id" +
@@ -299,7 +300,7 @@ private ContainerInfo allocateContainer(final Pipeline pipeline,
@Override
public void updateContainerState(final ContainerID cid,
final LifeCycleEvent event)
- throws IOException, InvalidStateTransitionException {
+ throws IOException, InvalidStateTransitionException, TimeoutException {
HddsProtos.ContainerID protoId = cid.getProtobuf();
lock.lock();
try {
@@ -435,7 +436,7 @@ public void notifyContainerReportProcessing(final boolean isFullReport,
@Override
public void deleteContainer(final ContainerID cid)
- throws IOException {
+ throws IOException, TimeoutException {
HddsProtos.ContainerID protoId = cid.getProtobuf();
lock.lock();
try {
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReportHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReportHandler.java
index 5cdefcf38c2e..8f1eececa4e5 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReportHandler.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReportHandler.java
@@ -39,6 +39,7 @@
import java.io.IOException;
import java.util.List;
import java.util.Set;
+import java.util.concurrent.TimeoutException;
/**
* Handles container reports from datanode.
@@ -231,7 +232,8 @@ private void processSingleReplica(final DatanodeDetails datanodeDetails,
try {
processContainerReplica(
datanodeDetails, container, replicaProto, publisher);
- } catch (IOException | InvalidStateTransitionException e) {
+ } catch (IOException | InvalidStateTransitionException |
+ TimeoutException e) {
LOG.error("Exception while processing container report for container" +
" {} from datanode {}.", replicaProto.getContainerID(),
datanodeDetails, e);
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java
index 9f741068a86e..1a544ce0b146 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java
@@ -21,6 +21,7 @@
import java.util.Map;
import java.util.NavigableSet;
import java.util.Set;
+import java.util.concurrent.TimeoutException;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ContainerInfoProto;
@@ -141,7 +142,7 @@ void removeContainerReplica(ContainerID id,
*/
@Replicate
void addContainer(ContainerInfoProto containerInfo)
- throws IOException;
+ throws IOException, TimeoutException;
/**
*
@@ -149,7 +150,7 @@ void addContainer(ContainerInfoProto containerInfo)
@Replicate
void updateContainerState(HddsProtos.ContainerID id,
HddsProtos.LifeCycleEvent event)
- throws IOException, InvalidStateTransitionException;
+ throws IOException, InvalidStateTransitionException, TimeoutException;
/**
*
@@ -170,7 +171,7 @@ ContainerInfo getMatchingContainer(long size, String owner,
*/
@Replicate
void removeContainer(HddsProtos.ContainerID containerInfo)
- throws IOException;
+ throws IOException, TimeoutException;
/**
* Reinitialize the ContainerStateManager with container store.
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/IncrementalContainerReportHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/IncrementalContainerReportHandler.java
index c494e2d6abc8..20a5d9adf6ca 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/IncrementalContainerReportHandler.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/IncrementalContainerReportHandler.java
@@ -19,6 +19,7 @@
package org.apache.hadoop.hdds.scm.container;
import java.io.IOException;
+import java.util.concurrent.TimeoutException;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos
@@ -103,7 +104,8 @@ public void onMessage(final IncrementalContainerReportFromDatanode report,
success = false;
LOG.warn("Container {} replica not found!",
replicaProto.getContainerID());
- } catch (IOException | InvalidStateTransitionException e) {
+ } catch (IOException | InvalidStateTransitionException |
+ TimeoutException e) {
success = false;
LOG.error("Exception while processing ICR for container {}",
replicaProto.getContainerID(), e);
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java
index a77f7a8c0591..a29be341b08e 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java
@@ -217,7 +217,7 @@ private void balance() {
if (iR == IterationResult.ITERATION_COMPLETED) {
try {
saveConfiguration(config, true, i + 1);
- } catch (IOException e) {
+ } catch (IOException | TimeoutException e) {
LOG.warn("Could not persist next iteration index value for " +
"ContainerBalancer after completing an iteration", e);
}
@@ -665,7 +665,7 @@ private boolean moveContainer(DatanodeDetails source,
LOG.warn("Could not find Container {} for container move",
containerID, e);
return false;
- } catch (NodeNotFoundException e) {
+ } catch (NodeNotFoundException | TimeoutException e) {
LOG.warn("Container move failed for container {}", containerID, e);
return false;
}
@@ -981,7 +981,8 @@ public void start() throws IllegalContainerBalancerStateException,
*/
public void startBalancer(ContainerBalancerConfiguration configuration)
throws IllegalContainerBalancerStateException,
- InvalidContainerBalancerConfigurationException, IOException {
+ InvalidContainerBalancerConfigurationException, IOException,
+ TimeoutException {
lock.lock();
try {
// validates state, config, and then saves config
@@ -1090,7 +1091,8 @@ public void stop() {
* "stop" command.
*/
public void stopBalancer()
- throws IOException, IllegalContainerBalancerStateException {
+ throws IOException, IllegalContainerBalancerStateException,
+ TimeoutException {
lock.lock();
try {
// should be leader, out of safe mode, and currently running
@@ -1113,7 +1115,8 @@ private void tryStopBalancer(String stopReason) {
LOG.info("Stopping ContainerBalancer. Reason for stopping: {}",
stopReason);
stopBalancer();
- } catch (IllegalContainerBalancerStateException | IOException e) {
+ } catch (IllegalContainerBalancerStateException | IOException |
+ TimeoutException e) {
LOG.warn("Tried to stop ContainerBalancer but failed. Reason for " +
"stopping: {}", stopReason, e);
}
@@ -1121,7 +1124,7 @@ private void tryStopBalancer(String stopReason) {
private void saveConfiguration(ContainerBalancerConfiguration configuration,
boolean shouldRun, int index)
- throws IOException {
+ throws IOException, TimeoutException {
lock.lock();
try {
saveConfiguration(configuration.toProtobufBuilder()
@@ -1201,7 +1204,7 @@ public void setOzoneConfiguration(
private void setBalancerConfigOnStartBalancer(
ContainerBalancerConfiguration configuration)
throws InvalidContainerBalancerConfigurationException,
- IllegalContainerBalancerStateException, IOException {
+ IllegalContainerBalancerStateException, IOException, TimeoutException {
validateState(false);
validateConfiguration(configuration);
saveConfiguration(configuration, true, 0);
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/LegacyReplicationManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/LegacyReplicationManager.java
index 536c21a37dc5..92b26379c9f9 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/LegacyReplicationManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/LegacyReplicationManager.java
@@ -86,6 +86,7 @@
import java.util.UUID;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Consumer;
import java.util.function.Predicate;
@@ -597,7 +598,7 @@ private void updateInflightAction(final ContainerInfo container,
final InflightMap inflightActions,
final Predicate filter,
final Runnable timeoutCounter,
- final Consumer completedCounter) {
+ final Consumer completedCounter) throws TimeoutException {
final ContainerID id = container.containerID();
final long deadline = clock.millis() - rmConf.getEventTimeout();
inflightActions.iterate(id, a -> updateInflightAction(
@@ -635,6 +636,8 @@ private boolean updateInflightAction(final ContainerInfo container,
// Should not happen, but if it does, just remove the action as the
// node somehow does not exist;
remove = true;
+ } catch (TimeoutException e) {
+ LOG.error("Got exception while updating.", e);
}
return remove;
}
@@ -654,7 +657,7 @@ private void updateMoveIfNeeded(final boolean isUnhealthy,
final boolean isNotInService,
final ContainerInfo container, final DatanodeDetails dn,
final boolean isInflightReplication)
- throws ContainerNotFoundException {
+ throws ContainerNotFoundException, TimeoutException {
// make sure inflightMove contains the container
ContainerID id = container.containerID();
@@ -743,7 +746,8 @@ private void updateMoveIfNeeded(final boolean isUnhealthy,
*/
public CompletableFuture move(ContainerID cid,
DatanodeDetails src, DatanodeDetails tgt)
- throws ContainerNotFoundException, NodeNotFoundException {
+ throws ContainerNotFoundException, NodeNotFoundException,
+ TimeoutException {
return move(cid, new MoveDataNodePair(src, tgt));
}
@@ -754,8 +758,8 @@ public CompletableFuture move(ContainerID cid,
* @param mp MoveDataNodePair which contains source and target datanodes
*/
private CompletableFuture move(ContainerID cid,
- MoveDataNodePair mp)
- throws ContainerNotFoundException, NodeNotFoundException {
+ MoveDataNodePair mp) throws ContainerNotFoundException,
+ NodeNotFoundException, TimeoutException {
CompletableFuture ret = new CompletableFuture<>();
if (!scmContext.isLeader()) {
@@ -1031,7 +1035,7 @@ private boolean canForceCloseContainer(final ContainerInfo container,
*/
private void deleteContainerReplicas(final ContainerInfo container,
final Set replicas) throws IOException,
- InvalidStateTransitionException {
+ InvalidStateTransitionException, TimeoutException {
Preconditions.assertTrue(container.getState() ==
LifeCycleState.CLOSED);
Preconditions.assertTrue(container.getNumberOfKeys() == 0);
@@ -1054,7 +1058,7 @@ private void deleteContainerReplicas(final ContainerInfo container,
*/
private void handleContainerUnderDelete(final ContainerInfo container,
final Set replicas) throws IOException,
- InvalidStateTransitionException {
+ InvalidStateTransitionException, TimeoutException {
if (replicas.size() == 0) {
containerManager.updateContainerState(container.containerID(),
HddsProtos.LifeCycleEvent.CLEANUP);
@@ -1305,7 +1309,8 @@ private void handleOverReplicatedContainer(final ContainerInfo container,
* @param replicaSet An Set of replicas, which may have excess replicas
*/
private void deleteSrcDnForMove(final ContainerInfo cif,
- final Set replicaSet) {
+ final Set replicaSet)
+ throws TimeoutException {
final ContainerID cid = cif.containerID();
MoveDataNodePair movePair = moveScheduler.getMoveDataNodePair(cid);
if (movePair == null) {
@@ -1828,7 +1833,8 @@ public interface MoveScheduler {
* @param contianerIDProto Container to which the move option is finished
*/
@Replicate
- void completeMove(HddsProtos.ContainerID contianerIDProto);
+ void completeMove(HddsProtos.ContainerID contianerIDProto)
+ throws TimeoutException;
/**
* start a move action for a given container.
@@ -1838,7 +1844,8 @@ public interface MoveScheduler {
*/
@Replicate
void startMove(HddsProtos.ContainerID contianerIDProto,
- HddsProtos.MoveDataNodePairProto mp) throws IOException;
+ HddsProtos.MoveDataNodePairProto mp)
+ throws IOException, TimeoutException;
/**
* get the MoveDataNodePair of the giver container.
@@ -2025,7 +2032,11 @@ private void onLeaderReadyAndOutOfSafeMode() {
if (isTgtExist) {
//the former scm leader may or may not send the deletion command
//before reelection.here, we just try to send the command again.
- deleteSrcDnForMove(cif, replicas);
+ try {
+ deleteSrcDnForMove(cif, replicas);
+ } catch (TimeoutException ex) {
+ LOG.error("Exception while cleaning up excess replicas.", ex);
+ }
} else {
// resenting replication command is ok , no matter whether there is an
// on-going replication
@@ -2040,7 +2051,13 @@ private void onLeaderReadyAndOutOfSafeMode() {
}
});
- needToRemove.forEach(moveScheduler::completeMove);
+ for (HddsProtos.ContainerID containerID : needToRemove) {
+ try {
+ moveScheduler.completeMove(containerID);
+ } catch (TimeoutException ex) {
+ LOG.error("Exception while moving container.", ex);
+ }
+ }
}
/**
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManager.java
index 59585b4d7920..7464cc960591 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManager.java
@@ -56,6 +56,7 @@
import java.util.Set;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
@@ -533,7 +534,8 @@ public ReplicationManagerMetrics getMetrics() {
*/
public CompletableFuture move(
ContainerID cid, DatanodeDetails src, DatanodeDetails tgt)
- throws NodeNotFoundException, ContainerNotFoundException {
+ throws NodeNotFoundException, ContainerNotFoundException,
+ TimeoutException {
CompletableFuture ret =
new CompletableFuture<>();
if (!isRunning()) {
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/SequenceIdGenerator.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/SequenceIdGenerator.java
index 1b0316bbbc21..cd7689298c75 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/SequenceIdGenerator.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/SequenceIdGenerator.java
@@ -38,6 +38,7 @@
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
@@ -104,7 +105,7 @@ public SequenceIdGenerator(ConfigurationSource conf,
* @param sequenceIdName : name of the sequenceId
* @return : next id of this sequenceId.
*/
- public long getNextId(String sequenceIdName) {
+ public long getNextId(String sequenceIdName) throws TimeoutException {
lock.lock();
try {
Batch batch = sequenceIdToBatchMap.computeIfAbsent(
@@ -186,7 +187,8 @@ interface StateManager {
*/
@Replicate
Boolean allocateBatch(String sequenceIdName,
- Long expectedLastId, Long newLastId);
+ Long expectedLastId, Long newLastId)
+ throws TimeoutException;
/**
* @param sequenceIdName : name of the sequence id.
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/StatefulService.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/StatefulService.java
index 69df7c740fbc..3c73d11050a6 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/StatefulService.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/StatefulService.java
@@ -23,6 +23,7 @@
import java.io.IOException;
import java.lang.reflect.InvocationTargetException;
+import java.util.concurrent.TimeoutException;
/**
* A StatefulService is an SCMService that persists configuration to RocksDB.
@@ -48,7 +49,7 @@ protected StatefulService(StatefulServiceStateManager stateManager) {
* @throws IOException on failure to persist configuration
*/
protected final void saveConfiguration(GeneratedMessage configurationMessage)
- throws IOException {
+ throws IOException, TimeoutException {
stateManager.saveConfiguration(getServiceName(),
configurationMessage.toByteString());
}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/StatefulServiceStateManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/StatefulServiceStateManager.java
index a6a8a50ae5d5..dbb0f5d6cb27 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/StatefulServiceStateManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/StatefulServiceStateManager.java
@@ -23,6 +23,7 @@
import org.apache.hadoop.hdds.utils.db.Table;
import java.io.IOException;
+import java.util.concurrent.TimeoutException;
/**
* This interface defines an API for saving and reading configurations of a
@@ -41,7 +42,7 @@ public interface StatefulServiceStateManager {
*/
@Replicate
void saveConfiguration(String serviceName, ByteString bytes)
- throws IOException;
+ throws IOException, TimeoutException;
/**
* Reads the persisted configuration mapped to the specified serviceName.
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/DeadNodeHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/DeadNodeHandler.java
index 26d3ff4b06da..55ae6d1b0f58 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/DeadNodeHandler.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/DeadNodeHandler.java
@@ -20,6 +20,7 @@
import java.io.IOException;
import java.util.Optional;
+import java.util.concurrent.TimeoutException;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
@@ -119,7 +120,7 @@ private void destroyPipelines(final DatanodeDetails datanodeDetails) {
} catch (PipelineNotFoundException ignore) {
// Pipeline is not there in pipeline manager,
// should we care?
- } catch (IOException ex) {
+ } catch (IOException | TimeoutException ex) {
LOG.warn("Exception while finalizing pipeline {}",
id, ex);
}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/HealthyReadOnlyNodeHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/HealthyReadOnlyNodeHandler.java
index 90120187f928..4f764cef73ed 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/HealthyReadOnlyNodeHandler.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/HealthyReadOnlyNodeHandler.java
@@ -19,6 +19,7 @@
import java.io.IOException;
import java.util.Set;
+import java.util.concurrent.TimeoutException;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
@@ -91,7 +92,7 @@ public void onMessage(DatanodeDetails datanodeDetails,
HddsProtos.NodeState.HEALTHY_READONLY,
datanodeDetails.getUuidString());
pipelineManager.closePipeline(pipeline, true);
- } catch (IOException ex) {
+ } catch (IOException | TimeoutException ex) {
LOG.error("Failed to close pipeline {} which uses HEALTHY READONLY " +
"datanode {}: ", pipelineID, datanodeDetails, ex);
}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/StaleNodeHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/StaleNodeHandler.java
index dd8cea366975..e01949973a99 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/StaleNodeHandler.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/StaleNodeHandler.java
@@ -31,6 +31,7 @@
import java.io.IOException;
import java.util.Set;
+import java.util.concurrent.TimeoutException;
/**
* Handles Stale node event.
@@ -61,7 +62,7 @@ public void onMessage(DatanodeDetails datanodeDetails,
try {
Pipeline pipeline = pipelineManager.getPipeline(pipelineID);
pipelineManager.closePipeline(pipeline, true);
- } catch (IOException e) {
+ } catch (IOException | TimeoutException e) {
LOG.info("Could not finalize pipeline={} for dn={}", pipelineID,
datanodeDetails);
}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/StartDatanodeAdminHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/StartDatanodeAdminHandler.java
index c71b1c93ca31..6535ae48273d 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/StartDatanodeAdminHandler.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/StartDatanodeAdminHandler.java
@@ -29,6 +29,7 @@
import java.io.IOException;
import java.util.Set;
+import java.util.concurrent.TimeoutException;
/**
* Handler which is fired when a datanode starts admin (decommission or
@@ -59,7 +60,7 @@ public void onMessage(DatanodeDetails datanodeDetails,
try {
Pipeline pipeline = pipelineManager.getPipeline(pipelineID);
pipelineManager.closePipeline(pipeline, false);
- } catch (IOException e) {
+ } catch (IOException | TimeoutException e) {
LOG.info("Could not finalize pipeline={} for dn={}", pipelineID,
datanodeDetails);
}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineActionHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineActionHandler.java
index e33f256a4476..7b76cf8d2260 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineActionHandler.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineActionHandler.java
@@ -36,6 +36,7 @@
import org.slf4j.LoggerFactory;
import java.io.IOException;
+import java.util.concurrent.TimeoutException;
/**
* Handles pipeline actions from datanode.
@@ -102,7 +103,7 @@ private void processPipelineAction(final DatanodeDetails datanode,
}
publisher.fireEvent(SCMEvents.DATANODE_COMMAND,
new CommandForDatanode<>(datanode.getUuid(), command));
- } catch (IOException ioe) {
+ } catch (IOException | TimeoutException ioe) {
LOG.error("Could not execute pipeline action={} pipeline={}",
action, pid, ioe);
}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManager.java
index afc663a893a2..09816b6b9212 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManager.java
@@ -24,6 +24,7 @@
import java.util.List;
import java.util.NavigableSet;
import java.util.Set;
+import java.util.concurrent.TimeoutException;
import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
@@ -36,14 +37,13 @@
*/
public interface PipelineManager extends Closeable, PipelineManagerMXBean {
- Pipeline createPipeline(
- ReplicationConfig replicationConfig
- )
- throws IOException;
+ Pipeline createPipeline(ReplicationConfig replicationConfig)
+ throws IOException, TimeoutException;
- Pipeline createPipeline(
- ReplicationConfig replicationConfig, List excludedNodes,
- List favoredNodes) throws IOException;
+ Pipeline createPipeline(ReplicationConfig replicationConfig,
+ List excludedNodes,
+ List favoredNodes)
+ throws IOException, TimeoutException;
Pipeline createPipeline(
@@ -107,13 +107,14 @@ NavigableSet getContainersInPipeline(PipelineID pipelineID)
int getNumberOfContainers(PipelineID pipelineID) throws IOException;
- void openPipeline(PipelineID pipelineId) throws IOException;
+ void openPipeline(PipelineID pipelineId) throws IOException, TimeoutException;
- void closePipeline(Pipeline pipeline, boolean onTimeout) throws IOException;
+ void closePipeline(Pipeline pipeline, boolean onTimeout)
+ throws IOException, TimeoutException;
void closeStalePipelines(DatanodeDetails datanodeDetails);
- void scrubPipelines() throws IOException;
+ void scrubPipelines() throws IOException, TimeoutException;
void startPipelineCreator();
@@ -131,7 +132,8 @@ NavigableSet getContainersInPipeline(PipelineID pipelineID)
* @param pipelineID ID of the pipeline to activate.
* @throws IOException in case of any Exception
*/
- void activatePipeline(PipelineID pipelineID) throws IOException;
+ void activatePipeline(PipelineID pipelineID)
+ throws IOException, TimeoutException;
/**
* Deactivates an active pipeline.
@@ -139,7 +141,8 @@ NavigableSet getContainersInPipeline(PipelineID pipelineID)
* @param pipelineID ID of the pipeline to deactivate.
* @throws IOException in case of any Exception
*/
- void deactivatePipeline(PipelineID pipelineID) throws IOException;
+ void deactivatePipeline(PipelineID pipelineID)
+ throws IOException, TimeoutException;
/**
* Wait a pipeline to be OPEN.
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerImpl.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerImpl.java
index 4de7d658c28c..3e5d86d0af0f 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerImpl.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerImpl.java
@@ -61,6 +61,7 @@
import java.util.NavigableSet;
import java.util.Set;
import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.stream.Collectors;
@@ -172,7 +173,7 @@ public static PipelineManagerImpl newPipelineManager(
.setPeriodicalTask(() -> {
try {
pipelineManager.scrubPipelines();
- } catch (IOException e) {
+ } catch (IOException | TimeoutException e) {
LOG.error("Unexpected error during pipeline scrubbing", e);
}
}).build();
@@ -187,9 +188,8 @@ public static PipelineManagerImpl newPipelineManager(
}
@Override
- public Pipeline createPipeline(
- ReplicationConfig replicationConfig
- ) throws IOException {
+ public Pipeline createPipeline(ReplicationConfig replicationConfig)
+ throws IOException, TimeoutException {
return createPipeline(replicationConfig, Collections.emptyList(),
Collections.emptyList());
}
@@ -197,7 +197,7 @@ public Pipeline createPipeline(
@Override
public Pipeline createPipeline(ReplicationConfig replicationConfig,
List excludedNodes, List favoredNodes)
- throws IOException {
+ throws IOException, TimeoutException {
if (!isPipelineCreationAllowed() && !factorOne(replicationConfig)) {
LOG.debug("Pipeline creation is not allowed until safe mode prechecks " +
"complete");
@@ -220,7 +220,7 @@ public Pipeline createPipeline(ReplicationConfig replicationConfig,
ClientVersion.CURRENT_VERSION));
recordMetricsForPipeline(pipeline);
return pipeline;
- } catch (IOException ex) {
+ } catch (IOException | TimeoutException ex) {
LOG.debug("Failed to create pipeline with replicationConfig {}.",
replicationConfig, ex);
metrics.incNumPipelineCreationFailed();
@@ -347,7 +347,8 @@ public int getNumberOfContainers(PipelineID pipelineID) throws IOException {
}
@Override
- public void openPipeline(PipelineID pipelineId) throws IOException {
+ public void openPipeline(PipelineID pipelineId)
+ throws IOException, TimeoutException {
acquireWriteLock();
try {
Pipeline pipeline = stateManager.getPipeline(pipelineId);
@@ -372,7 +373,8 @@ public void openPipeline(PipelineID pipelineId) throws IOException {
* @param pipeline - pipeline to be removed
* @throws IOException
*/
- protected void removePipeline(Pipeline pipeline) throws IOException {
+ protected void removePipeline(Pipeline pipeline)
+ throws IOException, TimeoutException {
pipelineFactory.close(pipeline.getType(), pipeline);
PipelineID pipelineID = pipeline.getId();
acquireWriteLock();
@@ -393,7 +395,7 @@ protected void removePipeline(Pipeline pipeline) throws IOException {
* @throws IOException
*/
protected void closeContainersForPipeline(final PipelineID pipelineId)
- throws IOException {
+ throws IOException, TimeoutException {
Set containerIDs = stateManager.getContainers(pipelineId);
ContainerManager containerManager = scmContext.getScm()
.getContainerManager();
@@ -420,7 +422,7 @@ protected void closeContainersForPipeline(final PipelineID pipelineId)
*/
@Override
public void closePipeline(Pipeline pipeline, boolean onTimeout)
- throws IOException {
+ throws IOException, TimeoutException {
PipelineID pipelineID = pipeline.getId();
// close containers.
closeContainersForPipeline(pipelineID);
@@ -461,7 +463,7 @@ public void closeStalePipelines(DatanodeDetails datanodeDetails) {
LOG.info("Closing the stale pipeline: {}", p.getId());
closePipeline(p, false);
LOG.info("Closed the stale pipeline: {}", p.getId());
- } catch (IOException e) {
+ } catch (IOException | TimeoutException e) {
LOG.error("Closing the stale pipeline failed: {}", p, e);
}
});
@@ -485,7 +487,7 @@ List getStalePipelines(DatanodeDetails datanodeDetails) {
* Scrub pipelines.
*/
@Override
- public void scrubPipelines() throws IOException {
+ public void scrubPipelines() throws IOException, TimeoutException {
Instant currentTime = clock.instant();
Long pipelineScrubTimeoutInMills = conf.getTimeDuration(
ScmConfigKeys.OZONE_SCM_PIPELINE_ALLOCATED_TIMEOUT,
@@ -554,7 +556,7 @@ public int minPipelineLimit(Pipeline pipeline) {
*/
@Override
public void activatePipeline(PipelineID pipelineID)
- throws IOException {
+ throws IOException, TimeoutException {
acquireWriteLock();
try {
stateManager.updatePipelineState(pipelineID.getProtobuf(),
@@ -572,7 +574,7 @@ public void activatePipeline(PipelineID pipelineID)
*/
@Override
public void deactivatePipeline(PipelineID pipelineID)
- throws IOException {
+ throws IOException, TimeoutException {
acquireWriteLock();
try {
stateManager.updatePipelineState(pipelineID.getProtobuf(),
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineReportHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineReportHandler.java
index 64815a92d3ee..134544541f93 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineReportHandler.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineReportHandler.java
@@ -19,6 +19,7 @@
package org.apache.hadoop.hdds.scm.pipeline;
import java.io.IOException;
+import java.util.concurrent.TimeoutException;
import org.apache.hadoop.hdds.client.RatisReplicationConfig;
import org.apache.hadoop.hdds.conf.ConfigurationSource;
@@ -95,7 +96,7 @@ public void onMessage(PipelineReportFromDatanode pipelineReportFromDatanode,
// pipeline report on followers.
} catch (PipelineNotFoundException e) {
LOGGER.error("Could not find pipeline {}", report.getPipelineID());
- } catch (IOException e) {
+ } catch (IOException | TimeoutException e) {
LOGGER.error("Could not process pipeline report={} from dn={}.",
report, dn, e);
}
@@ -103,7 +104,8 @@ public void onMessage(PipelineReportFromDatanode pipelineReportFromDatanode,
}
protected void processPipelineReport(PipelineReport report,
- DatanodeDetails dn, EventPublisher publisher) throws IOException {
+ DatanodeDetails dn, EventPublisher publisher)
+ throws IOException, TimeoutException {
PipelineID pipelineID = PipelineID.getFromProtobuf(report.getPipelineID());
Pipeline pipeline;
try {
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineStateManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineStateManager.java
index 6180158f2043..53d12aa9d7d9 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineStateManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineStateManager.java
@@ -28,6 +28,7 @@
import java.util.Collection;
import java.util.List;
import java.util.NavigableSet;
+import java.util.concurrent.TimeoutException;
/**
* Manages the state of pipelines in SCM.
@@ -40,7 +41,8 @@ public interface PipelineStateManager {
* @throws IOException
*/
@Replicate
- void addPipeline(HddsProtos.Pipeline pipelineProto) throws IOException;
+ void addPipeline(HddsProtos.Pipeline pipelineProto)
+ throws IOException, TimeoutException;
/**
* Removing pipeline would be replicated to Ratis.
@@ -50,7 +52,7 @@ public interface PipelineStateManager {
*/
@Replicate
void removePipeline(HddsProtos.PipelineID pipelineIDProto)
- throws IOException;
+ throws IOException, TimeoutException;
/**
* Updating pipeline state would be replicated to Ratis.
@@ -59,11 +61,8 @@ void removePipeline(HddsProtos.PipelineID pipelineIDProto)
* @throws IOException
*/
@Replicate
- void updatePipelineState(
- HddsProtos.PipelineID pipelineIDProto,
- HddsProtos.PipelineState newState
- )
- throws IOException;
+ void updatePipelineState(HddsProtos.PipelineID pipelineIDProto,
+ HddsProtos.PipelineState newState) throws IOException, TimeoutException;
void addContainerToPipeline(
PipelineID pipelineID,
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableContainerFactory.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableContainerFactory.java
index 7d25ee98d3bd..4c2fe4d8c593 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableContainerFactory.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableContainerFactory.java
@@ -25,6 +25,7 @@
import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
import java.io.IOException;
+import java.util.concurrent.TimeoutException;
/**
* Factory class to obtain a container to which a block can be allocated for
@@ -48,7 +49,7 @@ public WritableContainerFactory(StorageContainerManager scm) {
public ContainerInfo getContainer(final long size,
ReplicationConfig repConfig, String owner, ExcludeList excludeList)
- throws IOException {
+ throws IOException, TimeoutException {
switch (repConfig.getReplicationType()) {
case STAND_ALONE:
return standaloneProvider
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableContainerProvider.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableContainerProvider.java
index d5d17769ab90..6a37610129ba 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableContainerProvider.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableContainerProvider.java
@@ -23,6 +23,7 @@
import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
import java.io.IOException;
+import java.util.concurrent.TimeoutException;
/**
* Interface used by the WritableContainerFactory to obtain a writable container
@@ -52,6 +53,7 @@ public interface WritableContainerProvider {
* @throws IOException
*/
ContainerInfo getContainer(long size, T repConfig,
- String owner, ExcludeList excludeList) throws IOException;
+ String owner, ExcludeList excludeList)
+ throws IOException, TimeoutException;
}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableECContainerProvider.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableECContainerProvider.java
index 7e8e3d1e1140..76e31d152414 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableECContainerProvider.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableECContainerProvider.java
@@ -42,6 +42,7 @@
import java.util.Collections;
import java.util.List;
import java.util.NavigableSet;
+import java.util.concurrent.TimeoutException;
import static org.apache.hadoop.hdds.conf.StorageUnit.BYTES;
@@ -91,7 +92,7 @@ public WritableECContainerProvider(ConfigurationSource conf,
@Override
public ContainerInfo getContainer(final long size,
ECReplicationConfig repConfig, String owner, ExcludeList excludeList)
- throws IOException {
+ throws IOException, TimeoutException {
// Bound this at a minimum of 1 byte in case a request is made for a very
// small size, which when divided by EC DataNum is zero.
long requiredSpace = Math.max(1, size / repConfig.getData());
@@ -162,7 +163,8 @@ public ContainerInfo getContainer(final long size,
}
private ContainerInfo allocateContainer(ReplicationConfig repConfig,
- long size, String owner, ExcludeList excludeList) throws IOException {
+ long size, String owner, ExcludeList excludeList)
+ throws IOException, TimeoutException {
List excludedNodes = Collections.emptyList();
if (excludeList.getDatanodes().size() > 0) {
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableRatisContainerProvider.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableRatisContainerProvider.java
index a113a0cd140f..c423177e8469 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableRatisContainerProvider.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableRatisContainerProvider.java
@@ -34,6 +34,7 @@
import java.io.IOException;
import java.util.List;
+import java.util.concurrent.TimeoutException;
import java.util.stream.Collectors;
/**
@@ -64,7 +65,7 @@ public WritableRatisContainerProvider(ConfigurationSource conf,
@Override
public ContainerInfo getContainer(final long size,
ReplicationConfig repConfig, String owner, ExcludeList excludeList)
- throws IOException {
+ throws IOException, TimeoutException {
/*
Here is the high level logic.
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMBlockProtocolServer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMBlockProtocolServer.java
index c5a80d95f122..de61874542fe 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMBlockProtocolServer.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMBlockProtocolServer.java
@@ -26,6 +26,7 @@
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
+import java.util.concurrent.TimeoutException;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.hdds.client.BlockID;
@@ -182,7 +183,6 @@ public List allocateBlock(
auditMap.put("replication", replicationConfig.toString());
auditMap.put("owner", owner);
List blocks = new ArrayList<>(num);
- boolean auditSuccess = true;
if (LOG.isDebugEnabled()) {
LOG.debug("Allocating {} blocks of size {}, with {}",
@@ -196,19 +196,17 @@ public List allocateBlock(
blocks.add(block);
}
}
+ AUDIT.logWriteSuccess(buildAuditMessageForSuccess(
+ SCMAction.ALLOCATE_BLOCK, auditMap));
return blocks;
+ } catch (TimeoutException ex) {
+ AUDIT.logWriteFailure(buildAuditMessageForFailure(
+ SCMAction.ALLOCATE_BLOCK, auditMap, ex));
+ throw new IOException(ex);
} catch (Exception ex) {
- auditSuccess = false;
- AUDIT.logWriteFailure(
- buildAuditMessageForFailure(SCMAction.ALLOCATE_BLOCK, auditMap, ex)
- );
+ AUDIT.logWriteFailure(buildAuditMessageForFailure(
+ SCMAction.ALLOCATE_BLOCK, auditMap, ex));
throw ex;
- } finally {
- if (auditSuccess) {
- AUDIT.logWriteSuccess(
- buildAuditMessageForSuccess(SCMAction.ALLOCATE_BLOCK, auditMap)
- );
- }
}
}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java
index 533e26a4b3af..8ccd343e2436 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java
@@ -91,6 +91,7 @@
import java.util.Optional;
import java.util.Set;
import java.util.TreeSet;
+import java.util.concurrent.TimeoutException;
import java.util.stream.Collectors;
import java.util.stream.Stream;
@@ -200,14 +201,18 @@ public ContainerWithPipeline allocateContainer(HddsProtos.ReplicationType
ResultCodes.SAFE_MODE_EXCEPTION);
}
getScm().checkAdminAccess(getRemoteUser());
-
- final ContainerInfo container = scm.getContainerManager()
- .allocateContainer(
- ReplicationConfig.fromProtoTypeAndFactor(replicationType, factor),
- owner);
- final Pipeline pipeline = scm.getPipelineManager()
- .getPipeline(container.getPipelineID());
- return new ContainerWithPipeline(container, pipeline);
+ try {
+ final ContainerInfo container = scm.getContainerManager()
+ .allocateContainer(
+ ReplicationConfig.fromProtoTypeAndFactor(replicationType, factor),
+ owner);
+ final Pipeline pipeline = scm.getPipelineManager()
+ .getPipeline(container.getPipelineID());
+ return new ContainerWithPipeline(container, pipeline);
+ } catch (TimeoutException e) {
+ throw new SCMException("Allocate Container TimeoutException",
+ ResultCodes.INTERNAL_ERROR);
+ }
}
@Override
@@ -545,7 +550,6 @@ public List listContainer(long startContainerID,
@Override
public void deleteContainer(long containerID) throws IOException {
- boolean auditSuccess = true;
Map auditMap = Maps.newHashMap();
auditMap.put("containerID", String.valueOf(containerID));
UserGroupInformation remoteUser = getRemoteUser();
@@ -554,18 +558,17 @@ public void deleteContainer(long containerID) throws IOException {
getScm().checkAdminAccess(remoteUser);
scm.getContainerManager().deleteContainer(
ContainerID.valueOf(containerID));
+ AUDIT.logWriteSuccess(buildAuditMessageForSuccess(
+ SCMAction.DELETE_CONTAINER, auditMap));
+ } catch (TimeoutException ex) {
+ AUDIT.logWriteFailure(buildAuditMessageForFailure(
+ SCMAction.DELETE_CONTAINER, auditMap, ex));
+ throw new SCMException("Delete Container TimeoutException",
+ ResultCodes.INTERNAL_ERROR);
} catch (Exception ex) {
- auditSuccess = false;
- AUDIT.logWriteFailure(
- buildAuditMessageForFailure(SCMAction.DELETE_CONTAINER, auditMap, ex)
- );
+ AUDIT.logWriteFailure(buildAuditMessageForFailure(
+ SCMAction.DELETE_CONTAINER, auditMap, ex));
throw ex;
- } finally {
- if (auditSuccess) {
- AUDIT.logWriteSuccess(
- buildAuditMessageForSuccess(SCMAction.DELETE_CONTAINER, auditMap)
- );
- }
}
}
@@ -663,11 +666,18 @@ public void closeContainer(long containerID) throws IOException {
public Pipeline createReplicationPipeline(HddsProtos.ReplicationType type,
HddsProtos.ReplicationFactor factor, HddsProtos.NodePool nodePool)
throws IOException {
- Pipeline result = scm.getPipelineManager()
- .createPipeline(ReplicationConfig.fromProtoTypeAndFactor(type, factor));
- AUDIT.logWriteSuccess(
- buildAuditMessageForSuccess(SCMAction.CREATE_PIPELINE, null));
- return result;
+ try {
+ Pipeline result = scm.getPipelineManager().createPipeline(
+ ReplicationConfig.fromProtoTypeAndFactor(type, factor));
+ AUDIT.logWriteSuccess(buildAuditMessageForSuccess(
+ SCMAction.CREATE_PIPELINE, null));
+ return result;
+ } catch (TimeoutException ex) {
+ AUDIT.logWriteFailure(buildAuditMessageForFailure(
+ SCMAction.CREATE_PIPELINE, null, ex));
+ throw new SCMException("Create Pipeline TimeoutException",
+ ResultCodes.INTERNAL_ERROR);
+ }
}
@Override
@@ -687,35 +697,55 @@ public Pipeline getPipeline(HddsProtos.PipelineID pipelineID)
@Override
public void activatePipeline(HddsProtos.PipelineID pipelineID)
throws IOException {
- AUDIT.logWriteSuccess(buildAuditMessageForSuccess(
- SCMAction.ACTIVATE_PIPELINE, null));
- scm.getPipelineManager().activatePipeline(
- PipelineID.getFromProtobuf(pipelineID));
+ try {
+ scm.getPipelineManager().activatePipeline(
+ PipelineID.getFromProtobuf(pipelineID));
+ AUDIT.logWriteSuccess(buildAuditMessageForSuccess(
+ SCMAction.ACTIVATE_PIPELINE, null));
+ } catch (TimeoutException ex) {
+ AUDIT.logWriteFailure(buildAuditMessageForFailure(
+ SCMAction.ACTIVATE_PIPELINE, null, ex));
+ throw new SCMException("Activate Pipeline TimeoutException",
+ ResultCodes.INTERNAL_ERROR);
+ }
}
@Override
public void deactivatePipeline(HddsProtos.PipelineID pipelineID)
throws IOException {
- getScm().checkAdminAccess(getRemoteUser());
- AUDIT.logWriteSuccess(buildAuditMessageForSuccess(
- SCMAction.DEACTIVATE_PIPELINE, null));
- scm.getPipelineManager().deactivatePipeline(
- PipelineID.getFromProtobuf(pipelineID));
+ try {
+ getScm().checkAdminAccess(getRemoteUser());
+ scm.getPipelineManager().deactivatePipeline(
+ PipelineID.getFromProtobuf(pipelineID));
+ AUDIT.logWriteSuccess(buildAuditMessageForSuccess(
+ SCMAction.DEACTIVATE_PIPELINE, null));
+ } catch (TimeoutException ex) {
+ AUDIT.logWriteFailure(buildAuditMessageForFailure(
+ SCMAction.DEACTIVATE_PIPELINE, null, ex));
+ throw new SCMException("DeActivate Pipeline TimeoutException",
+ ResultCodes.INTERNAL_ERROR);
+ }
}
@Override
public void closePipeline(HddsProtos.PipelineID pipelineID)
throws IOException {
- getScm().checkAdminAccess(getRemoteUser());
Map auditMap = Maps.newHashMap();
- auditMap.put("pipelineID", pipelineID.getId());
- PipelineManager pipelineManager = scm.getPipelineManager();
- Pipeline pipeline =
- pipelineManager.getPipeline(PipelineID.getFromProtobuf(pipelineID));
- pipelineManager.closePipeline(pipeline, true);
- AUDIT.logWriteSuccess(
- buildAuditMessageForSuccess(SCMAction.CLOSE_PIPELINE, null)
- );
+ try {
+ getScm().checkAdminAccess(getRemoteUser());
+ auditMap.put("pipelineID", pipelineID.getId());
+ PipelineManager pipelineManager = scm.getPipelineManager();
+ Pipeline pipeline =
+ pipelineManager.getPipeline(PipelineID.getFromProtobuf(pipelineID));
+ pipelineManager.closePipeline(pipeline, true);
+ AUDIT.logWriteSuccess(buildAuditMessageForSuccess(
+ SCMAction.CLOSE_PIPELINE, auditMap));
+ } catch (TimeoutException ex) {
+ AUDIT.logWriteFailure(buildAuditMessageForFailure(
+ SCMAction.CLOSE_PIPELINE, auditMap, ex));
+ throw new SCMException("Close Pipeline TimeoutException",
+ ResultCodes.INTERNAL_ERROR);
+ }
}
@Override
@@ -914,7 +944,7 @@ public StartContainerBalancerResponseProto startContainerBalancer(
try {
containerBalancer.startBalancer(cbc);
} catch (IllegalContainerBalancerStateException | IOException |
- InvalidContainerBalancerConfigurationException e) {
+ InvalidContainerBalancerConfigurationException | TimeoutException e) {
AUDIT.logWriteFailure(buildAuditMessageForFailure(
SCMAction.START_CONTAINER_BALANCER, null, e));
return StartContainerBalancerResponseProto.newBuilder()
@@ -936,7 +966,7 @@ public void stopContainerBalancer() throws IOException {
scm.getContainerBalancer().stopBalancer();
AUDIT.logWriteSuccess(buildAuditMessageForSuccess(
SCMAction.STOP_CONTAINER_BALANCER, null));
- } catch (IllegalContainerBalancerStateException e) {
+ } catch (IllegalContainerBalancerStateException | TimeoutException e) {
AUDIT.logWriteFailure(buildAuditMessageForFailure(
SCMAction.STOP_CONTAINER_BALANCER, null, e));
}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeProtocolServer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeProtocolServer.java
index 625a59e0ec0c..f2654fac027b 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeProtocolServer.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeProtocolServer.java
@@ -26,6 +26,7 @@
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
+import java.util.concurrent.TimeoutException;
import java.util.stream.Collectors;
import org.apache.hadoop.fs.CommonConfigurationKeys;
@@ -268,7 +269,7 @@ public static SCMRegisteredResponseProto getRegisteredResponse(
@Override
public SCMHeartbeatResponseProto sendHeartbeat(
- SCMHeartbeatRequestProto heartbeat) throws IOException {
+ SCMHeartbeatRequestProto heartbeat) throws IOException, TimeoutException {
List cmdResponses = new ArrayList<>();
for (SCMCommand cmd : heartbeatDispatcher.dispatch(heartbeat)) {
cmdResponses.add(getCommandResponse(cmd));
@@ -305,7 +306,7 @@ public SCMHeartbeatResponseProto sendHeartbeat(
*/
@VisibleForTesting
public SCMCommandProto getCommandResponse(SCMCommand cmd)
- throws IOException {
+ throws IOException, TimeoutException {
SCMCommandProto.Builder builder = SCMCommandProto.newBuilder()
.setEncodedToken(cmd.getEncodedToken());
@@ -331,6 +332,11 @@ public SCMCommandProto getCommandResponse(SCMCommand cmd)
.stream()
.map(tx -> tx.getTxID())
.collect(Collectors.toList());
+ /*
+ * TODO: Can we avoid this?
+ * This introduces a Ratis call while processing datanode heartbeat,
+ * which is not good.
+ */
scm.getScmBlockManager().getDeletedBlockLog().incrementCount(txs);
return builder
.setCommandType(deleteBlocksCommand)
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/upgrade/FinalizationStateManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/upgrade/FinalizationStateManager.java
index 362ea77168c8..44868715f889 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/upgrade/FinalizationStateManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/upgrade/FinalizationStateManager.java
@@ -19,6 +19,7 @@
import org.apache.hadoop.hdds.scm.metadata.Replicate;
import java.io.IOException;
+import java.util.concurrent.TimeoutException;
/**
* Manages the state of finalization in SCM.
@@ -26,13 +27,14 @@
public interface FinalizationStateManager {
@Replicate
- void addFinalizingMark() throws IOException;
+ void addFinalizingMark() throws IOException, TimeoutException;
@Replicate
- void removeFinalizingMark() throws IOException;
+ void removeFinalizingMark() throws IOException, TimeoutException;
@Replicate
- void finalizeLayoutFeature(Integer layoutVersion) throws IOException;
+ void finalizeLayoutFeature(Integer layoutVersion)
+ throws IOException, TimeoutException;
/**
* @param query The checkpoint to check for being crossed.
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/upgrade/SCMUpgradeFinalizer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/upgrade/SCMUpgradeFinalizer.java
index b80c9d260ccd..fa3520ee38ac 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/upgrade/SCMUpgradeFinalizer.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/upgrade/SCMUpgradeFinalizer.java
@@ -21,6 +21,7 @@
import static org.apache.hadoop.hdds.scm.pipeline.Pipeline.PipelineState.CLOSED;
import java.io.IOException;
+import java.util.concurrent.TimeoutException;
import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
@@ -74,17 +75,22 @@ protected boolean componentFinishedFinalizationSteps(
@Override
public void preFinalizeUpgrade(SCMUpgradeFinalizationContext context)
throws IOException {
- FinalizationStateManager stateManager =
- context.getFinalizationStateManager();
- if (!stateManager.crossedCheckpoint(
- FinalizationCheckpoint.FINALIZATION_STARTED)) {
- context.getFinalizationStateManager().addFinalizingMark();
- }
- logCheckpointCrossed(FinalizationCheckpoint.FINALIZATION_STARTED);
+ try {
+ FinalizationStateManager stateManager =
+ context.getFinalizationStateManager();
+ if (!stateManager.crossedCheckpoint(
+ FinalizationCheckpoint.FINALIZATION_STARTED)) {
+ context.getFinalizationStateManager().addFinalizingMark();
+ }
+ logCheckpointCrossed(FinalizationCheckpoint.FINALIZATION_STARTED);
- if (!stateManager.crossedCheckpoint(
- FinalizationCheckpoint.MLV_EQUALS_SLV)) {
- closePipelinesBeforeFinalization(context.getPipelineManager());
+ if (!stateManager.crossedCheckpoint(
+ FinalizationCheckpoint.MLV_EQUALS_SLV)) {
+ closePipelinesBeforeFinalization(context.getPipelineManager());
+ }
+ } catch (TimeoutException ex) {
+ LOG.error("TimeoutException during preFinalizeUpgrade", ex);
+ throw new IOException(ex);
}
}
@@ -96,7 +102,7 @@ public void finalizeLayoutFeature(LayoutFeature lf,
try {
context.getFinalizationStateManager()
.finalizeLayoutFeature(lf.layoutVersion());
- } catch (IOException ex) {
+ } catch (IOException | TimeoutException ex) {
throw new UpgradeException(ex,
UpgradeException.ResultCodes.LAYOUT_FEATURE_FINALIZATION_FAILED);
}
@@ -129,17 +135,22 @@ void replicatedFinalizationSteps(HDDSLayoutFeature lf,
public void postFinalizeUpgrade(SCMUpgradeFinalizationContext context)
throws IOException {
- // If we reached this phase of finalization, all layout features should
- // be finalized.
- logCheckpointCrossed(FinalizationCheckpoint.MLV_EQUALS_SLV);
- FinalizationStateManager stateManager =
- context.getFinalizationStateManager();
- if (!stateManager.crossedCheckpoint(
- FinalizationCheckpoint.FINALIZATION_COMPLETE)) {
- createPipelinesAfterFinalization(context.getPipelineManager());
- stateManager.removeFinalizingMark();
+ try {
+ // If we reached this phase of finalization, all layout features should
+ // be finalized.
+ logCheckpointCrossed(FinalizationCheckpoint.MLV_EQUALS_SLV);
+ FinalizationStateManager stateManager =
+ context.getFinalizationStateManager();
+ if (!stateManager.crossedCheckpoint(
+ FinalizationCheckpoint.FINALIZATION_COMPLETE)) {
+ createPipelinesAfterFinalization(context.getPipelineManager());
+ stateManager.removeFinalizingMark();
+ }
+ logCheckpointCrossed(FinalizationCheckpoint.FINALIZATION_COMPLETE);
+ } catch (TimeoutException ex) {
+ LOG.error("TimeoutException during postFinalizeUpgrade", ex);
+ throw new IOException(ex);
}
- logCheckpointCrossed(FinalizationCheckpoint.FINALIZATION_COMPLETE);
}
@Override
@@ -150,7 +161,7 @@ public void runPrefinalizeStateActions(Storage storage,
}
private void closePipelinesBeforeFinalization(PipelineManager pipelineManager)
- throws IOException {
+ throws IOException, TimeoutException {
/*
* Before we can call finalize the feature, we need to make sure that
* all existing pipelines are closed and pipeline Manger would freeze
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/HddsTestUtils.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/HddsTestUtils.java
index 3d0dc963e828..bf7cd07f2575 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/HddsTestUtils.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/HddsTestUtils.java
@@ -89,6 +89,7 @@
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeoutException;
/**
* Stateless helper functions for Hdds tests.
@@ -397,7 +398,7 @@ public static PipelineReportFromDatanode getPipelineReportFromDatanode(
}
public static void openAllRatisPipelines(PipelineManager pipelineManager)
- throws IOException {
+ throws IOException, TimeoutException {
// Pipeline is created by background thread
for (ReplicationFactor factor : ReplicationFactor.values()) {
// Trigger the processed pipeline report event
@@ -532,7 +533,7 @@ public static CRLStatusReport createCRLStatusReport(
public static org.apache.hadoop.hdds.scm.container.ContainerInfo
allocateContainer(ContainerManager containerManager)
- throws IOException {
+ throws IOException, TimeoutException {
return containerManager
.allocateContainer(RatisReplicationConfig
.getInstance(ReplicationFactor.THREE),
@@ -541,7 +542,8 @@ public static CRLStatusReport createCRLStatusReport(
}
public static void closeContainer(ContainerManager containerManager,
- ContainerID id) throws IOException, InvalidStateTransitionException {
+ ContainerID id) throws IOException,
+ InvalidStateTransitionException, TimeoutException {
containerManager.updateContainerState(
id, HddsProtos.LifeCycleEvent.FINALIZE);
containerManager.updateContainerState(
@@ -556,7 +558,8 @@ public static void closeContainer(ContainerManager containerManager,
* @throws IOException
*/
public static void quasiCloseContainer(ContainerManager containerManager,
- ContainerID id) throws IOException, InvalidStateTransitionException {
+ ContainerID id) throws IOException,
+ InvalidStateTransitionException, TimeoutException {
containerManager.updateContainerState(
id, HddsProtos.LifeCycleEvent.FINALIZE);
containerManager.updateContainerState(
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java
index 59916637018d..908a21e162d0 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java
@@ -257,7 +257,7 @@ public void testAllocateBlockInParallel() {
.allocateBlock(DEFAULT_BLOCK_SIZE, replicationConfig,
OzoneConsts.OZONE,
new ExcludeList()));
- } catch (IOException e) {
+ } catch (IOException | TimeoutException e) {
future.completeExceptionally(e);
}
return future;
@@ -307,7 +307,7 @@ public void testBlockDistribution() throws Exception {
blockList.add(block);
allocatedBlockMap.put(containerId, blockList);
future.complete(block);
- } catch (IOException e) {
+ } catch (IOException | TimeoutException e) {
future.completeExceptionally(e);
}
return future;
@@ -368,7 +368,7 @@ public void testBlockDistributionWithMultipleDisks() throws Exception {
blockList.add(block);
allocatedBlockMap.put(containerId, blockList);
future.complete(block);
- } catch (IOException e) {
+ } catch (IOException | TimeoutException e) {
future.completeExceptionally(e);
}
return future;
@@ -436,7 +436,7 @@ public void testBlockDistributionWithMultipleRaftLogDisks() throws Exception {
blockList.add(block);
allocatedBlockMap.put(containerId, blockList);
future.complete(block);
- } catch (IOException e) {
+ } catch (IOException | TimeoutException e) {
future.completeExceptionally(e);
}
return future;
@@ -515,7 +515,7 @@ public void testMultipleBlockAllocation()
new ExcludeList());
return !block.getPipeline().getId()
.equals(allocatedBlock.getPipeline().getId());
- } catch (IOException e) {
+ } catch (IOException | TimeoutException e) {
}
return false;
}, 100, 1000);
@@ -560,7 +560,7 @@ public void testMultipleBlockAllocationWithClosedContainer()
.allocateBlock(DEFAULT_BLOCK_SIZE, replicationConfig,
OzoneConsts.OZONE,
new ExcludeList());
- } catch (IOException e) {
+ } catch (IOException | TimeoutException e) {
}
return verifyNumberOfContainersInPipelines(
numContainerPerOwnerInPipeline);
@@ -585,7 +585,7 @@ public void testMultipleBlockAllocationWithClosedContainer()
.allocateBlock(DEFAULT_BLOCK_SIZE, replicationConfig,
OzoneConsts.OZONE,
new ExcludeList());
- } catch (IOException e) {
+ } catch (IOException | TimeoutException e) {
}
return verifyNumberOfContainersInPipelines(
numContainerPerOwnerInPipeline);
@@ -595,7 +595,7 @@ public void testMultipleBlockAllocationWithClosedContainer()
@Test
@Timeout(100)
public void testBlockAllocationWithNoAvailablePipelines()
- throws IOException {
+ throws IOException, TimeoutException {
for (Pipeline pipeline : pipelineManager.getPipelines()) {
pipelineManager.closePipeline(pipeline, false);
}
@@ -618,7 +618,7 @@ public void onMessage(final CommandForDatanode command,
(CreatePipelineCommand) command.getCommand();
try {
pipelineManager.openPipeline(createCommand.getPipelineID());
- } catch (IOException e) {
+ } catch (IOException | TimeoutException e) {
}
}
}
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestDeletedBlockLog.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestDeletedBlockLog.java
index de164cd2a9d3..1a5a48390c1b 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestDeletedBlockLog.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestDeletedBlockLog.java
@@ -65,6 +65,7 @@
import java.util.Random;
import java.util.Set;
import java.util.UUID;
+import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.stream.Collectors;
@@ -209,14 +210,15 @@ private Map> generateData(int dataSize) throws IOException {
private void addTransactions(Map> containerBlocksMap,
boolean shouldFlush)
- throws IOException {
+ throws IOException, TimeoutException {
deletedBlockLog.addTransactions(containerBlocksMap);
if (shouldFlush) {
scmHADBTransactionBuffer.flush();
}
}
- private void incrementCount(List txIDs) throws IOException {
+ private void incrementCount(List txIDs)
+ throws IOException, TimeoutException {
deletedBlockLog.incrementCount(txIDs);
scmHADBTransactionBuffer.flush();
}
@@ -262,7 +264,7 @@ private DeleteBlockTransactionResult createDeleteBlockTransactionResult(
}
private List getTransactions(
- int maximumAllowedBlocksNum) throws IOException {
+ int maximumAllowedBlocksNum) throws IOException, TimeoutException {
DatanodeDeletedBlockTransactions transactions =
deletedBlockLog.getTransactions(maximumAllowedBlocksNum);
List txns = new LinkedList<>();
@@ -432,7 +434,8 @@ public void testPersistence() throws Exception {
}
@Test
- public void testDeletedBlockTransactions() throws IOException {
+ public void testDeletedBlockTransactions()
+ throws IOException, TimeoutException {
int txNum = 10;
List blocks;
DatanodeDetails dnId1 = dnList.get(0), dnId2 = dnList.get(1);
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestCloseContainerEventHandler.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestCloseContainerEventHandler.java
index 449c6f7e8db0..d80265696828 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestCloseContainerEventHandler.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestCloseContainerEventHandler.java
@@ -22,6 +22,7 @@
import java.util.List;
import java.util.Set;
import java.util.UUID;
+import java.util.concurrent.TimeoutException;
import java.util.stream.Collectors;
import org.apache.hadoop.hdds.client.ECReplicationConfig;
@@ -113,19 +114,19 @@ public void testCloseContainerInInvalidState()
@Test
public void testCloseContainerEventWithRatisContainers()
- throws IOException, InvalidStateTransitionException {
+ throws IOException, InvalidStateTransitionException, TimeoutException {
closeContainerForValidContainer(RATIS_REP_CONFIG, 3, false);
}
@Test
public void testCloseContainerEventECContainer()
- throws InvalidStateTransitionException, IOException {
+ throws InvalidStateTransitionException, IOException, TimeoutException {
closeContainerForValidContainer(EC_REP_CONFIG, 5, true);
}
private void closeContainerForValidContainer(ReplicationConfig repConfig,
int nodeCount, boolean forceClose)
- throws IOException, InvalidStateTransitionException {
+ throws IOException, InvalidStateTransitionException, TimeoutException {
final Pipeline pipeline = createPipeline(repConfig, nodeCount);
final ContainerInfo container =
createContainer(repConfig, pipeline.getId());
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerManagerImpl.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerManagerImpl.java
index dbf4974ae1cb..7c0a59e070fe 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerManagerImpl.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerManagerImpl.java
@@ -20,6 +20,7 @@
import java.io.File;
import java.io.IOException;
import java.util.UUID;
+import java.util.concurrent.TimeoutException;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdds.HddsConfigKeys;
@@ -189,7 +190,8 @@ public void testAllocateContainersWithECReplicationConfig() throws Exception {
}
@Test
- public void testUpdateContainerReplicaInvokesPendingOp() throws IOException {
+ public void testUpdateContainerReplicaInvokesPendingOp()
+ throws IOException, TimeoutException {
final ContainerInfo container = containerManager.allocateContainer(
RatisReplicationConfig.getInstance(
ReplicationFactor.THREE), "admin");
@@ -209,7 +211,8 @@ public void testUpdateContainerReplicaInvokesPendingOp() throws IOException {
}
@Test
- public void testRemoveContainerReplicaInvokesPendingOp() throws IOException {
+ public void testRemoveContainerReplicaInvokesPendingOp()
+ throws IOException, TimeoutException {
final ContainerInfo container = containerManager.allocateContainer(
RatisReplicationConfig.getInstance(
ReplicationFactor.THREE), "admin");
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerReportHandler.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerReportHandler.java
index 623b1511e11c..894276008eda 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerReportHandler.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerReportHandler.java
@@ -59,6 +59,7 @@
import java.util.Map;
import java.util.Set;
import java.util.UUID;
+import java.util.concurrent.TimeoutException;
import java.util.stream.Collectors;
import java.util.stream.Stream;
@@ -82,7 +83,8 @@ public class TestContainerReportHandler {
private PipelineManager pipelineManager;
@BeforeEach
- public void setup() throws IOException, InvalidStateTransitionException {
+ public void setup() throws IOException, InvalidStateTransitionException,
+ TimeoutException {
final OzoneConfiguration conf = SCMTestUtils.getConf();
nodeManager = new MockNodeManager(true, 10);
containerManager = Mockito.mock(ContainerManager.class);
@@ -155,7 +157,7 @@ public void tearDown() throws Exception {
@Test
public void testUnderReplicatedContainer()
- throws NodeNotFoundException, IOException {
+ throws NodeNotFoundException, IOException, TimeoutException {
final ContainerReportHandler reportHandler = new ContainerReportHandler(
nodeManager, containerManager);
final Iterator nodeIterator = nodeManager.getNodes(
@@ -210,7 +212,7 @@ public void testUnderReplicatedContainer()
@Test
public void testOverReplicatedContainer() throws NodeNotFoundException,
- IOException {
+ IOException, TimeoutException {
final ContainerReportHandler reportHandler = new ContainerReportHandler(
nodeManager, containerManager);
@@ -269,7 +271,8 @@ public void testOverReplicatedContainer() throws NodeNotFoundException,
@Test
- public void testClosingToClosed() throws NodeNotFoundException, IOException {
+ public void testClosingToClosed() throws NodeNotFoundException, IOException,
+ TimeoutException {
/*
* The container is in CLOSING state and all the replicas are in
* OPEN/CLOSING state.
@@ -339,7 +342,7 @@ public void testClosingToClosed() throws NodeNotFoundException, IOException {
@Test
public void testClosingToQuasiClosed()
- throws NodeNotFoundException, IOException {
+ throws NodeNotFoundException, IOException, TimeoutException {
/*
* The container is in CLOSING state and all the replicas are in
* OPEN/CLOSING state.
@@ -407,7 +410,7 @@ public void testClosingToQuasiClosed()
@Test
public void testQuasiClosedToClosed()
- throws NodeNotFoundException, IOException {
+ throws NodeNotFoundException, IOException, TimeoutException {
/*
* The container is in QUASI_CLOSED state.
* - One of the replica is in QUASI_CLOSED state
@@ -479,7 +482,7 @@ public void testQuasiClosedToClosed()
@Test
public void openContainerKeyAndBytesUsedUpdatedToMinimumOfAllReplicas()
- throws IOException {
+ throws IOException, TimeoutException {
final ContainerReportHandler reportHandler = new ContainerReportHandler(
nodeManager, containerManager);
final Iterator nodeIterator = nodeManager.getNodes(
@@ -561,7 +564,7 @@ public void openContainerKeyAndBytesUsedUpdatedToMinimumOfAllReplicas()
@Test
public void notOpenContainerKeyAndBytesUsedUpdatedToMaximumOfAllReplicas()
- throws IOException {
+ throws IOException, TimeoutException {
final ContainerReportHandler reportHandler = new ContainerReportHandler(
nodeManager, containerManager);
final Iterator nodeIterator = nodeManager.getNodes(
@@ -637,7 +640,7 @@ public void notOpenContainerKeyAndBytesUsedUpdatedToMaximumOfAllReplicas()
@Test
public void openECContainerKeyAndBytesUsedUpdatedToMinimumOfAllReplicas()
- throws IOException {
+ throws IOException, TimeoutException {
final ECReplicationConfig repConfig = new ECReplicationConfig(3, 2);
final ContainerReportHandler reportHandler = new ContainerReportHandler(
nodeManager, containerManager);
@@ -713,7 +716,7 @@ public void openECContainerKeyAndBytesUsedUpdatedToMinimumOfAllReplicas()
@Test
public void closedECContainerKeyAndBytesUsedUpdatedToMinimumOfAllReplicas()
- throws IOException {
+ throws IOException, TimeoutException {
final ECReplicationConfig repConfig = new ECReplicationConfig(3, 2);
final ContainerReportHandler reportHandler = new ContainerReportHandler(
nodeManager, containerManager);
@@ -789,7 +792,7 @@ public void closedECContainerKeyAndBytesUsedUpdatedToMinimumOfAllReplicas()
@Test
public void testStaleReplicaOfDeletedContainer() throws NodeNotFoundException,
- IOException {
+ IOException, TimeoutException {
final ContainerReportHandler reportHandler = new ContainerReportHandler(
nodeManager, containerManager);
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerStateManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerStateManager.java
index 299e11c1c29a..e731e90b3381 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerStateManager.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerStateManager.java
@@ -22,6 +22,7 @@
import java.util.ArrayList;
import java.util.Set;
import java.util.UUID;
+import java.util.concurrent.TimeoutException;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdds.HddsConfigKeys;
@@ -65,7 +66,7 @@ public class TestContainerStateManager {
private Pipeline pipeline;
@BeforeEach
- public void init() throws IOException {
+ public void init() throws IOException, TimeoutException {
OzoneConfiguration conf = new OzoneConfiguration();
scmhaManager = SCMHAManagerStub.getInstance(true);
testDir = GenericTestUtils.getTestDir(
@@ -106,7 +107,8 @@ public void tearDown() throws Exception {
}
@Test
- public void checkReplicationStateOK() throws IOException {
+ public void checkReplicationStateOK()
+ throws IOException, TimeoutException {
//GIVEN
ContainerInfo c1 = allocateContainer();
@@ -127,7 +129,8 @@ public void checkReplicationStateOK() throws IOException {
}
@Test
- public void checkReplicationStateMissingReplica() throws IOException {
+ public void checkReplicationStateMissingReplica()
+ throws IOException, TimeoutException {
//GIVEN
ContainerInfo c1 = allocateContainer();
@@ -156,7 +159,8 @@ private void addReplica(ContainerInfo cont, DatanodeDetails node) {
.updateContainerReplica(cont.containerID(), replica);
}
- private ContainerInfo allocateContainer() throws IOException {
+ private ContainerInfo allocateContainer()
+ throws IOException, TimeoutException {
final ContainerInfo containerInfo = new ContainerInfo.Builder()
.setState(HddsProtos.LifeCycleState.OPEN)
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestIncrementalContainerReportHandler.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestIncrementalContainerReportHandler.java
index 756fb3474672..c7c7b6726088 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestIncrementalContainerReportHandler.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestIncrementalContainerReportHandler.java
@@ -66,6 +66,7 @@
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeoutException;
import static org.apache.hadoop.hdds.protocol.MockDatanodeDetails.randomDatanodeDetails;
import static org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto.State.CLOSED;
@@ -90,7 +91,8 @@ public class TestIncrementalContainerReportHandler {
private SCMHAManager scmhaManager;
@BeforeEach
- public void setup() throws IOException, InvalidStateTransitionException {
+ public void setup() throws IOException, InvalidStateTransitionException,
+ TimeoutException {
final OzoneConfiguration conf = new OzoneConfiguration();
final String path =
GenericTestUtils.getTempPath(UUID.randomUUID().toString());
@@ -183,7 +185,7 @@ public void tearDown() throws Exception {
@Test
- public void testClosingToClosed() throws IOException {
+ public void testClosingToClosed() throws IOException, TimeoutException {
final IncrementalContainerReportHandler reportHandler =
new IncrementalContainerReportHandler(
nodeManager, containerManager, scmContext);
@@ -216,7 +218,7 @@ public void testClosingToClosed() throws IOException {
}
@Test
- public void testClosingToQuasiClosed() throws IOException {
+ public void testClosingToQuasiClosed() throws IOException, TimeoutException {
final IncrementalContainerReportHandler reportHandler =
new IncrementalContainerReportHandler(
nodeManager, containerManager, scmContext);
@@ -250,7 +252,7 @@ public void testClosingToQuasiClosed() throws IOException {
}
@Test
- public void testQuasiClosedToClosed() throws IOException {
+ public void testQuasiClosedToClosed() throws IOException, TimeoutException {
final IncrementalContainerReportHandler reportHandler =
new IncrementalContainerReportHandler(
nodeManager, containerManager, scmContext);
@@ -287,7 +289,7 @@ public void testQuasiClosedToClosed() throws IOException {
}
@Test
- public void testDeleteContainer() throws IOException {
+ public void testDeleteContainer() throws IOException, TimeoutException {
final IncrementalContainerReportHandler reportHandler =
new IncrementalContainerReportHandler(
nodeManager, containerManager, scmContext);
@@ -324,7 +326,7 @@ public void testDeleteContainer() throws IOException {
// HDDS-5249 - This test reproduces the race condition mentioned in the Jira
// until the code was changed to fix the race condition.
public void testICRFCRRace() throws IOException, NodeNotFoundException,
- ExecutionException, InterruptedException {
+ ExecutionException, InterruptedException, TimeoutException {
final IncrementalContainerReportHandler reportHandler =
new IncrementalContainerReportHandler(
nodeManager, containerManager, scmContext);
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancer.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancer.java
index dd27e8f446ae..a6a11d8f440e 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancer.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancer.java
@@ -68,6 +68,7 @@
import java.util.Set;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeoutException;
import java.util.stream.Collectors;
import static org.mockito.Mockito.mock;
@@ -107,7 +108,8 @@ public class TestContainerBalancer {
* Sets up configuration values and creates a mock cluster.
*/
@BeforeEach
- public void setup() throws IOException, NodeNotFoundException {
+ public void setup() throws IOException, NodeNotFoundException,
+ TimeoutException {
conf = new OzoneConfiguration();
scm = Mockito.mock(StorageContainerManager.class);
containerManager = Mockito.mock(ContainerManager.class);
@@ -214,7 +216,7 @@ public void testCalculationOfUtilization() {
public void
initializeIterationShouldUpdateUnBalancedNodesWhenThresholdChanges()
throws IllegalContainerBalancerStateException, IOException,
- InvalidContainerBalancerConfigurationException {
+ InvalidContainerBalancerConfigurationException, TimeoutException {
List expectedUnBalancedNodes;
List unBalancedNodesAccordingToBalancer;
@@ -257,7 +259,7 @@ public void testCalculationOfUtilization() {
@Test
public void unBalancedNodesListShouldBeEmptyWhenClusterIsBalanced()
throws IllegalContainerBalancerStateException, IOException,
- InvalidContainerBalancerConfigurationException {
+ InvalidContainerBalancerConfigurationException, TimeoutException {
balancerConfiguration.setThreshold(99.99);
startBalancer(balancerConfiguration);
@@ -276,7 +278,7 @@ public void unBalancedNodesListShouldBeEmptyWhenClusterIsBalanced()
@Test
public void containerBalancerShouldObeyMaxDatanodesToInvolveLimit()
throws IllegalContainerBalancerStateException, IOException,
- InvalidContainerBalancerConfigurationException {
+ InvalidContainerBalancerConfigurationException, TimeoutException {
int percent = 20;
balancerConfiguration.setMaxDatanodesPercentageToInvolvePerIteration(
percent);
@@ -301,7 +303,7 @@ public void containerBalancerShouldObeyMaxDatanodesToInvolveLimit()
@Test
public void containerBalancerShouldSelectOnlyClosedContainers()
throws IllegalContainerBalancerStateException, IOException,
- InvalidContainerBalancerConfigurationException {
+ InvalidContainerBalancerConfigurationException, TimeoutException {
// make all containers open, balancer should not select any of them
for (ContainerInfo containerInfo : cidToInfoMap.values()) {
containerInfo.setState(HddsProtos.LifeCycleState.OPEN);
@@ -343,7 +345,7 @@ public void containerBalancerShouldSelectOnlyClosedContainers()
@Test
public void containerBalancerShouldObeyMaxSizeToMoveLimit()
throws IllegalContainerBalancerStateException, IOException,
- InvalidContainerBalancerConfigurationException {
+ InvalidContainerBalancerConfigurationException, TimeoutException {
balancerConfiguration.setThreshold(1);
balancerConfiguration.setMaxSizeToMovePerIteration(10 * OzoneConsts.GB);
balancerConfiguration.setIterations(1);
@@ -365,7 +367,7 @@ public void containerBalancerShouldObeyMaxSizeToMoveLimit()
@Test
public void targetDatanodeShouldNotAlreadyContainSelectedContainer()
throws IllegalContainerBalancerStateException, IOException,
- InvalidContainerBalancerConfigurationException {
+ InvalidContainerBalancerConfigurationException, TimeoutException {
balancerConfiguration.setThreshold(10);
balancerConfiguration.setMaxSizeToMovePerIteration(100 * OzoneConsts.GB);
balancerConfiguration.setMaxDatanodesPercentageToInvolvePerIteration(100);
@@ -394,7 +396,7 @@ public void targetDatanodeShouldNotAlreadyContainSelectedContainer()
@Test
public void containerMoveSelectionShouldFollowPlacementPolicy()
throws IllegalContainerBalancerStateException, IOException,
- InvalidContainerBalancerConfigurationException {
+ InvalidContainerBalancerConfigurationException, TimeoutException {
balancerConfiguration.setThreshold(10);
balancerConfiguration.setMaxSizeToMovePerIteration(50 * OzoneConsts.GB);
balancerConfiguration.setMaxDatanodesPercentageToInvolvePerIteration(100);
@@ -437,7 +439,8 @@ public void containerMoveSelectionShouldFollowPlacementPolicy()
@Test
public void targetDatanodeShouldBeInServiceHealthy()
throws NodeNotFoundException, IllegalContainerBalancerStateException,
- IOException, InvalidContainerBalancerConfigurationException {
+ IOException, InvalidContainerBalancerConfigurationException,
+ TimeoutException {
balancerConfiguration.setThreshold(10);
balancerConfiguration.setMaxDatanodesPercentageToInvolvePerIteration(100);
balancerConfiguration.setMaxSizeToMovePerIteration(50 * OzoneConsts.GB);
@@ -466,7 +469,7 @@ public void targetDatanodeShouldBeInServiceHealthy()
@Test
public void selectedContainerShouldNotAlreadyHaveBeenSelected()
throws IllegalContainerBalancerStateException, IOException,
- InvalidContainerBalancerConfigurationException {
+ InvalidContainerBalancerConfigurationException, TimeoutException {
balancerConfiguration.setThreshold(10);
balancerConfiguration.setMaxDatanodesPercentageToInvolvePerIteration(100);
balancerConfiguration.setMaxSizeToMovePerIteration(50 * OzoneConsts.GB);
@@ -494,7 +497,7 @@ public void selectedContainerShouldNotAlreadyHaveBeenSelected()
@Test
public void balancerShouldNotSelectConfiguredExcludeContainers()
throws IllegalContainerBalancerStateException, IOException,
- InvalidContainerBalancerConfigurationException {
+ InvalidContainerBalancerConfigurationException, TimeoutException {
balancerConfiguration.setThreshold(10);
balancerConfiguration.setMaxDatanodesPercentageToInvolvePerIteration(100);
balancerConfiguration.setMaxSizeToMovePerIteration(50 * OzoneConsts.GB);
@@ -523,7 +526,7 @@ public void balancerShouldNotSelectConfiguredExcludeContainers()
@Test
public void balancerShouldObeyMaxSizeEnteringTargetLimit()
throws IllegalContainerBalancerStateException, IOException,
- InvalidContainerBalancerConfigurationException {
+ InvalidContainerBalancerConfigurationException, TimeoutException {
conf.set("ozone.scm.container.size", "1MB");
balancerConfiguration =
conf.getObject(ContainerBalancerConfiguration.class);
@@ -557,7 +560,7 @@ public void balancerShouldObeyMaxSizeEnteringTargetLimit()
@Test
public void testMetrics()
throws IllegalContainerBalancerStateException, IOException,
- InvalidContainerBalancerConfigurationException {
+ InvalidContainerBalancerConfigurationException, TimeoutException {
conf.set("hdds.datanode.du.refresh.period", "1ms");
balancerConfiguration.setBalancingInterval(Duration.ofMillis(2));
balancerConfiguration.setThreshold(10);
@@ -590,7 +593,7 @@ public void testMetrics()
@Test
public void balancerShouldFollowExcludeAndIncludeDatanodesConfigurations()
throws IllegalContainerBalancerStateException, IOException,
- InvalidContainerBalancerConfigurationException {
+ InvalidContainerBalancerConfigurationException, TimeoutException {
balancerConfiguration.setThreshold(10);
balancerConfiguration.setIterations(1);
balancerConfiguration.setMaxSizeEnteringTarget(10 * OzoneConsts.GB);
@@ -663,7 +666,8 @@ public void testContainerBalancerConfiguration() {
public void checkIterationResult()
throws NodeNotFoundException, IOException,
IllegalContainerBalancerStateException,
- InvalidContainerBalancerConfigurationException {
+ InvalidContainerBalancerConfigurationException,
+ TimeoutException {
balancerConfiguration.setThreshold(10);
balancerConfiguration.setIterations(1);
balancerConfiguration.setMaxSizeEnteringTarget(10 * OzoneConsts.GB);
@@ -706,7 +710,8 @@ public void checkIterationResult()
public void checkIterationResultTimeout()
throws NodeNotFoundException, IOException,
IllegalContainerBalancerStateException,
- InvalidContainerBalancerConfigurationException {
+ InvalidContainerBalancerConfigurationException,
+ TimeoutException {
Mockito.when(replicationManager.move(Mockito.any(ContainerID.class),
Mockito.any(DatanodeDetails.class),
@@ -743,7 +748,7 @@ public void checkIterationResultTimeout()
public void checkIterationResultTimeoutFromReplicationManager()
throws NodeNotFoundException, IOException,
IllegalContainerBalancerStateException,
- InvalidContainerBalancerConfigurationException {
+ InvalidContainerBalancerConfigurationException, TimeoutException {
CompletableFuture future
= CompletableFuture.supplyAsync(() ->
MoveResult.REPLICATION_FAIL_TIME_OUT);
@@ -773,7 +778,7 @@ public void checkIterationResultTimeoutFromReplicationManager()
@Test
public void testStartAndImmediateStopForDeadlock()
throws IllegalContainerBalancerStateException, IOException,
- InvalidContainerBalancerConfigurationException {
+ InvalidContainerBalancerConfigurationException, TimeoutException {
startBalancer(balancerConfiguration);
stopBalancer();
Assertions.assertFalse(containerBalancer.isBalancerRunning());
@@ -963,7 +968,7 @@ private void sleepWhileBalancing(long millis) {
private void startBalancer(ContainerBalancerConfiguration config)
throws IllegalContainerBalancerStateException, IOException,
- InvalidContainerBalancerConfigurationException {
+ InvalidContainerBalancerConfigurationException, TimeoutException {
containerBalancer.startBalancer(config);
}
@@ -972,7 +977,8 @@ private void stopBalancer() {
if (containerBalancer.isBalancerRunning()) {
containerBalancer.stopBalancer();
}
- } catch (IOException | IllegalContainerBalancerStateException e) {
+ } catch (IOException | IllegalContainerBalancerStateException |
+ TimeoutException e) {
LOG.warn("Failed to stop balancer", e);
}
}
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestLegacyReplicationManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestLegacyReplicationManager.java
index 66863ce55366..fe2c32a2af0b 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestLegacyReplicationManager.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestLegacyReplicationManager.java
@@ -309,7 +309,7 @@ public void testReplicationManagerRestart() throws InterruptedException {
* any action on OPEN containers.
*/
@Test
- public void testOpenContainer() throws IOException {
+ public void testOpenContainer() throws IOException, TimeoutException {
final ContainerInfo container = getContainer(LifeCycleState.OPEN);
containerStateManager.addContainer(container.getProtobuf());
replicationManager.processAll();
@@ -324,7 +324,7 @@ public void testOpenContainer() throws IOException {
* to all the datanodes.
*/
@Test
- public void testClosingContainer() throws IOException {
+ public void testClosingContainer() throws IOException, TimeoutException {
final ContainerInfo container = getContainer(LifeCycleState.CLOSING);
final ContainerID id = container.containerID();
@@ -371,7 +371,8 @@ public void testClosingContainer() throws IOException {
* datanodes.
*/
@Test
- public void testQuasiClosedContainerWithTwoOpenReplica() throws IOException {
+ public void testQuasiClosedContainerWithTwoOpenReplica()
+ throws IOException, TimeoutException {
final ContainerInfo container = getContainer(LifeCycleState.QUASI_CLOSED);
final ContainerID id = container.containerID();
final UUID originNodeId = UUID.randomUUID();
@@ -412,7 +413,8 @@ public void testQuasiClosedContainerWithTwoOpenReplica() throws IOException {
* the container, ReplicationManager will not do anything.
*/
@Test
- public void testHealthyQuasiClosedContainer() throws IOException {
+ public void testHealthyQuasiClosedContainer()
+ throws IOException, TimeoutException {
final ContainerInfo container = getContainer(LifeCycleState.QUASI_CLOSED);
final ContainerID id = container.containerID();
final UUID originNodeId = UUID.randomUUID();
@@ -451,7 +453,7 @@ public void testHealthyQuasiClosedContainer() throws IOException {
*/
@Test
public void testQuasiClosedContainerWithUnhealthyReplica()
- throws IOException {
+ throws IOException, TimeoutException {
final ContainerInfo container = getContainer(LifeCycleState.QUASI_CLOSED);
container.setUsedBytes(100);
final ContainerID id = container.containerID();
@@ -565,7 +567,8 @@ public void testQuasiClosedContainerWithUnhealthyReplica()
* deletes the excess replicas.
*/
@Test
- public void testOverReplicatedQuasiClosedContainer() throws IOException {
+ public void testOverReplicatedQuasiClosedContainer()
+ throws IOException, TimeoutException {
final ContainerInfo container = getContainer(LifeCycleState.QUASI_CLOSED);
container.setUsedBytes(101);
final ContainerID id = container.containerID();
@@ -655,7 +658,7 @@ public void testOverReplicatedQuasiClosedContainer() throws IOException {
*/
@Test
public void testOverReplicatedQuasiClosedContainerWithUnhealthyReplica()
- throws IOException {
+ throws IOException, TimeoutException {
final ContainerInfo container = getContainer(LifeCycleState.QUASI_CLOSED);
final ContainerID id = container.containerID();
final UUID originNodeId = UUID.randomUUID();
@@ -726,7 +729,8 @@ public void testOverReplicatedQuasiClosedContainerWithUnhealthyReplica()
* under replicated.
*/
@Test
- public void testUnderReplicatedQuasiClosedContainer() throws IOException {
+ public void testUnderReplicatedQuasiClosedContainer()
+ throws IOException, TimeoutException {
final ContainerInfo container = getContainer(LifeCycleState.QUASI_CLOSED);
container.setUsedBytes(100);
final ContainerID id = container.containerID();
@@ -946,7 +950,7 @@ public void testUnderReplicatedQuasiClosedContainerWithUnhealthyReplica()
* highest BCSID.
*/
@Test
- public void testQuasiClosedToClosed() throws IOException {
+ public void testQuasiClosedToClosed() throws IOException, TimeoutException {
final ContainerInfo container = getContainer(LifeCycleState.QUASI_CLOSED);
final ContainerID id = container.containerID();
final Set replicas = getReplicas(id, State.QUASI_CLOSED,
@@ -980,7 +984,8 @@ public void testQuasiClosedToClosed() throws IOException {
* CLOSED and healthy.
*/
@Test
- public void testHealthyClosedContainer() throws IOException {
+ public void testHealthyClosedContainer()
+ throws IOException, TimeoutException {
final ContainerInfo container = getContainer(LifeCycleState.CLOSED);
final ContainerID id = container.containerID();
final Set replicas = getReplicas(id, State.CLOSED,
@@ -1009,7 +1014,8 @@ public void testHealthyClosedContainer() throws IOException {
* ReplicationManager should close the unhealthy OPEN container.
*/
@Test
- public void testUnhealthyOpenContainer() throws IOException {
+ public void testUnhealthyOpenContainer()
+ throws IOException, TimeoutException {
final ContainerInfo container = getContainer(LifeCycleState.OPEN);
final ContainerID id = container.containerID();
final Set replicas = getReplicas(id, State.OPEN,
@@ -1041,7 +1047,8 @@ public void testUnhealthyOpenContainer() throws IOException {
* ReplicationManager should skip send close command to unhealthy replica.
*/
@Test
- public void testCloseUnhealthyReplica() throws IOException {
+ public void testCloseUnhealthyReplica()
+ throws IOException, TimeoutException {
final ContainerInfo container = getContainer(LifeCycleState.CLOSING);
final ContainerID id = container.containerID();
final Set replicas = getReplicas(id, State.UNHEALTHY,
@@ -1073,7 +1080,8 @@ public void testGeneratedConfig() {
}
@Test
- public void additionalReplicaScheduledWhenMisReplicated() throws IOException {
+ public void additionalReplicaScheduledWhenMisReplicated()
+ throws IOException, TimeoutException {
final ContainerInfo container = getContainer(LifeCycleState.CLOSED);
container.setUsedBytes(100);
final ContainerID id = container.containerID();
@@ -1155,7 +1163,8 @@ public void additionalReplicaScheduledWhenMisReplicated() throws IOException {
}
@Test
- public void overReplicatedButRemovingMakesMisReplicated() throws IOException {
+ public void overReplicatedButRemovingMakesMisReplicated()
+ throws IOException, TimeoutException {
// In this test, the excess replica should not be removed.
final ContainerInfo container = getContainer(LifeCycleState.CLOSED);
final ContainerID id = container.containerID();
@@ -1211,7 +1220,8 @@ public void overReplicatedButRemovingMakesMisReplicated() throws IOException {
}
@Test
- public void testOverReplicatedAndPolicySatisfied() throws IOException {
+ public void testOverReplicatedAndPolicySatisfied()
+ throws IOException, TimeoutException {
final ContainerInfo container = getContainer(LifeCycleState.CLOSED);
final ContainerID id = container.containerID();
final UUID originNodeId = UUID.randomUUID();
@@ -1255,8 +1265,8 @@ public void testOverReplicatedAndPolicySatisfied() throws IOException {
}
@Test
- public void testOverReplicatedAndPolicyUnSatisfiedAndDeleted() throws
- IOException {
+ public void testOverReplicatedAndPolicyUnSatisfiedAndDeleted()
+ throws IOException, TimeoutException {
final ContainerInfo container = getContainer(LifeCycleState.CLOSED);
final ContainerID id = container.containerID();
final UUID originNodeId = UUID.randomUUID();
@@ -1305,7 +1315,8 @@ public void testOverReplicatedAndPolicyUnSatisfiedAndDeleted() throws
* decommissioned replicas.
*/
@Test
- public void testUnderReplicatedDueToDecommission() throws IOException {
+ public void testUnderReplicatedDueToDecommission()
+ throws IOException, TimeoutException {
final ContainerInfo container = createContainer(LifeCycleState.CLOSED);
addReplica(container, new NodeStatus(IN_SERVICE, HEALTHY), CLOSED);
addReplica(container, new NodeStatus(DECOMMISSIONING, HEALTHY), CLOSED);
@@ -1319,12 +1330,13 @@ public void testUnderReplicatedDueToDecommission() throws IOException {
* are decommissioning.
*/
@Test
- public void testUnderReplicatedDueToAllDecommission() throws IOException {
+ public void testUnderReplicatedDueToAllDecommission()
+ throws IOException, TimeoutException {
runTestUnderReplicatedDueToAllDecommission(3);
}
Void runTestUnderReplicatedDueToAllDecommission(int expectedReplication)
- throws IOException {
+ throws IOException, TimeoutException {
final ContainerInfo container = createContainer(LifeCycleState.CLOSED);
addReplica(container, new NodeStatus(DECOMMISSIONING, HEALTHY), CLOSED);
addReplica(container, new NodeStatus(DECOMMISSIONING, HEALTHY), CLOSED);
@@ -1365,7 +1377,8 @@ void runTestLimit(int replicationLimit, int deletionLimit,
* correctly replicated with decommissioned replicas still present.
*/
@Test
- public void testCorrectlyReplicatedWithDecommission() throws IOException {
+ public void testCorrectlyReplicatedWithDecommission()
+ throws IOException, TimeoutException {
final ContainerInfo container = createContainer(LifeCycleState.CLOSED);
addReplica(container, new NodeStatus(IN_SERVICE, HEALTHY), CLOSED);
addReplica(container, new NodeStatus(IN_SERVICE, HEALTHY), CLOSED);
@@ -1380,7 +1393,8 @@ public void testCorrectlyReplicatedWithDecommission() throws IOException {
* is not met for maintenance.
*/
@Test
- public void testUnderReplicatedDueToMaintenance() throws IOException {
+ public void testUnderReplicatedDueToMaintenance()
+ throws IOException, TimeoutException {
final ContainerInfo container = createContainer(LifeCycleState.CLOSED);
addReplica(container, new NodeStatus(IN_SERVICE, HEALTHY), CLOSED);
addReplica(container, new NodeStatus(IN_MAINTENANCE, HEALTHY), CLOSED);
@@ -1436,7 +1450,8 @@ public void testUnderReplicatedDueToMaintenanceMinRepOne()
* are going into maintenance.
*/
@Test
- public void testUnderReplicatedDueToAllMaintenance() throws IOException {
+ public void testUnderReplicatedDueToAllMaintenance()
+ throws IOException, TimeoutException {
final ContainerInfo container = createContainer(LifeCycleState.CLOSED);
addReplica(container, new NodeStatus(IN_MAINTENANCE, HEALTHY), CLOSED);
addReplica(container, new NodeStatus(IN_MAINTENANCE, HEALTHY), CLOSED);
@@ -1450,7 +1465,8 @@ public void testUnderReplicatedDueToAllMaintenance() throws IOException {
* replica are available.
*/
@Test
- public void testCorrectlyReplicatedWithMaintenance() throws IOException {
+ public void testCorrectlyReplicatedWithMaintenance()
+ throws IOException, TimeoutException {
final ContainerInfo container = createContainer(LifeCycleState.CLOSED);
addReplica(container, new NodeStatus(IN_SERVICE, HEALTHY), CLOSED);
addReplica(container, new NodeStatus(IN_SERVICE, HEALTHY), CLOSED);
@@ -1466,7 +1482,7 @@ public void testCorrectlyReplicatedWithMaintenance() throws IOException {
*/
@Test
public void testUnderReplicatedWithDecommissionAndMaintenance()
- throws IOException {
+ throws IOException, TimeoutException {
final ContainerInfo container = createContainer(LifeCycleState.CLOSED);
addReplica(container, new NodeStatus(DECOMMISSIONED, HEALTHY), CLOSED);
addReplica(container, new NodeStatus(DECOMMISSIONED, HEALTHY), CLOSED);
@@ -1482,7 +1498,7 @@ public void testUnderReplicatedWithDecommissionAndMaintenance()
*/
@Test
public void testContainerWithMissingReplicas()
- throws IOException {
+ throws IOException, TimeoutException {
createContainer(LifeCycleState.CLOSED);
assertReplicaScheduled(0);
assertUnderReplicatedCount(1);
@@ -1496,7 +1512,7 @@ public void testContainerWithMissingReplicas()
*/
@Test
public void testOverReplicatedClosedContainerWithDecomAndMaint()
- throws IOException {
+ throws IOException, TimeoutException {
final ContainerInfo container = createContainer(LifeCycleState.CLOSED);
addReplica(container, NodeStatus.inServiceHealthy(), CLOSED);
addReplica(container, new NodeStatus(DECOMMISSIONED, HEALTHY), CLOSED);
@@ -1542,7 +1558,8 @@ public void testOverReplicatedClosedContainerWithDecomAndMaint()
* scheduled.
*/
@Test
- public void testUnderReplicatedNotHealthySource() throws IOException {
+ public void testUnderReplicatedNotHealthySource()
+ throws IOException, TimeoutException {
final ContainerInfo container = createContainer(LifeCycleState.CLOSED);
addReplica(container, NodeStatus.inServiceStale(), CLOSED);
addReplica(container, new NodeStatus(DECOMMISSIONED, STALE), CLOSED);
@@ -1558,7 +1575,7 @@ public void testUnderReplicatedNotHealthySource() throws IOException {
*/
@Test
public void testMove() throws IOException, NodeNotFoundException,
- InterruptedException, ExecutionException {
+ InterruptedException, ExecutionException, TimeoutException {
final ContainerInfo container = createContainer(LifeCycleState.CLOSED);
ContainerID id = container.containerID();
ContainerReplica dn1 = addReplica(container,
@@ -1600,7 +1617,7 @@ public void testMove() throws IOException, NodeNotFoundException,
*/
@Test
public void testMoveCrashAndRestart() throws IOException,
- NodeNotFoundException, InterruptedException {
+ NodeNotFoundException, InterruptedException, TimeoutException {
final ContainerInfo container = createContainer(LifeCycleState.CLOSED);
ContainerID id = container.containerID();
ContainerReplica dn1 = addReplica(container,
@@ -1694,7 +1711,7 @@ public void testMoveCrashAndRestart() throws IOException,
@Test
public void testMoveNotDeleteSrcIfPolicyNotSatisfied()
throws IOException, NodeNotFoundException,
- InterruptedException, ExecutionException {
+ InterruptedException, ExecutionException, TimeoutException {
final ContainerInfo container = createContainer(LifeCycleState.CLOSED);
ContainerID id = container.containerID();
ContainerReplica dn1 = addReplica(container,
@@ -1736,7 +1753,8 @@ public void testMoveNotDeleteSrcIfPolicyNotSatisfied()
*/
@Test
public void testDnBecameUnhealthyWhenMoving() throws IOException,
- NodeNotFoundException, InterruptedException, ExecutionException {
+ NodeNotFoundException, InterruptedException, ExecutionException,
+ TimeoutException {
final ContainerInfo container = createContainer(LifeCycleState.CLOSED);
ContainerID id = container.containerID();
ContainerReplica dn1 = addReplica(container,
@@ -1779,7 +1797,7 @@ public void testDnBecameUnhealthyWhenMoving() throws IOException,
@Test
public void testMovePrerequisites() throws IOException, NodeNotFoundException,
InterruptedException, ExecutionException,
- InvalidStateTransitionException {
+ InvalidStateTransitionException, TimeoutException {
//all conditions is met
final ContainerInfo container = createContainer(LifeCycleState.OPEN);
ContainerID id = container.containerID();
@@ -1895,7 +1913,8 @@ public void testMovePrerequisites() throws IOException, NodeNotFoundException,
}
@Test
- public void testReplicateCommandTimeout() throws IOException {
+ public void testReplicateCommandTimeout()
+ throws IOException, TimeoutException {
long timeout = new ReplicationManagerConfiguration().getEventTimeout();
final ContainerInfo container = createContainer(LifeCycleState.CLOSED);
@@ -1915,8 +1934,8 @@ public void testReplicateCommandTimeout() throws IOException {
}
@Test
- public void testDeleteCommandTimeout() throws
- IOException, InterruptedException {
+ public void testDeleteCommandTimeout()
+ throws IOException, TimeoutException {
long timeout = new ReplicationManagerConfiguration().getEventTimeout();
final ContainerInfo container = createContainer(LifeCycleState.CLOSED);
@@ -1983,13 +2002,13 @@ public void testDeleteEmptyContainerNonEmptyReplica() throws Exception {
}
private ContainerInfo createContainer(LifeCycleState containerState)
- throws IOException {
+ throws IOException, TimeoutException {
return createContainer(containerState, CONTAINER_USED_BYTES_DEFAULT,
CONTAINER_NUM_KEYS_DEFAULT);
}
private ContainerInfo createContainer(LifeCycleState containerState,
- long usedBytes, long numKeys) throws IOException {
+ long usedBytes, long numKeys) throws IOException, TimeoutException {
final ContainerInfo container = getContainer(containerState);
container.setUsedBytes(usedBytes);
container.setNumberOfKeys(numKeys);
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/ha/TestBackgroundSCMService.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/ha/TestBackgroundSCMService.java
index e4ef1609130a..3810f0191550 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/ha/TestBackgroundSCMService.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/ha/TestBackgroundSCMService.java
@@ -27,6 +27,7 @@
import java.io.IOException;
import java.time.Instant;
import java.time.ZoneOffset;
+import java.util.concurrent.TimeoutException;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertTrue;
@@ -45,7 +46,7 @@ public class TestBackgroundSCMService {
private PipelineManager pipelineManager;
@BeforeEach
- public void setup() throws IOException {
+ public void setup() throws IOException, TimeoutException {
testClock = new TestClock(Instant.now(), ZoneOffset.UTC);
scmContext = SCMContext.emptyContext();
this.pipelineManager = mock(PipelineManager.class);
@@ -59,7 +60,7 @@ public void setup() throws IOException {
.setPeriodicalTask(() -> {
try {
pipelineManager.scrubPipelines();
- } catch (IOException e) {
+ } catch (IOException | TimeoutException e) {
throw new RuntimeException(e);
}
}).build();
@@ -98,7 +99,7 @@ public void testNotifyStatusChanged() {
}
@Test
- public void testRun() throws IOException {
+ public void testRun() throws IOException, TimeoutException {
assertFalse(backgroundSCMService.shouldRun());
// kick a run
synchronized (backgroundSCMService) {
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/ha/TestStatefulServiceStateManagerImpl.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/ha/TestStatefulServiceStateManagerImpl.java
index 7c47e7ec3dbf..3ea459419bf2 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/ha/TestStatefulServiceStateManagerImpl.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/ha/TestStatefulServiceStateManagerImpl.java
@@ -77,7 +77,7 @@ public void cleanup() throws Exception {
* @throws IOException on failure
*/
@Test
- public void testSaveConfiguration() throws IOException {
+ public void testSaveConfiguration() throws Exception {
String serviceName = "test";
String message = "message_string";
stateManager.saveConfiguration(serviceName,
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestContainerPlacement.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestContainerPlacement.java
index f82f8f3d547c..97575c55829f 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestContainerPlacement.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestContainerPlacement.java
@@ -23,6 +23,7 @@
import java.time.ZoneId;
import java.util.List;
import java.util.UUID;
+import java.util.concurrent.TimeoutException;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdds.HddsConfigKeys;
@@ -174,7 +175,7 @@ ContainerManager createContainerManager()
@Test
@Disabled
public void testContainerPlacementCapacity() throws IOException,
- InterruptedException {
+ InterruptedException, TimeoutException {
final int nodeCount = 4;
final long capacity = 10L * OzoneConsts.GB;
final long used = 2L * OzoneConsts.GB;
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/MockPipelineManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/MockPipelineManager.java
index f805470a25e8..efa4bed92ed6 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/MockPipelineManager.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/MockPipelineManager.java
@@ -39,6 +39,7 @@
import java.util.Map;
import java.util.NavigableSet;
import java.util.Set;
+import java.util.concurrent.TimeoutException;
import java.util.stream.Collectors;
import java.util.stream.Stream;
@@ -61,7 +62,7 @@ public MockPipelineManager(DBStore dbStore, SCMHAManager scmhaManager,
@Override
public Pipeline createPipeline(ReplicationConfig replicationConfig)
- throws IOException {
+ throws IOException, TimeoutException {
return createPipeline(replicationConfig, Collections.emptyList(),
Collections.emptyList());
}
@@ -69,7 +70,7 @@ public Pipeline createPipeline(ReplicationConfig replicationConfig)
@Override
public Pipeline createPipeline(ReplicationConfig replicationConfig,
List excludedNodes, List favoredNodes)
- throws IOException {
+ throws IOException, TimeoutException {
final List nodes = Stream.generate(
MockDatanodeDetails::randomDatanodeDetails)
.limit(replicationConfig.getRequiredNodes())
@@ -205,14 +206,14 @@ public int getNumberOfContainers(final PipelineID pipelineID)
@Override
public void openPipeline(final PipelineID pipelineId)
- throws IOException {
+ throws IOException, TimeoutException {
stateManager.updatePipelineState(
pipelineId.getProtobuf(), HddsProtos.PipelineState.PIPELINE_OPEN);
}
@Override
public void closePipeline(final Pipeline pipeline, final boolean onTimeout)
- throws IOException {
+ throws IOException, TimeoutException {
stateManager.updatePipelineState(pipeline.getId().getProtobuf(),
HddsProtos.PipelineState.PIPELINE_CLOSED);
}
@@ -259,7 +260,7 @@ public void activatePipeline(final PipelineID pipelineID)
@Override
public void deactivatePipeline(final PipelineID pipelineID)
- throws IOException {
+ throws IOException, TimeoutException {
stateManager.updatePipelineState(pipelineID.getProtobuf(),
HddsProtos.PipelineState.PIPELINE_DORMANT);
}
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineDatanodesIntersection.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineDatanodesIntersection.java
index dcc3829ccee8..ae4411cdf0c6 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineDatanodesIntersection.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineDatanodesIntersection.java
@@ -49,6 +49,7 @@
import java.io.IOException;
import java.util.List;
import java.util.UUID;
+import java.util.concurrent.TimeoutException;
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_DATANODE_PIPELINE_LIMIT;
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_PIPELINE_AUTO_CREATE_FACTOR_ONE;
@@ -145,7 +146,7 @@ public void testPipelineDatanodesIntersection(int nodeCount,
createdPipelineCount++;
} catch (SCMException e) {
end = true;
- } catch (IOException e) {
+ } catch (IOException | TimeoutException e) {
end = true;
// Should not throw regular IOException.
Assertions.fail();
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java
index 0e60921da56e..8fa45f584233 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java
@@ -75,6 +75,7 @@
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
import java.util.stream.Collectors;
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_DATANODE_PIPELINE_LIMIT;
@@ -760,7 +761,7 @@ public void testAddContainerWithClosedPipeline() throws Exception {
}
@Test
- public void testPipelineCloseFlow() throws IOException {
+ public void testPipelineCloseFlow() throws IOException, TimeoutException {
GenericTestUtils.LogCapturer logCapturer = GenericTestUtils.LogCapturer
.captureLogs(LoggerFactory.getLogger(PipelineManagerImpl.class));
PipelineManagerImpl pipelineManager = createPipelineManager(true);
@@ -866,7 +867,7 @@ public void testGetStalePipelines() throws IOException {
}
@Test
- public void testCloseStalePipelines() throws IOException {
+ public void testCloseStalePipelines() throws IOException, TimeoutException {
SCMHADBTransactionBuffer buffer =
new SCMHADBTransactionBufferStub(dbStore);
PipelineManagerImpl pipelineManager =
@@ -889,7 +890,8 @@ public void testCloseStalePipelines() throws IOException {
}
@Test
- public void testWaitForAllocatedPipeline() throws IOException {
+ public void testWaitForAllocatedPipeline()
+ throws IOException, TimeoutException {
SCMHADBTransactionBuffer buffer =
new SCMHADBTransactionBufferStub(dbStore);
PipelineManagerImpl pipelineManager =
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelinePlacementPolicy.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelinePlacementPolicy.java
index 446be34189d0..9d5cadeb2d38 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelinePlacementPolicy.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelinePlacementPolicy.java
@@ -24,6 +24,7 @@
import java.util.List;
import java.util.Set;
import java.util.UUID;
+import java.util.concurrent.TimeoutException;
import java.util.stream.Collectors;
import org.apache.hadoop.fs.FileUtil;
@@ -265,7 +266,7 @@ public void testChooseNodeNotEnoughSpace() throws IOException {
}
@Test
- public void testPickLowestLoadAnchor() throws IOException {
+ public void testPickLowestLoadAnchor() throws IOException, TimeoutException {
List healthyNodes = nodeManager
.getNodes(NodeStatus.inServiceHealthy());
@@ -426,7 +427,7 @@ private List overWriteLocationInNodes(
@Test
public void testHeavyNodeShouldBeExcludedWithMinorityHeavy()
- throws IOException {
+ throws IOException, TimeoutException {
List healthyNodes =
nodeManager.getNodes(NodeStatus.inServiceHealthy());
int nodesRequired = HddsProtos.ReplicationFactor.THREE.getNumber();
@@ -456,7 +457,7 @@ public void testHeavyNodeShouldBeExcludedWithMinorityHeavy()
@Test
public void testHeavyNodeShouldBeExcludedWithMajorityHeavy()
- throws IOException {
+ throws IOException, TimeoutException {
List healthyNodes =
nodeManager.getNodes(NodeStatus.inServiceHealthy());
int nodesRequired = HddsProtos.ReplicationFactor.THREE.getNumber();
@@ -636,7 +637,8 @@ private Set mockPipelineIDs(int count) {
}
private void insertHeavyNodesIntoNodeManager(
- List nodes, int heavyNodeCount) throws IOException {
+ List nodes, int heavyNodeCount)
+ throws IOException, TimeoutException {
if (nodes == null) {
throw new SCMException("",
SCMException.ResultCodes.FAILED_TO_FIND_SUITABLE_NODE);
@@ -679,7 +681,7 @@ private void insertHeavyNodesIntoNodeManager(
@Test
public void testCurrentRatisThreePipelineCount()
- throws IOException {
+ throws IOException, TimeoutException {
List healthyNodes = nodeManager
.getNodes(NodeStatus.inServiceHealthy());
int pipelineCount;
@@ -741,7 +743,7 @@ private void createPipelineWithReplicationConfig(List dnList,
replicationType,
ReplicationFactor
replicationFactor)
- throws IOException {
+ throws IOException, TimeoutException {
Pipeline pipeline = Pipeline.newBuilder()
.setId(PipelineID.randomId())
.setState(Pipeline.PipelineState.OPEN)
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineStateManagerImpl.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineStateManagerImpl.java
index 83f78a6eb2de..7b69a9cd8a6b 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineStateManagerImpl.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineStateManagerImpl.java
@@ -53,6 +53,7 @@
import java.util.List;
import java.util.Set;
import java.util.UUID;
+import java.util.concurrent.TimeoutException;
/**
* Test for PipelineStateManagerImpl.
@@ -113,7 +114,7 @@ private Pipeline createDummyPipeline(HddsProtos.ReplicationType type,
}
@Test
- public void testAddAndGetPipeline() throws IOException {
+ public void testAddAndGetPipeline() throws IOException, TimeoutException {
Pipeline pipeline = createDummyPipeline(0);
HddsProtos.Pipeline pipelineProto = pipeline.getProtobufMessage(
ClientVersion.CURRENT_VERSION);
@@ -148,7 +149,7 @@ public void testAddAndGetPipeline() throws IOException {
}
@Test
- public void testGetPipelines() throws IOException {
+ public void testGetPipelines() throws IOException, TimeoutException {
// In start there should be no pipelines
Assertions.assertTrue(stateManager.getPipelines().isEmpty());
@@ -178,7 +179,8 @@ public void testGetPipelines() throws IOException {
}
@Test
- public void testGetPipelinesByTypeAndFactor() throws IOException {
+ public void testGetPipelinesByTypeAndFactor()
+ throws IOException, TimeoutException {
Set pipelines = new HashSet<>();
for (HddsProtos.ReplicationType type : new ReplicationType[] {
ReplicationType.RATIS, ReplicationType.STAND_ALONE}) {
@@ -230,7 +232,8 @@ public void testGetPipelinesByTypeAndFactor() throws IOException {
}
@Test
- public void testGetPipelinesByTypeFactorAndState() throws IOException {
+ public void testGetPipelinesByTypeFactorAndState()
+ throws IOException, TimeoutException {
Set pipelines = new HashSet<>();
for (HddsProtos.ReplicationType type : new ReplicationType[] {
ReplicationType.RATIS, ReplicationType.STAND_ALONE}) {
@@ -296,7 +299,7 @@ public void testGetPipelinesByTypeFactorAndState() throws IOException {
}
@Test
- public void testAddAndGetContainer() throws IOException {
+ public void testAddAndGetContainer() throws IOException, TimeoutException {
long containerID = 0;
Pipeline pipeline = createDummyPipeline(1);
HddsProtos.Pipeline pipelineProto = pipeline
@@ -331,7 +334,7 @@ public void testAddAndGetContainer() throws IOException {
}
@Test
- public void testRemovePipeline() throws IOException {
+ public void testRemovePipeline() throws IOException, TimeoutException {
Pipeline pipeline = createDummyPipeline(1);
HddsProtos.Pipeline pipelineProto = pipeline
.getProtobufMessage(ClientVersion.CURRENT_VERSION);
@@ -356,7 +359,7 @@ public void testRemovePipeline() throws IOException {
}
@Test
- public void testRemoveContainer() throws IOException {
+ public void testRemoveContainer() throws IOException, TimeoutException {
long containerID = 1;
Pipeline pipeline = createDummyPipeline(1);
HddsProtos.Pipeline pipelineProto = pipeline
@@ -397,7 +400,7 @@ public void testRemoveContainer() throws IOException {
}
@Test
- public void testFinalizePipeline() throws IOException {
+ public void testFinalizePipeline() throws IOException, TimeoutException {
Pipeline pipeline = createDummyPipeline(1);
HddsProtos.Pipeline pipelineProto = pipeline
.getProtobufMessage(ClientVersion.CURRENT_VERSION);
@@ -436,7 +439,7 @@ public void testFinalizePipeline() throws IOException {
}
@Test
- public void testOpenPipeline() throws IOException {
+ public void testOpenPipeline() throws IOException, TimeoutException {
Pipeline pipeline = createDummyPipeline(1);
HddsProtos.Pipeline pipelineProto = pipeline
.getProtobufMessage(ClientVersion.CURRENT_VERSION);
@@ -456,7 +459,7 @@ public void testOpenPipeline() throws IOException {
}
@Test
- public void testQueryPipeline() throws IOException {
+ public void testQueryPipeline() throws IOException, TimeoutException {
Pipeline pipeline = createDummyPipeline(HddsProtos.ReplicationType.RATIS,
HddsProtos.ReplicationFactor.THREE, 3);
// pipeline in allocated state should not be reported
@@ -507,23 +510,25 @@ public void testQueryPipeline() throws IOException {
removePipeline(pipelineProto2);
}
- private void removePipeline(HddsProtos.Pipeline pipeline) throws IOException {
+ private void removePipeline(HddsProtos.Pipeline pipeline)
+ throws IOException, TimeoutException {
stateManager.removePipeline(pipeline.getId());
}
- private void openPipeline(HddsProtos.Pipeline pipeline) throws IOException {
+ private void openPipeline(HddsProtos.Pipeline pipeline)
+ throws IOException, TimeoutException {
stateManager.updatePipelineState(pipeline.getId(),
HddsProtos.PipelineState.PIPELINE_OPEN);
}
private void finalizePipeline(HddsProtos.Pipeline pipeline)
- throws IOException {
+ throws IOException, TimeoutException {
stateManager.updatePipelineState(pipeline.getId(),
HddsProtos.PipelineState.PIPELINE_CLOSED);
}
private void deactivatePipeline(HddsProtos.Pipeline pipeline)
- throws IOException {
+ throws IOException, TimeoutException {
stateManager.updatePipelineState(pipeline.getId(),
HddsProtos.PipelineState.PIPELINE_DORMANT);
}
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestRatisPipelineProvider.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestRatisPipelineProvider.java
index c44e5d8fcbfa..b788fd713eb4 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestRatisPipelineProvider.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestRatisPipelineProvider.java
@@ -54,6 +54,7 @@
import java.util.List;
import java.util.Set;
import java.util.UUID;
+import java.util.concurrent.TimeoutException;
import java.util.stream.Collectors;
import static org.apache.commons.collections.CollectionUtils.intersection;
@@ -124,7 +125,8 @@ private static void assertPipelineProperties(
}
private void createPipelineAndAssertions(
- HddsProtos.ReplicationFactor factor) throws IOException {
+ HddsProtos.ReplicationFactor factor)
+ throws IOException, TimeoutException {
Pipeline pipeline = provider.create(RatisReplicationConfig
.getInstance(factor));
assertPipelineProperties(pipeline, factor, REPLICATION_TYPE,
@@ -359,7 +361,7 @@ public void testCreatePipelinesWhenNotEnoughSpace() throws Exception {
private void addPipeline(
List dns,
Pipeline.PipelineState open, ReplicationConfig replicationConfig)
- throws IOException {
+ throws IOException, TimeoutException {
Pipeline openPipeline = Pipeline.newBuilder()
.setReplicationConfig(replicationConfig)
.setNodes(dns)
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestSimplePipelineProvider.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestSimplePipelineProvider.java
index 2c4d7e6d4202..1888c1c4dd8c 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestSimplePipelineProvider.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestSimplePipelineProvider.java
@@ -87,7 +87,7 @@ public void cleanup() throws Exception {
}
@Test
- public void testCreatePipelineWithFactor() throws IOException {
+ public void testCreatePipelineWithFactor() throws Exception {
HddsProtos.ReplicationFactor factor = HddsProtos.ReplicationFactor.THREE;
Pipeline pipeline =
provider.create(StandaloneReplicationConfig.getInstance(factor));
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestWritableECContainerProvider.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestWritableECContainerProvider.java
index 9c0ddc180f3d..09ae77581f17 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestWritableECContainerProvider.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestWritableECContainerProvider.java
@@ -59,6 +59,7 @@
import java.util.NavigableSet;
import java.util.Set;
import java.util.UUID;
+import java.util.concurrent.TimeoutException;
import static org.apache.hadoop.hdds.conf.StorageUnit.BYTES;
import static org.apache.hadoop.hdds.scm.pipeline.Pipeline.PipelineState.CLOSED;
@@ -134,7 +135,7 @@ public void setup() throws IOException {
@Test
public void testPipelinesCreatedUpToMinLimitAndRandomPipelineReturned()
- throws IOException {
+ throws IOException, TimeoutException {
// The first 5 calls should return a different container
Set allocatedContainers = new HashSet<>();
for (int i = 0; i < minPipelines; i++) {
@@ -160,7 +161,8 @@ public void testPipelinesCreatedUpToMinLimitAndRandomPipelineReturned()
}
@Test
- public void testPiplineLimitIgnoresExcludedPipelines() throws IOException {
+ public void testPiplineLimitIgnoresExcludedPipelines()
+ throws IOException, TimeoutException {
Set allocatedContainers = new HashSet<>();
for (int i = 0; i < minPipelines; i++) {
ContainerInfo container = provider.getContainer(
@@ -182,7 +184,7 @@ public void testPiplineLimitIgnoresExcludedPipelines() throws IOException {
@Test
public void testNewPipelineCreatedIfAllPipelinesExcluded()
- throws IOException {
+ throws IOException, TimeoutException {
Set allocatedContainers = new HashSet<>();
for (int i = 0; i < minPipelines; i++) {
ContainerInfo container = provider.getContainer(
@@ -202,7 +204,7 @@ public void testNewPipelineCreatedIfAllPipelinesExcluded()
@Test
public void testNewPipelineCreatedIfAllContainersExcluded()
- throws IOException {
+ throws IOException, TimeoutException {
Set allocatedContainers = new HashSet<>();
for (int i = 0; i < minPipelines; i++) {
ContainerInfo container = provider.getContainer(
@@ -238,14 +240,14 @@ public Pipeline createPipeline(ReplicationConfig repConf,
try {
provider.getContainer(1, repConfig, OWNER, new ExcludeList());
Assert.fail();
- } catch (IOException ex) {
+ } catch (IOException | TimeoutException ex) {
GenericTestUtils.assertExceptionContains("Cannot create pipelines", ex);
}
}
@Test
public void testExistingPipelineReturnedWhenNewCannotBeCreated()
- throws IOException {
+ throws IOException, TimeoutException {
pipelineManager = new MockPipelineManager(
dbStore, scmhaManager, nodeManager) {
@@ -254,7 +256,8 @@ public void testExistingPipelineReturnedWhenNewCannotBeCreated()
@Override
public Pipeline createPipeline(ReplicationConfig repConf,
List excludedNodes,
- List favoredNodes) throws IOException {
+ List favoredNodes)
+ throws IOException, TimeoutException {
if (throwError) {
throw new IOException("Cannot create pipelines");
}
@@ -284,7 +287,7 @@ public Pipeline createPipeline(ReplicationConfig repConf,
@Test
public void testNewContainerAllocatedAndPipelinesClosedIfNoSpaceInExisting()
- throws IOException {
+ throws IOException, TimeoutException {
Set allocatedContainers = new HashSet<>();
for (int i = 0; i < minPipelines; i++) {
ContainerInfo container =
@@ -322,7 +325,7 @@ public void testNewContainerAllocatedAndPipelinesClosedIfNoSpaceInExisting()
@Test
public void testPipelineNotFoundWhenAttemptingToUseExisting()
- throws IOException {
+ throws IOException, TimeoutException {
// Ensure PM throws PNF exception when we ask for the containers in the
// pipeline
pipelineManager = new MockPipelineManager(
@@ -354,7 +357,7 @@ public NavigableSet getContainersInPipeline(
@Test
public void testContainerNotFoundWhenAttemptingToUseExisting()
- throws IOException {
+ throws IOException, TimeoutException {
Set allocatedContainers = new HashSet<>();
for (int i = 0; i < minPipelines; i++) {
ContainerInfo container =
@@ -382,7 +385,8 @@ public void testContainerNotFoundWhenAttemptingToUseExisting()
}
@Test
- public void testPipelineOpenButContainerRemovedFromIt() throws IOException {
+ public void testPipelineOpenButContainerRemovedFromIt()
+ throws IOException, TimeoutException {
// This can happen if the container close process is triggered from the DN.
// When tha happens, CM will change the container state to CLOSING and
// remove it from the container list in pipeline Manager.
@@ -407,7 +411,7 @@ public void testPipelineOpenButContainerRemovedFromIt() throws IOException {
@Test
public void testExcludedNodesPassedToCreatePipelineIfProvided()
- throws IOException {
+ throws IOException, TimeoutException {
PipelineManager pipelineManagerSpy = Mockito.spy(pipelineManager);
provider = new WritableECContainerProvider(
conf, pipelineManagerSpy, containerManager, pipelineChoosingPolicy);
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/update/server/MockCRLStore.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/update/server/MockCRLStore.java
index 661d0e419161..d67af90f0dad 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/update/server/MockCRLStore.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/update/server/MockCRLStore.java
@@ -48,6 +48,7 @@
import java.util.Date;
import java.util.List;
import java.util.Optional;
+import java.util.concurrent.TimeoutException;
/**
* Mock CRL Store impl for test.
@@ -97,7 +98,8 @@ public BigInteger issueCert() throws Exception {
}
public Optional revokeCert(List certs,
- Instant revokeTime) throws IOException {
+ Instant revokeTime)
+ throws IOException, TimeoutException {
log.debug("Revoke certs: {}", certs);
Optional crlId = scmCertStore.revokeCertificates(certs,
caCertificateHolder,
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/update/server/TestSCMUpdateServiceGrpcServer.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/update/server/TestSCMUpdateServiceGrpcServer.java
index 257ac3128d30..f1b641a36f8d 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/update/server/TestSCMUpdateServiceGrpcServer.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/update/server/TestSCMUpdateServiceGrpcServer.java
@@ -45,6 +45,7 @@
import java.util.Arrays;
import java.util.List;
import java.util.Optional;
+import java.util.concurrent.TimeoutException;
/**
* Tests for SCM update Service.
@@ -198,13 +199,14 @@ public void testClientUpdateWithDelayedRevoke() throws Exception {
}
private Long revokeCert(BigInteger certId, Instant revokeTime)
- throws IOException {
+ throws IOException, TimeoutException {
Optional crlId =
mockCRLStore.revokeCert(Arrays.asList(certId), revokeTime);
return crlId.get();
}
- private Long revokeCertNow(BigInteger certId) throws IOException {
+ private Long revokeCertNow(BigInteger certId)
+ throws IOException, TimeoutException {
Optional crlId =
mockCRLStore.revokeCert(Arrays.asList(certId), Instant.now());
return crlId.get();
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerStateManagerIntegration.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerStateManagerIntegration.java
index ea26ad2e2a97..81bb05496f13 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerStateManagerIntegration.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerStateManagerIntegration.java
@@ -268,7 +268,7 @@ public void testGetMatchingContainerMultipleThreads()
@Test
public void testUpdateContainerState() throws IOException,
- InvalidStateTransitionException {
+ InvalidStateTransitionException, TimeoutException {
Set containerList = containerStateManager
.getContainerIDs(HddsProtos.LifeCycleState.OPEN);
int containers = containerList == null ? 0 : containerList.size();
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/container/metrics/TestSCMContainerManagerMetrics.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/container/metrics/TestSCMContainerManagerMetrics.java
index ad49310824ad..2ac274d2a38f 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/container/metrics/TestSCMContainerManagerMetrics.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/container/metrics/TestSCMContainerManagerMetrics.java
@@ -82,7 +82,7 @@ public void teardown() {
}
@Test
- public void testContainerOpsMetrics() throws IOException {
+ public void testContainerOpsMetrics() throws Exception {
MetricsRecordBuilder metrics;
ContainerManager containerManager = scm.getContainerManager();
metrics = getMetrics(SCMContainerManagerMetrics.class.getSimpleName());
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestNode2PipelineMap.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestNode2PipelineMap.java
index abbb6d1a969c..d891caa024b0 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestNode2PipelineMap.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestNode2PipelineMap.java
@@ -38,6 +38,7 @@
import java.io.IOException;
import java.util.List;
import java.util.Set;
+import java.util.concurrent.TimeoutException;
import org.junit.Rule;
import org.junit.rules.Timeout;
@@ -93,7 +94,7 @@ public void shutdown() {
@Test
public void testPipelineMap() throws IOException,
- InvalidStateTransitionException {
+ InvalidStateTransitionException, TimeoutException {
Set set = pipelineManager
.getContainersInPipeline(ratisContainer.getPipeline().getId());
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineClose.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineClose.java
index e212692d2a37..5e7b2df26bb1 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineClose.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineClose.java
@@ -115,7 +115,7 @@ public void shutdown() {
@Test
public void testPipelineCloseWithClosedContainer() throws IOException,
- InvalidStateTransitionException {
+ InvalidStateTransitionException, TimeoutException {
Set set = pipelineManager
.getContainersInPipeline(ratisContainer.getPipeline().getId());
@@ -199,7 +199,8 @@ public void testPipelineCloseWithPipelineAction() throws Exception {
@Test
@Flaky("HDDS-5604")
- public void testPipelineCloseWithLogFailure() throws IOException {
+ public void testPipelineCloseWithLogFailure()
+ throws IOException, TimeoutException {
EventQueue eventQ = (EventQueue) scm.getEventQueue();
PipelineActionHandler pipelineActionTest =
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestSCMRestart.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestSCMRestart.java
index 5bc59542d430..0d46111d4548 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestSCMRestart.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestSCMRestart.java
@@ -33,6 +33,7 @@
import java.io.IOException;
import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_PIPELINE_REPORT_INTERVAL;
import org.junit.Rule;
@@ -110,7 +111,8 @@ public static void shutdown() {
}
@Test
- public void testPipelineWithScmRestart() throws IOException {
+ public void testPipelineWithScmRestart()
+ throws IOException, TimeoutException {
// After restart make sure that the pipeline are still present
Pipeline ratisPipeline1AfterRestart =
pipelineManager.getPipeline(ratisPipeline1.getId());
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/upgrade/TestHDDSUpgrade.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/upgrade/TestHDDSUpgrade.java
index 85bb9aa9000e..766c47915542 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/upgrade/TestHDDSUpgrade.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/upgrade/TestHDDSUpgrade.java
@@ -367,7 +367,8 @@ private void testPostUpgradeConditionsDataNodes(
/*
* Helper function to test that we can create new pipelines Post-Upgrade.
*/
- private void testPostUpgradePipelineCreation() throws IOException {
+ private void testPostUpgradePipelineCreation()
+ throws IOException, TimeoutException {
Pipeline ratisPipeline1 = scmPipelineManager.createPipeline(RATIS_THREE);
scmPipelineManager.openPipeline(ratisPipeline1.getId());
Assert.assertEquals(0,
@@ -417,7 +418,7 @@ private void waitForPipelineCreated() throws Exception {
/*
* Helper function for container creation.
*/
- private void createTestContainers() throws IOException {
+ private void createTestContainers() throws IOException, TimeoutException {
XceiverClientManager xceiverClientManager = new XceiverClientManager(conf);
ContainerInfo ci1 = scmContainerManager.allocateContainer(
RATIS_THREE, "Owner1");
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java
index d352233a682d..2b6d4b6f7a6f 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java
@@ -107,6 +107,7 @@
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.stream.Stream;
@@ -438,7 +439,8 @@ private Map> createDeleteTXLog(
StorageContainerManager scm,
DeletedBlockLog delLog,
Map keyLocations,
- TestStorageContainerManagerHelper helper) throws IOException {
+ TestStorageContainerManagerHelper helper)
+ throws IOException, TimeoutException {
// These keys will be written into a bunch of containers,
// gets a set of container names, verify container containerBlocks
// on datanodes.
@@ -831,7 +833,7 @@ public void testCloseContainerCommandOnRestart() throws Exception {
private void addTransactions(StorageContainerManager scm,
DeletedBlockLog delLog,
Map> containerBlocksMap)
- throws IOException {
+ throws IOException, TimeoutException {
delLog.addTransactions(containerBlocksMap);
if (SCMHAUtils.isSCMHAEnabled(scm.getConfiguration())) {
scm.getScmHAManager().asSCMHADBTransactionBuffer().flush();
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestScmSafeMode.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestScmSafeMode.java
index d4e14f63acaa..ade08e46eb38 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestScmSafeMode.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestScmSafeMode.java
@@ -57,6 +57,7 @@
import java.util.HashMap;
import java.util.List;
import java.util.Map;
+import java.util.concurrent.TimeoutException;
import static org.apache.hadoop.hdds.client.ReplicationType.RATIS;
import static org.apache.hadoop.hdds.client.ReplicationFactor.ONE;
@@ -247,7 +248,8 @@ public void testSCMSafeMode() throws Exception {
HddsProtos.LifeCycleEvent.FINALIZE);
mapping.updateContainerState(c.containerID(),
LifeCycleEvent.CLOSE);
- } catch (IOException | InvalidStateTransitionException e) {
+ } catch (IOException | InvalidStateTransitionException |
+ TimeoutException e) {
LOG.info("Failed to change state of open containers.", e);
}
});
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMContainerPlacementPolicyMetrics.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMContainerPlacementPolicyMetrics.java
index 30efd2b38633..20c1572c7f57 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMContainerPlacementPolicyMetrics.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMContainerPlacementPolicyMetrics.java
@@ -54,6 +54,7 @@
import java.util.List;
import java.util.UUID;
import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
import java.util.stream.Collectors;
import static java.nio.charset.StandardCharsets.UTF_8;
@@ -101,7 +102,7 @@ public void setup() throws Exception {
* Verifies container placement metric.
*/
@Test @Timeout(unit = TimeUnit.MILLISECONDS, value = 60000)
- public void test() throws IOException {
+ public void test() throws IOException, TimeoutException {
String volumeName = UUID.randomUUID().toString();
String bucketName = UUID.randomUUID().toString();
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMInstallSnapshotWithHA.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMInstallSnapshotWithHA.java
index 94ac63f84b24..084d3f08c169 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMInstallSnapshotWithHA.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMInstallSnapshotWithHA.java
@@ -23,6 +23,7 @@
import java.util.Iterator;
import java.util.List;
import java.util.UUID;
+import java.util.concurrent.TimeoutException;
import org.apache.commons.io.FileUtils;
import org.apache.hadoop.hdds.client.RatisReplicationConfig;
@@ -284,7 +285,7 @@ public void testInstallCorruptedCheckpointFailure() throws Exception {
private List writeToIncreaseLogIndex(
StorageContainerManager scm, long targetLogIndex)
- throws IOException, InterruptedException {
+ throws IOException, InterruptedException, TimeoutException {
List containers = new ArrayList<>();
SCMStateMachine stateMachine =
scm.getScmHAManager().getRatisServer().getSCMStateMachine();
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/pipeline/TestSCMPipelineMetrics.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/pipeline/TestSCMPipelineMetrics.java
index 503b3a1030e2..ed323ec0a3cf 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/pipeline/TestSCMPipelineMetrics.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/pipeline/TestSCMPipelineMetrics.java
@@ -37,6 +37,7 @@
import java.io.IOException;
import java.util.Optional;
+import java.util.concurrent.TimeoutException;
import org.junit.Rule;
import org.junit.rules.Timeout;
@@ -95,7 +96,7 @@ public void testPipelineDestroy() {
cluster.getStorageContainerManager()
.getPipelineManager()
.closePipeline(pipeline.get(), false);
- } catch (IOException e) {
+ } catch (IOException | TimeoutException e) {
e.printStackTrace();
Assert.fail();
}
@@ -105,7 +106,7 @@ public void testPipelineDestroy() {
}
@Test
- public void testNumBlocksAllocated() throws IOException {
+ public void testNumBlocksAllocated() throws IOException, TimeoutException {
AllocatedBlock block =
cluster.getStorageContainerManager().getScmBlockManager()
.allocateBlock(5,
diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/fsck/ContainerHealthTask.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/fsck/ContainerHealthTask.java
index d29bf5074a55..4badac672e7e 100644
--- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/fsck/ContainerHealthTask.java
+++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/fsck/ContainerHealthTask.java
@@ -23,6 +23,7 @@
import java.util.HashSet;
import java.util.List;
import java.util.Set;
+import java.util.concurrent.TimeoutException;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.PlacementPolicy;
@@ -228,7 +229,7 @@ private boolean containerDeletedInSCM(ContainerInfo containerInfo) {
} catch (InvalidStateTransitionException e) {
LOG.error("Failed to transition Container state while processing " +
"container in Container Health task", e);
- } catch (IOException e) {
+ } catch (IOException | TimeoutException e) {
LOG.error("Got exception while processing container in" +
" Container Health task", e);
}
diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconContainerManager.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconContainerManager.java
index 7e5f8d828f93..cccd48124d9b 100644
--- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconContainerManager.java
+++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconContainerManager.java
@@ -28,6 +28,7 @@
import java.util.Map;
import java.util.UUID;
import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeoutException;
import java.util.stream.Collectors;
import org.apache.hadoop.conf.Configuration;
@@ -169,7 +170,7 @@ public void checkAndAddNewContainerBatch(
for (ContainerWithPipeline cwp : verifiedContainerPipeline) {
try {
addNewContainer(cwp);
- } catch (IOException ioe) {
+ } catch (IOException | TimeoutException ioe) {
LOG.error("Exception while checking and adding new container.", ioe);
}
}
@@ -233,7 +234,7 @@ private boolean isHealthy(ContainerReplicaProto.State replicaState) {
* @throws IOException on Error.
*/
public void addNewContainer(ContainerWithPipeline containerWithPipeline)
- throws IOException {
+ throws IOException, TimeoutException {
ContainerInfo containerInfo = containerWithPipeline.getContainerInfo();
try {
if (containerInfo.getState().equals(HddsProtos.LifeCycleState.OPEN)) {
@@ -260,7 +261,7 @@ public void addNewContainer(ContainerWithPipeline containerWithPipeline)
LOG.info("Successfully added no open container {} to Recon.",
containerInfo.containerID());
}
- } catch (IOException ex) {
+ } catch (IOException | TimeoutException ex) {
LOG.info("Exception while adding container {} .",
containerInfo.containerID(), ex);
pipelineManager.removeContainerFromPipeline(
diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconIncrementalContainerReportHandler.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconIncrementalContainerReportHandler.java
index fd1c4ad895d4..d7359a997ab8 100644
--- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconIncrementalContainerReportHandler.java
+++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconIncrementalContainerReportHandler.java
@@ -19,6 +19,7 @@
package org.apache.hadoop.ozone.recon.scm;
import java.io.IOException;
+import java.util.concurrent.TimeoutException;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
@@ -90,7 +91,8 @@ public void onMessage(final IncrementalContainerReportFromDatanode report,
success = false;
LOG.error("Received ICR from unknown datanode {}.",
report.getDatanodeDetails(), ex);
- } catch (IOException | InvalidStateTransitionException e) {
+ } catch (IOException | InvalidStateTransitionException |
+ TimeoutException e) {
success = false;
LOG.error("Exception while processing ICR for container {}",
replicaProto.getContainerID());
diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconPipelineManager.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconPipelineManager.java
index 93dcae08f844..41559d130a31 100644
--- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconPipelineManager.java
+++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconPipelineManager.java
@@ -21,6 +21,7 @@
import java.io.IOException;
import java.time.ZoneOffset;
import java.util.List;
+import java.util.concurrent.TimeoutException;
import java.util.stream.Collectors;
import org.apache.hadoop.hdds.conf.ConfigurationSource;
@@ -95,7 +96,8 @@ public static ReconPipelineManager newReconPipelineManager(
* @param pipelinesFromScm pipelines from SCM.
* @throws IOException on exception.
*/
- void initializePipelines(List pipelinesFromScm) throws IOException {
+ void initializePipelines(List pipelinesFromScm)
+ throws IOException, TimeoutException {
acquireWriteLock();
try {
@@ -141,7 +143,7 @@ public void removeInvalidPipelines(List pipelinesFromScm) {
getStateManager().updatePipelineState(
pipelineID.getProtobuf(),
HddsProtos.PipelineState.PIPELINE_CLOSED);
- } catch (IOException e) {
+ } catch (IOException | TimeoutException e) {
LOG.warn("Pipeline {} not found while updating state. ",
p.getId(), e);
}
@@ -149,7 +151,7 @@ public void removeInvalidPipelines(List pipelinesFromScm) {
try {
LOG.info("Removing invalid pipeline {} from Recon.", pipelineID);
closePipeline(p, false);
- } catch (IOException e) {
+ } catch (IOException | TimeoutException e) {
LOG.warn("Unable to remove pipeline {}", pipelineID, e);
}
});
@@ -163,7 +165,8 @@ public void removeInvalidPipelines(List pipelinesFromScm) {
* @throws IOException
*/
@VisibleForTesting
- public void addPipeline(Pipeline pipeline) throws IOException {
+ public void addPipeline(Pipeline pipeline)
+ throws IOException, TimeoutException {
acquireWriteLock();
try {
getStateManager().addPipeline(
diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconPipelineReportHandler.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconPipelineReportHandler.java
index 78c8469f208a..bf280b37e777 100644
--- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconPipelineReportHandler.java
+++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconPipelineReportHandler.java
@@ -19,6 +19,7 @@
package org.apache.hadoop.ozone.recon.scm;
import java.io.IOException;
+import java.util.concurrent.TimeoutException;
import org.apache.hadoop.hdds.conf.ConfigurationSource;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
@@ -57,7 +58,8 @@ public ReconPipelineReportHandler(SafeModeManager scmSafeModeManager,
@Override
protected void processPipelineReport(PipelineReport report,
- DatanodeDetails dn, EventPublisher publisher) throws IOException {
+ DatanodeDetails dn, EventPublisher publisher)
+ throws IOException, TimeoutException {
ReconPipelineManager reconPipelineManager =
(ReconPipelineManager)getPipelineManager();
diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconStorageContainerManagerFacade.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconStorageContainerManagerFacade.java
index 09b91f6f4477..f619e29703b3 100644
--- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconStorageContainerManagerFacade.java
+++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconStorageContainerManagerFacade.java
@@ -28,6 +28,7 @@
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeoutException;
import org.apache.commons.io.FileUtils;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
@@ -363,7 +364,7 @@ private void initializePipelinesFromScm() {
List pipelinesFromScm = scmServiceProvider.getPipelines();
LOG.info("Obtained {} pipelines from SCM.", pipelinesFromScm.size());
pipelineManager.initializePipelines(pipelinesFromScm);
- } catch (IOException ioEx) {
+ } catch (IOException | TimeoutException ioEx) {
LOG.error("Exception encountered while getting pipelines from SCM.",
ioEx);
}
diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java
index d968904aab0e..85da3dd5c93d 100644
--- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java
+++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java
@@ -41,6 +41,7 @@
import java.util.Map;
import java.util.Set;
import java.util.UUID;
+import java.util.concurrent.TimeoutException;
import java.util.stream.Collectors;
import javax.ws.rs.WebApplicationException;
import javax.ws.rs.core.Response;
@@ -409,7 +410,7 @@ public void testGetContainersWithPrevKey() {
}
@Test
- public void testGetMissingContainers() throws IOException {
+ public void testGetMissingContainers() throws IOException, TimeoutException {
Response response = containerEndpoint.getMissingContainers();
MissingContainersResponse responseObject =
@@ -477,7 +478,7 @@ ContainerInfo newContainerInfo(long containerId) {
.build();
}
- void putContainerInfos(int num) throws IOException {
+ void putContainerInfos(int num) throws IOException, TimeoutException {
for (int i = 1; i <= num; i++) {
final ContainerInfo info = newContainerInfo(i);
reconContainerManager.addNewContainer(
@@ -486,7 +487,7 @@ void putContainerInfos(int num) throws IOException {
}
@Test
- public void testUnhealthyContainers() throws IOException {
+ public void testUnhealthyContainers() throws IOException, TimeoutException {
Response response = containerEndpoint.getUnhealthyContainers(1000, 1);
UnhealthyContainersResponse responseObject =
@@ -580,7 +581,8 @@ public void testUnhealthyContainers() throws IOException {
}
@Test
- public void testUnhealthyContainersFilteredResponse() throws IOException {
+ public void testUnhealthyContainersFilteredResponse()
+ throws IOException, TimeoutException {
String missing = UnHealthyContainerStates.MISSING.toString();
Response response = containerEndpoint
@@ -633,7 +635,8 @@ public void testUnhealthyContainersInvalidState() {
}
@Test
- public void testUnhealthyContainersPaging() throws IOException {
+ public void testUnhealthyContainersPaging()
+ throws IOException, TimeoutException {
putContainerInfos(6);
uuid1 = newDatanode("host1", "127.0.0.1");
uuid2 = newDatanode("host2", "127.0.0.2");
diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/scm/AbstractReconContainerManagerTest.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/scm/AbstractReconContainerManagerTest.java
index d6a7a13c3e22..9fac275b337a 100644
--- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/scm/AbstractReconContainerManagerTest.java
+++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/scm/AbstractReconContainerManagerTest.java
@@ -22,6 +22,7 @@
import java.time.ZoneId;
import java.util.LinkedList;
import java.util.List;
+import java.util.concurrent.TimeoutException;
import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
@@ -150,7 +151,7 @@ protected ReconContainerManager getContainerManager() {
}
private StorageContainerServiceProvider getScmServiceProvider()
- throws IOException {
+ throws IOException, TimeoutException {
Pipeline pipeline = getRandomPipeline();
getPipelineManager().addPipeline(pipeline);
@@ -206,7 +207,7 @@ protected Table getContainerTable()
}
protected ContainerWithPipeline getTestContainer(LifeCycleState state)
- throws IOException {
+ throws IOException, TimeoutException {
ContainerID containerID = ContainerID.valueOf(100L);
Pipeline pipeline = getRandomPipeline();
pipelineManager.addPipeline(pipeline);
@@ -224,7 +225,7 @@ protected ContainerWithPipeline getTestContainer(LifeCycleState state)
protected ContainerWithPipeline getTestContainer(long id,
LifeCycleState state)
- throws IOException {
+ throws IOException, TimeoutException {
ContainerID containerID = ContainerID.valueOf(id);
Pipeline pipeline = getRandomPipeline();
pipelineManager.addPipeline(pipeline);
diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/scm/TestReconContainerManager.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/scm/TestReconContainerManager.java
index 2612a7db246a..7a03929cf865 100644
--- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/scm/TestReconContainerManager.java
+++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/scm/TestReconContainerManager.java
@@ -33,6 +33,7 @@
import java.util.Map;
import java.util.NavigableSet;
import java.util.UUID;
+import java.util.concurrent.TimeoutException;
import org.apache.hadoop.hdds.client.RatisReplicationConfig;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
@@ -55,7 +56,7 @@
public class TestReconContainerManager
extends AbstractReconContainerManagerTest {
@Test
- public void testAddNewOpenContainer() throws IOException {
+ public void testAddNewOpenContainer() throws IOException, TimeoutException {
ContainerWithPipeline containerWithPipeline =
getTestContainer(LifeCycleState.OPEN);
ContainerID containerID =
@@ -87,7 +88,8 @@ public void testAddNewOpenContainer() throws IOException {
}
@Test
- public void testAddNewClosedContainer() throws IOException {
+ public void testAddNewClosedContainer()
+ throws IOException, TimeoutException {
ContainerWithPipeline containerWithPipeline = getTestContainer(CLOSED);
ContainerID containerID =
containerWithPipeline.getContainerInfo().containerID();
@@ -195,7 +197,8 @@ ContainerInfo newContainerInfo(long containerId, Pipeline pipeline) {
}
@Test
- public void testUpdateAndRemoveContainerReplica() throws IOException {
+ public void testUpdateAndRemoveContainerReplica()
+ throws IOException, TimeoutException {
// Sanity checking updateContainerReplica and ContainerReplicaHistory
// Init Container 1
diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/scm/TestReconIncrementalContainerReportHandler.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/scm/TestReconIncrementalContainerReportHandler.java
index ecbdbbace2cd..228244645fc4 100644
--- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/scm/TestReconIncrementalContainerReportHandler.java
+++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/scm/TestReconIncrementalContainerReportHandler.java
@@ -32,6 +32,7 @@
import java.nio.file.Paths;
import java.util.Arrays;
import java.util.UUID;
+import java.util.concurrent.TimeoutException;
import org.apache.hadoop.hdds.HddsConfigKeys;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
@@ -112,7 +113,8 @@ public void testProcessICR() throws IOException, NodeNotFoundException {
}
@Test
- public void testProcessICRStateMismatch() throws IOException {
+ public void testProcessICRStateMismatch()
+ throws IOException, TimeoutException {
// Recon container state is "OPEN".
// Replica state could be any Non OPEN state.
diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/scm/TestReconPipelineManager.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/scm/TestReconPipelineManager.java
index 343e07eaef78..2cd7211c95c9 100644
--- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/scm/TestReconPipelineManager.java
+++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/scm/TestReconPipelineManager.java
@@ -22,6 +22,7 @@
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
+import java.util.concurrent.TimeoutException;
import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
@@ -99,7 +100,7 @@ public void tearDown() throws Exception {
}
@Test
- public void testInitialize() throws IOException {
+ public void testInitialize() throws IOException, TimeoutException {
// Get 3 OPEN pipelines from SCM.
List pipelinesFromScm = getPipelines(3);
@@ -173,7 +174,7 @@ public void testInitialize() throws IOException {
}
@Test
- public void testAddPipeline() throws IOException {
+ public void testAddPipeline() throws IOException, TimeoutException {
Pipeline pipeline = getRandomPipeline();
NetworkTopology clusterMap = new NetworkTopologyImpl(conf);
diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/scm/TestReconPipelineReportHandler.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/scm/TestReconPipelineReportHandler.java
index fcf67a4721fa..049559efec26 100644
--- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/scm/TestReconPipelineReportHandler.java
+++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/scm/TestReconPipelineReportHandler.java
@@ -25,6 +25,7 @@
import static org.mockito.Mockito.when;
import java.io.IOException;
+import java.util.concurrent.TimeoutException;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
@@ -43,7 +44,8 @@
public class TestReconPipelineReportHandler {
@Test
- public void testProcessPipelineReport() throws IOException {
+ public void testProcessPipelineReport()
+ throws IOException, TimeoutException {
// Check with pipeline which does not exist in Recon.
Pipeline pipeline = getRandomPipeline();
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/SCMThroughputBenchmark.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/SCMThroughputBenchmark.java
index b3887783c4e5..4fe7d4af97e5 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/SCMThroughputBenchmark.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/SCMThroughputBenchmark.java
@@ -79,6 +79,7 @@
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.stream.Collectors;
@@ -762,7 +763,7 @@ public void run() {
try {
datanode.sendHeartbeat();
succReportSendCounter.incrementAndGet();
- } catch (IOException e) {
+ } catch (IOException | TimeoutException e) {
LOG.error("{}", e);
failReportSendCounter.incrementAndGet();
}
@@ -802,7 +803,7 @@ public void register() throws IOException {
}
}
- public void sendHeartbeat() throws IOException {
+ public void sendHeartbeat() throws IOException, TimeoutException {
SCMHeartbeatRequestProto heartbeatRequest = SCMHeartbeatRequestProto
.newBuilder()
.setDatanodeDetails(datanodeDetails.getProtoBufMessage())