adoroszlai commented on code in PR #7748: URL: https://github.com/apache/ozone/pull/7748#discussion_r1963827922
########## hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/replicas/ReplicasUtils.java: ########## @@ -0,0 +1,124 @@ +/* + * 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.io.IOException; +import java.util.Iterator; +import java.util.function.BiConsumer; +import org.apache.hadoop.ozone.client.ObjectStore; +import org.apache.hadoop.ozone.client.OzoneBucket; +import org.apache.hadoop.ozone.client.OzoneClient; +import org.apache.hadoop.ozone.client.OzoneKey; +import org.apache.hadoop.ozone.client.OzoneVolume; +import org.apache.hadoop.ozone.shell.OzoneAddress; + +/** + * Utils class for the replicas package. + */ + +public final class ReplicasUtils { + private ReplicasUtils() { + } + + static class KeyParts { + private String volumeName; + private String bucketName; + private String keyName; + + KeyParts(String volumeName, String bucketName, String keyName) { + this.volumeName = volumeName; + this.bucketName = bucketName; + this.keyName = keyName; + } + + public String getBucketName() { + return bucketName; + } + + public String getVolumeName() { + return volumeName; + } + + public void setVolumeName(String volumeName) { + this.volumeName = volumeName; + } + + public void setBucketName(String bucketName) { + this.bucketName = bucketName; + } Review Comment: nit: Are mutators needed? If not, we should prefer immutable objects. ########## hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/replicas/Checksums.java: ########## @@ -81,70 +82,22 @@ public class ReadReplicas extends KeyHandler implements DebugSubcommand { private static final String JSON_PROPERTY_REPLICA_UUID = "uuid"; private static final String JSON_PROPERTY_REPLICA_EXCEPTION = "exception"; - @Override - protected void execute(OzoneClient client, OzoneAddress address) - throws IOException { - - address.ensureKeyAddress(); - - boolean isChecksumVerifyEnabled - = getConf().getBoolean("ozone.client.verify.checksum", true); - OzoneConfiguration configuration = new OzoneConfiguration(getConf()); - configuration.setBoolean("ozone.client.verify.checksum", - !isChecksumVerifyEnabled); - - RpcClient newClient = new RpcClient(configuration, null); - try { - ClientProtocol noChecksumClient; - ClientProtocol checksumClient; - if (isChecksumVerifyEnabled) { - checksumClient = client.getObjectStore().getClientProxy(); - noChecksumClient = newClient; - } else { - checksumClient = newClient; - noChecksumClient = client.getObjectStore().getClientProxy(); - } - - String volumeName = address.getVolumeName(); - String bucketName = address.getBucketName(); - String keyName = address.getKeyName(); - // Multilevel keys will have a '/' in their names. This interferes with - // directory and file creation process. Flatten the keys to fix this. - String sanitizedKeyName = address.getKeyName().replace("/", "_"); - - File dir = createDirectory(volumeName, bucketName, sanitizedKeyName); + @CommandLine.Parameters(arity = "1", + description = Shell.OZONE_URI_DESCRIPTION) + private String uri; - OzoneKeyDetails keyInfoDetails - = checksumClient.getKeyDetails(volumeName, bucketName, keyName); + private RpcClient rpcClient = null; - Map<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>> replicas = - checksumClient.getKeysEveryReplicas(volumeName, bucketName, keyName); - - Map<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>> - replicasWithoutChecksum = noChecksumClient - .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(sanitizedKeyName, replicas, - replicasWithoutChecksum, dir, blocks); - result.set(JSON_PROPERTY_FILE_BLOCKS, blocks); + @Override + protected OzoneAddress getAddress() throws OzoneClientException { + return new OzoneAddress(uri); + } - String prettyJson = JsonUtils.toJsonStringWithDefaultPrettyPrinter(result); + @Override + protected void execute(OzoneClient client, OzoneAddress address) + throws IOException { - String manifestFileName = sanitizedKeyName + "_manifest"; - System.out.println("Writing manifest file : " + manifestFileName); - File manifestFile - = new File(dir, manifestFileName); - Files.write(manifestFile.toPath(), - prettyJson.getBytes(StandardCharsets.UTF_8)); - } finally { - newClient.close(); - } + ReplicasUtils.findCandidateKeys(client, address, processKeyConsumer); Review Comment: `Checksums` uses two clients, `FindMissingPadding` needs only one. The callback accepts only one client, so `Checksums` creates its own other client there. It tries to avoid creating many clients by keeping reference in `rpcClient` member variable. However, the client is closed after processing one item (see `finally` block). So `rpcClient` references a "closed" client after processing the first item. I think: - additional client should be created and closed in `Checksums.execute`, not in the callback - the callback should: - have one parameter: `KeyParts` - be defined as lambda locally in `execute`, having access to one or both clients, depending on the class High-level suggestion for `Checksums.execute`: ```java void execute(OzoneClient client, OzoneAddress address) { conf = ... try (RpcClient otherClient = createOtherClient(conf)) { ReplicasUtils.findCandidateKeys(client, address, key -> { ... }); } } ``` -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
