saxenapranav commented on code in PR #6069:
URL: https://github.com/apache/hadoop/pull/6069#discussion_r1330030917


##########
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemChecksum.java:
##########
@@ -0,0 +1,130 @@
+/**
+ * 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.fs.azurebfs;
+
+import java.security.SecureRandom;
+import java.util.Arrays;
+import java.util.HashSet;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.impl.OpenFileParameters;
+
+import static 
org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_BUFFERED_PREAD_DISABLE;
+import static 
org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+
+/**
+ * Test For Verifying Checksum Related Operations
+ */
+public class ITestAzureBlobFileSystemChecksum extends 
AbstractAbfsIntegrationTest {

Review Comment:
   Some tests around
   ```
     private int optimisedRead(final byte[] b, final int off, final int len,
         final long readFrom, final long actualLen) throws IOException {
   ```
   since it can fire with offset!=0
   
   ITests which directly call client.append() and client.read() with different 
offsets would be awesome.



##########
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemChecksum.java:
##########
@@ -0,0 +1,130 @@
+/**
+ * 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.fs.azurebfs;
+
+import java.security.SecureRandom;
+import java.util.Arrays;
+import java.util.HashSet;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.impl.OpenFileParameters;
+
+import static 
org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_BUFFERED_PREAD_DISABLE;
+import static 
org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+
+/**
+ * Test For Verifying Checksum Related Operations
+ */
+public class ITestAzureBlobFileSystemChecksum extends 
AbstractAbfsIntegrationTest {
+
+  public ITestAzureBlobFileSystemChecksum() throws Exception {
+    super();
+  }
+
+  @Test
+  public void testWriteReadWithChecksum() throws Exception {
+    testWriteReadWithChecksumInternal(true);
+    testWriteReadWithChecksumInternal(false);
+  }
+
+  private void testWriteReadWithChecksumInternal(final boolean 
readAheadEnabled)
+      throws Exception {
+    AzureBlobFileSystem fs = getFileSystem();
+    AbfsConfiguration conf = fs.getAbfsStore().getAbfsConfiguration();

Review Comment:
   This conf would be shared across thread in multiple -thread run. Better we 
take clone of the conf and create a new FS for the test, so other tests are not 
interfered.



##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java:
##########
@@ -119,6 +119,7 @@ public final class FileSystemConfigurations {
   public static final int STREAM_ID_LEN = 12;
   public static final boolean DEFAULT_ENABLE_ABFS_LIST_ITERATOR = true;
   public static final boolean DEFAULT_ENABLE_ABFS_RENAME_RESILIENCE = true;
+  public static final boolean DEFAULT_ENABLE_ABFS_CHECKSUM_VALIDATION = false;

Review Comment:
   Lets have AbstractAbfsIntegrationTest enforce this property as true in the 
integeration tests we run. So, in future, if someone tries to change append 
flow , read flow. Checksum checks would be enforced and would highlight if 
something goes bad with the new change.



##########
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemChecksum.java:
##########
@@ -0,0 +1,130 @@
+/**
+ * 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.fs.azurebfs;
+
+import java.security.SecureRandom;
+import java.util.Arrays;
+import java.util.HashSet;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.impl.OpenFileParameters;
+
+import static 
org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_BUFFERED_PREAD_DISABLE;
+import static 
org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+
+/**
+ * Test For Verifying Checksum Related Operations
+ */
+public class ITestAzureBlobFileSystemChecksum extends 
AbstractAbfsIntegrationTest {
+
+  public ITestAzureBlobFileSystemChecksum() throws Exception {
+    super();
+  }
+
+  @Test
+  public void testWriteReadWithChecksum() throws Exception {
+    testWriteReadWithChecksumInternal(true);
+    testWriteReadWithChecksumInternal(false);
+  }
+
+  private void testWriteReadWithChecksumInternal(final boolean 
readAheadEnabled)
+      throws Exception {
+    AzureBlobFileSystem fs = getFileSystem();
+    AbfsConfiguration conf = fs.getAbfsStore().getAbfsConfiguration();

Review Comment:
   Also, lets keep path different.



##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java:
##########
@@ -1412,6 +1428,81 @@ private void appendIfNotEmpty(StringBuilder sb, String 
regEx,
     }
   }
 
+  /**
+   * Add MD5 hash as checksum request header to the append request
+   * @param requestHeaders
+   * @param reqParams
+   * @param buffer
+   */
+  private void addCheckSumHeaderForWrite(List<AbfsHttpHeader> requestHeaders,
+      final AppendRequestParameters reqParams, final byte[] buffer) {
+    try {
+      MessageDigest md5Digest = MessageDigest.getInstance("MD5");
+      byte[] md5Bytes = md5Digest.digest(
+          Arrays.copyOfRange(buffer, reqParams.getoffset(), 
reqParams.getLength()));
+      String md5Hash = Base64.getEncoder().encodeToString(md5Bytes);
+      requestHeaders.add(new AbfsHttpHeader(CONTENT_MD5, md5Hash));
+    } catch (NoSuchAlgorithmException e) {
+      e.printStackTrace();
+    }
+  }
+
+  /**
+   * T verify the checksum information received from server for the data read
+   * @param buffer stores the data received from server
+   * @param result HTTP Operation Result
+   * @param bufferOffset Position where data returned by server is saved in 
buffer
+   * @throws AbfsRestOperationException
+   */
+  private void verifyCheckSumForRead(final byte[] buffer, final 
AbfsHttpOperation result, final int bufferOffset)
+      throws AbfsRestOperationException{
+    // Number of bytes returned by server could be less than or equal to what
+    // caller requests. In case it is less, extra bytes will be initialized to 0
+    // Server returned MD5 Hash will be computed on what server returned.
+    // We need to get exact data that server returned and compute its md5 hash
+    // Computed hash should be equal to what server returned
+    int numberOfBytesRead = (int)result.getBytesReceived();
+    if (numberOfBytesRead == 0) {
+      return;
+    }
+    byte[] dataRead = new byte[numberOfBytesRead];
+    System.arraycopy(buffer, bufferOffset, dataRead, 0, numberOfBytesRead);
+
+    try {
+      MessageDigest md5Digest = MessageDigest.getInstance(MD5);
+      byte[] md5Bytes = md5Digest.digest(dataRead);
+      String md5HashComputed = Base64.getEncoder().encodeToString(md5Bytes);
+      String md5HashActual = result.getResponseHeader(CONTENT_MD5);
+      if (!md5HashComputed.equals(md5HashActual)) {
+        throw new InvalidChecksumException(null);
+      }
+    } catch (NoSuchAlgorithmException e) {
+      throw new InvalidChecksumException(e);

Review Comment:
   Having said above, here no need to handle much. Reason being, it is more of 
a non-possible exception, since our algo used is constant. If it breaks, it 
breaks for all. Cant break from some cases.



##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java:
##########
@@ -1412,6 +1428,81 @@ private void appendIfNotEmpty(StringBuilder sb, String 
regEx,
     }
   }
 
+  /**
+   * Add MD5 hash as checksum request header to the append request
+   * @param requestHeaders
+   * @param reqParams
+   * @param buffer
+   */
+  private void addCheckSumHeaderForWrite(List<AbfsHttpHeader> requestHeaders,
+      final AppendRequestParameters reqParams, final byte[] buffer) {
+    try {
+      MessageDigest md5Digest = MessageDigest.getInstance("MD5");
+      byte[] md5Bytes = md5Digest.digest(
+          Arrays.copyOfRange(buffer, reqParams.getoffset(), 
reqParams.getLength()));
+      String md5Hash = Base64.getEncoder().encodeToString(md5Bytes);
+      requestHeaders.add(new AbfsHttpHeader(CONTENT_MD5, md5Hash));
+    } catch (NoSuchAlgorithmException e) {
+      e.printStackTrace();
+    }
+  }
+
+  /**
+   * T verify the checksum information received from server for the data read
+   * @param buffer stores the data received from server
+   * @param result HTTP Operation Result
+   * @param bufferOffset Position where data returned by server is saved in 
buffer
+   * @throws AbfsRestOperationException
+   */
+  private void verifyCheckSumForRead(final byte[] buffer, final 
AbfsHttpOperation result, final int bufferOffset)
+      throws AbfsRestOperationException{
+    // Number of bytes returned by server could be less than or equal to what
+    // caller requests. In case it is less, extra bytes will be initialized to 0
+    // Server returned MD5 Hash will be computed on what server returned.
+    // We need to get exact data that server returned and compute its md5 hash
+    // Computed hash should be equal to what server returned
+    int numberOfBytesRead = (int)result.getBytesReceived();
+    if (numberOfBytesRead == 0) {
+      return;
+    }
+    byte[] dataRead = new byte[numberOfBytesRead];
+    System.arraycopy(buffer, bufferOffset, dataRead, 0, numberOfBytesRead);
+
+    try {
+      MessageDigest md5Digest = MessageDigest.getInstance(MD5);
+      byte[] md5Bytes = md5Digest.digest(dataRead);
+      String md5HashComputed = Base64.getEncoder().encodeToString(md5Bytes);
+      String md5HashActual = result.getResponseHeader(CONTENT_MD5);
+      if (!md5HashComputed.equals(md5HashActual)) {
+        throw new InvalidChecksumException(null);

Review Comment:
   Instead of null, I feel we should flow in PathIOException(pathStr). That 
would be more inline to @anmolanmol1234 's prior comment.



-- 
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: common-issues-unsubscr...@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org

Reply via email to