Skip to content

Commit a5e1cd0

Browse files
authored
HDDS-13533. Show the summary of replicas verify checks (apache#8898)
1 parent 1d42e20 commit a5e1cd0

File tree

2 files changed

+163
-12
lines changed

2 files changed

+163
-12
lines changed

hadoop-ozone/dist/src/main/smoketest/debug/ozone-debug-keywords.robot

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -36,7 +36,8 @@ Execute replicas verify container state debug tool
3636

3737
Parse replicas verify JSON output
3838
[Arguments] ${output}
39-
${json} = Evaluate json.loads('''${output}''') json
39+
${json_split} = Evaluate '''${output}'''.split('***')[0].strip()
40+
${json} = Evaluate json.loads('''${json_split}''') json
4041
[Return] ${json}
4142

4243
Check to Verify Replicas

hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/replicas/ReplicasVerify.java

Lines changed: 161 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -17,14 +17,25 @@
1717

1818
package org.apache.hadoop.ozone.debug.replicas;
1919

20+
import static org.apache.hadoop.ozone.conf.OzoneServiceConfig.DEFAULT_SHUTDOWN_HOOK_PRIORITY;
21+
2022
import com.fasterxml.jackson.databind.node.ArrayNode;
2123
import com.fasterxml.jackson.databind.node.ObjectNode;
2224
import java.io.IOException;
25+
import java.io.PrintStream;
2326
import java.util.ArrayList;
27+
import java.util.HashSet;
2428
import java.util.Iterator;
2529
import java.util.List;
30+
import java.util.Map;
2631
import java.util.Optional;
32+
import java.util.Set;
33+
import java.util.concurrent.ConcurrentHashMap;
34+
import java.util.concurrent.TimeUnit;
2735
import java.util.concurrent.atomic.AtomicBoolean;
36+
import java.util.concurrent.atomic.AtomicInteger;
37+
import java.util.function.Supplier;
38+
import org.apache.commons.lang3.time.DurationFormatUtils;
2839
import org.apache.hadoop.hdds.client.ReplicationConfig;
2940
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
3041
import org.apache.hadoop.hdds.scm.cli.ScmOption;
@@ -41,6 +52,7 @@
4152
import org.apache.hadoop.ozone.shell.OzoneAddress;
4253
import org.apache.hadoop.ozone.shell.Shell;
4354
import org.apache.hadoop.ozone.shell.ShellReplicationOptions;
55+
import org.apache.hadoop.ozone.util.ShutdownHookManager;
4456
import picocli.CommandLine;
4557

4658
/**
@@ -79,22 +91,80 @@ public class ReplicasVerify extends Handler {
7991

8092
private List<ReplicaVerifier> replicaVerifiers;
8193

94+
private static final String DURATION_FORMAT = "HH:mm:ss,SSS";
95+
private long startTime;
96+
private long endTime;
97+
private String verificationScope;
98+
private final List<String> verificationTypes = new ArrayList<>();
99+
private final AtomicInteger volumesProcessed = new AtomicInteger(0);
100+
private final AtomicInteger bucketsProcessed = new AtomicInteger(0);
101+
private final AtomicInteger keysProcessed = new AtomicInteger(0);
102+
private final AtomicInteger keysPassed = new AtomicInteger(0);
103+
private final AtomicInteger keysFailed = new AtomicInteger(0);
104+
private final Map<String, AtomicInteger> failuresByType = new ConcurrentHashMap<>();
105+
private volatile Throwable exception;
106+
107+
private void addVerifier(boolean condition, Supplier<ReplicaVerifier> verifierSupplier) {
108+
if (condition) {
109+
ReplicaVerifier verifier = verifierSupplier.get();
110+
replicaVerifiers.add(verifier);
111+
String verifierType = verifier.getType();
112+
verificationTypes.add(verifierType);
113+
failuresByType.put(verifierType, new AtomicInteger(0));
114+
}
115+
}
116+
82117
@Override
83118
protected void execute(OzoneClient client, OzoneAddress address) throws IOException {
119+
startTime = System.nanoTime();
120+
121+
if (!address.getKeyName().isEmpty()) {
122+
verificationScope = "Key";
123+
} else if (!address.getBucketName().isEmpty()) {
124+
verificationScope = "Bucket";
125+
} else if (!address.getVolumeName().isEmpty()) {
126+
verificationScope = "Volume";
127+
} else {
128+
verificationScope = "All Volumes";
129+
}
130+
84131
replicaVerifiers = new ArrayList<>();
85132

86-
if (verification.doExecuteChecksums) {
87-
replicaVerifiers.add(new ChecksumVerifier(getConf()));
88-
}
133+
addVerifier(verification.doExecuteChecksums, () -> {
134+
try {
135+
return new ChecksumVerifier(getConf());
136+
} catch (IOException e) {
137+
throw new RuntimeException(e);
138+
}
139+
});
89140

90-
if (verification.doExecuteBlockExistence) {
91-
replicaVerifiers.add(new BlockExistenceVerifier(getConf()));
92-
}
93-
if (verification.doExecuteReplicaState) {
94-
replicaVerifiers.add(new ContainerStateVerifier(getConf(), containerCacheSize));
95-
}
141+
addVerifier(verification.doExecuteBlockExistence, () -> {
142+
try {
143+
return new BlockExistenceVerifier(getConf());
144+
} catch (IOException e) {
145+
throw new RuntimeException(e);
146+
}
147+
});
96148

97-
findCandidateKeys(client, address);
149+
addVerifier(verification.doExecuteReplicaState, () -> {
150+
try {
151+
return new ContainerStateVerifier(getConf(), containerCacheSize);
152+
} catch (IOException e) {
153+
throw new RuntimeException(e);
154+
}
155+
});
156+
157+
// Add shutdown hook to ensure summary is printed even if interrupted
158+
addShutdownHook();
159+
160+
try {
161+
findCandidateKeys(client, address);
162+
} catch (Exception e) {
163+
exception = e;
164+
throw e;
165+
} finally {
166+
endTime = System.nanoTime();
167+
}
98168
}
99169

100170
@Override
@@ -133,6 +203,7 @@ void findCandidateKeys(OzoneClient ozoneClient, OzoneAddress address) throws IOE
133203

134204
void checkVolume(OzoneClient ozoneClient, OzoneVolume volume, ArrayNode keysArray, AtomicBoolean allKeysPassed)
135205
throws IOException {
206+
volumesProcessed.incrementAndGet();
136207
for (Iterator<? extends OzoneBucket> it = volume.listBuckets(null); it.hasNext();) {
137208
OzoneBucket bucket = it.next();
138209
checkBucket(ozoneClient, bucket, keysArray, allKeysPassed);
@@ -141,6 +212,7 @@ void checkVolume(OzoneClient ozoneClient, OzoneVolume volume, ArrayNode keysArra
141212

142213
void checkBucket(OzoneClient ozoneClient, OzoneBucket bucket, ArrayNode keysArray, AtomicBoolean allKeysPassed)
143214
throws IOException {
215+
bucketsProcessed.incrementAndGet();
144216
for (Iterator<? extends OzoneKey> it = bucket.listKeys(null); it.hasNext();) {
145217
OzoneKey key = it.next();
146218
// TODO: Remove this check once HDDS-12094 is fixed
@@ -152,6 +224,7 @@ void checkBucket(OzoneClient ozoneClient, OzoneBucket bucket, ArrayNode keysArra
152224

153225
void processKey(OzoneClient ozoneClient, String volumeName, String bucketName, String keyName,
154226
ArrayNode keysArray, AtomicBoolean allKeysPassed) throws IOException {
227+
keysProcessed.incrementAndGet();
155228
OmKeyInfo keyInfo = ozoneClient.getProxy().getKeyInfo(
156229
volumeName, bucketName, keyName, false);
157230

@@ -167,6 +240,7 @@ void processKey(OzoneClient ozoneClient, String volumeName, String bucketName, S
167240

168241
ArrayNode blocksArray = keyNode.putArray("blocks");
169242
boolean keyPass = true;
243+
Set<String> failedVerificationTypes = new HashSet<>();
170244

171245
for (OmKeyLocationInfo keyLocation : keyInfo.getLatestVersionLocations().getBlocksLatestVersionOnly()) {
172246
long containerID = keyLocation.getContainerID();
@@ -205,6 +279,7 @@ void processKey(OzoneClient ozoneClient, String volumeName, String bucketName, S
205279

206280
if (!result.passed()) {
207281
replicaPass = false;
282+
failedVerificationTypes.add(verifier.getType());
208283
}
209284
}
210285

@@ -219,15 +294,90 @@ void processKey(OzoneClient ozoneClient, String volumeName, String bucketName, S
219294
}
220295

221296
keyNode.put("pass", keyPass);
222-
if (!keyPass) {
297+
if (keyPass) {
298+
keysPassed.incrementAndGet();
299+
} else {
300+
keysFailed.incrementAndGet();
223301
allKeysPassed.set(false);
302+
failedVerificationTypes.forEach(failedType -> failuresByType
303+
.computeIfAbsent(failedType, k -> new AtomicInteger(0))
304+
.incrementAndGet()
305+
);
224306
}
225307

226308
if (!keyPass || allResults) {
227309
keysArray.add(keyNode);
228310
}
229311
}
230312

313+
/**
314+
* Adds ShutdownHook to print summary statistics.
315+
*/
316+
private void addShutdownHook() {
317+
ShutdownHookManager.get().addShutdownHook(() -> {
318+
if (endTime == 0) {
319+
endTime = System.nanoTime();
320+
}
321+
printSummary(System.err);
322+
}, DEFAULT_SHUTDOWN_HOOK_PRIORITY);
323+
}
324+
325+
/**
326+
* Prints summary of replica verification run.
327+
*
328+
* @param out PrintStream
329+
*/
330+
void printSummary(PrintStream out) {
331+
if (endTime == 0) {
332+
endTime = System.nanoTime();
333+
}
334+
335+
long execTimeNanos = endTime - startTime;
336+
String execTime = DurationFormatUtils.formatDuration(TimeUnit.NANOSECONDS.toMillis(execTimeNanos), DURATION_FORMAT);
337+
338+
long totalKeysProcessed = keysProcessed.get();
339+
long totalKeysPassed = keysPassed.get();
340+
long totalKeysFailed = keysFailed.get();
341+
342+
out.println();
343+
out.println("***************************************************");
344+
out.println("REPLICA VERIFICATION SUMMARY");
345+
out.println("***************************************************");
346+
out.println("Status: " + (exception != null ? "Failed" :
347+
(totalKeysFailed == 0 ? "Success" : "Completed with failures")));
348+
out.println("Verification Scope: " + verificationScope);
349+
out.println("Verification Types: " + String.join(", ", verificationTypes));
350+
out.println("URI: " + uri);
351+
out.println();
352+
out.println("Number of Volumes processed: " + volumesProcessed.get());
353+
out.println("Number of Buckets processed: " + bucketsProcessed.get());
354+
out.println("Number of Keys processed: " + totalKeysProcessed);
355+
out.println();
356+
out.println("Keys passed verification: " + totalKeysPassed);
357+
out.println("Keys failed verification: " + totalKeysFailed);
358+
359+
if (!failuresByType.isEmpty() && totalKeysFailed > 0) {
360+
out.println();
361+
for (String verificationType : verificationTypes) {
362+
long typeFailures = failuresByType.get(verificationType).get();
363+
if (typeFailures > 0) {
364+
out.println("Keys failed " + verificationType + " verification: " + typeFailures);
365+
}
366+
}
367+
out.println("Note: A key may fail multiple verification types, so total may exceed overall failures.");
368+
}
369+
370+
out.println();
371+
out.println("Total Execution time: " + execTime);
372+
373+
if (exception != null) {
374+
out.println();
375+
out.println("Exception: " + exception.getClass().getSimpleName() + ": " + exception.getMessage());
376+
}
377+
378+
out.println("***************************************************");
379+
}
380+
231381
/**
232382
* Check if the key should be processed based on replication config.
233383
* @param keyInfo the key to check

0 commit comments

Comments
 (0)