errose28 commented on code in PR #8248: URL: https://github.com/apache/ozone/pull/8248#discussion_r2049743412
########## 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: Steps to implement the checksum verifier with only `OmKeyLocationInfo` and `DatanodeDetails`: 1. Call `BlockInputStreamFactory#create` with - `ReplicationConfig` set to 1, since we are just reading one block from one datanode. - The `OmKeyLocationInfo` object, since this extends `BlockLocationInfo`. - A single node `Pipeline` object built from the `DatanodeDetails`. - `Token` extracted from `OmKeyLocationInfo` - The `XceiverClientManager` obtained from the `RpcClient` instance that this verifier should be maintaining to do its checks. - `null` refresh function - The [refresh function](https://github.com/apache/ozone/blob/a0c07c5a0aae76b2249370cdd6a305e690417210/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java#L2230) is used to fetch updated location information if a block is not present, which could be caused by a stale OM location cache. - We will need to handle this in the `ReplicaVerifier` driver, where if any of the block checks for a key come back as not present, it retries `getKeyInfo` and all the verifications for that key with a container cache refresh. - We may provide more control over this in HDDS-12206. - We can handle cache misses in a follow-up change, no need to implement this part here. - `conf.getObject(OzoneClientConfig.class)` where `conf` is the configuration instance shared for the whole command. 2. Exhaust the stream using `IOUtils#copyLarge` just like we are already doing. Any checksum exceptions should be thrown from there. -- 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