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 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 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())