Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -39,11 +39,9 @@
import org.apache.hadoop.ozone.container.common.SCMTestUtils;
import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
import org.apache.ratis.protocol.exceptions.NotLeaderException;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.mockito.ArgumentCaptor;
import org.mockito.Mockito;

import java.io.IOException;
import java.util.ArrayList;
Expand Down Expand Up @@ -605,11 +603,11 @@ public void testUnderReplicationWithVulnerableReplicasOnUniqueOrigins() throws I
DECOMMISSIONING, State.UNHEALTHY, sequenceID);
replicas.add(unhealthyReplica);
UnderReplicatedHealthResult result = getUnderReplicatedHealthResult();
Mockito.when(result.hasVulnerableUnhealthy()).thenReturn(true);
when(result.hasVulnerableUnhealthy()).thenReturn(true);

final Set<Pair<DatanodeDetails, SCMCommand<?>>> commands = testProcessing(replicas, Collections.emptyList(),
result, 2, 1);
Assertions.assertEquals(unhealthyReplica.getDatanodeDetails(), commands.iterator().next().getKey());
assertEquals(unhealthyReplica.getDatanodeDetails(), commands.iterator().next().getKey());
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,6 @@
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.EnumSource;
import org.mockito.ArgumentCaptor;
import org.mockito.Mockito;

