errose28 commented on code in PR #8248: URL: https://github.com/apache/ozone/pull/8248#discussion_r2036403381
########## hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/replicas/ReplicaVerifier.java: ########## @@ -17,12 +17,14 @@ package org.apache.hadoop.ozone.debug.replicas; -import org.apache.hadoop.ozone.client.OzoneKeyDetails; +import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.ozone.client.io.OzoneInputStream; +import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; /** - * Functional interface for implementing a key verifier. + * Functional interface for implementing a block verifier. */ @FunctionalInterface public interface ReplicaVerifier { - void verifyKey(OzoneKeyDetails keyDetails); + BlockVerificationResult verifyBlock(DatanodeDetails datanode, OzoneInputStream stream, OmKeyLocationInfo keyLocation); Review Comment: We should let each verifier open an input stream if needed. Only the checksums check will need it for now, and they won't be thread safe when we start parallelizing the verification. ```suggestion BlockVerificationResult verifyBlock(DatanodeDetails datanode, OmKeyLocationInfo keyLocation); ``` ########## hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/replicas/Checksums.java: ########## @@ -44,121 +32,23 @@ * downloaded replicas. */ public class Checksums implements ReplicaVerifier { Review Comment: ```suggestion public class ChecksumVerifier implements ReplicaVerifier { ``` ########## hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/replicas/BlockVerificationResult.java: ########## @@ -0,0 +1,91 @@ +/* + * 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 com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ArrayNode; +import com.fasterxml.jackson.databind.node.ObjectNode; +import java.util.List; + +/** + * Json structure for replicas to pass through each check and give output. + */ +public class BlockVerificationResult { + + private final String type; + private final boolean pass; + private final List<FailureDetail> failures; + + public BlockVerificationResult(String type, boolean pass, List<FailureDetail> failures) { + this.type = type; + this.pass = pass; + this.failures = failures; + } + + public String getType() { + return type; + } + + public boolean isPass() { + return pass; + } + + public List<FailureDetail> getFailures() { + return failures; + } + + /** + * Details about the check failure. + */ + public static class FailureDetail { + private final boolean present; Review Comment: Now that I'm looking at this in context it would probably be better to call this something like `accessible` or maybe `completed`? The goal is to quickly indicate whether we finished the check and it failed, or we were unable to finish the check due to connection or other issues. ########## hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/replicas/BlockVerificationResult.java: ########## @@ -0,0 +1,91 @@ +/* + * 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 com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ArrayNode; +import com.fasterxml.jackson.databind.node.ObjectNode; +import java.util.List; + +/** + * Json structure for replicas to pass through each check and give output. + */ +public class BlockVerificationResult { + + private final String type; + private final boolean pass; + private final List<FailureDetail> failures; + + public BlockVerificationResult(String type, boolean pass, List<FailureDetail> failures) { + this.type = type; + this.pass = pass; + this.failures = failures; + } + + public String getType() { + return type; + } + + public boolean isPass() { Review Comment: ```suggestion public boolean passed() { ``` ########## hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/replicas/BlockExistenceVerifier.java: ########## @@ -0,0 +1,86 @@ +/* + * 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 java.util.Map; +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.protocol.proto.HddsProtos; +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.client.io.OzoneInputStream; +import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; + +/** + * Checks block existence using GetBlock calls to the Datanodes. + */ +public class BlockExistenceVerifier implements ReplicaVerifier { + + private OzoneConfiguration conf; + private static final String CHECKTYPE = "blockExistence"; + + public BlockExistenceVerifier(OzoneConfiguration conf) { + this.conf = conf; + } + + @Override + public BlockVerificationResult verifyBlock(DatanodeDetails datanode, OzoneInputStream stream, + OmKeyLocationInfo keyLocation) { + try (ContainerOperationClient containerClient = new ContainerOperationClient(conf); + XceiverClientManager xceiverClientManager = containerClient.getXceiverClientManager()) { + + Pipeline keyPipeline = keyLocation.getPipeline(); + boolean isECKey = keyPipeline.getReplicationConfig().getReplicationType() == HddsProtos.ReplicationType.EC; + Pipeline pipeline = isECKey ? keyPipeline : Review Comment: This check (most/all the checks actually) will act the same whether it is EC or Ratis data, since they are just working on individual blocks without a bigger picture. ########## hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/replicas/BlockExistenceVerifier.java: ########## @@ -0,0 +1,86 @@ +/* + * 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 java.util.Map; +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.protocol.proto.HddsProtos; +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.client.io.OzoneInputStream; +import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; + +/** + * Checks block existence using GetBlock calls to the Datanodes. + */ +public class BlockExistenceVerifier implements ReplicaVerifier { + + private OzoneConfiguration conf; + private static final String CHECKTYPE = "blockExistence"; Review Comment: ```suggestion private final OzoneConfiguration conf; private static final String CHECK_TYPE = "blockExistence"; ``` ########## hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/replicas/BlockVerificationResult.java: ########## @@ -0,0 +1,91 @@ +/* + * 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 com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ArrayNode; +import com.fasterxml.jackson.databind.node.ObjectNode; +import java.util.List; + +/** + * Json structure for replicas to pass through each check and give output. + */ +public class BlockVerificationResult { + + private final String type; + private final boolean pass; + private final List<FailureDetail> failures; + + public BlockVerificationResult(String type, boolean pass, List<FailureDetail> failures) { Review Comment: The getter for the list of failures can return an `Optional` which allows the list to remain null in the common case where verification passed. ########## hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/replicas/BlockExistenceVerifier.java: ########## @@ -0,0 +1,86 @@ +/* + * 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 java.util.Map; +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.protocol.proto.HddsProtos; +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.client.io.OzoneInputStream; +import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; + +/** + * Checks block existence using GetBlock calls to the Datanodes. + */ +public class BlockExistenceVerifier implements ReplicaVerifier { + + private OzoneConfiguration conf; + private static final String CHECKTYPE = "blockExistence"; + + public BlockExistenceVerifier(OzoneConfiguration conf) { + this.conf = conf; + } + + @Override + public BlockVerificationResult verifyBlock(DatanodeDetails datanode, OzoneInputStream stream, + OmKeyLocationInfo keyLocation) { + try (ContainerOperationClient containerClient = new ContainerOperationClient(conf); + XceiverClientManager xceiverClientManager = containerClient.getXceiverClientManager()) { Review Comment: We should init the clients in the constructor so they can be reused across checks. This allows the xceiver client cache to remain populated between block checks. In the parallelized version, each thread will have their own set of verifier instances to use, so we won't need to worry about these classes being thread safe. ########## hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/replicas/Checksums.java: ########## @@ -44,121 +32,23 @@ * downloaded replicas. */ public class Checksums implements ReplicaVerifier { + private static final String CHECKTYPE = "checksum"; - private static final String JSON_PROPERTY_FILE_NAME = "filename"; - private static final String JSON_PROPERTY_FILE_SIZE = "datasize"; - private static final String JSON_PROPERTY_FILE_BLOCKS = "blocks"; - private static final String JSON_PROPERTY_BLOCK_INDEX = "blockIndex"; - private static final String JSON_PROPERTY_BLOCK_CONTAINERID = "containerId"; - private static final String JSON_PROPERTY_BLOCK_LOCALID = "localId"; - private static final String JSON_PROPERTY_BLOCK_LENGTH = "length"; - private static final String JSON_PROPERTY_BLOCK_OFFSET = "offset"; - private static final String JSON_PROPERTY_BLOCK_REPLICAS = "replicas"; - private static final String JSON_PROPERTY_REPLICA_HOSTNAME = "hostname"; - private static final String JSON_PROPERTY_REPLICA_UUID = "uuid"; - private static final String JSON_PROPERTY_REPLICA_EXCEPTION = "exception"; - - - private String outputDir; - private OzoneClient client; - - public Checksums(OzoneClient client, String outputDir) { - this.client = client; - this.outputDir = outputDir; - } - - private void downloadReplicasAndCreateManifest( - Map<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>> replicas, - ArrayNode blocks) throws IOException { - int blockIndex = 0; - - for (Map.Entry<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>> - block : replicas.entrySet()) { - ObjectNode blockJson = JsonUtils.createObjectNode(null); - ArrayNode replicasJson = JsonUtils.createArrayNode(); - - blockIndex += 1; - OmKeyLocationInfo locationInfo = block.getKey(); - blockJson.put(JSON_PROPERTY_BLOCK_INDEX, blockIndex); - blockJson.put(JSON_PROPERTY_BLOCK_CONTAINERID, locationInfo.getContainerID()); - blockJson.put(JSON_PROPERTY_BLOCK_LOCALID, locationInfo.getLocalID()); - blockJson.put(JSON_PROPERTY_BLOCK_LENGTH, locationInfo.getLength()); - blockJson.put(JSON_PROPERTY_BLOCK_OFFSET, locationInfo.getOffset()); - - for (Map.Entry<DatanodeDetails, OzoneInputStream> - replica : block.getValue().entrySet()) { - DatanodeDetails datanode = replica.getKey(); - - ObjectNode replicaJson = JsonUtils.createObjectNode(null); - - replicaJson.put(JSON_PROPERTY_REPLICA_HOSTNAME, datanode.getHostName()); - replicaJson.put(JSON_PROPERTY_REPLICA_UUID, datanode.getUuidString()); - - try (InputStream is = replica.getValue()) { - IOUtils.copyLarge(is, NullOutputStream.INSTANCE); - } catch (IOException e) { - replicaJson.put(JSON_PROPERTY_REPLICA_EXCEPTION, e.getMessage()); - } - replicasJson.add(replicaJson); - } - blockJson.set(JSON_PROPERTY_BLOCK_REPLICAS, replicasJson); - blocks.add(blockJson); - } - } - - @Nonnull - private File createDirectory(String volumeName, String bucketName, - String keyName) throws IOException { - String fileSuffix - = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date()); - String directoryName = volumeName + "_" + bucketName + "_" + keyName + - "_" + fileSuffix; - System.out.println("Creating directory : " + directoryName); - File dir = new File(outputDir, directoryName); - if (!dir.exists()) { - if (dir.mkdirs()) { - System.out.println("Successfully created!"); - } else { - throw new IOException(String.format( - "Failed to create directory %s.", dir)); - } - } - return dir; + public Checksums() { } @Override - public void verifyKey(OzoneKeyDetails keyDetails) { - String volumeName = keyDetails.getVolumeName(); - String bucketName = keyDetails.getBucketName(); - String keyName = keyDetails.getName(); - System.out.println("Processing key : " + volumeName + "/" + bucketName + "/" + keyName); - try { - ClientProtocol checksumClient = client.getObjectStore().getClientProxy(); - - // Multilevel keys will have a '/' in their names. This interferes with - // directory and file creation process. Flatten the keys to fix this. - String sanitizedKeyName = keyName.replace("/", "_"); + public BlockVerificationResult verifyBlock(DatanodeDetails datanode, OzoneInputStream stream, + OmKeyLocationInfo keyLocation) { + try (InputStream is = stream) { + IOUtils.copyLarge(is, NullOutputStream.INSTANCE); - File dir = createDirectory(volumeName, bucketName, sanitizedKeyName); - OzoneKeyDetails keyInfoDetails = checksumClient.getKeyDetails(volumeName, bucketName, keyName); - Map<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>> replicas = - checksumClient.getKeysEveryReplicas(volumeName, bucketName, keyName); - - ObjectNode result = JsonUtils.createObjectNode(null); - result.put(JSON_PROPERTY_FILE_NAME, volumeName + "/" + bucketName + "/" + keyName); - result.put(JSON_PROPERTY_FILE_SIZE, keyInfoDetails.getDataSize()); - - ArrayNode blocks = JsonUtils.createArrayNode(); - downloadReplicasAndCreateManifest(replicas, blocks); - result.set(JSON_PROPERTY_FILE_BLOCKS, blocks); - - String prettyJson = JsonUtils.toJsonStringWithDefaultPrettyPrinter(result); - - String manifestFileName = sanitizedKeyName + "_manifest"; - File manifestFile = new File(dir, manifestFileName); - Files.write(manifestFile.toPath(), prettyJson.getBytes(StandardCharsets.UTF_8)); + return new BlockVerificationResult(CHECKTYPE, true, Collections.emptyList()); } catch (IOException e) { Review Comment: We should filter out the checksum specific exception type and if that is thrown the check fails with `present = true`. If we get a different exception we should set that flag to `false` to indicate that we could not complete the check. ########## hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/replicas/BlockVerificationResult.java: ########## @@ -0,0 +1,91 @@ +/* + * 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 com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ArrayNode; +import com.fasterxml.jackson.databind.node.ObjectNode; +import java.util.List; + +/** + * Json structure for replicas to pass through each check and give output. + */ +public class BlockVerificationResult { + + private final String type; + private final boolean pass; + private final List<FailureDetail> failures; + + public BlockVerificationResult(String type, boolean pass, List<FailureDetail> failures) { + this.type = type; + this.pass = pass; + this.failures = failures; + } + + public String getType() { + return type; + } + + public boolean isPass() { + return pass; + } + + public List<FailureDetail> getFailures() { + return failures; + } + + /** + * Details about the check failure. + */ + public static class FailureDetail { + private final boolean present; + private final String message; + + public FailureDetail(boolean present, String message) { + this.present = present; + this.message = message; + } + + public boolean isPresent() { + return present; + } + + public String getFailureMessage() { + return message; + } + + } + + public ObjectNode toJson(ObjectMapper mapper) { Review Comment: Can we move this to `ReplicasVerify` instead? It would be better to decouple this so this class just provides information, and the CLI driver handles how it is represented. ########## hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/replicas/BlockExistenceVerifier.java: ########## @@ -0,0 +1,86 @@ +/* + * 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 java.util.Map; +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.protocol.proto.HddsProtos; +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.client.io.OzoneInputStream; +import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; + +/** + * Checks block existence using GetBlock calls to the Datanodes. + */ +public class BlockExistenceVerifier implements ReplicaVerifier { + + private OzoneConfiguration conf; + private static final String CHECKTYPE = "blockExistence"; + + public BlockExistenceVerifier(OzoneConfiguration conf) { + this.conf = conf; + } + + @Override + public BlockVerificationResult verifyBlock(DatanodeDetails datanode, OzoneInputStream stream, + OmKeyLocationInfo keyLocation) { + try (ContainerOperationClient containerClient = new ContainerOperationClient(conf); + XceiverClientManager xceiverClientManager = containerClient.getXceiverClientManager()) { + + Pipeline keyPipeline = keyLocation.getPipeline(); + boolean isECKey = keyPipeline.getReplicationConfig().getReplicationType() == HddsProtos.ReplicationType.EC; + Pipeline pipeline = isECKey ? keyPipeline : + Pipeline.newBuilder(keyPipeline) + .setReplicationConfig(StandaloneReplicationConfig.getInstance(ONE)) + .build(); + + try (XceiverClientSpi client = xceiverClientManager.acquireClientForReadData(pipeline)) { + + Map<DatanodeDetails, ContainerProtos.GetBlockResponseProto> responses = + ContainerProtocolCalls.getBlockFromAllNodes( + client, + keyLocation.getBlockID().getDatanodeBlockIDProtobuf(), + keyLocation.getToken() + ); + + ContainerProtos.GetBlockResponseProto response = responses.get(datanode); + boolean hasBlock = response != null && response.hasBlockData(); + + return new BlockVerificationResult(CHECKTYPE, hasBlock, hasBlock ? Collections.emptyList() : + Collections.singletonList(new BlockVerificationResult.FailureDetail( + true, "Block does not exist on this replica"))); + } + + } catch (IOException | InterruptedException e) { + BlockVerificationResult.FailureDetail failure = new BlockVerificationResult.FailureDetail(true, e.getMessage()); Review Comment: In this case the present/accessible/completed flag (whatever we decide to call it) should be set to false, since the check was unable to finish due to an exception. ########## hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/replicas/BlockVerificationResult.java: ########## @@ -0,0 +1,91 @@ +/* + * 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 com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ArrayNode; +import com.fasterxml.jackson.databind.node.ObjectNode; +import java.util.List; + +/** + * Json structure for replicas to pass through each check and give output. + */ +public class BlockVerificationResult { + + private final String type; + private final boolean pass; + private final List<FailureDetail> failures; + + public BlockVerificationResult(String type, boolean pass, List<FailureDetail> failures) { Review Comment: We can use two different pass/fail static constructors to handle the different parameter requirements. This way we don't need to init a list if there are no failures and it is clearer to the caller. See the container scanner's healthy/unhealthy [ScanResult](https://github.com/apache/ozone/blob/39437eae0cbc4de6078ca16f80d5c579d5a93fbf/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java#L73) for an example of this. ########## hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/replicas/ReplicasVerify.java: ########## @@ -88,41 +110,130 @@ void findCandidateKeys(OzoneClient ozoneClient, OzoneAddress address) throws IOE String volumeName = address.getVolumeName(); String bucketName = address.getBucketName(); String keyName = address.getKeyName(); + + ObjectMapper mapper = new ObjectMapper(); + ObjectNode root = mapper.createObjectNode(); + ArrayNode keysArray = mapper.createArrayNode(); + if (!keyName.isEmpty()) { OzoneKeyDetails keyDetails = ozoneClient.getProxy().getKeyDetails(volumeName, bucketName, keyName); - processKey(keyDetails); + processKey(ozoneClient, keyDetails, 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); } } + + root.set("keys", keysArray); + System.out.println(mapper.writerWithDefaultPrettyPrinter().writeValueAsString(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())); + processKey(ozoneClient, bucket.getKey(key.getName()), keysArray); } } } - void processKey(OzoneKeyDetails keyDetails) { - replicaVerifiers.forEach(verifier -> verifier.verifyKey(keyDetails)); + void processKey(OzoneClient ozoneClient, OzoneKeyDetails keyDetails, ArrayNode keysArray) { + ObjectMapper mapper = new ObjectMapper(); Review Comment: To create the initial json object, use `JsonUtils#createObjectNode` which uses the default mapper. - To add individual fields to this object, use `put` - To add an object under this or another sub-object, use the return from `putObject` - To add a list under this or another sub-object, use the return from `putArray` ########## hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/replicas/ReplicasVerify.java: ########## @@ -54,6 +63,10 @@ public class ReplicasVerify extends Handler { required = true) private String outputDir; + @CommandLine.Option(names = {"--all"}, Review Comment: We will probably eventually have options to print results for `all` keys, and run `all` possible checks, so we should distinguish them here. Maybe call this `--all-results`. I'm hesitant to call this `--all-keys` because we may add filters later for things like replication config or ctime and it would be confusing whether this option overrides those filters (it wouldn't). Side notes while I'm thinking about "all": 1. Should the command run all checks by default if no checks are specified? 2. I'm wondering if it would be better to make `checks` one flag that takes a list instead of individual flags, for example: ``` ozone debug replicas verify --checks=block-existence,checksums ozone debug replicas verify --checks=checksums ozone debug replicas verify --checks=all ``` @adoroszlai what do you think makes more sense for the CLI? ########## hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/replicas/BlockExistenceVerifier.java: ########## @@ -0,0 +1,86 @@ +/* + * 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 java.util.Map; +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.protocol.proto.HddsProtos; +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.client.io.OzoneInputStream; +import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; + +/** + * Checks block existence using GetBlock calls to the Datanodes. + */ +public class BlockExistenceVerifier implements ReplicaVerifier { + + private OzoneConfiguration conf; + private static final String CHECKTYPE = "blockExistence"; + + public BlockExistenceVerifier(OzoneConfiguration conf) { + this.conf = conf; + } + + @Override + public BlockVerificationResult verifyBlock(DatanodeDetails datanode, OzoneInputStream stream, + OmKeyLocationInfo keyLocation) { + try (ContainerOperationClient containerClient = new ContainerOperationClient(conf); + XceiverClientManager xceiverClientManager = containerClient.getXceiverClientManager()) { + + Pipeline keyPipeline = keyLocation.getPipeline(); + boolean isECKey = keyPipeline.getReplicationConfig().getReplicationType() == HddsProtos.ReplicationType.EC; + Pipeline pipeline = isECKey ? keyPipeline : + Pipeline.newBuilder(keyPipeline) + .setReplicationConfig(StandaloneReplicationConfig.getInstance(ONE)) + .build(); + + try (XceiverClientSpi client = xceiverClientManager.acquireClientForReadData(pipeline)) { + + Map<DatanodeDetails, ContainerProtos.GetBlockResponseProto> responses = + ContainerProtocolCalls.getBlockFromAllNodes( Review Comment: We should only make this call on the specified datanode, instead of making the call on all datanodes and dropping the results we don't need later. ########## hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/replicas/ReplicasVerify.java: ########## @@ -88,41 +110,130 @@ void findCandidateKeys(OzoneClient ozoneClient, OzoneAddress address) throws IOE String volumeName = address.getVolumeName(); String bucketName = address.getBucketName(); String keyName = address.getKeyName(); + + ObjectMapper mapper = new ObjectMapper(); + ObjectNode root = mapper.createObjectNode(); + ArrayNode keysArray = mapper.createArrayNode(); + if (!keyName.isEmpty()) { OzoneKeyDetails keyDetails = ozoneClient.getProxy().getKeyDetails(volumeName, bucketName, keyName); - processKey(keyDetails); + processKey(ozoneClient, keyDetails, 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); } } + + root.set("keys", keysArray); + System.out.println(mapper.writerWithDefaultPrettyPrinter().writeValueAsString(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())); + processKey(ozoneClient, bucket.getKey(key.getName()), keysArray); } } } - void processKey(OzoneKeyDetails keyDetails) { - replicaVerifiers.forEach(verifier -> verifier.verifyKey(keyDetails)); + void processKey(OzoneClient ozoneClient, OzoneKeyDetails keyDetails, ArrayNode keysArray) { + ObjectMapper mapper = new ObjectMapper(); + + String volumeName = keyDetails.getVolumeName(); + String bucketName = keyDetails.getBucketName(); + String keyName = keyDetails.getName(); + + ObjectNode keyNode = JsonUtils.createObjectNode(null); + keyNode.put("volumeName", volumeName); + keyNode.put("bucketName", bucketName); + keyNode.put("name", keyName); + + ArrayNode blocksArray = mapper.createArrayNode(); + boolean keyPass = true; //to check if all checks are passed + + try { + ClientProtocol clientProtocol = ozoneClient.getObjectStore().getClientProxy(); + Map<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>> replicas = + clientProtocol.getKeysEveryReplicas(volumeName, bucketName, keyName); Review Comment: I think we should make [RpcClient#getKeyInfo](https://github.com/apache/ozone/blob/a0c07c5a0aae76b2249370cdd6a305e690417210/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java#L1853) public and that should be the only call we make to the OM for each key we get from the listing. The ability to specify whether or not to use the container cache will be helpful for HDDS-12208. This could be used to identify if there is a stale cache bug that is causing replicas to not be returned to the client. ########## hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/replicas/BlockVerificationResult.java: ########## @@ -0,0 +1,91 @@ +/* + * 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 com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ArrayNode; +import com.fasterxml.jackson.databind.node.ObjectNode; +import java.util.List; + +/** + * Json structure for replicas to pass through each check and give output. + */ +public class BlockVerificationResult { + + private final String type; + private final boolean pass; + private final List<FailureDetail> failures; + + public BlockVerificationResult(String type, boolean pass, List<FailureDetail> failures) { + this.type = type; + this.pass = pass; + this.failures = failures; + } + + public String getType() { + return type; + } + + public boolean isPass() { + return pass; + } + + public List<FailureDetail> getFailures() { + return failures; + } + + /** + * Details about the check failure. + */ + public static class FailureDetail { + private final boolean present; + private final String message; + + public FailureDetail(boolean present, String message) { + this.present = present; + this.message = message; + } + + public boolean isPresent() { + return present; + } + + public String getFailureMessage() { + return message; + } + + } + + public ObjectNode toJson(ObjectMapper mapper) { Review Comment: With this change we could add a `getType` method to the `ReplicaVerifier` interface instead of putting it here, and the driver can assemble the information into one json object. Then we don't need each verifier to reuse the `CHECK_TYPE` constant every time they create a verification result. -- 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