HDDS-125. Cleanup HDDS CheckStyle issues.

Contributed by Anu Engineer.
This commit is contained in:
Anu Engineer 2018-05-29 09:54:06 -07:00
parent 17aa40f669
commit 9502b47bd2
24 changed files with 263 additions and 95 deletions

View File

@ -41,7 +41,6 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.UUID;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;

View File

@ -190,7 +190,7 @@ public class DeletedBlockLogImpl implements DeletedBlockLog {
try {
for(Long txID : txIDs) {
try {
byte [] deleteBlockBytes =
byte[] deleteBlockBytes =
deletedStore.get(Longs.toByteArray(txID));
if (deleteBlockBytes == null) {
LOG.warn("Delete txID {} not found", txID);

View File

@ -152,7 +152,8 @@ public class ContainerMapping implements Mapping {
ContainerInfo containerInfo;
lock.lock();
try {
byte[] containerBytes = containerStore.get(Longs.toByteArray(containerID));
byte[] containerBytes = containerStore.get(
Longs.toByteArray(containerID));
if (containerBytes == null) {
throw new SCMException(
"Specified key does not exist. key : " + containerID,
@ -229,7 +230,8 @@ public class ContainerMapping implements Mapping {
containerStateManager.allocateContainer(
pipelineSelector, type, replicationFactor, owner);
byte[] containerIDBytes = Longs.toByteArray(containerInfo.getContainerID());
byte[] containerIDBytes = Longs.toByteArray(
containerInfo.getContainerID());
containerStore.put(containerIDBytes, containerInfo.getProtobuf()
.toByteArray());
} finally {

View File

@ -230,18 +230,18 @@ public class ContainerStateManager implements Closeable {
*
* Container State Flow:
*
* [ALLOCATED]------->[CREATING]--------->[OPEN]---------->[CLOSING]------->[CLOSED]
* (CREATE) | (CREATED) (FINALIZE) (CLOSE) |
* | |
* | |
* |(TIMEOUT) (DELETE)|
* | |
* +------------------> [DELETING] <-------------------+
* |
* |
* (CLEANUP)|
* |
* [DELETED]
* [ALLOCATED]---->[CREATING]------>[OPEN]-------->[CLOSING]------->[CLOSED]
* (CREATE) | (CREATED) (FINALIZE) (CLOSE) |
* | |
* | |
* |(TIMEOUT) (DELETE)|
* | |
* +-------------> [DELETING] <-------------------+
* |
* |
* (CLEANUP)|
* |
* [DELETED]
*/
private void initializeStateMachine() {
stateMachine.addTransition(LifeCycleState.ALLOCATED,

View File

@ -45,7 +45,8 @@ public interface Mapping extends Closeable {
* The max size of the searching range cannot exceed the
* value of count.
*
* @param startContainerID start containerID, >=0, start searching at the head if 0.
* @param startContainerID start containerID, >=0,
* start searching at the head if 0.
* @param count count must be >= 0
* Usually the count will be replace with a very big
* value instead of being unlimited in case the db is very big.
@ -53,7 +54,8 @@ public interface Mapping extends Closeable {
* @return a list of container.
* @throws IOException
*/
List<ContainerInfo> listContainer(long startContainerID, int count) throws IOException;
List<ContainerInfo> listContainer(long startContainerID, int count)
throws IOException;
/**
* Allocates a new container for a given keyName and replication factor.
@ -64,7 +66,8 @@ public interface Mapping extends Closeable {
* @throws IOException
*/
ContainerInfo allocateContainer(HddsProtos.ReplicationType type,
HddsProtos.ReplicationFactor replicationFactor, String owner) throws IOException;
HddsProtos.ReplicationFactor replicationFactor, String owner)
throws IOException;
/**
* Deletes a container from SCM.

View File

@ -31,7 +31,7 @@ import java.util.UUID;
@InterfaceAudience.Private
public interface SCMNodeStorageStatMXBean {
/**
* Get the capacity of the dataNode
* Get the capacity of the dataNode.
* @param datanodeID Datanode Id
* @return long
*/
@ -52,7 +52,7 @@ public interface SCMNodeStorageStatMXBean {
long getUsedSpace(UUID datanodeId);
/**
* Returns the total capacity of all dataNodes
* Returns the total capacity of all dataNodes.
* @return long
*/
long getTotalCapacity();

View File

@ -56,7 +56,7 @@ public class SCMNodeStorageStatMap implements SCMNodeStorageStatMXBean {
// NodeStorageInfo MXBean
private ObjectName scmNodeStorageInfoBean;
/**
* constructs the scmNodeStorageReportMap object
* constructs the scmNodeStorageReportMap object.
*/
public SCMNodeStorageStatMap(OzoneConfiguration conf) {
// scmNodeStorageReportMap = new ConcurrentHashMap<>();
@ -73,6 +73,9 @@ public class SCMNodeStorageStatMap implements SCMNodeStorageStatMXBean {
HDDS_DATANODE_STORAGE_UTILIZATION_CRITICAL_THRESHOLD_DEFAULT);
}
/**
* Enum that Describes what we should do at various thresholds.
*/
public enum UtilizationThreshold {
NORMAL, WARN, CRITICAL;
}
@ -107,8 +110,8 @@ public class SCMNodeStorageStatMap implements SCMNodeStorageStatMXBean {
* @param datanodeID -- Datanode UUID
* @param report - set if StorageReports.
*/
public void insertNewDatanode(UUID datanodeID, Set<StorageLocationReport> report)
throws SCMException {
public void insertNewDatanode(UUID datanodeID,
Set<StorageLocationReport> report) throws SCMException {
Preconditions.checkNotNull(report);
Preconditions.checkState(report.size() != 0);
Preconditions.checkNotNull(datanodeID);
@ -142,8 +145,8 @@ public class SCMNodeStorageStatMap implements SCMNodeStorageStatMXBean {
* @throws SCMException - if we don't know about this datanode, for new DN
* use insertNewDatanode.
*/
public void updateDatanodeMap(UUID datanodeID, Set<StorageLocationReport> report)
throws SCMException {
public void updateDatanodeMap(UUID datanodeID,
Set<StorageLocationReport> report) throws SCMException {
Preconditions.checkNotNull(datanodeID);
Preconditions.checkNotNull(report);
Preconditions.checkState(report.size() != 0);
@ -301,7 +304,7 @@ public class SCMNodeStorageStatMap implements SCMNodeStorageStatMXBean {
}
/**
* removes the dataNode from scmNodeStorageReportMap
* removes the dataNode from scmNodeStorageReportMap.
* @param datanodeID
* @throws SCMException in case the dataNode is not found in the map.
*/
@ -339,11 +342,11 @@ public class SCMNodeStorageStatMap implements SCMNodeStorageStatMXBean {
}
/**
* get the scmUsed ratio
* get the scmUsed ratio.
*/
public double getScmUsedratio(long scmUsed, long capacity) {
double scmUsedRatio =
truncateDecimals (scmUsed / (double) capacity);
truncateDecimals(scmUsed / (double) capacity);
return scmUsedRatio;
}
/**

View File

@ -69,14 +69,14 @@ public class StorageReportResult {
}
public ReportResultBuilder setFullVolumeSet(
Set<StorageLocationReport> fullVolumes) {
this.fullVolumes = fullVolumes;
Set<StorageLocationReport> fullVolumesSet) {
this.fullVolumes = fullVolumesSet;
return this;
}
public ReportResultBuilder setFailedVolumeSet(
Set<StorageLocationReport> failedVolumes) {
this.failedVolumes = failedVolumes;
Set<StorageLocationReport> failedVolumesSet) {
this.failedVolumes = failedVolumesSet;
return this;
}

View File

@ -92,7 +92,7 @@ public class Node2ContainerMap {
}
/**
* Removes datanode Entry from the map
* Removes datanode Entry from the map.
* @param datanodeID - Datanode ID.
*/
public void removeDatanode(UUID datanodeID) {

View File

@ -170,8 +170,9 @@ public class PipelineSelector {
throws IOException {
PipelineManager manager = getPipelineManager(replicationType);
Preconditions.checkNotNull(manager, "Found invalid pipeline manager");
LOG.debug("Getting replication pipeline forReplicationType {} : ReplicationFactor {}",
replicationType.toString(), replicationFactor.toString());
LOG.debug("Getting replication pipeline forReplicationType {} :" +
" ReplicationFactor {}", replicationType.toString(),
replicationFactor.toString());
return manager.
getPipeline(replicationFactor, replicationType);
}

View File

@ -45,7 +45,6 @@ import org.apache.hadoop.hdds.scm.node.SCMNodeManager;
import org.apache.hadoop.hdds.server.ServiceRuntimeInfoImpl;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.ipc.ProtobufRpcEngine;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
import org.apache.hadoop.metrics2.util.MBeans;
@ -87,7 +86,7 @@ import static org.apache.hadoop.util.ExitUtil.terminate;
* create a container, which then can be used to store data.
*/
@InterfaceAudience.LimitedPrivate({"HDFS", "CBLOCK", "OZONE", "HBASE"})
public class StorageContainerManager extends ServiceRuntimeInfoImpl
public final class StorageContainerManager extends ServiceRuntimeInfoImpl
implements SCMMXBean {
private static final Logger LOG = LoggerFactory

View File

@ -39,7 +39,6 @@ import org.junit.runners.Parameterized.Parameters;
import java.io.File;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.net.MalformedURLException;
import java.net.URL;
import java.net.URLConnection;
import java.util.Arrays;

View File

@ -0,0 +1,23 @@
/*
* 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.
*
*/
/**
* Make checkstyle happy.
* */
package org.apache.hadoop.hdds.scm.block;

View File

@ -216,8 +216,10 @@ public class TestContainerMapping {
mapping.processContainerReports(crBuilder.build());
ContainerInfo updatedContainer = mapping.getContainer(info.getContainerID());
Assert.assertEquals(100000000L, updatedContainer.getNumberOfKeys());
ContainerInfo updatedContainer =
mapping.getContainer(info.getContainerID());
Assert.assertEquals(100000000L,
updatedContainer.getNumberOfKeys());
Assert.assertEquals(2000000000L, updatedContainer.getUsedBytes());
}
@ -251,8 +253,10 @@ public class TestContainerMapping {
mapping.processContainerReports(crBuilder.build());
ContainerInfo updatedContainer = mapping.getContainer(info.getContainerID());
Assert.assertEquals(500000000L, updatedContainer.getNumberOfKeys());
ContainerInfo updatedContainer =
mapping.getContainer(info.getContainerID());
Assert.assertEquals(500000000L,
updatedContainer.getNumberOfKeys());
Assert.assertEquals(5368705120L, updatedContainer.getUsedBytes());
NavigableSet<ContainerID> pendingCloseContainers = mapping.getStateManager()
.getMatchingContainerIDs(

View File

@ -0,0 +1,22 @@
/*
* 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.
*
*/
/**
* Make CheckStyle happy.
*/
package org.apache.hadoop.hdds.scm.container.closer;

View File

@ -0,0 +1,22 @@
/*
* 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.
*
*/
/**
* Make CheckStyle Happy.
*/
package org.apache.hadoop.hdds.scm.container;

View File

@ -0,0 +1,22 @@
/*
* 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.
*
*/
/**
* Make CheckStyle Happy.
*/
package org.apache.hadoop.hdds.scm.container.states;

View File

@ -510,42 +510,42 @@ public class TestNodeManager {
* @throws InterruptedException
* @throws TimeoutException
*/
/**
* These values are very important. Here is what it means so you don't
* have to look it up while reading this code.
*
* OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL - This the frequency of the
* HB processing thread that is running in the SCM. This thread must run
* for the SCM to process the Heartbeats.
*
* OZONE_SCM_HEARTBEAT_INTERVAL - This is the frequency at which
* datanodes will send heartbeats to SCM. Please note: This is the only
* config value for node manager that is specified in seconds. We don't
* want SCM heartbeat resolution to be more than in seconds.
* In this test it is not used, but we are forced to set it because we
* have validation code that checks Stale Node interval and Dead Node
* interval is larger than the value of
* OZONE_SCM_HEARTBEAT_INTERVAL.
*
* OZONE_SCM_STALENODE_INTERVAL - This is the time that must elapse
* from the last heartbeat for us to mark a node as stale. In this test
* we set that to 3. That is if a node has not heartbeat SCM for last 3
* seconds we will mark it as stale.
*
* OZONE_SCM_DEADNODE_INTERVAL - This is the time that must elapse
* from the last heartbeat for a node to be marked dead. We have an
* additional constraint that this must be at least 2 times bigger than
* Stale node Interval.
*
* With these we are trying to explore the state of this cluster with
* various timeouts. Each section is commented so that you can keep
* track of the state of the cluster nodes.
*
*/
@Test
public void testScmClusterIsInExpectedState1() throws IOException,
InterruptedException, TimeoutException {
/**
* These values are very important. Here is what it means so you don't
* have to look it up while reading this code.
*
* OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL - This the frequency of the
* HB processing thread that is running in the SCM. This thread must run
* for the SCM to process the Heartbeats.
*
* OZONE_SCM_HEARTBEAT_INTERVAL - This is the frequency at which
* datanodes will send heartbeats to SCM. Please note: This is the only
* config value for node manager that is specified in seconds. We don't
* want SCM heartbeat resolution to be more than in seconds.
* In this test it is not used, but we are forced to set it because we
* have validation code that checks Stale Node interval and Dead Node
* interval is larger than the value of
* OZONE_SCM_HEARTBEAT_INTERVAL.
*
* OZONE_SCM_STALENODE_INTERVAL - This is the time that must elapse
* from the last heartbeat for us to mark a node as stale. In this test
* we set that to 3. That is if a node has not heartbeat SCM for last 3
* seconds we will mark it as stale.
*
* OZONE_SCM_DEADNODE_INTERVAL - This is the time that must elapse
* from the last heartbeat for a node to be marked dead. We have an
* additional constraint that this must be at least 2 times bigger than
* Stale node Interval.
*
* With these we are trying to explore the state of this cluster with
* various timeouts. Each section is commented so that you can keep
* track of the state of the cluster nodes.
*
*/
OzoneConfiguration conf = getConf();
conf.setTimeDuration(OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL, 100,
MILLISECONDS);

View File

@ -42,11 +42,14 @@ import java.util.HashSet;
import java.io.IOException;
import java.util.concurrent.ConcurrentHashMap;
/**
* Test Node Storage Map.
*/
public class TestSCMNodeStorageStatMap {
private final static int DATANODE_COUNT = 100;
final long capacity = 10L * OzoneConsts.GB;
final long used = 2L * OzoneConsts.GB;
final long remaining = capacity - used;
private final long capacity = 10L * OzoneConsts.GB;
private final long used = 2L * OzoneConsts.GB;
private final long remaining = capacity - used;
private static OzoneConfiguration conf = new OzoneConfiguration();
private final Map<UUID, Set<StorageLocationReport>> testData =
new ConcurrentHashMap<>();
@ -59,9 +62,10 @@ public class TestSCMNodeStorageStatMap {
UUID dnId = UUID.randomUUID();
Set<StorageLocationReport> reportSet = new HashSet<>();
String path = GenericTestUtils.getTempPath(
TestSCMNodeStorageStatMap.class.getSimpleName() + "-" + Integer
.toString(dnIndex));
StorageLocationReport.Builder builder = StorageLocationReport.newBuilder();
TestSCMNodeStorageStatMap.class.getSimpleName() + "-" +
Integer.toString(dnIndex));
StorageLocationReport.Builder builder =
StorageLocationReport.newBuilder();
builder.setStorageType(StorageType.DISK).setId(dnId.toString())
.setStorageLocation(path).setScmUsed(used).setRemaining(remaining)
.setCapacity(capacity).setFailed(false);
@ -139,12 +143,12 @@ public class TestSCMNodeStorageStatMap {
String path =
GenericTestUtils.getRandomizedTempPath().concat("/" + storageId);
StorageLocationReport report = reportSet.iterator().next();
long capacity = report.getCapacity();
long used = report.getScmUsed();
long remaining = report.getRemaining();
long reportCapacity = report.getCapacity();
long reportScmUsed = report.getScmUsed();
long reportRemaining = report.getRemaining();
List<SCMStorageReport> reports = TestUtils
.createStorageReport(capacity, used, remaining, path, null, storageId,
1);
.createStorageReport(reportCapacity, reportScmUsed, reportRemaining,
path, null, storageId, 1);
StorageReportResult result =
map.processNodeReport(key, TestUtils.createNodeReport(reports));
Assert.assertEquals(result.getStatus(),
@ -158,7 +162,7 @@ public class TestSCMNodeStorageStatMap {
SCMNodeStorageStatMap.ReportStatus.ALL_IS_WELL);
reportList.add(TestUtils
.createStorageReport(capacity, capacity, 0, path, null,
.createStorageReport(reportCapacity, reportCapacity, 0, path, null,
UUID.randomUUID().toString(), 1).get(0));
result = map.processNodeReport(key, TestUtils.createNodeReport(reportList));
Assert.assertEquals(result.getStatus(),
@ -166,8 +170,8 @@ public class TestSCMNodeStorageStatMap {
// Mark a disk failed
SCMStorageReport srb2 = SCMStorageReport.newBuilder()
.setStorageUuid(UUID.randomUUID().toString())
.setStorageLocation(srb.getStorageLocation()).setScmUsed(capacity)
.setCapacity(capacity).setRemaining(0).setFailed(true).build();
.setStorageLocation(srb.getStorageLocation()).setScmUsed(reportCapacity)
.setCapacity(reportCapacity).setRemaining(0).setFailed(true).build();
reportList.add(srb2);
nrb.addAllStorageReport(reportList);
result = map.processNodeReport(key, nrb.addStorageReport(srb).build());

View File

@ -0,0 +1,22 @@
/*
* 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.
*
*/
/**
* Make CheckStyle Happy.
*/
package org.apache.hadoop.hdds.scm.node;

View File

@ -32,8 +32,6 @@ import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReportsResponseProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMHeartbeatResponseProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMNodeReport;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMRegisteredCmdResponseProto;
import org.apache.hadoop.hdds.protocol.proto

View File

@ -0,0 +1,22 @@
/*
* 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.
*
*/
/**
* Make CheckStyle Happy.
*/
package org.apache.hadoop.ozone.container.common;

View File

@ -0,0 +1,22 @@
/*
* 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.
*
*/
/**
* Make CheckStyle Happy.
*/
package org.apache.hadoop.ozone.container.placement;

View File

@ -202,8 +202,8 @@ public class TestContainerSupervisor {
ppool.handleContainerReport(reportsProto);
}
clist = datanodeStateManager.getContainerReport(wayOverReplicatedContainerID,
ppool.getPool().getPoolName(), 7);
clist = datanodeStateManager.getContainerReport(
wayOverReplicatedContainerID, ppool.getPool().getPoolName(), 7);
for (ContainerReportsRequestProto reportsProto : clist) {
ppool.handleContainerReport(reportsProto);
@ -264,7 +264,8 @@ public class TestContainerSupervisor {
"PoolNew", 1);
containerSupervisor.handleContainerReport(clist.get(0));
GenericTestUtils.waitFor(() ->
inProgressLog.getOutput().contains(Long.toString(newContainerID)) && inProgressLog
inProgressLog.getOutput()
.contains(Long.toString(newContainerID)) && inProgressLog
.getOutput().contains(id.getUuidString()),
200, 10 * 1000);
} finally {