import java.io.IOException;
import java.time.Instant;
Expand Down Expand Up @@ -530,7 +529,7 @@ public void testQuasiClosedContainerWithUnhealthyReplicaOnDecommissioningNodeWit
ContainerReplicaProto.State.UNHEALTHY);
replicas.add(unhealthy);
storeContainerAndReplicas(container, replicas);
Mockito.when(replicationManager.getNodeStatus(any(DatanodeDetails.class)))
when(replicationManager.getNodeStatus(any(DatanodeDetails.class)))
.thenAnswer(invocation -> {
DatanodeDetails dn = invocation.getArgument(0);
if (dn.equals(unhealthy.getDatanodeDetails())) {
Expand All @@ -550,9 +549,9 @@ public void testQuasiClosedContainerWithUnhealthyReplicaOnDecommissioningNodeWit
assertEquals(0, repQueue.overReplicatedQueueSize());

// next, this test sets up some mocks to test if RatisUnderReplicationHandler will handle this container correctly
Mockito.when(ratisPlacementPolicy.chooseDatanodes(anyList(), anyList(), eq(null), eq(1), anyLong(),
when(ratisPlacementPolicy.chooseDatanodes(anyList(), anyList(), eq(null), eq(1), anyLong(),
anyLong())).thenAnswer(invocation -> ImmutableList.of(MockDatanodeDetails.randomDatanodeDetails()));
Mockito.when(nodeManager.getTotalDatanodeCommandCounts(any(DatanodeDetails.class), any(), any()))
when(nodeManager.getTotalDatanodeCommandCounts(any(DatanodeDetails.class), any(), any()))
.thenAnswer(invocation -> {
Map<SCMCommandProto.Type, Integer> map = new HashMap<>();
map.put(SCMCommandProto.Type.replicateContainerCommand, 0);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -66,7 +66,6 @@
import org.apache.ratis.util.function.CheckedBiFunction;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Assertions;

/**
* This class tests the metrics of ContainerStateMachine.
Expand Down Expand Up @@ -142,7 +141,7 @@ static void runContainerStateMachineMetrics(
pipeline, blockID, 1024);
ContainerCommandResponseProto response =
client.sendCommand(writeChunkRequest);
Assertions.assertEquals(ContainerProtos.Result.SUCCESS,
assertEquals(ContainerProtos.Result.SUCCESS,
response.getResult());

metric = getMetrics(CSMMetrics.SOURCE_NAME +
Expand All @@ -160,7 +159,7 @@ static void runContainerStateMachineMetrics(
ContainerTestHelper.getReadChunkRequest(pipeline, writeChunkRequest
.getWriteChunk());
response = client.sendCommand(readChunkRequest);
Assertions.assertEquals(ContainerProtos.Result.SUCCESS,
assertEquals(ContainerProtos.Result.SUCCESS,
response.getResult());

metric = getMetrics(CSMMetrics.SOURCE_NAME +
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -55,7 +55,8 @@
import static org.apache.ozone.test.MetricsAsserts.assertCounter;
import static org.apache.ozone.test.MetricsAsserts.assertQuantileGauges;
import static org.apache.ozone.test.MetricsAsserts.getMetrics;
import org.junit.jupiter.api.Assertions;
import static org.junit.jupiter.api.Assertions.assertEquals;

import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout;

Expand Down Expand Up @@ -121,15 +122,15 @@ public void testContainerMetrics() throws Exception {
pipeline, blockID, 1024);
ContainerCommandResponseProto response =
client.sendCommand(writeChunkRequest);
Assertions.assertEquals(ContainerProtos.Result.SUCCESS,
assertEquals(ContainerProtos.Result.SUCCESS,
response.getResult());

//Read Chunk
ContainerProtos.ContainerCommandRequestProto readChunkRequest =
ContainerTestHelper.getReadChunkRequest(pipeline, writeChunkRequest
.getWriteChunk());
response = client.sendCommand(readChunkRequest);
Assertions.assertEquals(ContainerProtos.Result.SUCCESS, response.getResult());
assertEquals(ContainerProtos.Result.SUCCESS, response.getResult());

MetricsRecordBuilder containerMetrics = getMetrics(
"StorageContainerMetrics");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,14 +17,12 @@

package org.apache.hadoop.ozone.container.metrics;

import org.apache.commons.text.WordUtils;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCommandProto;
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.MiniOzoneHAClusterImpl;
import org.apache.hadoop.ozone.container.common.statemachine.DatanodeQueueMetrics;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout;
Expand All @@ -34,10 +32,12 @@
import java.io.IOException;
import java.util.UUID;

import static org.apache.commons.text.WordUtils.capitalize;
import static org.apache.hadoop.ozone.container.common.statemachine.DatanodeQueueMetrics.COMMAND_DISPATCHER_QUEUE_PREFIX;
import static org.apache.hadoop.ozone.container.common.statemachine.DatanodeQueueMetrics.STATE_CONTEXT_COMMAND_QUEUE_PREFIX;
import static org.apache.ozone.test.MetricsAsserts.getLongGauge;
import static org.apache.ozone.test.MetricsAsserts.getMetrics;
import static org.assertj.core.api.Assertions.assertThat;

/**
* Test for queue metrics of datanodes.
Expand Down Expand Up @@ -89,14 +89,12 @@ public void init() throws Exception {

@Test
public void testQueueMetrics() {

for (SCMCommandProto.Type type: SCMCommandProto.Type.values()) {
Assertions.assertTrue(
getGauge(STATE_CONTEXT_COMMAND_QUEUE_PREFIX +
WordUtils.capitalize(String.valueOf(type)) + "Size") >= 0);
Assertions.assertTrue(
getGauge(COMMAND_DISPATCHER_QUEUE_PREFIX +
WordUtils.capitalize(String.valueOf(type)) + "Size") >= 0);
String typeSize = capitalize(String.valueOf(type)) + "Size";
assertThat(getGauge(STATE_CONTEXT_COMMAND_QUEUE_PREFIX + typeSize))
.isGreaterThanOrEqualTo(0);
assertThat(getGauge(COMMAND_DISPATCHER_QUEUE_PREFIX + typeSize))
.isGreaterThanOrEqualTo(0);
}

}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,6 @@
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
import org.apache.ozone.test.GenericTestUtils.LogCapturer;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout;
Expand Down Expand Up @@ -352,20 +351,15 @@ private Token<ContainerTokenIdentifier> createContainer(
}

private long createAndCloseContainer(
XceiverClientSpi client, boolean useToken) {
XceiverClientSpi client, boolean useToken) throws IOException {
long id = getTestContainerID();
try {
Token<ContainerTokenIdentifier>
token = createContainer(client, useToken, id);

ContainerCommandRequestProto request =
getCloseContainer(client.getPipeline(), id, token);
ContainerCommandResponseProto response = client.sendCommand(request);
assertNotNull(response);
assertSame(response.getResult(), ContainerProtos.Result.SUCCESS);
} catch (Exception e) {
Assertions.fail(e);
}
Token<ContainerTokenIdentifier> token = createContainer(client, useToken, id);

ContainerCommandRequestProto request =
getCloseContainer(client.getPipeline(), id, token);
ContainerCommandResponseProto response = client.sendCommand(request);
assertNotNull(response);
assertSame(response.getResult(), ContainerProtos.Result.SUCCESS);
return id;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -67,14 +67,14 @@
import org.apache.ratis.util.function.CheckedBiConsumer;
import org.apache.ratis.util.function.CheckedBiFunction;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test;

import static org.apache.hadoop.hdds.HddsConfigKeys.OZONE_METADATA_DIRS;
import static org.apache.hadoop.hdds.protocol.MockDatanodeDetails.randomDatanodeDetails;
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.HDDS_DATANODE_DIR_KEY;
import static org.apache.ratis.rpc.SupportedRpcType.GRPC;
import static org.junit.jupiter.api.Assertions.assertNotNull;

/**
* Test Containers.
Expand Down Expand Up @@ -170,7 +170,7 @@ static void runTestClientServer(
ContainerTestHelper
.getCreateContainerRequest(
ContainerTestHelper.getTestContainerID(), pipeline);
Assertions.assertNotNull(request.getTraceID());
assertNotNull(request.getTraceID());

client.sendCommand(request);
} finally {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -95,24 +95,22 @@
import org.apache.commons.lang3.RandomUtils;
import org.apache.commons.lang3.exception.ExceptionUtils;
import org.apache.ratis.rpc.RpcType;

import static org.apache.ratis.rpc.SupportedRpcType.GRPC;

import org.apache.ratis.util.ExitUtils;
import org.apache.ratis.util.function.CheckedBiConsumer;
import org.apache.ratis.util.function.CheckedBiFunction;

import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test;

import static org.apache.ratis.rpc.SupportedRpcType.GRPC;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;

import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test;

/**
* Test Container servers when security is enabled.
*/
Expand Down Expand Up @@ -320,7 +318,7 @@ private static void assertFailsTokenVerification(XceiverClientSpi client,
String msg = response.getMessage();
assertTrue(msg.contains(BLOCK_TOKEN_VERIFICATION_FAILED.name()), msg);
} else {
final Throwable t = Assertions.assertThrows(Throwable.class,
final Throwable t = assertThrows(Throwable.class,
() -> client.sendCommand(request));
assertRootCauseMessage(BLOCK_TOKEN_VERIFICATION_FAILED.name(), t);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,12 +30,12 @@

import org.apache.ozone.test.GenericTestUtils;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout;

import static java.nio.charset.StandardCharsets.UTF_8;
import static org.assertj.core.api.Assertions.assertThat;

/**
* Test Datanode Ratis log parser.
Expand Down Expand Up @@ -78,14 +78,14 @@ public void testRatisLogParsing() throws Exception {
File currentDir = new File(pipelineDir, "current");
File logFile = new File(currentDir, "log_inprogress_0");
GenericTestUtils.waitFor(logFile::exists, 100, 15000);
Assertions.assertTrue(logFile.isFile());
assertThat(logFile).isFile();

DatanodeRatisLogParser datanodeRatisLogParser =
new DatanodeRatisLogParser();
datanodeRatisLogParser.setSegmentFile(logFile);
datanodeRatisLogParser.parseRatisLogs(
DatanodeRatisLogParser::smToContainerLogString);
Assertions.assertTrue(out.toString(StandardCharsets.UTF_8.name())
.contains("Num Total Entries:"));
assertThat(out.toString(StandardCharsets.UTF_8.name()))
.contains("Num Total Entries:");
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,6 @@
import org.apache.ozone.test.GenericTestUtils;
import org.apache.ozone.test.GenericTestUtils.LogCapturer;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout;
Expand All @@ -56,6 +55,7 @@
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL;
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_STALENODE_INTERVAL;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_REPLICATION;
import static org.assertj.core.api.Assertions.assertThat;

/**
* This class tests datanode can tolerate configured num of failed volumes.
Expand Down Expand Up @@ -141,8 +141,8 @@ public void testDNCorrectlyHandlesVolumeFailureOnStartup() throws Exception {
// cluster.
GenericTestUtils.waitFor(() -> exitCapturer.getOutput()
.contains("Exiting with status 1: ExitException"), 500, 60000);
Assertions.assertTrue(dsmCapturer.getOutput()
.contains("DatanodeStateMachine Shutdown due to too many bad volumes"));
assertThat(dsmCapturer.getOutput())
.contains("DatanodeStateMachine Shutdown due to too many bad volumes");

// restore bad volumes
DatanodeTestUtils.restoreBadRootDir(volRootDir0);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,6 @@
import org.apache.ozone.test.GenericTestUtils;
import org.apache.ratis.server.RaftServer;
import org.apache.ratis.server.raftlog.RaftLog;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.slf4j.Logger;
Expand All @@ -46,6 +45,9 @@
import java.io.IOException;
import java.net.URI;

import static org.assertj.core.api.Assertions.assertThat;
import static org.junit.jupiter.api.Assertions.assertEquals;

/**
* Test for OmBucketReadWriteFileOps.
*/
Expand Down Expand Up @@ -207,7 +209,7 @@ private void verifyFileCreation(int expectedCount, FileStatus[] fileStatuses,
}
}
}
Assertions.assertEquals(expectedCount, actual, "Mismatch Count!");
assertEquals(expectedCount, actual, "Mismatch Count!");
}

private void verifyOMLockMetrics(OMLockMetrics omLockMetrics) {
Expand All @@ -218,15 +220,15 @@ private void verifyOMLockMetrics(OMLockMetrics omLockMetrics) {
omLockMetrics.getLongestReadLockWaitingTimeMs());
int readWaitingSamples =
Integer.parseInt(readLockWaitingTimeMsStat.split(" ")[2]);
Assertions.assertTrue(readWaitingSamples > 0, "Read Lock Waiting Samples should be positive");
assertThat(readWaitingSamples).isPositive();

String readLockHeldTimeMsStat = omLockMetrics.getReadLockHeldTimeMsStat();
LOG.info("Read Lock Held Time Stat: " + readLockHeldTimeMsStat);
LOG.info("Longest Read Lock Held Time (ms): " +
omLockMetrics.getLongestReadLockHeldTimeMs());
int readHeldSamples =
Integer.parseInt(readLockHeldTimeMsStat.split(" ")[2]);
Assertions.assertTrue(readHeldSamples > 0, "Read Lock Held Samples should be positive");
assertThat(readHeldSamples).isPositive();

String writeLockWaitingTimeMsStat =
omLockMetrics.getWriteLockWaitingTimeMsStat();
Expand All @@ -235,15 +237,15 @@ private void verifyOMLockMetrics(OMLockMetrics omLockMetrics) {
omLockMetrics.getLongestWriteLockWaitingTimeMs());
int writeWaitingSamples =
Integer.parseInt(writeLockWaitingTimeMsStat.split(" ")[2]);
Assertions.assertTrue(writeWaitingSamples > 0, "Write Lock Waiting Samples should be positive");
assertThat(writeWaitingSamples).isPositive();

String writeLockHeldTimeMsStat = omLockMetrics.getWriteLockHeldTimeMsStat();
LOG.info("Write Lock Held Time Stat: " + writeLockHeldTimeMsStat);
LOG.info("Longest Write Lock Held Time (ms): " +
omLockMetrics.getLongestWriteLockHeldTimeMs());
int writeHeldSamples =
Integer.parseInt(writeLockHeldTimeMsStat.split(" ")[2]);
Assertions.assertTrue(writeHeldSamples > 0, "Write Lock Held Samples should be positive");
assertThat(writeHeldSamples).isPositive();
}

private static class ParameterBuilder {
Expand Down
Loading