errose28 commented on code in PR #8248: URL: https://github.com/apache/ozone/pull/8248#discussion_r2053034843
########## hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/replicas/ReplicasVerify.java: ########## @@ -88,41 +109,122 @@ void findCandidateKeys(OzoneClient ozoneClient, OzoneAddress address) throws IOE String volumeName = address.getVolumeName(); String bucketName = address.getBucketName(); String keyName = address.getKeyName(); + + ObjectNode root = JsonUtils.createObjectNode(null); + ArrayNode keysArray = root.putArray("keys"); + if (!keyName.isEmpty()) { - OzoneKeyDetails keyDetails = ozoneClient.getProxy().getKeyDetails(volumeName, bucketName, keyName); - processKey(keyDetails); + OmKeyInfo keyInfo = ((RpcClient) ozoneClient.getProxy()).getKeyInfo(volumeName, bucketName, keyName, false); Review Comment: We should make this part of the `ClientProtocol` too so the cast is not required. ########## hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/replicas/ReplicasVerify.java: ########## @@ -88,41 +109,122 @@ void findCandidateKeys(OzoneClient ozoneClient, OzoneAddress address) throws IOE String volumeName = address.getVolumeName(); String bucketName = address.getBucketName(); String keyName = address.getKeyName(); + + ObjectNode root = JsonUtils.createObjectNode(null); + ArrayNode keysArray = root.putArray("keys"); + if (!keyName.isEmpty()) { - OzoneKeyDetails keyDetails = ozoneClient.getProxy().getKeyDetails(volumeName, bucketName, keyName); - processKey(keyDetails); + OmKeyInfo keyInfo = ((RpcClient) ozoneClient.getProxy()).getKeyInfo(volumeName, bucketName, keyName, false); + processKey(ozoneClient, keyInfo, keysArray); } else if (!bucketName.isEmpty()) { OzoneVolume volume = objectStore.getVolume(volumeName); OzoneBucket bucket = volume.getBucket(bucketName); - checkBucket(bucket); + checkBucket(ozoneClient, bucket, keysArray); } else if (!volumeName.isEmpty()) { OzoneVolume volume = objectStore.getVolume(volumeName); - checkVolume(volume); + checkVolume(ozoneClient, volume, keysArray); } else { for (Iterator<? extends OzoneVolume> it = objectStore.listVolumes(null); it.hasNext();) { - checkVolume(it.next()); + checkVolume(ozoneClient, it.next(), keysArray); } } + + System.out.println(JsonUtils.toJsonStringWithDefaultPrettyPrinter(root)); } - void checkVolume(OzoneVolume volume) throws IOException { + void checkVolume(OzoneClient ozoneClient, OzoneVolume volume, ArrayNode keysArray) throws IOException { for (Iterator<? extends OzoneBucket> it = volume.listBuckets(null); it.hasNext();) { OzoneBucket bucket = it.next(); - checkBucket(bucket); + checkBucket(ozoneClient, bucket, keysArray); } } - void checkBucket(OzoneBucket bucket) throws IOException { + void checkBucket(OzoneClient ozoneClient, OzoneBucket bucket, ArrayNode keysArray) throws IOException { for (Iterator<? extends OzoneKey> it = bucket.listKeys(null); it.hasNext();) { OzoneKey key = it.next(); // TODO: Remove this check once HDDS-12094 is fixed if (!key.getName().endsWith("/")) { - processKey(bucket.getKey(key.getName())); + OmKeyInfo keyInfo = ((RpcClient) ozoneClient.getProxy()).getKeyInfo( + bucket.getVolumeName(), bucket.getName(), key.getName(), false); + processKey(ozoneClient, keyInfo, keysArray); } } } - void processKey(OzoneKeyDetails keyDetails) { - replicaVerifiers.forEach(verifier -> verifier.verifyKey(keyDetails)); + void processKey(OzoneClient ozoneClient, OmKeyInfo keyInfo, ArrayNode keysArray) { + String volumeName = keyInfo.getVolumeName(); + String bucketName = keyInfo.getBucketName(); + String keyName = keyInfo.getKeyName(); + + ObjectNode keyNode = JsonUtils.createObjectNode(null); + keyNode.put("volumeName", volumeName); + keyNode.put("bucketName", bucketName); + keyNode.put("name", keyName); + + ArrayNode blocksArray = keyNode.putArray("blocks"); + boolean keyPass = true; + + for (OmKeyLocationInfoGroup keyLocationInfoGroup : keyInfo.getKeyLocationVersions()) { Review Comment: We can just use `getLatestVersionLocations` here since we don't have any plans to support key versioning on the server side, and the client side output will not render it in an intuitive way right now. ########## hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/replicas/ReplicasVerify.java: ########## @@ -88,41 +109,122 @@ void findCandidateKeys(OzoneClient ozoneClient, OzoneAddress address) throws IOE String volumeName = address.getVolumeName(); String bucketName = address.getBucketName(); String keyName = address.getKeyName(); + + ObjectNode root = JsonUtils.createObjectNode(null); + ArrayNode keysArray = root.putArray("keys"); + if (!keyName.isEmpty()) { - OzoneKeyDetails keyDetails = ozoneClient.getProxy().getKeyDetails(volumeName, bucketName, keyName); - processKey(keyDetails); + OmKeyInfo keyInfo = ((RpcClient) ozoneClient.getProxy()).getKeyInfo(volumeName, bucketName, keyName, false); + processKey(ozoneClient, keyInfo, keysArray); } else if (!bucketName.isEmpty()) { OzoneVolume volume = objectStore.getVolume(volumeName); OzoneBucket bucket = volume.getBucket(bucketName); - checkBucket(bucket); + checkBucket(ozoneClient, bucket, keysArray); } else if (!volumeName.isEmpty()) { OzoneVolume volume = objectStore.getVolume(volumeName); - checkVolume(volume); + checkVolume(ozoneClient, volume, keysArray); } else { for (Iterator<? extends OzoneVolume> it = objectStore.listVolumes(null); it.hasNext();) { - checkVolume(it.next()); + checkVolume(ozoneClient, it.next(), keysArray); } } + + System.out.println(JsonUtils.toJsonStringWithDefaultPrettyPrinter(root)); } - void checkVolume(OzoneVolume volume) throws IOException { + void checkVolume(OzoneClient ozoneClient, OzoneVolume volume, ArrayNode keysArray) throws IOException { for (Iterator<? extends OzoneBucket> it = volume.listBuckets(null); it.hasNext();) { OzoneBucket bucket = it.next(); - checkBucket(bucket); + checkBucket(ozoneClient, bucket, keysArray); } } - void checkBucket(OzoneBucket bucket) throws IOException { + void checkBucket(OzoneClient ozoneClient, OzoneBucket bucket, ArrayNode keysArray) throws IOException { for (Iterator<? extends OzoneKey> it = bucket.listKeys(null); it.hasNext();) { OzoneKey key = it.next(); // TODO: Remove this check once HDDS-12094 is fixed if (!key.getName().endsWith("/")) { - processKey(bucket.getKey(key.getName())); + OmKeyInfo keyInfo = ((RpcClient) ozoneClient.getProxy()).getKeyInfo( + bucket.getVolumeName(), bucket.getName(), key.getName(), false); + processKey(ozoneClient, keyInfo, keysArray); } } } - void processKey(OzoneKeyDetails keyDetails) { - replicaVerifiers.forEach(verifier -> verifier.verifyKey(keyDetails)); + void processKey(OzoneClient ozoneClient, OmKeyInfo keyInfo, ArrayNode keysArray) { + String volumeName = keyInfo.getVolumeName(); + String bucketName = keyInfo.getBucketName(); + String keyName = keyInfo.getKeyName(); + + ObjectNode keyNode = JsonUtils.createObjectNode(null); + keyNode.put("volumeName", volumeName); + keyNode.put("bucketName", bucketName); + keyNode.put("name", keyName); + + ArrayNode blocksArray = keyNode.putArray("blocks"); + boolean keyPass = true; + + for (OmKeyLocationInfoGroup keyLocationInfoGroup : keyInfo.getKeyLocationVersions()) { + for (OmKeyLocationInfo keyLocation : keyLocationInfoGroup.getLocationList()) { + long containerID = keyLocation.getContainerID(); + long localID = keyLocation.getLocalID(); + + ObjectNode blockNode = JsonUtils.createObjectNode(null); + blockNode.put("containerID", containerID); + blockNode.put("localID", localID); Review Comment: I would prefer to call this `blockID`. I know we use the terms interchangeably in the code but I think `blockID` is clearer to what is being referenced. ########## hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/replicas/ReplicasVerify.java: ########## @@ -88,41 +109,122 @@ void findCandidateKeys(OzoneClient ozoneClient, OzoneAddress address) throws IOE String volumeName = address.getVolumeName(); String bucketName = address.getBucketName(); String keyName = address.getKeyName(); + + ObjectNode root = JsonUtils.createObjectNode(null); + ArrayNode keysArray = root.putArray("keys"); + if (!keyName.isEmpty()) { - OzoneKeyDetails keyDetails = ozoneClient.getProxy().getKeyDetails(volumeName, bucketName, keyName); - processKey(keyDetails); + OmKeyInfo keyInfo = ((RpcClient) ozoneClient.getProxy()).getKeyInfo(volumeName, bucketName, keyName, false); + processKey(ozoneClient, keyInfo, keysArray); } else if (!bucketName.isEmpty()) { OzoneVolume volume = objectStore.getVolume(volumeName); OzoneBucket bucket = volume.getBucket(bucketName); - checkBucket(bucket); + checkBucket(ozoneClient, bucket, keysArray); } else if (!volumeName.isEmpty()) { OzoneVolume volume = objectStore.getVolume(volumeName); - checkVolume(volume); + checkVolume(ozoneClient, volume, keysArray); } else { for (Iterator<? extends OzoneVolume> it = objectStore.listVolumes(null); it.hasNext();) { - checkVolume(it.next()); + checkVolume(ozoneClient, it.next(), keysArray); } } + + System.out.println(JsonUtils.toJsonStringWithDefaultPrettyPrinter(root)); } - void checkVolume(OzoneVolume volume) throws IOException { + void checkVolume(OzoneClient ozoneClient, OzoneVolume volume, ArrayNode keysArray) throws IOException { for (Iterator<? extends OzoneBucket> it = volume.listBuckets(null); it.hasNext();) { OzoneBucket bucket = it.next(); - checkBucket(bucket); + checkBucket(ozoneClient, bucket, keysArray); } } - void checkBucket(OzoneBucket bucket) throws IOException { + void checkBucket(OzoneClient ozoneClient, OzoneBucket bucket, ArrayNode keysArray) throws IOException { for (Iterator<? extends OzoneKey> it = bucket.listKeys(null); it.hasNext();) { OzoneKey key = it.next(); // TODO: Remove this check once HDDS-12094 is fixed if (!key.getName().endsWith("/")) { - processKey(bucket.getKey(key.getName())); + OmKeyInfo keyInfo = ((RpcClient) ozoneClient.getProxy()).getKeyInfo( + bucket.getVolumeName(), bucket.getName(), key.getName(), false); + processKey(ozoneClient, keyInfo, keysArray); } } } - void processKey(OzoneKeyDetails keyDetails) { - replicaVerifiers.forEach(verifier -> verifier.verifyKey(keyDetails)); + void processKey(OzoneClient ozoneClient, OmKeyInfo keyInfo, ArrayNode keysArray) { + String volumeName = keyInfo.getVolumeName(); + String bucketName = keyInfo.getBucketName(); + String keyName = keyInfo.getKeyName(); + + ObjectNode keyNode = JsonUtils.createObjectNode(null); + keyNode.put("volumeName", volumeName); + keyNode.put("bucketName", bucketName); + keyNode.put("name", keyName); + + ArrayNode blocksArray = keyNode.putArray("blocks"); + boolean keyPass = true; + + for (OmKeyLocationInfoGroup keyLocationInfoGroup : keyInfo.getKeyLocationVersions()) { + for (OmKeyLocationInfo keyLocation : keyLocationInfoGroup.getLocationList()) { + long containerID = keyLocation.getContainerID(); + long localID = keyLocation.getLocalID(); + + ObjectNode blockNode = JsonUtils.createObjectNode(null); + blockNode.put("containerID", containerID); + blockNode.put("localID", localID); + + ArrayNode replicasArray = blockNode.putArray("replicas"); + boolean blockPass = true; + + for (DatanodeDetails datanode : keyLocation.getPipeline().getNodes()) { + ObjectNode datanodeNode = JsonUtils.createObjectNode(null); + datanodeNode.put("uuid", datanode.getUuidString()); + datanodeNode.put("hostname", datanode.getHostName()); + + ArrayNode checksArray = JsonUtils.createArrayNode(); + boolean replicaPass = true; + + for (ReplicaVerifier verifier : replicaVerifiers) { + BlockVerificationResult result = verifier.verifyBlock(datanode, keyLocation); + ObjectNode checkNode = JsonUtils.createObjectNode(null); + checkNode.put("type", verifier.getType()); + checkNode.put("pass", result.passed()); + + ArrayNode failuresArray = checkNode.putArray("failures"); + for (BlockVerificationResult.FailureDetail failure : result.getFailures().orElse(Collections.emptyList())) { Review Comment: Looking at the way this is actually used my previous suggestion may not have been the best. Since `Collections#emptyList` is a singleton, we can probably just use that as the default value for the failure list inside the `FailureDetail` object instead of null. Then we can return regular list instead of optional. ########## hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/replicas/BlockVerificationResult.java: ########## @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.ozone.debug.replicas; + +import java.util.Collections; +import java.util.List; +import java.util.Optional; + +/** + * Json structure for replicas to pass through each check and give output. + */ +public class BlockVerificationResult { + + private final boolean pass; + private final List<FailureDetail> failures; + + public BlockVerificationResult(boolean pass, List<FailureDetail> failures) { + this.pass = pass; + this.failures = failures; + } + + public static BlockVerificationResult pass() { + return new BlockVerificationResult(true, null); + } + + public static BlockVerificationResult failCheck(String message) { + return new BlockVerificationResult(false, + Collections.singletonList(new FailureDetail(true, message))); + } + + public static BlockVerificationResult failIncomplete(String message) { + return new BlockVerificationResult(false, + Collections.singletonList(new FailureDetail(false, message))); + } + + public boolean passed() { + return pass; + } + + public Optional<List<FailureDetail>> getFailures() { + return Optional.ofNullable(failures); + } + + /** + * Details about the check failure. + */ + public static class FailureDetail { + // indicates whether the check finished and failed, + // or it was unable to finish due to connection or other issues + private final boolean completed; + private final String message; + + public FailureDetail(boolean completed, String message) { + this.completed = completed; Review Comment: Now that I'm looking at the whole output it looks like `complete` should actually be a top level property of the `BlockVerificationResult`. Then the failures can just be returned as a list of strings for now. When rendering it as json we can keep it as an object with a `message` field to keep the output format extensible later. This internal format is easy enough to expand later if just a list of strings is not enough. ########## hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/replicas/ReplicasVerify.java: ########## @@ -88,41 +109,122 @@ void findCandidateKeys(OzoneClient ozoneClient, OzoneAddress address) throws IOE String volumeName = address.getVolumeName(); String bucketName = address.getBucketName(); String keyName = address.getKeyName(); + + ObjectNode root = JsonUtils.createObjectNode(null); + ArrayNode keysArray = root.putArray("keys"); + if (!keyName.isEmpty()) { - OzoneKeyDetails keyDetails = ozoneClient.getProxy().getKeyDetails(volumeName, bucketName, keyName); - processKey(keyDetails); + OmKeyInfo keyInfo = ((RpcClient) ozoneClient.getProxy()).getKeyInfo(volumeName, bucketName, keyName, false); + processKey(ozoneClient, keyInfo, keysArray); } else if (!bucketName.isEmpty()) { OzoneVolume volume = objectStore.getVolume(volumeName); OzoneBucket bucket = volume.getBucket(bucketName); - checkBucket(bucket); + checkBucket(ozoneClient, bucket, keysArray); } else if (!volumeName.isEmpty()) { OzoneVolume volume = objectStore.getVolume(volumeName); - checkVolume(volume); + checkVolume(ozoneClient, volume, keysArray); } else { for (Iterator<? extends OzoneVolume> it = objectStore.listVolumes(null); it.hasNext();) { - checkVolume(it.next()); + checkVolume(ozoneClient, it.next(), keysArray); } } + + System.out.println(JsonUtils.toJsonStringWithDefaultPrettyPrinter(root)); } - void checkVolume(OzoneVolume volume) throws IOException { + void checkVolume(OzoneClient ozoneClient, OzoneVolume volume, ArrayNode keysArray) throws IOException { for (Iterator<? extends OzoneBucket> it = volume.listBuckets(null); it.hasNext();) { OzoneBucket bucket = it.next(); - checkBucket(bucket); + checkBucket(ozoneClient, bucket, keysArray); } } - void checkBucket(OzoneBucket bucket) throws IOException { + void checkBucket(OzoneClient ozoneClient, OzoneBucket bucket, ArrayNode keysArray) throws IOException { for (Iterator<? extends OzoneKey> it = bucket.listKeys(null); it.hasNext();) { OzoneKey key = it.next(); // TODO: Remove this check once HDDS-12094 is fixed if (!key.getName().endsWith("/")) { - processKey(bucket.getKey(key.getName())); + OmKeyInfo keyInfo = ((RpcClient) ozoneClient.getProxy()).getKeyInfo( + bucket.getVolumeName(), bucket.getName(), key.getName(), false); + processKey(ozoneClient, keyInfo, keysArray); } } } - void processKey(OzoneKeyDetails keyDetails) { - replicaVerifiers.forEach(verifier -> verifier.verifyKey(keyDetails)); + void processKey(OzoneClient ozoneClient, OmKeyInfo keyInfo, ArrayNode keysArray) { Review Comment: `ozoneClient` parameter is unused, we can remove it. Alternatively we can make this method take separate volume, bucket, key parameters and do the `getKeyInfo` call itself. ########## hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/replicas/ReplicasVerify.java: ########## @@ -88,41 +109,122 @@ void findCandidateKeys(OzoneClient ozoneClient, OzoneAddress address) throws IOE String volumeName = address.getVolumeName(); String bucketName = address.getBucketName(); String keyName = address.getKeyName(); + + ObjectNode root = JsonUtils.createObjectNode(null); + ArrayNode keysArray = root.putArray("keys"); + if (!keyName.isEmpty()) { - OzoneKeyDetails keyDetails = ozoneClient.getProxy().getKeyDetails(volumeName, bucketName, keyName); - processKey(keyDetails); + OmKeyInfo keyInfo = ((RpcClient) ozoneClient.getProxy()).getKeyInfo(volumeName, bucketName, keyName, false); + processKey(ozoneClient, keyInfo, keysArray); } else if (!bucketName.isEmpty()) { OzoneVolume volume = objectStore.getVolume(volumeName); OzoneBucket bucket = volume.getBucket(bucketName); - checkBucket(bucket); + checkBucket(ozoneClient, bucket, keysArray); } else if (!volumeName.isEmpty()) { OzoneVolume volume = objectStore.getVolume(volumeName); - checkVolume(volume); + checkVolume(ozoneClient, volume, keysArray); } else { for (Iterator<? extends OzoneVolume> it = objectStore.listVolumes(null); it.hasNext();) { - checkVolume(it.next()); + checkVolume(ozoneClient, it.next(), keysArray); } } + + System.out.println(JsonUtils.toJsonStringWithDefaultPrettyPrinter(root)); Review Comment: At the end of the top level object (at the same level as the `keys` list) let's add one more `pass` flag set to true only if all keys passed verification. This will be useful when `--all-results` is passed so we don't have to filter all the keys searched to see if any failed. ########## hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/replicas/ReplicasVerify.java: ########## @@ -88,41 +109,122 @@ void findCandidateKeys(OzoneClient ozoneClient, OzoneAddress address) throws IOE String volumeName = address.getVolumeName(); String bucketName = address.getBucketName(); String keyName = address.getKeyName(); + + ObjectNode root = JsonUtils.createObjectNode(null); + ArrayNode keysArray = root.putArray("keys"); + if (!keyName.isEmpty()) { - OzoneKeyDetails keyDetails = ozoneClient.getProxy().getKeyDetails(volumeName, bucketName, keyName); - processKey(keyDetails); + OmKeyInfo keyInfo = ((RpcClient) ozoneClient.getProxy()).getKeyInfo(volumeName, bucketName, keyName, false); + processKey(ozoneClient, keyInfo, keysArray); } else if (!bucketName.isEmpty()) { OzoneVolume volume = objectStore.getVolume(volumeName); OzoneBucket bucket = volume.getBucket(bucketName); - checkBucket(bucket); + checkBucket(ozoneClient, bucket, keysArray); } else if (!volumeName.isEmpty()) { OzoneVolume volume = objectStore.getVolume(volumeName); - checkVolume(volume); + checkVolume(ozoneClient, volume, keysArray); } else { for (Iterator<? extends OzoneVolume> it = objectStore.listVolumes(null); it.hasNext();) { - checkVolume(it.next()); + checkVolume(ozoneClient, it.next(), keysArray); } } + + System.out.println(JsonUtils.toJsonStringWithDefaultPrettyPrinter(root)); } - void checkVolume(OzoneVolume volume) throws IOException { + void checkVolume(OzoneClient ozoneClient, OzoneVolume volume, ArrayNode keysArray) throws IOException { for (Iterator<? extends OzoneBucket> it = volume.listBuckets(null); it.hasNext();) { OzoneBucket bucket = it.next(); - checkBucket(bucket); + checkBucket(ozoneClient, bucket, keysArray); } } - void checkBucket(OzoneBucket bucket) throws IOException { + void checkBucket(OzoneClient ozoneClient, OzoneBucket bucket, ArrayNode keysArray) throws IOException { for (Iterator<? extends OzoneKey> it = bucket.listKeys(null); it.hasNext();) { OzoneKey key = it.next(); // TODO: Remove this check once HDDS-12094 is fixed if (!key.getName().endsWith("/")) { - processKey(bucket.getKey(key.getName())); + OmKeyInfo keyInfo = ((RpcClient) ozoneClient.getProxy()).getKeyInfo( + bucket.getVolumeName(), bucket.getName(), key.getName(), false); + processKey(ozoneClient, keyInfo, keysArray); } } } - void processKey(OzoneKeyDetails keyDetails) { - replicaVerifiers.forEach(verifier -> verifier.verifyKey(keyDetails)); + void processKey(OzoneClient ozoneClient, OmKeyInfo keyInfo, ArrayNode keysArray) { + String volumeName = keyInfo.getVolumeName(); + String bucketName = keyInfo.getBucketName(); + String keyName = keyInfo.getKeyName(); + + ObjectNode keyNode = JsonUtils.createObjectNode(null); + keyNode.put("volumeName", volumeName); + keyNode.put("bucketName", bucketName); + keyNode.put("name", keyName); + + ArrayNode blocksArray = keyNode.putArray("blocks"); + boolean keyPass = true; + + for (OmKeyLocationInfoGroup keyLocationInfoGroup : keyInfo.getKeyLocationVersions()) { + for (OmKeyLocationInfo keyLocation : keyLocationInfoGroup.getLocationList()) { + long containerID = keyLocation.getContainerID(); + long localID = keyLocation.getLocalID(); + + ObjectNode blockNode = JsonUtils.createObjectNode(null); Review Comment: After creating the top level object node, all further Json objects should be created and added to an existing Json object with one call to `putObject` or `addObject` for arrays. ########## hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/replicas/BlockExistenceVerifier.java: ########## @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.ozone.debug.replicas; + +import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.ONE; + +import java.io.IOException; +import java.util.Collections; +import org.apache.hadoop.hdds.client.StandaloneReplicationConfig; +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; +import org.apache.hadoop.hdds.scm.XceiverClientManager; +import org.apache.hadoop.hdds.scm.XceiverClientSpi; +import org.apache.hadoop.hdds.scm.cli.ContainerOperationClient; +import org.apache.hadoop.hdds.scm.pipeline.Pipeline; +import org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls; +import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; + +/** + * Verifies block existence by making getBlock calls to the datanode. + */ +public class BlockExistenceVerifier implements ReplicaVerifier { + private final ContainerOperationClient containerClient; + private final XceiverClientManager xceiverClientManager; + private static final String CHECK_TYPE = "blockExistence"; + + @Override + public String getType() { + return CHECK_TYPE; + } + + public BlockExistenceVerifier(OzoneConfiguration conf) throws IOException { + this.containerClient = new ContainerOperationClient(conf); + this.xceiverClientManager = containerClient.getXceiverClientManager(); + } + + @Override + public BlockVerificationResult verifyBlock(DatanodeDetails datanode, OmKeyLocationInfo keyLocation) { + try { + Pipeline pipeline = Pipeline.newBuilder(keyLocation.getPipeline()) + .setReplicationConfig(StandaloneReplicationConfig.getInstance(ONE)) + .setNodes(Collections.singletonList(datanode)) + .build(); + + XceiverClientSpi client = xceiverClientManager.acquireClientForReadData(pipeline); + ContainerProtos.GetBlockResponseProto response = ContainerProtocolCalls.getBlock( + client, + keyLocation.getBlockID(), + keyLocation.getToken(), + Collections.singletonMap(datanode, 1) Review Comment: I think we will need to set this to the correct replica index for EC key data to be read properly. That would need to be added to the pipeline object. We will need to check how to obtain this information. It would probably also be useful to print the replica index in our main output for EC data. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@ozone.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@ozone.apache.org For additional commands, e-mail: issues-h...@ozone.apache.org