HDDS-382. Remove RatisTestHelper#RatisTestSuite constructor argument and fix checkstyle in ContainerTestHelper, GenericTestUtils

Contributed by Nandakumar.
This commit is contained in:
Anu Engineer 2018-08-28 14:06:19 -07:00
parent 33f42efc94
commit c5629d546d
80 changed files with 226 additions and 282 deletions

View File

@ -38,7 +38,6 @@
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Semaphore;
@ -188,13 +187,12 @@ public ContainerProtos.ContainerCommandResponseProto sendCommand(
/**
* Create a pipeline.
*
* @param pipeline - pipeline to be created.
* @param ignored - pipeline to be created.
*/
@Override
public void createPipeline(Pipeline pipeline)
public void createPipeline(Pipeline ignored)
throws IOException {
// For stand alone pipeline, there is no notion called setup pipeline.
return;
}
/**

View File

@ -38,7 +38,6 @@
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Semaphore;
@ -218,13 +217,12 @@ private void reconnect() throws IOException {
/**
* Create a pipeline.
*
* @param pipeline - pipeline to be created.
* @param ignored - pipeline to be created.
*/
@Override
public void createPipeline(Pipeline pipeline)
public void createPipeline(Pipeline ignored)
throws IOException {
// For stand alone pipeline, there is no notion called setup pipeline.
return;
}
/**

View File

@ -154,7 +154,7 @@ public XceiverClientSpi call() throws Exception {
break;
case CHAINED:
default:
throw new IOException ("not implemented" + pipeline.getType());
throw new IOException("not implemented" + pipeline.getType());
}
client.connect();
return client;

View File

@ -65,7 +65,8 @@ public class ChunkInputStream extends InputStream implements Seekable {
* @param chunks list of chunks to read
* @param traceID container protocol call traceID
*/
public ChunkInputStream(BlockID blockID, XceiverClientManager xceiverClientManager,
public ChunkInputStream(
BlockID blockID, XceiverClientManager xceiverClientManager,
XceiverClientSpi xceiverClient, List<ChunkInfo> chunks, String traceID) {
this.blockID = blockID;
this.traceID = traceID;
@ -211,8 +212,8 @@ public synchronized void seek(long pos) throws IOException {
if (pos < 0 || (chunks.size() == 0 && pos > 0)
|| pos >= chunkOffset[chunks.size() - 1] + chunks.get(chunks.size() - 1)
.getLen()) {
throw new EOFException(
"EOF encountered pos: " + pos + " container key: " + blockID.getLocalID());
throw new EOFException("EOF encountered pos: " + pos + " container key: "
+ blockID.getLocalID());
}
if (chunkIndex == -1) {
chunkIndex = Arrays.binarySearch(chunkOffset, pos);

View File

@ -76,8 +76,8 @@ public class ChunkOutputStream extends OutputStream {
* @param chunkSize chunk size
*/
public ChunkOutputStream(BlockID blockID, String key,
XceiverClientManager xceiverClientManager, XceiverClientSpi xceiverClient,
String traceID, int chunkSize) {
XceiverClientManager xceiverClientManager,
XceiverClientSpi xceiverClient, String traceID, int chunkSize) {
this.blockID = blockID;
this.key = key;
this.traceID = traceID;

View File

@ -23,7 +23,7 @@
import java.util.Objects;
/**
* BlockID of ozone (containerID localID)
* BlockID of ozone (containerID localID).
*/
public class BlockID {
private long containerID;
@ -65,7 +65,8 @@ public ContainerProtos.DatanodeBlockID getDatanodeBlockIDProtobuf() {
setContainerID(containerID).setLocalID(localID).build();
}
public static BlockID getFromProtobuf(ContainerProtos.DatanodeBlockID blockID) {
public static BlockID getFromProtobuf(
ContainerProtos.DatanodeBlockID blockID) {
return new BlockID(blockID.getContainerID(),
blockID.getLocalID());
}

View File

@ -20,7 +20,6 @@
import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.ContainerCommandRequestProto;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
@ -29,7 +28,6 @@
import java.io.Closeable;
import java.io.IOException;
import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.atomic.AtomicInteger;

View File

@ -43,8 +43,8 @@ public Builder setPipeline(Pipeline p) {
return this;
}
public Builder setBlockID(BlockID blockID) {
this.blockID = blockID;
public Builder setBlockID(BlockID blockId) {
this.blockID = blockId;
return this;
}

View File

@ -396,13 +396,13 @@ public static class Builder {
private ReplicationType replicationType;
public Builder setReplicationType(
ReplicationType replicationType) {
this.replicationType = replicationType;
ReplicationType repType) {
this.replicationType = repType;
return this;
}
public Builder setPipelineID(PipelineID pipelineID) {
this.pipelineID = pipelineID;
public Builder setPipelineID(PipelineID pipelineId) {
this.pipelineID = pipelineId;
return this;
}
@ -447,8 +447,8 @@ public Builder setOwner(String containerOwner) {
return this;
}
public Builder setDeleteTransactionId(long deleteTransactionId) {
this.deleteTransactionId = deleteTransactionId;
public Builder setDeleteTransactionId(long deleteTransactionID) {
this.deleteTransactionId = deleteTransactionID;
return this;
}

View File

@ -26,8 +26,8 @@
/**
* Class wraps ozone container info.
*/
public class ContainerWithPipeline
implements Comparator<ContainerWithPipeline>, Comparable<ContainerWithPipeline> {
public class ContainerWithPipeline implements Comparator<ContainerWithPipeline>,
Comparable<ContainerWithPipeline> {
private final ContainerInfo containerInfo;
private final Pipeline pipeline;
@ -45,7 +45,8 @@ public Pipeline getPipeline() {
return pipeline;
}
public static ContainerWithPipeline fromProtobuf(HddsProtos.ContainerWithPipeline allocatedContainer) {
public static ContainerWithPipeline fromProtobuf(
HddsProtos.ContainerWithPipeline allocatedContainer) {
return new ContainerWithPipeline(
ContainerInfo.fromProtobuf(allocatedContainer.getContainerInfo()),
Pipeline.getFromProtoBuf(allocatedContainer.getPipeline()));

View File

@ -169,8 +169,8 @@ public List<DatanodeDetails> getMachines() {
*/
public List<String> getDatanodeHosts() {
List<String> dataHosts = new ArrayList<>();
for (DatanodeDetails id :getDatanodes().values()) {
dataHosts.add(id.getHostName());
for (DatanodeDetails datanode : getDatanodes().values()) {
dataHosts.add(datanode.getHostName());
}
return dataHosts;
}
@ -219,7 +219,7 @@ public HddsProtos.LifeCycleState getLifeCycleState() {
* Update the State of the pipeline.
*/
public void setLifeCycleState(HddsProtos.LifeCycleState nextState) {
lifeCycleState = nextState;
lifeCycleState = nextState;
}
/**
@ -244,9 +244,8 @@ public HddsProtos.ReplicationType getType() {
public String toString() {
final StringBuilder b = new StringBuilder(getClass().getSimpleName())
.append("[");
getDatanodes().keySet().stream()
.forEach(id -> b.
append(id.endsWith(getLeaderID()) ? "*" + id : id));
getDatanodes().keySet().forEach(
node -> b.append(node.endsWith(getLeaderID()) ? "*" + id : id));
b.append(" id:").append(id);
if (getType() != null) {
b.append(" type:").append(getType().toString());

View File

@ -38,7 +38,8 @@ public interface StorageContainerLocationProtocol {
* set of datanodes that should be used creating this container.
*
*/
ContainerWithPipeline allocateContainer(HddsProtos.ReplicationType replicationType,
ContainerWithPipeline allocateContainer(
HddsProtos.ReplicationType replicationType,
HddsProtos.ReplicationFactor factor, String owner)
throws IOException;
@ -61,7 +62,8 @@ ContainerWithPipeline allocateContainer(HddsProtos.ReplicationType replicationTy
* @return ContainerWithPipeline - the container info with the pipeline.
* @throws IOException
*/
ContainerWithPipeline getContainerWithPipeline(long containerID) throws IOException;
ContainerWithPipeline getContainerWithPipeline(long containerID)
throws IOException;
/**
* Ask SCM a list of containers with a range of container names

View File

@ -97,8 +97,9 @@ public StorageContainerLocationProtocolClientSideTranslatorPB(
* @throws IOException
*/
@Override
public ContainerWithPipeline allocateContainer(HddsProtos.ReplicationType type,
HddsProtos.ReplicationFactor factor, String owner) throws IOException {
public ContainerWithPipeline allocateContainer(
HddsProtos.ReplicationType type, HddsProtos.ReplicationFactor factor,
String owner) throws IOException {
ContainerRequestProto request = ContainerRequestProto.newBuilder()
.setReplicationFactor(factor)
@ -116,7 +117,8 @@ public ContainerWithPipeline allocateContainer(HddsProtos.ReplicationType type,
throw new IOException(response.hasErrorMessage() ?
response.getErrorMessage() : "Allocate container failed.");
}
return ContainerWithPipeline.fromProtobuf(response.getContainerWithPipeline());
return ContainerWithPipeline.fromProtobuf(
response.getContainerWithPipeline());
}
public ContainerInfo getContainer(long containerID) throws IOException {
@ -138,17 +140,18 @@ public ContainerInfo getContainer(long containerID) throws IOException {
/**
* {@inheritDoc}
*/
public ContainerWithPipeline getContainerWithPipeline(long containerID) throws IOException {
public ContainerWithPipeline getContainerWithPipeline(long containerID)
throws IOException {
Preconditions.checkState(containerID >= 0,
"Container ID cannot be negative");
GetContainerWithPipelineRequestProto request = GetContainerWithPipelineRequestProto
.newBuilder()
.setContainerID(containerID)
.build();
GetContainerWithPipelineRequestProto request =
GetContainerWithPipelineRequestProto.newBuilder()
.setContainerID(containerID).build();
try {
GetContainerWithPipelineResponseProto response =
rpcProxy.getContainerWithPipeline(NULL_RPC_CONTROLLER, request);
return ContainerWithPipeline.fromProtobuf(response.getContainerWithPipeline());
return ContainerWithPipeline.fromProtobuf(
response.getContainerWithPipeline());
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
}

View File

@ -113,8 +113,8 @@ public static GetKeyResponseProto getKey(XceiverClientSpi xceiverClient,
* @throws IOException if there is an I/O error while performing the call
*/
public static ContainerProtos.GetCommittedBlockLengthResponseProto
getCommittedBlockLength(
XceiverClientSpi xceiverClient, BlockID blockID, String traceID)
getCommittedBlockLength(
XceiverClientSpi xceiverClient, BlockID blockID, String traceID)
throws IOException {
ContainerProtos.GetCommittedBlockLengthRequestProto.Builder
getBlockLengthRequestBuilder =
@ -375,7 +375,7 @@ public static ReadContainerResponseProto readContainer(
}
/**
* Reads the data given the blockID
* Reads the data given the blockID.
*
* @param client
* @param blockID - ID of the block

View File

@ -110,11 +110,6 @@ public static Versioning getVersioning(boolean versioning) {
}
}
/**
* Ozone handler types.
*/
public static final String OZONE_HANDLER_DISTRIBUTED = "distributed";
public static final String DELETING_KEY_PREFIX = "#deleting#";
public static final String DELETED_KEY_PREFIX = "#deleted#";
public static final String DELETE_TRANSACTION_KEY_PREFIX = "#delTX#";

View File

@ -37,7 +37,8 @@ public class KeyData {
/**
* Represent a list of chunks.
* In order to reduce memory usage, chunkList is declared as an {@link Object}.
* In order to reduce memory usage, chunkList is declared as an
* {@link Object}.
* When #elements == 0, chunkList is null.
* When #elements == 1, chunkList refers to the only element.
* When #elements > 1, chunkList refers to the list.
@ -157,7 +158,7 @@ public List<ContainerProtos.ChunkInfo> getChunks() {
}
/**
* Adds chinkInfo to the list
* Adds chinkInfo to the list.
*/
public void addChunk(ContainerProtos.ChunkInfo chunkInfo) {
if (chunkList == null) {
@ -237,7 +238,8 @@ public void setChunks(List<ContainerProtos.ChunkInfo> chunks) {
} else {
final int n = chunks.size();
chunkList = n == 0? null: n == 1? chunks.get(0): chunks;
size = chunks.parallelStream().mapToLong(ContainerProtos.ChunkInfo::getLen).sum();
size = chunks.parallelStream().mapToLong(
ContainerProtos.ChunkInfo::getLen).sum();
}
}

View File

@ -36,7 +36,8 @@
@InterfaceAudience.Public
@InterfaceStability.Stable
public class HddsVersionInfo {
private static final Logger LOG = LoggerFactory.getLogger(HddsVersionInfo.class);
private static final Logger LOG = LoggerFactory.getLogger(
HddsVersionInfo.class);
private Properties info;
@ -95,7 +96,8 @@ protected String _getProtocVersion() {
return info.getProperty("protocVersion", "Unknown");
}
private static HddsVersionInfo HDDS_VERSION_INFO = new HddsVersionInfo("hdds");
private static final HddsVersionInfo HDDS_VERSION_INFO =
new HddsVersionInfo("hdds");
/**
* Get the HDDS version.
* @return the Hdds version string, eg. "0.6.3-dev"

View File

@ -55,7 +55,6 @@
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;

View File

@ -232,7 +232,8 @@ public static HddsDatanodeService createHddsDatanodeService(
public static void main(String[] args) {
try {
if (DFSUtil.parseHelpArgument(args, "Starts HDDS Datanode", System.out, false)) {
if (DFSUtil.parseHelpArgument(
args, "Starts HDDS Datanode", System.out, false)) {
System.exit(0);
}
Configuration conf = new OzoneConfiguration();

View File

@ -43,7 +43,6 @@
import java.io.FileInputStream;
import java.io.FileOutputStream;
import java.io.IOException;
import java.nio.file.Path;
import java.nio.file.Paths;
import org.yaml.snakeyaml.Yaml;
@ -54,8 +53,6 @@
.Result.NO_SUCH_ALGORITHM;
import static org.apache.hadoop.ozone.container.common.impl.ContainerData
.CHARSET_ENCODING;
import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_EXTENSION;
/**
* A set of helper functions to create proper responses.
@ -75,14 +72,13 @@ private ContainerUtils() {
* @return ContainerCommand Response Builder.
*/
public static ContainerCommandResponseProto.Builder
getContainerCommandResponse(
ContainerCommandRequestProto request, Result result, String message) {
return
ContainerCommandResponseProto.newBuilder()
.setCmdType(request.getCmdType())
.setTraceID(request.getTraceID())
.setResult(result)
.setMessage(message);
getContainerCommandResponse(
ContainerCommandRequestProto request, Result result, String message) {
return ContainerCommandResponseProto.newBuilder()
.setCmdType(request.getCmdType())
.setTraceID(request.getTraceID())
.setResult(result)
.setMessage(message);
}
/**
@ -287,7 +283,7 @@ public static String getChecksum(String containerDataYamlStr)
}
/**
* Get the .container file from the containerBaseDir
* Get the .container file from the containerBaseDir.
* @param containerBaseDir container base directory. The name of this
* directory is same as the containerID
* @return the .container file
@ -301,7 +297,7 @@ public static File getContainerFile(File containerBaseDir) {
}
/**
* ContainerID can be decoded from the container base directory name
* ContainerID can be decoded from the container base directory name.
*/
public static long getContainerID(File containerBaseDir) {
return Long.parseLong(containerBaseDir.getName());

View File

@ -132,7 +132,7 @@ public Iterator<Map.Entry<Long, Container>> getContainerMapIterator() {
}
/**
* Return a copy of the containerMap
* Return a copy of the containerMap.
* @return containerMap
*/
public Map<Long, Container> getContainerMap() {

View File

@ -45,10 +45,12 @@ public class OpenContainerBlockMap {
/**
* Map: localId -> KeyData.
*
* In order to support {@link #getAll()}, the update operations are synchronized.
* In order to support {@link #getAll()}, the update operations are
* synchronized.
*/
static class KeyDataMap {
private final ConcurrentMap<Long, KeyData> blocks = new ConcurrentHashMap<>();
private final ConcurrentMap<Long, KeyData> blocks =
new ConcurrentHashMap<>();
KeyData get(long localId) {
return blocks.get(localId);
@ -59,7 +61,8 @@ synchronized int removeAndGetSize(long localId) {
return blocks.size();
}
synchronized KeyData computeIfAbsent(long localId, Function<Long, KeyData> f) {
synchronized KeyData computeIfAbsent(
long localId, Function<Long, KeyData> f) {
return blocks.computeIfAbsent(localId, f);
}
@ -76,7 +79,8 @@ synchronized List<KeyData> getAll() {
*
* For now, we will track all open blocks of a container in the blockMap.
*/
private final ConcurrentMap<Long, KeyDataMap> containers = new ConcurrentHashMap<>();
private final ConcurrentMap<Long, KeyDataMap> containers =
new ConcurrentHashMap<>();
/**
* Removes the Container matching with specified containerId.
@ -109,7 +113,7 @@ public void removeChunk(BlockID blockID, ChunkInfo chunkInfo) {
}
/**
* returns the list of open to the openContainerBlockMap
* Returns the list of open to the openContainerBlockMap.
* @param containerId container id
* @return List of open Keys(blocks)
*/
@ -130,15 +134,14 @@ public void removeFromKeyMap(BlockID blockID) {
}
/**
* Returns true if the block exists in the map, false otherwise
* Returns true if the block exists in the map, false otherwise.
*
* @param blockID
* @return True, if it exists, false otherwise
*/
public boolean checkIfBlockExists(BlockID blockID) {
KeyDataMap keyDataMap = containers.get(blockID.getContainerID());
return keyDataMap == null ? false :
keyDataMap.get(blockID.getLocalID()) != null;
return keyDataMap != null && keyDataMap.get(blockID.getLocalID()) != null;
}
@VisibleForTesting

View File

@ -71,10 +71,10 @@
*/
public final class XceiverServerRatis implements XceiverServerSpi {
static final Logger LOG = LoggerFactory.getLogger(XceiverServerRatis.class);
private static final AtomicLong callIdCounter = new AtomicLong();
private static final AtomicLong CALL_ID_COUNTER = new AtomicLong();
private static long nextCallId() {
return callIdCounter.getAndIncrement() & Long.MAX_VALUE;
return CALL_ID_COUNTER.getAndIncrement() & Long.MAX_VALUE;
}
private final int port;
@ -307,6 +307,6 @@ private RaftClientRequest createRaftClientRequest(
RaftClientRequest.Type type) {
return new RaftClientRequest(clientId, server.getId(),
PipelineID.getFromProtobuf(pipelineID).getRaftGroupID(),
nextCallId(),0, Message.valueOf(request.toByteString()), type);
nextCallId(), 0, Message.valueOf(request.toByteString()), type);
}
}

View File

@ -67,8 +67,8 @@ public interface KeyManager {
* @param count - Number of keys to return.
* @return List of Keys that match the criteria.
*/
List<KeyData> listKey(Container container, long startLocalID, int count) throws
IOException;
List<KeyData> listKey(Container container, long startLocalID, int count)
throws IOException;
/**
* Returns the last committed block length for the block.

View File

@ -109,23 +109,23 @@ public static CommandStatusBuilder newBuilder() {
return new CommandStatusBuilder();
}
public CommandStatusBuilder setType(Type type) {
this.type = type;
public CommandStatusBuilder setType(Type commandType) {
this.type = commandType;
return this;
}
public CommandStatusBuilder setCmdId(Long cmdId) {
this.cmdId = cmdId;
public CommandStatusBuilder setCmdId(Long commandId) {
this.cmdId = commandId;
return this;
}
public CommandStatusBuilder setStatus(Status status) {
this.status = status;
public CommandStatusBuilder setStatus(Status commandStatus) {
this.status = commandStatus;
return this;
}
public CommandStatusBuilder setMsg(String msg) {
this.msg = msg;
public CommandStatusBuilder setMsg(String message) {
this.msg = message;
return this;
}

View File

@ -193,11 +193,13 @@ private void sleepIfNeeded() {
rpcCount.incrementAndGet();
heartbeatCount.incrementAndGet();
if(heartbeat.hasCommandStatusReport()){
cmdStatusList.addAll(heartbeat.getCommandStatusReport().getCmdStatusList());
cmdStatusList.addAll(heartbeat.getCommandStatusReport()
.getCmdStatusList());
commandStatusReport.incrementAndGet();
}
sleepIfNeeded();
return SCMHeartbeatResponseProto.newBuilder().addAllCommands(scmCommandRequests)
return SCMHeartbeatResponseProto.newBuilder().addAllCommands(
scmCommandRequests)
.setDatanodeUUID(heartbeat.getDatanodeDetails().getUuid())
.build();
}

View File

@ -19,17 +19,12 @@
package org.apache.hadoop.ozone.container.common.interfaces;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher;
import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
import org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler;
import org.apache.hadoop.test.GenericTestUtils;
import org.junit.Assert;
import static org.junit.Assert.fail;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
@ -37,8 +32,6 @@
import org.junit.rules.Timeout;
import org.mockito.Mockito;
import java.util.UUID;
/**
* Tests Handler interface.
*/

View File

@ -21,8 +21,6 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerInfo;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerAction;
import org.apache.hadoop.hdds.protocol.proto

View File

@ -20,7 +20,6 @@
import org.apache.hadoop.fs.GetSpaceUsed;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
@ -43,8 +42,8 @@ public class TestRoundRobinVolumeChoosingPolicy {
private List<HddsVolume> volumes;
private final String baseDir = MiniDFSCluster.getBaseDirectory();
private final String volume1 = baseDir + "disk1";
private final String volume2 = baseDir + "disk2";
private final String volume1 = baseDir + "disk1";
private final String volume2 = baseDir + "disk2";
private static final String DUMMY_IP_ADDR = "0.0.0.0";
@Before

View File

@ -62,7 +62,8 @@ public void setUp() throws Exception {
conf = new OzoneConfiguration();
conf.set(ScmConfigKeys.HDDS_DATANODE_DIR_KEY, folder.getRoot()
.getAbsolutePath());
conf.set(OzoneConfigKeys.OZONE_METADATA_DIRS, folder.newFolder().getAbsolutePath());
conf.set(OzoneConfigKeys.OZONE_METADATA_DIRS,
folder.newFolder().getAbsolutePath());
}
@Test

View File

@ -180,9 +180,11 @@ public synchronized boolean remove(TIMEOUT_PAYLOAD payload) {
}
protected abstract void onTimeout(EventPublisher publisher, TIMEOUT_PAYLOAD payload);
protected abstract void onTimeout(
EventPublisher publisher, TIMEOUT_PAYLOAD payload);
protected abstract void onFinished(EventPublisher publisher, TIMEOUT_PAYLOAD payload);
protected abstract void onFinished(
EventPublisher publisher, TIMEOUT_PAYLOAD payload);
public List<TIMEOUT_PAYLOAD> getTimeoutEvents(
Predicate<? super TIMEOUT_PAYLOAD> predicate) {

View File

@ -22,9 +22,6 @@
import org.junit.Before;
import org.junit.Test;
import java.util.Set;
import java.util.stream.Collectors;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
/**

View File

@ -161,11 +161,11 @@ private void preAllocateContainers(int count, ReplicationType type,
lock.lock();
try {
for (int i = 0; i < count; i++) {
ContainerWithPipeline containerWithPipeline = null;
ContainerWithPipeline containerWithPipeline;
try {
// TODO: Fix this later when Ratis is made the Default.
containerWithPipeline = containerManager.allocateContainer(type, factor,
owner);
containerWithPipeline = containerManager.allocateContainer(
type, factor, owner);
if (containerWithPipeline == null) {
LOG.warn("Unable to allocate container.");
@ -293,12 +293,12 @@ public AllocatedBlock allocateBlock(final long size,
private String getChannelName(ReplicationType type) {
switch (type) {
case RATIS:
return "RA" + UUID.randomUUID().toString().substring(3);
case STAND_ALONE:
return "SA" + UUID.randomUUID().toString().substring(3);
default:
return "RA" + UUID.randomUUID().toString().substring(3);
case RATIS:
return "RA" + UUID.randomUUID().toString().substring(3);
case STAND_ALONE:
return "SA" + UUID.randomUUID().toString().substring(3);
default:
return "RA" + UUID.randomUUID().toString().substring(3);
}
}

View File

@ -232,7 +232,8 @@ public void commitTransactions(
lock.lock();
try {
Set<UUID> dnsWithCommittedTxn;
for (DeleteBlockTransactionResult transactionResult : transactionResults) {
for (DeleteBlockTransactionResult transactionResult :
transactionResults) {
if (isTransactionFailed(transactionResult)) {
continue;
}

View File

@ -109,8 +109,8 @@ public BackgroundTaskQueue getTasks() {
public void handlePendingDeletes(PendingDeleteStatusList deletionStatusList) {
DatanodeDetails dnDetails = deletionStatusList.getDatanodeDetails();
for (PendingDeleteStatusList.PendingDeleteStatus deletionStatus : deletionStatusList
.getPendingDeleteStatuses()) {
for (PendingDeleteStatusList.PendingDeleteStatus deletionStatus :
deletionStatusList.getPendingDeleteStatuses()) {
LOG.info(
"Block deletion txnID mismatch in datanode {} for containerID {}."
+ " Datanode delete txnID: {}, SCM txnID: {}",

View File

@ -62,8 +62,8 @@ public void onMessage(ContainerID containerID, EventPublisher publisher) {
containerManager.getContainerWithPipeline(containerID.getId());
info = containerWithPipeline.getContainerInfo();
if (info == null) {
LOG.error("Failed to update the container state. Container with id : {} "
+ "does not exist", containerID.getId());
LOG.error("Failed to update the container state. Container with id : {}"
+ " does not exist", containerID.getId());
return;
}
} catch (IOException e) {

View File

@ -600,8 +600,8 @@ private HddsProtos.SCMContainerInfo reconcileState(
.setReplicationType(knownState.getReplicationType())
.setReplicationFactor(knownState.getReplicationFactor());
// TODO: If current state doesn't have this DN in list of DataNodes with replica
// then add it in list of replicas.
// TODO: If current state doesn't have this DN in list of DataNodes with
// replica then add it in list of replicas.
// If used size is greater than allocated size, we will be updating
// allocated size with used size. This update is done as a fallback

View File

@ -288,9 +288,10 @@ private void initializeStateMachine() {
* @return ContainerWithPipeline
* @throws IOException on Failure.
*/
public ContainerWithPipeline allocateContainer(PipelineSelector selector, HddsProtos
.ReplicationType type, HddsProtos.ReplicationFactor replicationFactor,
String owner) throws IOException {
public ContainerWithPipeline allocateContainer(PipelineSelector selector,
HddsProtos.ReplicationType type,
HddsProtos.ReplicationFactor replicationFactor, String owner)
throws IOException {
Pipeline pipeline = selector.getReplicationPipeline(type,
replicationFactor);

View File

@ -182,7 +182,7 @@ public void stop() {
}
/**
* Event for the ReplicationCommandWatcher to repeate the embedded request
* Event for the ReplicationCommandWatcher to repeate the embedded request.
* in case fof timeout.
*/
public static class ReplicationRequestToRepeat

View File

@ -381,7 +381,7 @@ NavigableSet<ContainerID> getContainerIDsByType(ReplicationType type) {
}
/**
* Returns Open containers in the SCM by the Pipeline
* Returns Open containers in the SCM by the Pipeline.
*
* @param pipelineID - Pipeline id.
* @return NavigableSet<ContainerID>

View File

@ -84,8 +84,8 @@ public void insertNewDatanode(UUID datanodeID, Set<ContainerID> containerIDs)
* @throws SCMException - if we don't know about this datanode, for new DN
* use insertNewDatanode.
*/
public void setContainersForDatanode(UUID datanodeID, Set<ContainerID> containers)
throws SCMException {
public void setContainersForDatanode(UUID datanodeID,
Set<ContainerID> containers) throws SCMException {
Preconditions.checkNotNull(datanodeID);
Preconditions.checkNotNull(containers);
if (dn2ContainerMap

View File

@ -17,8 +17,6 @@
package org.apache.hadoop.hdds.scm.node.states;
import java.io.IOException;
/**
* This exception represents that the node that is being accessed does not
* exist in NodeStateMap.

View File

@ -94,7 +94,8 @@ ReportResult build() {
if (nullSafeMissingContainers == null) {
nullSafeMissingContainers = Collections.emptySet();
}
return new ReportResult(status, nullSafeMissingContainers, nullSafeNewContainers);
return new ReportResult(status, nullSafeMissingContainers,
nullSafeNewContainers);
}
}
}

View File

@ -23,39 +23,31 @@
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
import java.util.Collections;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import java.util.UUID;
import java.util.Map;
import java.util.HashSet;
import java.util.Collections;
import java.util.concurrent.ConcurrentHashMap;
import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes
.DUPLICATE_DATANODE;
import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes.DUPLICATE_DATANODE;
/**
* This data structure maintains the list of pipelines which the given datanode
* is a part of.
* This information will be added whenever a new pipeline allocation happens.
* This data structure maintains the list of pipelines which the given datanode is a part of. This
* information will be added whenever a new pipeline allocation happens.
*
* TODO: this information needs to be regenerated from pipeline reports on
* SCM restart
* <p>TODO: this information needs to be regenerated from pipeline reports on SCM restart
*/
public class Node2PipelineMap {
private final Map<UUID, Set<Pipeline>> dn2PipelineMap;
/**
* Constructs a Node2PipelineMap Object.
*/
/** Constructs a Node2PipelineMap Object. */
public Node2PipelineMap() {
dn2PipelineMap = new ConcurrentHashMap<>();
}
/**
* Returns true if this a datanode that is already tracked by
* Node2PipelineMap.
* Returns true if this a datanode that is already tracked by Node2PipelineMap.
*
* @param datanodeID - UUID of the Datanode.
* @return True if this is tracked, false if this map does not know about it.
@ -71,18 +63,17 @@ private boolean isKnownDatanode(UUID datanodeID) {
* @param datanodeID -- Datanode UUID
* @param pipelines - set of pipelines.
*/
private void insertNewDatanode(UUID datanodeID, Set<Pipeline> pipelines)
throws SCMException {
private void insertNewDatanode(UUID datanodeID, Set<Pipeline> pipelines) throws SCMException {
Preconditions.checkNotNull(pipelines);
Preconditions.checkNotNull(datanodeID);
if(dn2PipelineMap.putIfAbsent(datanodeID, pipelines) != null) {
throw new SCMException("Node already exists in the map",
DUPLICATE_DATANODE);
if (dn2PipelineMap.putIfAbsent(datanodeID, pipelines) != null) {
throw new SCMException("Node already exists in the map", DUPLICATE_DATANODE);
}
}
/**
* Removes datanode Entry from the map.
*
* @param datanodeID - Datanode ID.
*/
public synchronized void removeDatanode(UUID datanodeID) {
@ -98,20 +89,19 @@ public synchronized void removeDatanode(UUID datanodeID) {
*/
public Set<Pipeline> getPipelines(UUID datanode) {
Preconditions.checkNotNull(datanode);
return dn2PipelineMap.computeIfPresent(datanode, (k, v) ->
Collections.unmodifiableSet(v));
return dn2PipelineMap.computeIfPresent(datanode, (k, v) -> Collections.unmodifiableSet(v));
}
/**
* Adds a pipeline entry to a given dataNode in the map.
*
* @param pipeline Pipeline to be added
*/
public synchronized void addPipeline(Pipeline pipeline) {
for (DatanodeDetails details : pipeline.getDatanodes().values()) {
UUID dnId = details.getUuid();
dn2PipelineMap
.computeIfAbsent(dnId,
k -> Collections.synchronizedSet(new HashSet<>()))
.computeIfAbsent(dnId, k -> Collections.synchronizedSet(new HashSet<>()))
.add(pipeline);
}
}
@ -119,8 +109,12 @@ public synchronized void addPipeline(Pipeline pipeline) {
public synchronized void removePipeline(Pipeline pipeline) {
for (DatanodeDetails details : pipeline.getDatanodes().values()) {
UUID dnId = details.getUuid();
dn2PipelineMap.computeIfPresent(dnId,
(k, v) -> {v.remove(pipeline); return v;});
dn2PipelineMap.computeIfPresent(
dnId,
(k, v) -> {
v.remove(pipeline);
return v;
});
}
}

View File

@ -111,7 +111,7 @@ public abstract Pipeline allocatePipeline(
ReplicationFactor replicationFactor);
/**
* Initialize the pipeline
* Initialize the pipeline.
* TODO: move the initialization to Ozone Client later
*/
public abstract void initializePipeline(Pipeline pipeline) throws IOException;
@ -176,7 +176,7 @@ public Pipeline createPipeline(ReplicationFactor replicationFactor,
}
/**
* Remove the pipeline from active allocation
* Remove the pipeline from active allocation.
* @param pipeline pipeline to be finalized
*/
public synchronized void finalizePipeline(Pipeline pipeline) {
@ -193,7 +193,7 @@ public void closePipeline(Pipeline pipeline) {
}
/**
* list members in the pipeline .
* list members in the pipeline.
* @return the datanode
*/
public abstract List<DatanodeDetails> getMembers(PipelineID pipelineID)

View File

@ -126,7 +126,7 @@ public PipelineSelector(NodeManager nodeManager,
}
/**
* Event and State Transition Mapping:
* Event and State Transition Mapping.
*
* State: ALLOCATED ---------------> CREATING
* Event: CREATE
@ -293,7 +293,7 @@ public Pipeline getReplicationPipeline(ReplicationType replicationType,
pipeline = manager.getPipeline(replicationFactor, replicationType);
} else {
// if a new pipeline is created, initialize its state machine
updatePipelineState(pipeline,HddsProtos.LifeCycleEvent.CREATE);
updatePipelineState(pipeline, HddsProtos.LifeCycleEvent.CREATE);
//TODO: move the initialization of pipeline to Ozone Client
manager.initializePipeline(pipeline);
@ -334,7 +334,8 @@ public void finalizePipeline(Pipeline pipeline) throws IOException {
/**
* Close a given pipeline.
*/
public void closePipelineIfNoOpenContainers(Pipeline pipeline) throws IOException {
public void closePipelineIfNoOpenContainers(Pipeline pipeline)
throws IOException {
if (pipeline.getLifeCycleState() != LifeCycleState.CLOSING) {
return;
}

View File

@ -165,7 +165,8 @@ public ContainerInfo getContainer(long containerID) throws IOException {
}
@Override
public ContainerWithPipeline getContainerWithPipeline(long containerID) throws IOException {
public ContainerWithPipeline getContainerWithPipeline(long containerID)
throws IOException {
String remoteUser = getRpcRemoteUsername();
getScm().checkAdminAccess(remoteUser);
return scm.getScmContainerManager()

View File

@ -74,7 +74,7 @@ private static DatanodeDetails createDatanodeDetails(UUID uuid) {
+ "." + random.nextInt(256)
+ "." + random.nextInt(256)
+ "." + random.nextInt(256);
return createDatanodeDetails(uuid.toString(), "localhost", ipAddress);
return createDatanodeDetails(uuid.toString(), "localhost", ipAddress);
}
/**
@ -259,12 +259,12 @@ public static StorageReportProto createStorageReport(UUID nodeId, String path,
StorageTypeProto storageTypeProto =
type == null ? StorageTypeProto.DISK : type;
srb.setStorageType(storageTypeProto);
return srb.build();
return srb.build();
}
/**
* Generates random container reports
* Generates random container reports.
*
* @return ContainerReportsProto
*/
@ -281,7 +281,7 @@ public static ContainerReportsProto getRandomContainerReports() {
*/
public static ContainerReportsProto getRandomContainerReports(
int numberOfContainers) {
List<ContainerInfo> containerInfos = new ArrayList<>();
List<ContainerInfo> containerInfos = new ArrayList<>();
for (int i = 0; i < numberOfContainers; i++) {
containerInfos.add(getRandomContainerInfo(i));
}

View File

@ -39,7 +39,6 @@
import java.io.File;
import java.io.IOException;
import java.nio.file.Paths;
import java.util.Collections;
import static org.apache.hadoop.ozone.OzoneConsts.GB;
import static org.apache.hadoop.ozone.OzoneConsts.MB;

View File

@ -102,8 +102,8 @@ private void setupContainerManager() throws IOException {
ContainerInfo containerInfo =
new ContainerInfo.Builder().setContainerID(1).build();
Pipeline pipeline =
new Pipeline(null, LifeCycleState.CLOSED, ReplicationType.RATIS,
ReplicationFactor.THREE, null);
new Pipeline(null, LifeCycleState.CLOSED,
ReplicationType.RATIS, ReplicationFactor.THREE, null);
pipeline.addMember(dnList.get(0));
pipeline.addMember(dnList.get(1));
pipeline.addMember(dnList.get(2));
@ -379,7 +379,8 @@ public void testDeletedBlockTransactions() throws IOException {
Assert.assertTrue(transactions.isFull());
}
private void mockContainerInfo(long containerID, DatanodeDetails dd) throws IOException {
private void mockContainerInfo(long containerID, DatanodeDetails dd)
throws IOException {
Pipeline pipeline =
new Pipeline("fake", LifeCycleState.OPEN,
ReplicationType.STAND_ALONE, ReplicationFactor.ONE,

View File

@ -48,12 +48,11 @@
public class TestCommandStatusReportHandler implements EventPublisher {
private static Logger LOG = LoggerFactory
private static final Logger LOG = LoggerFactory
.getLogger(TestCommandStatusReportHandler.class);
private CommandStatusReportHandler cmdStatusReportHandler;
private String storagePath = GenericTestUtils.getRandomizedTempPath()
.concat("/" + UUID.randomUUID().toString());
;
@Before
public void setup() {
@ -69,10 +68,9 @@ public void testCommandStatusReport() {
.emptyList());
cmdStatusReportHandler.onMessage(report, this);
assertFalse(logCapturer.getOutput().contains("DeleteBlockCommandStatus"));
assertFalse(logCapturer.getOutput().contains
("CloseContainerCommandStatus"));
assertFalse(logCapturer.getOutput().contains
("ReplicateCommandStatus"));
assertFalse(logCapturer.getOutput().contains(
"CloseContainerCommandStatus"));
assertFalse(logCapturer.getOutput().contains("ReplicateCommandStatus"));
report = this.getStatusReport(this.getCommandStatusList());
@ -93,13 +91,13 @@ public void testCommandStatusReport() {
}
private CommandStatusReportFromDatanode getStatusReport(List<CommandStatus>
reports) {
CommandStatusReportsProto report = TestUtils.createCommandStatusReport
(reports);
private CommandStatusReportFromDatanode getStatusReport(
List<CommandStatus> reports) {
CommandStatusReportsProto report = TestUtils.createCommandStatusReport(
reports);
DatanodeDetails dn = TestUtils.randomDatanodeDetails();
return new SCMDatanodeHeartbeatDispatcher.CommandStatusReportFromDatanode
(dn, report);
return new SCMDatanodeHeartbeatDispatcher.CommandStatusReportFromDatanode(
dn, report);
}
@Override

View File

@ -37,7 +37,6 @@
import java.io.File;
import java.io.IOException;
import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleEvent.CREATE;
import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleEvent.CREATED;
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT;
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE;

View File

@ -178,8 +178,8 @@ public void testGetContainerWithPipeline() throws Exception {
mapping
.updateContainerState(contInfo.getContainerID(), LifeCycleEvent.CLOSE);
ContainerInfo finalContInfo = contInfo;
LambdaTestUtils.intercept(SCMException.class,"No entry exist for "
+ "containerId:" , () -> mapping.getContainerWithPipeline(
LambdaTestUtils.intercept(SCMException.class, "No entry exist for "
+ "containerId:", () -> mapping.getContainerWithPipeline(
finalContInfo.getContainerID()));
mapping.getStateManager().getContainerStateMap()
@ -376,7 +376,8 @@ private ContainerInfo createContainer()
@Test
public void testFlushAllContainers() throws IOException {
ContainerInfo info = createContainer();
List<ContainerInfo> containers = mapping.getStateManager().getAllContainers();
List<ContainerInfo> containers = mapping.getStateManager()
.getAllContainers();
Assert.assertTrue(containers.size() > 0);
mapping.flushContainerInfo();
}

View File

@ -86,7 +86,7 @@ public void test() throws IOException {
.setContainerID((Long) invocation.getArguments()[0])
.setState(LifeCycleState.CLOSED)
.build()
);
);
ContainerStateManager containerStateManager =
new ContainerStateManager(conf, mapping);

View File

@ -52,13 +52,13 @@ public void chooseDatanodes() throws SCMException {
.thenReturn(new ArrayList<>(datanodes));
when(mockNodeManager.getNodeStat(anyObject()))
.thenReturn(new SCMNodeMetric(100l, 0L, 100L));
.thenReturn(new SCMNodeMetric(100L, 0L, 100L));
when(mockNodeManager.getNodeStat(datanodes.get(2)))
.thenReturn(new SCMNodeMetric(100l, 90L, 10L));
.thenReturn(new SCMNodeMetric(100L, 90L, 10L));
when(mockNodeManager.getNodeStat(datanodes.get(3)))
.thenReturn(new SCMNodeMetric(100l, 80L, 20L));
.thenReturn(new SCMNodeMetric(100L, 80L, 20L));
when(mockNodeManager.getNodeStat(datanodes.get(4)))
.thenReturn(new SCMNodeMetric(100l, 70L, 30L));
.thenReturn(new SCMNodeMetric(100L, 70L, 30L));
SCMContainerPlacementCapacity scmContainerPlacementRandom =
new SCMContainerPlacementCapacity(mockNodeManager, conf);

View File

@ -51,9 +51,9 @@ public void chooseDatanodes() throws SCMException {
.thenReturn(new ArrayList<>(datanodes));
when(mockNodeManager.getNodeStat(anyObject()))
.thenReturn(new SCMNodeMetric(100l, 0l, 100l));
.thenReturn(new SCMNodeMetric(100L, 0L, 100L));
when(mockNodeManager.getNodeStat(datanodes.get(2)))
.thenReturn(new SCMNodeMetric(100l, 90l, 10l));
.thenReturn(new SCMNodeMetric(100L, 90L, 10L));
SCMContainerPlacementRandom scmContainerPlacementRandom =
new SCMContainerPlacementRandom(mockNodeManager, conf);

View File

@ -21,7 +21,6 @@
import java.util.Iterator;
import java.util.List;
import java.util.Objects;
import java.util.UUID;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
@ -132,7 +131,7 @@ protected List<DatanodeDetails> getCurrentReplicas(
//WHEN
queue.fireEvent(SCMEvents.REPLICATE_CONTAINER,
new ReplicationRequest(1l, (short) 2, System.currentTimeMillis(),
new ReplicationRequest(1L, (short) 2, System.currentTimeMillis(),
(short) 3));
Thread.sleep(500L);
@ -159,10 +158,8 @@ public void testCommandWatcher() throws InterruptedException, IOException {
leaseManager.start();
ReplicationManager replicationManager =
new ReplicationManager(containerPlacementPolicy, containerStateManager,
queue, leaseManager) {
new ReplicationManager(containerPlacementPolicy,
containerStateManager, queue, leaseManager) {
@Override
protected List<DatanodeDetails> getCurrentReplicas(
ReplicationRequest request) throws IOException {
@ -172,7 +169,7 @@ protected List<DatanodeDetails> getCurrentReplicas(
replicationManager.start();
queue.fireEvent(SCMEvents.REPLICATE_CONTAINER,
new ReplicationRequest(1l, (short) 2, System.currentTimeMillis(),
new ReplicationRequest(1L, (short) 2, System.currentTimeMillis(),
(short) 3));
Thread.sleep(500L);

View File

@ -92,8 +92,8 @@ public void testPollOp() throws InterruptedException {
1, replicationQueue.size());
Assert.assertEquals(temp, msg5);
// Message 2 should be ordered before message 5 as both have same replication
// number but message 2 has earlier timestamp.
// Message 2 should be ordered before message 5 as both have same
// replication number but message 2 has earlier timestamp.
temp = replicationQueue.take();
Assert.assertEquals("Should have 0 objects",
replicationQueue.size(), 0);

View File

@ -32,8 +32,6 @@
.SCMContainerPlacementCapacity;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.StorageReportProto;
import org.apache.hadoop.hdds.scm.events.SCMEvents;
import org.apache.hadoop.hdds.server.events.EventQueue;
import org.apache.hadoop.ozone.OzoneConfigKeys;
@ -158,7 +156,8 @@ public void testContainerPlacementCapacity() throws IOException,
assertTrue(nodeManager.isOutOfChillMode());
ContainerWithPipeline containerWithPipeline = containerManager.allocateContainer(
ContainerWithPipeline containerWithPipeline = containerManager
.allocateContainer(
xceiverClientManager.getType(),
xceiverClientManager.getFactor(), "OZONE");
assertEquals(xceiverClientManager.getFactor().getNumber(),

View File

@ -424,7 +424,8 @@ public void testScmDetectStaleAndDeadNode() throws IOException,
List<DatanodeDetails> nodeList = createNodeSet(nodeManager, nodeCount);
DatanodeDetails staleNode = TestUtils.createRandomDatanodeAndRegister(nodeManager);
DatanodeDetails staleNode = TestUtils.createRandomDatanodeAndRegister(
nodeManager);
// Heartbeat once
nodeManager.processHeartbeat(staleNode);

View File

@ -37,13 +37,12 @@
public class TestNodeReportHandler implements EventPublisher {
private static Logger LOG = LoggerFactory
private static final Logger LOG = LoggerFactory
.getLogger(TestNodeReportHandler.class);
private NodeReportHandler nodeReportHandler;
private SCMNodeManager nodeManager;
private String storagePath = GenericTestUtils.getRandomizedTempPath()
.concat("/" + UUID.randomUUID().toString());
;
@Before
public void resetEventCollector() throws IOException {

View File

@ -121,7 +121,7 @@ public static void setUp() throws Exception {
config.set(OZONE_METADATA_DIRS, testDir.getAbsolutePath());
config
.setBoolean(OzoneConfigKeys.DFS_CONTAINER_RATIS_IPC_RANDOM_PORT, true);
config.set(HddsConfigKeys.HDDS_COMMAND_STATUS_REPORT_INTERVAL,"1s");
config.set(HddsConfigKeys.HDDS_COMMAND_STATUS_REPORT_INTERVAL, "1s");
}
@Test
@ -228,8 +228,8 @@ public void testGetVersionToInvalidEndpoint() throws Exception {
try (EndpointStateMachine rpcEndPoint = createEndpoint(conf,
nonExistentServerAddress, 1000)) {
rpcEndPoint.setState(EndpointStateMachine.EndPointStates.GETVERSION);
OzoneContainer ozoneContainer = new OzoneContainer(TestUtils.randomDatanodeDetails(),
conf, null);
OzoneContainer ozoneContainer = new OzoneContainer(
TestUtils.randomDatanodeDetails(), conf, null);
VersionEndpointTask versionTask = new VersionEndpointTask(rpcEndPoint,
conf, ozoneContainer);
EndpointStateMachine.EndPointStates newState = versionTask.call();
@ -405,7 +405,8 @@ public void testHeartbeatWithCommandStatusReport() throws Exception {
assertEquals(0, scmServerImpl.getCommandStatusReportCount());
// Send heartbeat again from heartbeat endpoint task
final StateContext stateContext = heartbeatTaskHelper(serverAddress, 3000);
final StateContext stateContext = heartbeatTaskHelper(
serverAddress, 3000);
Map<Long, CommandStatus> map = stateContext.getCommandStatusMap();
assertNotNull(map);
assertEquals("Should have 3 objects", 3, map.size());

View File

@ -87,11 +87,13 @@ public void testCapacityPlacementYieldsBetterDataDistribution() throws
for (int x = 0; x < opsCount; x++) {
long containerSize = random.nextInt(100) * OzoneConsts.GB;
List<DatanodeDetails> nodesCapacity =
capacityPlacer.chooseDatanodes(new ArrayList<>(), nodesRequired, containerSize);
capacityPlacer.chooseDatanodes(new ArrayList<>(), nodesRequired,
containerSize);
assertEquals(nodesRequired, nodesCapacity.size());
List<DatanodeDetails> nodesRandom =
randomPlacer.chooseDatanodes(nodesCapacity, nodesRequired, containerSize);
randomPlacer.chooseDatanodes(nodesCapacity, nodesRequired,
containerSize);
// One fifth of all calls are delete
if (x % 5 == 0) {

View File

@ -18,7 +18,6 @@
package org.apache.hadoop.ozone.client;
import com.google.common.base.Preconditions;
import com.google.common.base.Strings;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
@ -158,9 +157,9 @@ private class VolumeIterator implements Iterator<OzoneVolume> {
private OzoneVolume currentValue;
/**
* Creates an Iterator to iterate over all volumes after prevVolume of the user.
* If prevVolume is null it iterates from the first volume. The returned volumes
* match volume prefix.
* Creates an Iterator to iterate over all volumes after
* prevVolume of the user. If prevVolume is null it iterates from the
* first volume. The returned volumes match volume prefix.
* @param user user name
* @param volPrefix volume prefix to match
*/

View File

@ -28,10 +28,7 @@
import org.apache.hadoop.hdds.scm.container.states.ContainerStateMap;
import org.apache.hadoop.hdds.scm.pipelines.PipelineSelector;
import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
import org.apache.hadoop.ozone.HddsDatanodeService;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.container.common.impl.ContainerData;
import org.apache.hadoop.test.GenericTestUtils;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.BeforeClass;
@ -39,7 +36,6 @@
import java.io.IOException;
import java.util.NavigableSet;
import java.util.UUID;
import java.util.concurrent.TimeoutException;
import static org.apache.hadoop.hdds.protocol.proto.HddsProtos

View File

@ -23,7 +23,6 @@
import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
import org.apache.hadoop.ozone.client.rpc.RpcClient;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.ozone.container.ContainerTestHelper;
import org.apache.hadoop.ozone.client.rest.OzoneException;
import org.apache.ratis.rpc.RpcType;
import org.apache.ratis.rpc.SupportedRpcType;
@ -54,9 +53,9 @@ class RatisTestSuite implements Closeable {
* OZONE_ENABLED = true
* RATIS_ENABLED = true
*/
public RatisTestSuite(final Class<?> clazz)
public RatisTestSuite()
throws IOException, TimeoutException, InterruptedException {
conf = newOzoneConfiguration(clazz, RPC);
conf = newOzoneConfiguration(RPC);
cluster = newMiniOzoneCluster(NUM_DATANODES, conf);
}
@ -84,8 +83,7 @@ public int getDatanodeOzoneRestPort() {
}
}
static OzoneConfiguration newOzoneConfiguration(
Class<?> clazz, RpcType rpc) {
static OzoneConfiguration newOzoneConfiguration(RpcType rpc) {
final OzoneConfiguration conf = new OzoneConfiguration();
initRatisConf(rpc, conf);
return conf;

View File

@ -27,9 +27,7 @@
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.server.datanode.ObjectStoreHandler;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.ozone.container.common.impl.ContainerData;
import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyUtils;
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;

View File

@ -73,8 +73,7 @@ public class TestCloseContainerHandlingByClient {
/**
* Create a MiniDFSCluster for testing.
* <p>
* Ozone is made active by setting OZONE_ENABLED = true and
* OZONE_HANDLER_TYPE_KEY = "distributed"
* Ozone is made active by setting OZONE_ENABLED = true
*
* @throws IOException
*/

View File

@ -38,12 +38,10 @@
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
import org.apache.hadoop.ozone.container.common.helpers.KeyData;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.util.Time;
import org.junit.Assert;
import org.slf4j.Logger;

View File

@ -75,7 +75,6 @@
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Stage.COMBINED;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_ROOT_PREFIX;
import static org.apache.hadoop.ozone.container.ContainerTestHelper.getChunk;
import static org.apache.hadoop.ozone.container.ContainerTestHelper.getData;
import static org.apache.hadoop.ozone.container.ContainerTestHelper.setDataChecksum;

View File

@ -53,8 +53,7 @@ public class TestOzoneContainerRatis {
public Timeout testTimeout = new Timeout(300000);
static OzoneConfiguration newOzoneConfiguration() {
final OzoneConfiguration conf = new OzoneConfiguration();
return conf;
return new OzoneConfiguration();
}
private static void runTestOzoneContainerViaDataNodeRatis(

View File

@ -23,7 +23,6 @@
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.RatisTestHelper;
import org.apache.hadoop.ozone.container.ContainerTestHelper;
import org.apache.ratis.rpc.RpcType;
import org.apache.ratis.rpc.SupportedRpcType;
import org.junit.Ignore;
@ -46,8 +45,7 @@ public class TestRatisManager {
TestRatisManager.class);
static OzoneConfiguration newOzoneConfiguration() {
final OzoneConfiguration conf = new OzoneConfiguration();
return conf;
return new OzoneConfiguration();
}

View File

@ -30,8 +30,6 @@
import org.junit.Test;
import org.junit.rules.ExpectedException;
import java.io.IOException;
/**
* Test allocate container calls.
*/

View File

@ -23,7 +23,6 @@
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.client.rest.headers.Header;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.util.Time;
import org.apache.http.HttpResponse;
import org.apache.http.client.methods.HttpPost;

View File

@ -30,7 +30,6 @@
import org.apache.hadoop.ozone.client.rpc.RpcClient;
import org.apache.hadoop.ozone.web.request.OzoneQuota;
import org.apache.hadoop.ozone.web.utils.OzoneUtils;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.util.Time;
import org.junit.AfterClass;
import org.junit.Before;
@ -95,8 +94,6 @@ public static void init()
InterruptedException {
conf = new OzoneConfiguration();
String path = GenericTestUtils
.getTempPath(TestBuckets.class.getSimpleName());
cluster = MiniOzoneCluster.newBuilder(conf)
.setNumDatanodes(3)
.build();
@ -128,7 +125,7 @@ public void testCreateBucket() throws Exception {
}
static void runTestCreateBucket(ClientProtocol client)
throws OzoneException, IOException, ParseException {
throws IOException {
String volumeName = OzoneUtils.getRequestID().toLowerCase();
VolumeArgs volumeArgs = VolumeArgs.newBuilder()
.setOwner("bilbo")

View File

@ -61,7 +61,7 @@ public static Collection<Object[]> clientProtocol() {
@BeforeClass
public static void init() throws Exception {
suite = new RatisTestHelper.RatisTestSuite(TestBucketsRatis.class);
suite = new RatisTestHelper.RatisTestSuite();
conf = suite.getConf();
}

View File

@ -57,7 +57,7 @@ public class TestKeysRatis {
@BeforeClass
public static void init() throws Exception {
suite = new RatisTestHelper.RatisTestSuite(TestBucketsRatis.class);
suite = new RatisTestHelper.RatisTestSuite();
path = GenericTestUtils.getTempPath(TestKeysRatis.class.getSimpleName());
ozoneCluster = suite.getCluster();
ozoneCluster.waitForClusterToBeReady();

View File

@ -43,28 +43,15 @@
import java.util.Collections;
import java.util.List;
import java.util.Random;
import java.util.concurrent.TimeUnit;
import static org.apache.hadoop.ozone
.OzoneConfigKeys.DFS_CONTAINER_RATIS_ENABLED_DEFAULT;
import static org.apache.hadoop.ozone
.OzoneConfigKeys.DFS_CONTAINER_RATIS_ENABLED_KEY;
import static org.apache.hadoop.ozone
.OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_INTERVAL;
import static org.apache.hadoop.ozone
.OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_INTERVAL_DEFAULT;
import static org.apache.hadoop.ozone
.OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_TIMEOUT;
import static org.apache.hadoop.ozone
.OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_TIMEOUT_DEFAULT;
import static org.apache.hadoop.ozone
.OzoneConfigKeys.OZONE_KEY_PREALLOCATION_MAXSIZE;
import static org.apache.hadoop.ozone
.OzoneConfigKeys.OZONE_KEY_PREALLOCATION_MAXSIZE_DEFAULT;
import static org.apache.hadoop.ozone.OzoneConfigKeys
.OZONE_OPEN_KEY_CLEANUP_SERVICE_INTERVAL_SECONDS;
import static org.apache.hadoop.ozone.OzoneConfigKeys
.OZONE_OPEN_KEY_CLEANUP_SERVICE_INTERVAL_SECONDS_DEFAULT;
import static org.apache.hadoop.ozone
.OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_DEFAULT;
import static org.apache.hadoop.ozone

View File

@ -42,7 +42,6 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.ozone.client.ObjectStore;
import org.apache.hadoop.ozone.client.OzoneBucket;
import org.apache.hadoop.ozone.client.OzoneClient;
@ -60,7 +59,6 @@
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.Progressable;
import org.apache.hadoop.ozone.client.io.OzoneInputStream;
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
import static org.apache.hadoop.fs.ozone.Constants.OZONE_DEFAULT_USER;

View File

@ -19,7 +19,6 @@
import com.google.common.base.Preconditions;
import com.google.common.primitives.Longs;
import com.google.protobuf.ByteString;
import org.apache.commons.cli.BasicParser;
import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.Option;
@ -31,7 +30,6 @@
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.DFSUtilClient;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OzoneAclInfo;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.BucketInfo;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyInfo;
@ -56,8 +54,6 @@
import java.sql.DriverManager;
import java.sql.SQLException;
import java.sql.Statement;
import java.util.HashSet;
import java.util.Set;
import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_DB_SUFFIX;
import static org.apache.hadoop.ozone.OzoneConsts.OM_DB_NAME;