rakeshadr commented on a change in pull request #2919:
URL: https://github.com/apache/ozone/pull/2919#discussion_r770435999



##########
File path: 
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/BaseFileChecksumHelper.java
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.client;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.fs.FileChecksum;
+import org.apache.hadoop.fs.MD5MD5CRC32GzipFileChecksum;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientSpi;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.MD5Hash;
+import org.apache.hadoop.ozone.client.rpc.RpcClient;
+import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * The base class to support file checksum.
+ */
+public abstract class BaseFileChecksumHelper {
+  static final Logger LOG =
+      LoggerFactory.getLogger(BaseFileChecksumHelper.class);
+
+  private OzoneVolume volume;
+  private OzoneBucket bucket;
+  private String keyName;
+
+  private final long length;
+  protected RpcClient rpcClient;
+  protected XceiverClientFactory xceiverClientFactory;
+
+  //private final Options.ChecksumCombineMode combineMode;
+  //private final BlockChecksumType blockChecksumType;

Review comment:
       @jojochuang  I hope we would be following the similar approach from HDFS 
to handle the EC files as well. Could you please elaborate the overall plans. 
Thanks!
   
   Also, could you please remove the `commented code` everywhere, to make the 
patch cleaner.

##########
File path: 
hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/TestReplicatedFileChecksumHelper.java
##########
@@ -0,0 +1,230 @@
+/*
+ * 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.client;
+
+import org.apache.hadoop.fs.FileChecksum;
+import org.apache.hadoop.fs.MD5MD5CRC32GzipFileChecksum;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+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.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientGrpc;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.ozone.client.rpc.RpcClient;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.hadoop.util.DataChecksum;
+import org.apache.hadoop.util.Time;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+
+import static 
org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChecksumType.CRC32;
+import static org.junit.Assert.*;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.when;
+
+public class TestReplicatedFileChecksumHelper {
+
+  @Before

Review comment:
       Please remove setup and teardown as its empty method.

##########
File path: 
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/BaseFileChecksumHelper.java
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.client;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.fs.FileChecksum;
+import org.apache.hadoop.fs.MD5MD5CRC32GzipFileChecksum;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientSpi;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.MD5Hash;
+import org.apache.hadoop.ozone.client.rpc.RpcClient;
+import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * The base class to support file checksum.
+ */
+public abstract class BaseFileChecksumHelper {
+  static final Logger LOG =
+      LoggerFactory.getLogger(BaseFileChecksumHelper.class);
+
+  private OzoneVolume volume;
+  private OzoneBucket bucket;
+  private String keyName;
+
+  private final long length;

Review comment:
       I couldn't see length is used anywhere. Also, we need to adopt the idea 
of `remaining` from HDFS, right?
   
   remaining logic in hdfs 
[reference-1](https://github.com/apache/hadoop/blob/trunk/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/FileChecksumHelper.java#L86)
 , 
[reference-2](https://github.com/apache/hadoop/blob/trunk/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/FileChecksumHelper.java#L518)
 

##########
File path: 
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/BaseFileChecksumHelper.java
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.client;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.fs.FileChecksum;
+import org.apache.hadoop.fs.MD5MD5CRC32GzipFileChecksum;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientSpi;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.MD5Hash;
+import org.apache.hadoop.ozone.client.rpc.RpcClient;
+import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * The base class to support file checksum.
+ */
+public abstract class BaseFileChecksumHelper {
+  static final Logger LOG =
+      LoggerFactory.getLogger(BaseFileChecksumHelper.class);
+
+  private OzoneVolume volume;
+  private OzoneBucket bucket;
+  private String keyName;
+
+  private final long length;
+  protected RpcClient rpcClient;
+  protected XceiverClientFactory xceiverClientFactory;
+
+  //private final Options.ChecksumCombineMode combineMode;
+  //private final BlockChecksumType blockChecksumType;
+  protected final DataOutputBuffer blockChecksumBuf = new DataOutputBuffer();
+
+  private FileChecksum fileChecksum;
+  protected List<OmKeyLocationInfo> keyLocationInfos;
+
+  private int bytesPerCRC = -1;
+  //private DataChecksum.Type crcType = DataChecksum.Type.DEFAULT;
+  private long crcPerBlock = 0;
+
+  // initialization
+  BaseFileChecksumHelper(
+      OzoneVolume volume, OzoneBucket bucket, String keyName,
+      long length,
+      //Options.ChecksumCombineMode checksumCombineMode,
+      RpcClient rpcClient,
+      XceiverClientSpi xceiverClientGrpc) throws IOException {
+
+    this.volume = volume;
+    this.bucket = bucket;
+    this.keyName = keyName;
+    this.length = length;
+    this.rpcClient = rpcClient;
+    this.xceiverClientFactory = rpcClient.getXeiverClientManager();
+    refetchBlocks();
+  }
+
+  int getBytesPerCRC() {
+    return bytesPerCRC;
+  }
+
+  void setBytesPerCRC(int bytesPerCRC) {
+    this.bytesPerCRC = bytesPerCRC;
+  }
+
+  /**
+   * Request the blocks created in the most recent version from Ozone Manager.
+   *
+   * @throws IOException
+   */
+  void refetchBlocks() throws IOException {
+    OzoneManagerProtocol ozoneManagerClient = 
rpcClient.getOzoneManagerClient();
+    OmKeyArgs keyArgs = new OmKeyArgs.Builder()
+        .setVolumeName(volume.getName())
+        .setBucketName(bucket.getName())
+        .setKeyName(keyName)
+        .setRefreshPipeline(true)
+        .setSortDatanodesInPipeline(true)
+        .setLatestVersionLocation(true)
+        .build();
+    OmKeyInfo keyInfo = ozoneManagerClient.lookupKey(keyArgs);
+
+    // use OmKeyArgs to call Om.lookup() and get OmKeyInfo
+    keyLocationInfos = keyInfo
+        .getLatestVersionLocations().getBlocksLatestVersionOnly();
+  }
+
+  /**
+   * Compute file checksum given the list of chunk checksums requested earlier.
+   * @throws IOException
+   */
+  public void compute() throws IOException {
+    /**
+     * request length is 0 or the file is empty, return one with the
+     * magic entry that matches what previous hdfs versions return.
+     */
+    if (keyLocationInfos == null || keyLocationInfos.isEmpty()) {
+      // Explicitly specified here in case the default DataOutputBuffer
+      // buffer length value is changed in future. This matters because the
+      // fixed value 32 has to be used to repeat the magic value for previous
+      // HDFS version.
+      final int lenOfZeroBytes = 32;
+      byte[] emptyBlockMd5 = new byte[lenOfZeroBytes];
+      MD5Hash fileMD5 = MD5Hash.digest(emptyBlockMd5);
+      fileChecksum =  new MD5MD5CRC32GzipFileChecksum(0, 0, fileMD5);
+    } else {
+      checksumBlocks();
+      fileChecksum = makeFinalResult();
+    }
+  }
+
+  @VisibleForTesting
+  List<OmKeyLocationInfo> getKeyLocationInfos() {
+    return keyLocationInfos;
+  }
+
+
+  /**
+   * Compute block checksums block by block and append the raw bytes of the
+   * block checksums into getBlockChecksumBuf().
+   *
+   * @throws IOException
+   */
+  abstract void checksumBlocks() throws IOException;
+
+  /**
+   * Make final file checksum result given the per-block or per-block-group
+   * checksums collected into getBlockChecksumBuf().
+   */
+  FileChecksum makeFinalResult() throws IOException {
+    // TODO: support composite CRC
+    return makeMd5CrcResult();
+  }
+
+  FileChecksum makeMd5CrcResult() {

Review comment:
       Please make visibility to `private`

##########
File path: 
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/BaseFileChecksumHelper.java
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.client;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.fs.FileChecksum;
+import org.apache.hadoop.fs.MD5MD5CRC32GzipFileChecksum;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientSpi;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.MD5Hash;
+import org.apache.hadoop.ozone.client.rpc.RpcClient;
+import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * The base class to support file checksum.
+ */
+public abstract class BaseFileChecksumHelper {
+  static final Logger LOG =
+      LoggerFactory.getLogger(BaseFileChecksumHelper.class);
+
+  private OzoneVolume volume;
+  private OzoneBucket bucket;
+  private String keyName;
+
+  private final long length;
+  protected RpcClient rpcClient;
+  protected XceiverClientFactory xceiverClientFactory;
+
+  //private final Options.ChecksumCombineMode combineMode;
+  //private final BlockChecksumType blockChecksumType;
+  protected final DataOutputBuffer blockChecksumBuf = new DataOutputBuffer();
+
+  private FileChecksum fileChecksum;
+  protected List<OmKeyLocationInfo> keyLocationInfos;
+
+  private int bytesPerCRC = -1;
+  //private DataChecksum.Type crcType = DataChecksum.Type.DEFAULT;
+  private long crcPerBlock = 0;
+
+  // initialization
+  BaseFileChecksumHelper(
+      OzoneVolume volume, OzoneBucket bucket, String keyName,
+      long length,
+      //Options.ChecksumCombineMode checksumCombineMode,
+      RpcClient rpcClient,
+      XceiverClientSpi xceiverClientGrpc) throws IOException {

Review comment:
       please remove `XceiverClientSpi xceiverClientGrpc`, its nowhere used.

##########
File path: 
hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/TestReplicatedFileChecksumHelper.java
##########
@@ -0,0 +1,230 @@
+/*
+ * 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.client;
+
+import org.apache.hadoop.fs.FileChecksum;
+import org.apache.hadoop.fs.MD5MD5CRC32GzipFileChecksum;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+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.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientGrpc;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.ozone.client.rpc.RpcClient;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.hadoop.util.DataChecksum;
+import org.apache.hadoop.util.Time;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+
+import static 
org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChecksumType.CRC32;
+import static org.junit.Assert.*;

Review comment:
       Can you please replace `import static org.junit.Assert.*;`
   
   with specific imports.
   ```
   import static org.junit.Assert.assertEquals;
   import static org.junit.Assert.assertTrue;
   ```

##########
File path: 
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/BaseFileChecksumHelper.java
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.client;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.fs.FileChecksum;
+import org.apache.hadoop.fs.MD5MD5CRC32GzipFileChecksum;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientSpi;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.MD5Hash;
+import org.apache.hadoop.ozone.client.rpc.RpcClient;
+import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * The base class to support file checksum.
+ */
+public abstract class BaseFileChecksumHelper {
+  static final Logger LOG =
+      LoggerFactory.getLogger(BaseFileChecksumHelper.class);
+
+  private OzoneVolume volume;
+  private OzoneBucket bucket;
+  private String keyName;
+
+  private final long length;
+  protected RpcClient rpcClient;
+  protected XceiverClientFactory xceiverClientFactory;
+
+  //private final Options.ChecksumCombineMode combineMode;
+  //private final BlockChecksumType blockChecksumType;
+  protected final DataOutputBuffer blockChecksumBuf = new DataOutputBuffer();
+
+  private FileChecksum fileChecksum;
+  protected List<OmKeyLocationInfo> keyLocationInfos;
+
+  private int bytesPerCRC = -1;
+  //private DataChecksum.Type crcType = DataChecksum.Type.DEFAULT;
+  private long crcPerBlock = 0;
+
+  // initialization
+  BaseFileChecksumHelper(
+      OzoneVolume volume, OzoneBucket bucket, String keyName,
+      long length,
+      //Options.ChecksumCombineMode checksumCombineMode,
+      RpcClient rpcClient,
+      XceiverClientSpi xceiverClientGrpc) throws IOException {
+
+    this.volume = volume;
+    this.bucket = bucket;
+    this.keyName = keyName;
+    this.length = length;
+    this.rpcClient = rpcClient;
+    this.xceiverClientFactory = rpcClient.getXeiverClientManager();
+    refetchBlocks();
+  }
+
+  int getBytesPerCRC() {

Review comment:
       Please remove unused `getBytesPerCRC` method.

##########
File path: 
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/ReplicatedFileChecksumHelper.java
##########
@@ -0,0 +1,142 @@
+/*
+ * 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.client;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.XceiverClientSpi;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls;
+import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
+import org.apache.hadoop.io.MD5Hash;
+import org.apache.hadoop.ozone.client.rpc.RpcClient;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.security.token.Token;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+
+import java.io.IOException;
+import java.util.List;
+
+public class ReplicatedFileChecksumHelper extends BaseFileChecksumHelper {
+  ReplicatedFileChecksumHelper(
+      OzoneVolume volume, OzoneBucket bucket, String keyName, long length,
+      /*Options.ChecksumCombineMode checksumCombineMode, */RpcClient rpcClient,
+      XceiverClientSpi xceiverClientSpi) throws IOException {
+    super(volume, bucket, keyName, length, /*checksumCombineMode, */rpcClient, 
xceiverClientSpi);
+  }
+
+  @Override
+  void checksumBlocks() throws IOException {
+    for (OmKeyLocationInfo keyLocationInfo : keyLocationInfos) {
+      // for each block, send request
+      // TODO: retry multiple times with different replicas
+
+      List<ContainerProtos.ChunkInfo> chunkInfos =
+          getChunkInfos(keyLocationInfo);
+      ContainerProtos.ChecksumData checksumData =
+          chunkInfos.get(0).getChecksumData();
+      int bytesPerChecksum = checksumData.getBytesPerChecksum();
+      setBytesPerCRC(bytesPerChecksum);
+      List<ByteString> checksums = checksumData.getChecksumsList();
+
+      byte[] blockChecksum = getBlockChecksumFromChunkChecksums(checksums);
+      populateBlockChecksumBuf(blockChecksum);

Review comment:
       Can you please add log message similar to HDFS after invoking 
`populateBlockChecksumBuf()`
   
    ```
          String blockChecksumForDebug = populateBlockChecksumBuf(checksumData);
           LOG.debug("got reply from {}: blockChecksum={}, 
blockChecksumType={}",
               datanode, blockChecksumForDebug, getBlockChecksumType());
   ```




-- 
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]

Reply via email to