You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-issues@hadoop.apache.org by GitBox <gi...@apache.org> on 2020/03/06 07:54:36 UTC

[GitHub] [hadoop] mehakmeet opened a new pull request #1881: HADOOP-16910 Adding file system counters in ABFS

mehakmeet opened a new pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881
 
 
   - Write_ops
   - Read_ops
   - Bytes_written (already updated)
   - Bytes_Read (already updated)
   
   Change-Id: I77349fdd158babd66df665713201fa9c8606f191
   
   ## NOTICE
   
   Please create an issue in ASF JIRA before opening a pull request,
   and you need to set the title of the pull request which starts with
   the corresponding JIRA issue number. (e.g. HADOOP-XXXXX. Fix a typo in YYY.)
   For more details, please see https://cwiki.apache.org/confluence/display/HADOOP/How+To+Contribute
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r388858679
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java
 ##########
 @@ -143,7 +143,7 @@ public void testBlobDataReader() throws Exception {
 
     // TEST WRITE FILE
     try {
-      abfsStore.openFileForWrite(EXISTED_FILE_PATH, true);
+      abfsStore.openFileForWrite(EXISTED_FILE_PATH, fs.getFsStatistics(),true);
 
 Review comment:
   nit, add a space after the comma

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] aajisaka commented on issue #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
aajisaka commented on issue #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#issuecomment-603068969
 
 
   Hi @mehakmeet and @steveloughran 
   Failed to compile TestAbfsOutputStream.java after the merge. Would the fix it?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mehakmeet removed a comment on issue #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet removed a comment on issue #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#issuecomment-595684591
 
 
   test run : mvn -T 1C -Dparallel-tests=abfs clean verify
   Region : East US, West US
   
   Failed tests(Container configurations based) :
   `[ERROR] Failures: 
   [ERROR]   ITestGetNameSpaceEnabled.testNonXNSAccount:59->Assert.assertFalse:64->Assert.assertTrue:41->Assert.fail:88 Expecting getIsNamespaceEnabled() return false \n
   [ERROR] Errors: 
   [ERROR]   ITestGetNameSpaceEnabled.testFailedRequestWhenCredentialsNotCorrect:91->AbstractAbfsIntegrationTest.getFileSystem:197 ? InvalidConfigurationValue`

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mukund-thakur commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r396230414
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java
 ##########
 @@ -146,6 +146,8 @@ public void testBlobDataReader() throws Exception {
       abfsStore.openFileForWrite(EXISTED_FILE_PATH, fs.getFsStatistics(), true);
     } catch (AbfsRestOperationException e) {
       assertEquals(AzureServiceErrorCode.AUTHORIZATION_PERMISSION_MISS_MATCH, e.getErrorCode());
+    } finally {
+      abfsStore.close();
 
 Review comment:
   Use IOUtils.cleanupWithLogger() here as well. Or you can just say this code is old.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mehakmeet removed a comment on issue #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet removed a comment on issue #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#issuecomment-595674353
 
 
   test run : mvn -T 1C -Dparallel-tests=abfs clean verify
   Region : East US, West US
   
   Failed tests(Container configurations based) :
   `[ERROR] Failures: [ERROR] ITestGetNameSpaceEnabled.testNonXNSAccount:59->Assert.assertFalse:64->Assert.assertTrue:41->Assert.fail:88 Expecting getIsNamespaceEnabled() return false [ERROR] Errors: [ERROR] ITestAbfsIdentityTransformer.<init>:66 ? NullPointer [ERROR] ITestGetNameSpaceEnabled.testFailedRequestWhenCredentialsNotCorrect:91->AbstractAbfsIntegrationTest.getFileSystem:197 ? InvalidConfigurationValue`

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mehakmeet edited a comment on issue #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet edited a comment on issue #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#issuecomment-603101587
 
 
   @aajisaka 
   Yes, I'll be fixing it.
   https://issues.apache.org/jira/browse/HADOOP-16934

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mehakmeet commented on issue #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on issue #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#issuecomment-595672664
 
 
   test run : mvn -T 1C -Dparallel-tests=abfs clean verify
   
   Failed tests(Container configurations based) :
   `[ERROR] Failures: 
   [ERROR]   ITestGetNameSpaceEnabled.testNonXNSAccount:59->Assert.assertFalse:64->Assert.assertTrue:41->Assert.fail:88 Expecting getIsNamespaceEnabled() return false
   [ERROR] Errors: 
   [ERROR]   ITestAbfsIdentityTransformer.<init>:66 ? NullPointer
   [ERROR]   ITestGetNameSpaceEnabled.testFailedRequestWhenCredentialsNotCorrect:91->AbstractAbfsIntegrationTest.getFileSystem:197 ? InvalidConfigurationValue`

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mukund-thakur commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r389616042
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java
 ##########
 @@ -143,7 +143,7 @@ public void testBlobDataReader() throws Exception {
 
     // TEST WRITE FILE
     try {
-      abfsStore.openFileForWrite(EXISTED_FILE_PATH, true);
+      abfsStore.openFileForWrite(EXISTED_FILE_PATH, fs.getFsStatistics(),true);
 
 Review comment:
   @mehakmeet  You can check the checkstyle errors produced by Apache Yetus run and fix those before actual code review. https://builds.apache.org/job/hadoop-multibranch/job/PR-1881/1/artifact/out/diff-checkstyle-hadoop-tools_hadoop-azure.txt 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r388859651
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java
 ##########
 @@ -59,6 +62,59 @@ public void testEnsureFileCreatedImmediately() throws Exception {
     assertIsFile(fs, TEST_FILE_PATH);
   }
 
+  /**
+   * {@link AbfsOutputStream#incrementWriteOps()}
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testWriteOpsMetric() throws Exception {
 
 Review comment:
   move into the same tests as the reads for ease of backports

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r388855498
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java
 ##########
 @@ -252,6 +255,13 @@ int readRemote(long position, byte[] b, int offset, int length) throws IOExcepti
     return (int) bytesRead;
   }
 
+  /**
+   * Increment Read Operations
 
 Review comment:
   nit: trailing .

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mehakmeet commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r396241061
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java
 ##########
 @@ -146,6 +146,8 @@ public void testBlobDataReader() throws Exception {
       abfsStore.openFileForWrite(EXISTED_FILE_PATH, fs.getFsStatistics(), true);
     } catch (AbfsRestOperationException e) {
       assertEquals(AzureServiceErrorCode.AUTHORIZATION_PERMISSION_MISS_MATCH, e.getErrorCode());
+    } finally {
+      abfsStore.close();
 
 Review comment:
   .close() does this already.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mehakmeet commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r389364668
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java
 ##########
 @@ -59,6 +62,59 @@ public void testEnsureFileCreatedImmediately() throws Exception {
     assertIsFile(fs, TEST_FILE_PATH);
   }
 
+  /**
+   * {@link AbfsOutputStream#incrementWriteOps()}
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testWriteOpsMetric() throws Exception {
 
 Review comment:
   I'll remove this test and add write operations tests in testAbfsInputStreamReadOps

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mehakmeet commented on issue #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on issue #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#issuecomment-595674353
 
 
   test run : mvn -T 1C -Dparallel-tests=abfs clean verify
   Region : East US, West US
   
   Failed tests(Container configurations based) :
   `[ERROR] Failures: [ERROR] ITestGetNameSpaceEnabled.testNonXNSAccount:59->Assert.assertFalse:64->Assert.assertTrue:41->Assert.fail:88 Expecting getIsNamespaceEnabled() return false [ERROR] Errors: [ERROR] ITestAbfsIdentityTransformer.<init>:66 ? NullPointer [ERROR] ITestGetNameSpaceEnabled.testFailedRequestWhenCredentialsNotCorrect:91->AbstractAbfsIntegrationTest.getFileSystem:197 ? InvalidConfigurationValue`

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r388856133
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsTestWithTimeout.java
 ##########
 @@ -67,4 +77,42 @@ public void nameThread() {
   protected int getTestTimeoutMillis() {
     return TEST_TIMEOUT;
   }
+
+  /**
+   * Describe a test in the logs.
+   *
+   * @param text text to print
+   * @param args arguments to format in the printing
+   */
+  protected void describe(String text, Object... args) {
+    LOG.info("\n\n{}: {}\n",
+        methodName.getMethodName(),
+        String.format(text, args));
+  }
+
+  /**
+   * Validate Contents written on a file in Abfs
+   *
+   * @param fs AzureBlobFileSystem
+   * @param path Path of the file
+   * @param originalByteArray original byte array
+   * @return
+   * @throws IOException
+   */
+  protected boolean validateContent(AzureBlobFileSystem fs, Path path,
+      byte[] originalByteArray)
+      throws IOException {
+    FSDataInputStream in = fs.open(path);
+    byte[] contentByteArray = new byte[originalByteArray.length];
+    int seekPos = 0;
+    while (in.read() != -1) {
 
 Review comment:
   read() always moves the stream forward, so you don't need these seek/seekPos++; they can only slow things down

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mukund-thakur removed a comment on issue #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mukund-thakur removed a comment on issue #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#issuecomment-596485775
 
 
   Some minor changes. Rest all looks good. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r394273198
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStreamStatistics.java
 ##########
 @@ -0,0 +1,147 @@
+/**
+ * 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 org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+/**
+ * Test Abfs Stream.
+ */
+
+public class ITestAbfsStreamStatistics extends AbstractAbfsIntegrationTest {
+  public ITestAbfsStreamStatistics() throws Exception {
+  }
+
+  /***
+   * Testing {@code incrementReadOps()} in class {@code AbfsInputStream} and
+   * {@code incrementWriteOps()} in class {@code AbfsOutputStream}.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testAbfsStreamOps() throws Exception {
+    describe("Test to see correct population of read and write operations in "
+        + "Abfs");
+
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path smallOperationsFile = new Path("testOneReadWriteOps");
+    Path largeOperationsFile = new Path("testLargeReadWriteOps");
+    FileSystem.Statistics statistics = fs.getFsStatistics();
+    String testReadWriteOps = "test this";
+    statistics.reset();
+
+    //Test for zero write operation
+    assertReadWriteOps("write", 0, statistics.getWriteOps());
+
+    //Test for zero read operation
+    assertReadWriteOps("read", 0, statistics.getReadOps());
+
+    FSDataOutputStream outForOneOperation = null;
+    FSDataInputStream inForOneOperation = null;
+    try {
+      outForOneOperation = fs.create(smallOperationsFile);
+      statistics.reset();
+      outForOneOperation.write(testReadWriteOps.getBytes());
+
+      //Test for a single write operation
+      assertReadWriteOps("write", 1, statistics.getWriteOps());
+
+      inForOneOperation = fs.open(smallOperationsFile);
+      inForOneOperation.read(testReadWriteOps.getBytes(), 0,
+          testReadWriteOps.getBytes().length);
+
+      //Test for a single read operation
+      assertReadWriteOps("read", 1, statistics.getReadOps());
+
+    } finally {
+      if (inForOneOperation != null) {
+        inForOneOperation.close();
+      }
+      if (outForOneOperation != null) {
+        outForOneOperation.close();
+      }
+    }
+
+    //Validating if content is being written in the smallOperationsFile
+    Assert.assertTrue("Mismatch in content validation",
+        validateContent(fs, smallOperationsFile,
+            testReadWriteOps.getBytes()));
+
+    FSDataOutputStream outForLargeOperations = null;
+    FSDataInputStream inForLargeOperations = null;
+    StringBuilder largeOperationsValidationString = new StringBuilder();
+    try {
+      outForLargeOperations = fs.create(largeOperationsFile);
+      statistics.reset();
+      int largeValue = 1000000;
+      for (int i = 0; i < largeValue; i++) {
+        outForLargeOperations.write(testReadWriteOps.getBytes());
+
+        //Creating the String for content Validation
+        largeOperationsValidationString.append(testReadWriteOps);
+      }
+
+      //Test for 1000000 write operations
+      assertReadWriteOps("write", largeValue, statistics.getWriteOps());
+
+      inForLargeOperations = fs.open(largeOperationsFile);
+      for (int i = 0; i < largeValue; i++)
+        inForLargeOperations
+            .read(testReadWriteOps.getBytes(), 0,
+                testReadWriteOps.getBytes().length);
+
+      //Test for 1000000 read operations
+      assertReadWriteOps("read", largeValue, statistics.getReadOps());
+
+    } finally {
+      if (inForLargeOperations != null) {
 
 Review comment:
   ```
   IOUtils.cleanupWithLogger(LOG, inForLargeOperations, outForLargeOperations)
   ```
   that does close on all non-null arguments, catches failures so they don't get in the way of whatever was thrown earlier. We use this throughout the hadoop codebase to clean up robustly, so get used to it

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r392958582
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
 ##########
 @@ -181,6 +185,14 @@ public synchronized void write(final byte[] data, final int off, final int lengt
 
       writableBytes = bufferSize - bufferIndex;
     }
+    incrementWriteOps();
+  }
+
+  /**
+   * Increment Write Operations.
+   */
+  public void incrementWriteOps() {
+    statistics.incrementWriteOps(1);
 
 Review comment:
   1. skip if statistics == null
   2. make private

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mehakmeet commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r391786649
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStreamStatistics.java
 ##########
 @@ -0,0 +1,111 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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 org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream;
+
+/**
+ * Test Abfs Stream.
+ */
+
+public class ITestAbfsStreamStatistics extends AbstractAbfsIntegrationTest {
+  public ITestAbfsStreamStatistics() throws Exception {
+  }
+
+  /***
+   * {@link AbfsInputStream#incrementReadOps()}.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testAbfsStreamOps() throws Exception {
+    describe("Test to see correct population of read and write operations in "
+        + "Abfs");
+
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path smallOperaionsFile = new Path("testOneReadWriteOps");
+    Path largeOperationsFile = new Path("testLargeReadWriteOps");
+    FileSystem.Statistics statistics = fs.getFsStatistics();
+    String testReadWriteOps = "test this";
+    statistics.reset();
+
+    //Test for zero read and write operation
+    Assert.assertEquals("Mismatch in read operations", 0,
+        statistics.getReadOps());
+    Assert.assertEquals("Mismatch in write operations", 0,
+        statistics.getWriteOps());
+
+    FSDataOutputStream outForOneOperation = fs.create(smallOperaionsFile);
+    statistics.reset();
+    outForOneOperation.write(testReadWriteOps.getBytes());
+    FSDataInputStream inForOneCall = fs.open(smallOperaionsFile);
+    inForOneCall.read(testReadWriteOps.getBytes(), 0,
+        testReadWriteOps.getBytes().length);
+
+    //Test for one read and write operation
+    Assert.assertEquals("Mismatch in read operations", 1,
+        statistics.getReadOps());
+    Assert.assertEquals("Mismatch in write operations", 1,
+        statistics.getWriteOps());
+
+    outForOneOperation.close();
+    //Validating if Content is being written in the smallFile
+    Assert.assertEquals("Mismatch in content validation", true,
+        validateContent(fs, smallOperaionsFile,
+            testReadWriteOps.getBytes()));
+
+    FSDataOutputStream outForLargeOperations = fs.create(largeOperationsFile);
+    statistics.reset();
+
+    StringBuilder largeOperationsValidationString = new StringBuilder();
+    for (int i = 0; i < 1000000; i++) {
+      outForLargeOperations.write(testReadWriteOps.getBytes());
+
+      //Creating the String for content Validation
+      largeOperationsValidationString.append(testReadWriteOps);
+    }
+
+    FSDataInputStream inForLargeCalls = fs.open(largeOperationsFile);
+
+    for (int i = 0; i < 1000000; i++)
+      inForLargeCalls
+          .read(testReadWriteOps.getBytes(), 0,
+              testReadWriteOps.getBytes().length);
+
+    //Test for one million read and write operations
+    Assert.assertEquals("Mismatch in read operations", 1000000,
+        statistics.getReadOps());
+    Assert.assertEquals("Mismatch in write operations", 1000000,
+        statistics.getWriteOps());
+
+    outForLargeOperations.close();
+    //Validating if actually "test" is being written million times in largeOperationsFile
 
 Review comment:
   this comment needs to change

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mehakmeet edited a comment on issue #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet edited a comment on issue #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#issuecomment-595684938
 
 
   test run : mvn -T 1C -Dparallel-tests=abfs clean verify
   Region : East US, West US
   
   Failed tests(Container configurations based) :
   ```
   [ERROR] Failures: [ERROR] ITestGetNameSpaceEnabled.testNonXNSAccount:59->Assert.assertFalse:64->Assert.assertTrue:41->Assert.fail:88 Expecting getIsNamespaceEnabled() return false
   [ERROR] Errors: [ERROR] ITestGetNameSpaceEnabled.testFailedRequestWhenCredentialsNotCorrect:91->AbstractAbfsIntegrationTest.getFileSystem:197 ? InvalidConfigurationValue`

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r395740845
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStreamStatistics.java
 ##########
 @@ -76,12 +79,8 @@ public void testAbfsStreamOps() throws Exception {
       assertReadWriteOps("read", 1, statistics.getReadOps());
 
     } finally {
-      if (inForOneOperation != null) {
-        inForOneOperation.close();
-      }
-      if (outForOneOperation != null) {
-        outForOneOperation.close();
-      }
+      IOUtils.cleanupWithLogger(null, inForOneOperation,
 
 Review comment:
   you are going to need to create a logger in this test case and pass it down i'm afraid

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mukund-thakur commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r391613932
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStreamStatistics.java
 ##########
 @@ -0,0 +1,109 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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 org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream;
+
+/**
+ * Test Abfs Stream.
+ */
+
+public class ITestAbfsStreamStatistics extends AbstractAbfsIntegrationTest {
+  public ITestAbfsStreamStatistics() throws Exception {
+  }
+
+  /***
+   * {@link AbfsInputStream#incrementReadOps()}.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testAbfsStreamOps() throws Exception {
+    describe("Test to see correct population of read and write operations in "
+        + "Abfs");
+
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path smallOperaionsFile = new Path("testOneReadWriteOps");
+    Path largeOperationsFile = new Path("testLargeReadWriteOps");
+    FileSystem.Statistics statistics = fs.getFsStatistics();
+    String testReadWriteOps = "test this";
+    statistics.reset();
+
+    //Test for zero read and write operation
+    Assert.assertEquals("Zero read operations", 0, statistics.getReadOps());
+    Assert.assertEquals("Zero write operations", 0, statistics.getWriteOps());
+
+    FSDataOutputStream outForOneOperation = fs.create(smallOperaionsFile);
+    statistics.reset();
+    outForOneOperation.write(testReadWriteOps.getBytes());
+    FSDataInputStream inForOneCall = fs.open(smallOperaionsFile);
+    inForOneCall.read(testReadWriteOps.getBytes(), 0,
+        testReadWriteOps.getBytes().length);
+
+    //Test for one read and write operation
+    Assert.assertEquals("one read operation is performed", 1,
+        statistics.getReadOps());
+    Assert.assertEquals("one write operation is performed", 1,
+        statistics.getWriteOps());
+
+    outForOneOperation.close();
+    //validating Content of file
+    Assert.assertEquals("one operation Content validation", true,
+        validateContent(fs, smallOperaionsFile,
+            testReadWriteOps.getBytes()));
+
+    FSDataOutputStream outForLargeOperations = fs.create(largeOperationsFile);
+    statistics.reset();
+
+    StringBuilder largeOperationsValidationString = new StringBuilder();
+    for (int i = 0; i < 1000000; i++) {
+      outForLargeOperations.write(testReadWriteOps.getBytes());
+
+      //Creating the String for content Validation
+      largeOperationsValidationString.append(testReadWriteOps);
+    }
+
+    FSDataInputStream inForLargeCalls = fs.open(largeOperationsFile);
+
+    for (int i = 0; i < 1000000; i++)
+      inForLargeCalls
+          .read(testReadWriteOps.getBytes(), 0,
+              testReadWriteOps.getBytes().length);
+
+    //Test for one million read and write operations
+    Assert.assertEquals("Large read operations", 1000000,
 
 Review comment:
   yes.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mehakmeet commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r388940913
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsTestWithTimeout.java
 ##########
 @@ -67,4 +77,42 @@ public void nameThread() {
   protected int getTestTimeoutMillis() {
     return TEST_TIMEOUT;
   }
+
+  /**
+   * Describe a test in the logs.
+   *
+   * @param text text to print
+   * @param args arguments to format in the printing
+   */
+  protected void describe(String text, Object... args) {
+    LOG.info("\n\n{}: {}\n",
+        methodName.getMethodName(),
+        String.format(text, args));
+  }
+
+  /**
+   * Validate Contents written on a file in Abfs
+   *
+   * @param fs AzureBlobFileSystem
+   * @param path Path of the file
+   * @param originalByteArray original byte array
+   * @return
+   * @throws IOException
+   */
+  protected boolean validateContent(AzureBlobFileSystem fs, Path path,
+      byte[] originalByteArray)
+      throws IOException {
+    FSDataInputStream in = fs.open(path);
+    byte[] contentByteArray = new byte[originalByteArray.length];
+    int seekPos = 0;
+    while (in.read() != -1) {
 
 Review comment:
   I will still need seekPos for indexing of contentByteArray..
   seek() is however an extra method call, I can compare byte by byte with originalByteArray ?
   Something like this ,maybe ?
   
   ``` 
   while (valueOfContentAtPos != -1 && pos < lenOfOriginalByteArray) {
         if (originalByteArray[pos] != valueOfContentAtPos)
           return false;
         valueOfContentAtPos = (byte) in.read();
         pos++;
       }
       if (valueOfContentAtPos != -1)
         return false;
       return true;```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] hadoop-yetus commented on issue #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on issue #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#issuecomment-595688964
 
 
   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |  25m 39s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 4 new or modified test files.  |
   ||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  19m  4s |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 31s |  trunk passed  |
   | +1 :green_heart: |  checkstyle  |   0m 25s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 34s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  15m  4s |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 26s |  trunk passed  |
   | +0 :ok: |  spotbugs  |   0m 51s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 49s |  trunk passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 27s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 24s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 24s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 17s |  hadoop-tools/hadoop-azure: The patch generated 2 new + 1 unchanged - 0 fixed = 3 total (was 1)  |
   | +1 :green_heart: |  mvnsite  |   0m 26s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  13m 42s |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 23s |  the patch passed  |
   | +1 :green_heart: |  findbugs  |   0m 53s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 20s |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 32s |  The patch does not generate ASF License warnings.  |
   |  |   |  82m 33s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.7 Server=19.03.7 base: https://builds.apache.org/job/hadoop-multibranch/job/PR-1881/1/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/1881 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux 6db0065e9119 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | personality/hadoop.sh |
   | git revision | trunk / 004e955 |
   | Default Java | 1.8.0_242 |
   | checkstyle | https://builds.apache.org/job/hadoop-multibranch/job/PR-1881/1/artifact/out/diff-checkstyle-hadoop-tools_hadoop-azure.txt |
   |  Test Results | https://builds.apache.org/job/hadoop-multibranch/job/PR-1881/1/testReport/ |
   | Max. process+thread count | 422 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://builds.apache.org/job/hadoop-multibranch/job/PR-1881/1/console |
   | versions | git=2.7.4 maven=3.3.9 findbugs=3.1.0-RC1 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] steveloughran merged pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran merged pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881
 
 
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mukund-thakur commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r391602306
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStreamStatistics.java
 ##########
 @@ -0,0 +1,109 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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 org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream;
+
+/**
+ * Test Abfs Stream.
+ */
+
+public class ITestAbfsStreamStatistics extends AbstractAbfsIntegrationTest {
+  public ITestAbfsStreamStatistics() throws Exception {
+  }
+
+  /***
+   * {@link AbfsInputStream#incrementReadOps()}.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testAbfsStreamOps() throws Exception {
+    describe("Test to see correct population of read and write operations in "
+        + "Abfs");
+
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path smallOperaionsFile = new Path("testOneReadWriteOps");
+    Path largeOperationsFile = new Path("testLargeReadWriteOps");
+    FileSystem.Statistics statistics = fs.getFsStatistics();
+    String testReadWriteOps = "test this";
+    statistics.reset();
+
+    //Test for zero read and write operation
+    Assert.assertEquals("Zero read operations", 0, statistics.getReadOps());
+    Assert.assertEquals("Zero write operations", 0, statistics.getWriteOps());
+
+    FSDataOutputStream outForOneOperation = fs.create(smallOperaionsFile);
+    statistics.reset();
+    outForOneOperation.write(testReadWriteOps.getBytes());
+    FSDataInputStream inForOneCall = fs.open(smallOperaionsFile);
+    inForOneCall.read(testReadWriteOps.getBytes(), 0,
+        testReadWriteOps.getBytes().length);
+
+    //Test for one read and write operation
+    Assert.assertEquals("one read operation is performed", 1,
 
 Review comment:
   Change to "Mismatch in read operation count."

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r394273761
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java
 ##########
 @@ -143,7 +143,7 @@ public void testBlobDataReader() throws Exception {
 
     // TEST WRITE FILE
     try {
-      abfsStore.openFileForWrite(EXISTED_FILE_PATH, true);
+      abfsStore.openFileForWrite(EXISTED_FILE_PATH, fs.getFsStatistics(), true);
 
 Review comment:
   add a .close() at the end so if something went wrong and the file was opened, we close the stream

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mehakmeet commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r391613039
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStreamStatistics.java
 ##########
 @@ -0,0 +1,109 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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 org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream;
+
+/**
+ * Test Abfs Stream.
+ */
+
+public class ITestAbfsStreamStatistics extends AbstractAbfsIntegrationTest {
+  public ITestAbfsStreamStatistics() throws Exception {
+  }
+
+  /***
+   * {@link AbfsInputStream#incrementReadOps()}.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testAbfsStreamOps() throws Exception {
+    describe("Test to see correct population of read and write operations in "
+        + "Abfs");
+
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path smallOperaionsFile = new Path("testOneReadWriteOps");
+    Path largeOperationsFile = new Path("testLargeReadWriteOps");
+    FileSystem.Statistics statistics = fs.getFsStatistics();
+    String testReadWriteOps = "test this";
+    statistics.reset();
+
+    //Test for zero read and write operation
+    Assert.assertEquals("Zero read operations", 0, statistics.getReadOps());
+    Assert.assertEquals("Zero write operations", 0, statistics.getWriteOps());
+
+    FSDataOutputStream outForOneOperation = fs.create(smallOperaionsFile);
+    statistics.reset();
+    outForOneOperation.write(testReadWriteOps.getBytes());
+    FSDataInputStream inForOneCall = fs.open(smallOperaionsFile);
+    inForOneCall.read(testReadWriteOps.getBytes(), 0,
+        testReadWriteOps.getBytes().length);
+
+    //Test for one read and write operation
+    Assert.assertEquals("one read operation is performed", 1,
+        statistics.getReadOps());
+    Assert.assertEquals("one write operation is performed", 1,
+        statistics.getWriteOps());
+
+    outForOneOperation.close();
+    //validating Content of file
+    Assert.assertEquals("one operation Content validation", true,
+        validateContent(fs, smallOperaionsFile,
+            testReadWriteOps.getBytes()));
+
+    FSDataOutputStream outForLargeOperations = fs.create(largeOperationsFile);
+    statistics.reset();
+
+    StringBuilder largeOperationsValidationString = new StringBuilder();
+    for (int i = 0; i < 1000000; i++) {
+      outForLargeOperations.write(testReadWriteOps.getBytes());
+
+      //Creating the String for content Validation
+      largeOperationsValidationString.append(testReadWriteOps);
+    }
+
+    FSDataInputStream inForLargeCalls = fs.open(largeOperationsFile);
+
+    for (int i = 0; i < 1000000; i++)
+      inForLargeCalls
+          .read(testReadWriteOps.getBytes(), 0,
+              testReadWriteOps.getBytes().length);
+
+    //Test for one million read and write operations
+    Assert.assertEquals("Large read operations", 1000000,
 
 Review comment:
   Will both 1 op error and 1000 ops error have same message ?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mehakmeet commented on issue #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on issue #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#issuecomment-595684591
 
 
   test run : mvn -T 1C -Dparallel-tests=abfs clean verify
   Region : East US, West US
   
   Failed tests(Container configurations based) :
   `[ERROR] Failures: 
   [ERROR]   ITestGetNameSpaceEnabled.testNonXNSAccount:59->Assert.assertFalse:64->Assert.assertTrue:41->Assert.fail:88 Expecting getIsNamespaceEnabled() return false
   [ERROR] Errors: 
   [ERROR]   ITestGetNameSpaceEnabled.testFailedRequestWhenCredentialsNotCorrect:91->AbstractAbfsIntegrationTest.getFileSystem:197 ? InvalidConfigurationValue`

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r392957716
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStreamStatistics.java
 ##########
 @@ -0,0 +1,111 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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 org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream;
+
+/**
+ * Test Abfs Stream.
+ */
+
+public class ITestAbfsStreamStatistics extends AbstractAbfsIntegrationTest {
+  public ITestAbfsStreamStatistics() throws Exception {
+  }
+
+  /***
+   * {@link AbfsInputStream#incrementReadOps()}.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testAbfsStreamOps() throws Exception {
+    describe("Test to see correct population of read and write operations in "
+        + "Abfs");
+
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path smallOperationsFile = new Path("testOneReadWriteOps");
+    Path largeOperationsFile = new Path("testLargeReadWriteOps");
+    FileSystem.Statistics statistics = fs.getFsStatistics();
+    String testReadWriteOps = "test this";
+    statistics.reset();
+
+    //Test for zero read and write operation
+    Assert.assertEquals("Mismatch in read operations", 0,
+        statistics.getReadOps());
+    Assert.assertEquals("Mismatch in write operations", 0,
+        statistics.getWriteOps());
+
+    FSDataOutputStream outForOneOperation = fs.create(smallOperationsFile);
+    statistics.reset();
+    outForOneOperation.write(testReadWriteOps.getBytes());
+    FSDataInputStream inForOneCall = fs.open(smallOperationsFile);
+    inForOneCall.read(testReadWriteOps.getBytes(), 0,
+        testReadWriteOps.getBytes().length);
+
+    //Test for one read and write operation
 
 Review comment:
   factor into assertReadWriteOps(operation, stats, read, write) 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mehakmeet removed a comment on issue #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet removed a comment on issue #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#issuecomment-595672664
 
 
   test run : mvn -T 1C -Dparallel-tests=abfs clean verify
   
   Failed tests(Container configurations based) :
   `[ERROR] Failures: 
   [ERROR]   ITestGetNameSpaceEnabled.testNonXNSAccount:59->Assert.assertFalse:64->Assert.assertTrue:41->Assert.fail:88 Expecting getIsNamespaceEnabled() return false
   [ERROR] Errors: 
   [ERROR]   ITestAbfsIdentityTransformer.<init>:66 ? NullPointer
   [ERROR]   ITestGetNameSpaceEnabled.testFailedRequestWhenCredentialsNotCorrect:91->AbstractAbfsIntegrationTest.getFileSystem:197 ? InvalidConfigurationValue`

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mehakmeet commented on issue #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on issue #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#issuecomment-603101587
 
 
   > Hi @mehakmeet and @steveloughran
   > Failed to compile TestAbfsOutputStream.java after the merge. Would the fix it?
   Yes, I'll be fixing it.
   https://issues.apache.org/jira/browse/HADOOP-16934

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mehakmeet edited a comment on issue #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet edited a comment on issue #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#issuecomment-595684591
 
 
   test run : mvn -T 1C -Dparallel-tests=abfs clean verify
   Region : East US, West US
   
   Failed tests(Container configurations based) :
   `[ERROR] Failures: 
   [ERROR]   ITestGetNameSpaceEnabled.testNonXNSAccount:59->Assert.assertFalse:64->Assert.assertTrue:41->Assert.fail:88 Expecting getIsNamespaceEnabled() return false \n
   [ERROR] Errors: 
   [ERROR]   ITestGetNameSpaceEnabled.testFailedRequestWhenCredentialsNotCorrect:91->AbstractAbfsIntegrationTest.getFileSystem:197 ? InvalidConfigurationValue`

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mukund-thakur commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r389614616
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java
 ##########
 @@ -59,6 +62,59 @@ public void testEnsureFileCreatedImmediately() throws Exception {
     assertIsFile(fs, TEST_FILE_PATH);
   }
 
+  /**
+   * {@link AbfsOutputStream#incrementWriteOps()}
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testWriteOpsMetric() throws Exception {
+    describe("Test to see correct population of write operations in Abfs");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path smallFile = new Path("testOneCall");
+    Path largeFile = new Path("testLargeCalls");
+    String testWriteOps = "test";
+    FileSystem.Statistics statistics = fs.getFsStatistics();
+    statistics.reset();
+
+    //Test for zero write operation
+    Assert.assertEquals(0, statistics.getWriteOps());
+
+    FSDataOutputStream outForOneCall = fs.create(smallFile);
+    statistics.reset();
+    //Writing "test" 1 time
+    outForOneCall
+        .write(testWriteOps.getBytes(), 0, testWriteOps.getBytes().length);
+
+    //Test for one write operation
+    Assert.assertEquals(1, statistics.getWriteOps());
 
 Review comment:
   All assert calls should have meaningful message as suggested by Steve. This is important for debugging test failures easily. You can check the examples in PR I sent you. 
   Sorry I missed this in internal review. :)

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mehakmeet commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r389365473
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java
 ##########
 @@ -59,6 +62,59 @@ public void testEnsureFileCreatedImmediately() throws Exception {
     assertIsFile(fs, TEST_FILE_PATH);
   }
 
+  /**
+   * {@link AbfsOutputStream#incrementWriteOps()}
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testWriteOpsMetric() throws Exception {
 
 Review comment:
   I'll remove this test and add write operations and validating tests in testAbfsInputStreamReadOps(change it's name to testAbfsStreamOps) ?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r388858762
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStream.java
 ##########
 @@ -0,0 +1,81 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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 org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream;
+
+/**
+ * Test Abfs Input Stream.
+ */
+
+public class ITestAbfsInputStream extends AbstractAbfsIntegrationTest {
 
 Review comment:
   This is a good test.
   
   If you call it ITestAbfsStreamStatistics you can test the writes as well as the reads -and you should test those writes already in the test suite.. If you put the write ops test there, into a single test suite, backporting is easier -less conflict.
   
   For the assertions, I'd like every assert to have some meaningful string, 
   
   assertEquals("read ops", 1000, statistics.getReadOps)
   
   Imagine "what would you want in the error text if this failed on a jenkins run"?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mehakmeet commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r389364566
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStream.java
 ##########
 @@ -0,0 +1,81 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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 org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream;
+
+/**
+ * Test Abfs Input Stream.
+ */
+
+public class ITestAbfsInputStream extends AbstractAbfsIntegrationTest {
 
 Review comment:
   So, I guess 1 test method would be enough to check both read and write operations in all 3 conditions.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r388858948
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java
 ##########
 @@ -392,7 +392,7 @@ public void deleteFilesystem() throws AzureBlobFileSystemException {
     }
   }
 
-  public OutputStream createFile(final Path path, final boolean overwrite, final FsPermission permission,
+  public OutputStream createFile(final Path path, final FileSystem.Statistics statistics, final boolean overwrite, final FsPermission permission,
 
 Review comment:
   nit: this line is long enough its time to split

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mehakmeet removed a comment on issue #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet removed a comment on issue #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#issuecomment-595650634
 
 
   Tested on Azure account : mehakmeetdata
   Azure container : mehakmeet-abfs
   Tested by : mvn -T 1C -Dparallel-tests=abfs clean verify

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mukund-thakur commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r391602594
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStreamStatistics.java
 ##########
 @@ -0,0 +1,109 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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 org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream;
+
+/**
+ * Test Abfs Stream.
+ */
+
+public class ITestAbfsStreamStatistics extends AbstractAbfsIntegrationTest {
+  public ITestAbfsStreamStatistics() throws Exception {
+  }
+
+  /***
+   * {@link AbfsInputStream#incrementReadOps()}.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testAbfsStreamOps() throws Exception {
+    describe("Test to see correct population of read and write operations in "
+        + "Abfs");
+
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path smallOperaionsFile = new Path("testOneReadWriteOps");
+    Path largeOperationsFile = new Path("testLargeReadWriteOps");
+    FileSystem.Statistics statistics = fs.getFsStatistics();
+    String testReadWriteOps = "test this";
+    statistics.reset();
+
+    //Test for zero read and write operation
+    Assert.assertEquals("Zero read operations", 0, statistics.getReadOps());
+    Assert.assertEquals("Zero write operations", 0, statistics.getWriteOps());
+
+    FSDataOutputStream outForOneOperation = fs.create(smallOperaionsFile);
+    statistics.reset();
+    outForOneOperation.write(testReadWriteOps.getBytes());
+    FSDataInputStream inForOneCall = fs.open(smallOperaionsFile);
+    inForOneCall.read(testReadWriteOps.getBytes(), 0,
+        testReadWriteOps.getBytes().length);
+
+    //Test for one read and write operation
+    Assert.assertEquals("one read operation is performed", 1,
+        statistics.getReadOps());
+    Assert.assertEquals("one write operation is performed", 1,
+        statistics.getWriteOps());
+
+    outForOneOperation.close();
+    //validating Content of file
+    Assert.assertEquals("one operation Content validation", true,
+        validateContent(fs, smallOperaionsFile,
+            testReadWriteOps.getBytes()));
+
+    FSDataOutputStream outForLargeOperations = fs.create(largeOperationsFile);
+    statistics.reset();
+
+    StringBuilder largeOperationsValidationString = new StringBuilder();
+    for (int i = 0; i < 1000000; i++) {
+      outForLargeOperations.write(testReadWriteOps.getBytes());
+
+      //Creating the String for content Validation
+      largeOperationsValidationString.append(testReadWriteOps);
+    }
+
+    FSDataInputStream inForLargeCalls = fs.open(largeOperationsFile);
+
+    for (int i = 0; i < 1000000; i++)
+      inForLargeCalls
+          .read(testReadWriteOps.getBytes(), 0,
+              testReadWriteOps.getBytes().length);
+
+    //Test for one million read and write operations
+    Assert.assertEquals("Large read operations", 1000000,
 
 Review comment:
   Same here.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mukund-thakur commented on issue #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on issue #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#issuecomment-596485775
 
 
   Some minor changes. Rest all looks good. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] aajisaka removed a comment on issue #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
aajisaka removed a comment on issue #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#issuecomment-603068418
 
 
   Hi @mehakmeet and @steveloughran 
   Failed to compile hadoop-azure after the merge. Would you fix the test as well?
   ```
   [ERROR] Failed to execute goal org.apache.maven.plugins:maven-compiler-plugin:3.1:testCompile (default-testCompile) on project hadoop-azure: Compilation failure: Compilation failure: 
   [ERROR] /Users/aajisaka/git/ghe.corp/hadoop-mirror/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java:[71,28] constructor AbfsOutputStream in class org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream cannot be applied to given types;
   [ERROR]   required: org.apache.hadoop.fs.azurebfs.services.AbfsClient,org.apache.hadoop.fs.FileSystem.Statistics,java.lang.String,long,int,boolean,boolean,boolean,boolean
   [ERROR]   found:    org.apache.hadoop.fs.azurebfs.services.AbfsClient,java.lang.String,int,int,boolean,boolean,boolean,boolean
   [ERROR]   reason: actual and formal argument lists differ in length
   [ERROR] /Users/aajisaka/git/ghe.corp/hadoop-mirror/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java:[121,28] constructor AbfsOutputStream in class org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream cannot be applied to given types;
   [ERROR]   required: org.apache.hadoop.fs.azurebfs.services.AbfsClient,org.apache.hadoop.fs.FileSystem.Statistics,java.lang.String,long,int,boolean,boolean,boolean,boolean
   [ERROR]   found:    org.apache.hadoop.fs.azurebfs.services.AbfsClient,java.lang.String,int,int,boolean,boolean,boolean,boolean
   [ERROR]   reason: actual and formal argument lists differ in length
   [ERROR] /Users/aajisaka/git/ghe.corp/hadoop-mirror/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java:[168,28] constructor AbfsOutputStream in class org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream cannot be applied to given types;
   [ERROR]   required: org.apache.hadoop.fs.azurebfs.services.AbfsClient,org.apache.hadoop.fs.FileSystem.Statistics,java.lang.String,long,int,boolean,boolean,boolean,boolean
   [ERROR]   found:    org.apache.hadoop.fs.azurebfs.services.AbfsClient,java.lang.String,int,int,boolean,boolean,boolean,boolean
   [ERROR]   reason: actual and formal argument lists differ in length
   [ERROR] /Users/aajisaka/git/ghe.corp/hadoop-mirror/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java:[226,28] constructor AbfsOutputStream in class org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream cannot be applied to given types;
   [ERROR]   required: org.apache.hadoop.fs.azurebfs.services.AbfsClient,org.apache.hadoop.fs.FileSystem.Statistics,java.lang.String,long,int,boolean,boolean,boolean,boolean
   [ERROR]   found:    org.apache.hadoop.fs.azurebfs.services.AbfsClient,java.lang.String,int,int,boolean,boolean,boolean,boolean
   [ERROR]   reason: actual and formal argument lists differ in length
   [ERROR] /Users/aajisaka/git/ghe.corp/hadoop-mirror/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java:[273,28] constructor AbfsOutputStream in class org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream cannot be applied to given types;
   [ERROR]   required: org.apache.hadoop.fs.azurebfs.services.AbfsClient,org.apache.hadoop.fs.FileSystem.Statistics,java.lang.String,long,int,boolean,boolean,boolean,boolean
   [ERROR]   found:    org.apache.hadoop.fs.azurebfs.services.AbfsClient,java.lang.String,int,int,boolean,boolean,boolean,boolean
   [ERROR]   reason: actual and formal argument lists differ in length
   [ERROR] /Users/aajisaka/git/ghe.corp/hadoop-mirror/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java:[320,28] constructor AbfsOutputStream in class org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream cannot be applied to given types;
   [ERROR]   required: org.apache.hadoop.fs.azurebfs.services.AbfsClient,org.apache.hadoop.fs.FileSystem.Statistics,java.lang.String,long,int,boolean,boolean,boolean,boolean
   [ERROR]   found:    org.apache.hadoop.fs.azurebfs.services.AbfsClient,java.lang.String,int,int,boolean,boolean,boolean,boolean
   [ERROR]   reason: actual and formal argument lists differ in length
   [ERROR] /Users/aajisaka/git/ghe.corp/hadoop-mirror/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java:[377,28] constructor AbfsOutputStream in class org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream cannot be applied to given types;
   [ERROR]   required: org.apache.hadoop.fs.azurebfs.services.AbfsClient,org.apache.hadoop.fs.FileSystem.Statistics,java.lang.String,long,int,boolean,boolean,boolean,boolean
   [ERROR]   found:    org.apache.hadoop.fs.azurebfs.services.AbfsClient,java.lang.String,int,int,boolean,boolean,boolean,boolean
   [ERROR]   reason: actual and formal argument lists differ in length
   ```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r388854742
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsTestWithTimeout.java
 ##########
 @@ -17,12 +17,19 @@
  */
 package org.apache.hadoop.fs.azurebfs;
 
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.Path;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.rules.TestName;
 import org.junit.rules.Timeout;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
 
 Review comment:
   nit: topmost import block

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mehakmeet commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r389364668
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java
 ##########
 @@ -59,6 +62,59 @@ public void testEnsureFileCreatedImmediately() throws Exception {
     assertIsFile(fs, TEST_FILE_PATH);
   }
 
+  /**
+   * {@link AbfsOutputStream#incrementWriteOps()}
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testWriteOpsMetric() throws Exception {
 
 Review comment:
   I'll remove this test and add write operations tests in testAbfsInputStreamReadOps

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mukund-thakur commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r391602306
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStreamStatistics.java
 ##########
 @@ -0,0 +1,109 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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 org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream;
+
+/**
+ * Test Abfs Stream.
+ */
+
+public class ITestAbfsStreamStatistics extends AbstractAbfsIntegrationTest {
+  public ITestAbfsStreamStatistics() throws Exception {
+  }
+
+  /***
+   * {@link AbfsInputStream#incrementReadOps()}.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testAbfsStreamOps() throws Exception {
+    describe("Test to see correct population of read and write operations in "
+        + "Abfs");
+
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path smallOperaionsFile = new Path("testOneReadWriteOps");
+    Path largeOperationsFile = new Path("testLargeReadWriteOps");
+    FileSystem.Statistics statistics = fs.getFsStatistics();
+    String testReadWriteOps = "test this";
+    statistics.reset();
+
+    //Test for zero read and write operation
+    Assert.assertEquals("Zero read operations", 0, statistics.getReadOps());
+    Assert.assertEquals("Zero write operations", 0, statistics.getWriteOps());
+
+    FSDataOutputStream outForOneOperation = fs.create(smallOperaionsFile);
+    statistics.reset();
+    outForOneOperation.write(testReadWriteOps.getBytes());
+    FSDataInputStream inForOneCall = fs.open(smallOperaionsFile);
+    inForOneCall.read(testReadWriteOps.getBytes(), 0,
+        testReadWriteOps.getBytes().length);
+
+    //Test for one read and write operation
+    Assert.assertEquals("one read operation is performed", 1,
 
 Review comment:
   Mismatch in read operation count.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mehakmeet commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r391786077
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStreamStatistics.java
 ##########
 @@ -0,0 +1,111 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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 org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream;
+
+/**
+ * Test Abfs Stream.
+ */
+
+public class ITestAbfsStreamStatistics extends AbstractAbfsIntegrationTest {
+  public ITestAbfsStreamStatistics() throws Exception {
+  }
+
+  /***
+   * {@link AbfsInputStream#incrementReadOps()}.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testAbfsStreamOps() throws Exception {
+    describe("Test to see correct population of read and write operations in "
+        + "Abfs");
+
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path smallOperaionsFile = new Path("testOneReadWriteOps");
 
 Review comment:
   just saw this typo. will fix it in the next patch.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mukund-thakur commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r391602770
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStreamStatistics.java
 ##########
 @@ -0,0 +1,109 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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 org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream;
+
+/**
+ * Test Abfs Stream.
+ */
+
+public class ITestAbfsStreamStatistics extends AbstractAbfsIntegrationTest {
+  public ITestAbfsStreamStatistics() throws Exception {
+  }
+
+  /***
+   * {@link AbfsInputStream#incrementReadOps()}.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testAbfsStreamOps() throws Exception {
+    describe("Test to see correct population of read and write operations in "
+        + "Abfs");
+
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path smallOperaionsFile = new Path("testOneReadWriteOps");
+    Path largeOperationsFile = new Path("testLargeReadWriteOps");
+    FileSystem.Statistics statistics = fs.getFsStatistics();
+    String testReadWriteOps = "test this";
+    statistics.reset();
+
+    //Test for zero read and write operation
+    Assert.assertEquals("Zero read operations", 0, statistics.getReadOps());
+    Assert.assertEquals("Zero write operations", 0, statistics.getWriteOps());
+
+    FSDataOutputStream outForOneOperation = fs.create(smallOperaionsFile);
+    statistics.reset();
+    outForOneOperation.write(testReadWriteOps.getBytes());
+    FSDataInputStream inForOneCall = fs.open(smallOperaionsFile);
+    inForOneCall.read(testReadWriteOps.getBytes(), 0,
+        testReadWriteOps.getBytes().length);
+
+    //Test for one read and write operation
+    Assert.assertEquals("one read operation is performed", 1,
+        statistics.getReadOps());
+    Assert.assertEquals("one write operation is performed", 1,
+        statistics.getWriteOps());
+
+    outForOneOperation.close();
+    //validating Content of file
+    Assert.assertEquals("one operation Content validation", true,
 
 Review comment:
   Mismatch in file content.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mehakmeet commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r388940913
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsTestWithTimeout.java
 ##########
 @@ -67,4 +77,42 @@ public void nameThread() {
   protected int getTestTimeoutMillis() {
     return TEST_TIMEOUT;
   }
+
+  /**
+   * Describe a test in the logs.
+   *
+   * @param text text to print
+   * @param args arguments to format in the printing
+   */
+  protected void describe(String text, Object... args) {
+    LOG.info("\n\n{}: {}\n",
+        methodName.getMethodName(),
+        String.format(text, args));
+  }
+
+  /**
+   * Validate Contents written on a file in Abfs
+   *
+   * @param fs AzureBlobFileSystem
+   * @param path Path of the file
+   * @param originalByteArray original byte array
+   * @return
+   * @throws IOException
+   */
+  protected boolean validateContent(AzureBlobFileSystem fs, Path path,
+      byte[] originalByteArray)
+      throws IOException {
+    FSDataInputStream in = fs.open(path);
+    byte[] contentByteArray = new byte[originalByteArray.length];
+    int seekPos = 0;
+    while (in.read() != -1) {
 
 Review comment:
   I will still need seekPos for indexing of contentByteArray..
   seek() is however an extra method call, I can compare byte by byte with originalByteArray ?
   Something like this ,maybe ?
   
   ``` 
   while (valueOfContentAtPos != -1 && pos < lenOfOriginalByteArray) {
         if (originalByteArray[pos] != valueOfContentAtPos)
           return false;
         valueOfContentAtPos = (byte) in.read();
         pos++;
       }
       if (valueOfContentAtPos != -1)
         return false;
       return true;

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r388856604
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStream.java
 ##########
 @@ -0,0 +1,81 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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 org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream;
+
+/**
+ * Test Abfs Input Stream.
+ */
+
+public class ITestAbfsInputStream extends AbstractAbfsIntegrationTest {
+  public ITestAbfsInputStream() throws Exception {
+  }
+
+  /***
+   * {@link AbfsInputStream#incrementReadOps()}
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testAbfsInputStreamReadOps() throws Exception {
+    describe("Test to see correct population of Read operations in Abfs");
+
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path smallFile = new Path("testOneReadCall");
+    Path largeFile = new Path("testLargeReadCalls");
+    FileSystem.Statistics statistics = fs.getFsStatistics();
+    String testReadOps = "test this";
+    statistics.reset();
+
+    //Test for zero read operation
+    Assert.assertEquals(0, statistics.getReadOps());
+
+    FSDataOutputStream outForOneCall = fs.create(smallFile);
+    statistics.reset();
+    outForOneCall.write(testReadOps.getBytes());
 
 Review comment:
   add a test for writes here too

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r392964448
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStreamStatistics.java
 ##########
 @@ -0,0 +1,111 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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 org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
 
 Review comment:
   minor: nit: ordering

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r392958021
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStreamStatistics.java
 ##########
 @@ -0,0 +1,111 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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 org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream;
+
+/**
+ * Test Abfs Stream.
+ */
+
+public class ITestAbfsStreamStatistics extends AbstractAbfsIntegrationTest {
+  public ITestAbfsStreamStatistics() throws Exception {
+  }
+
+  /***
+   * {@link AbfsInputStream#incrementReadOps()}.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testAbfsStreamOps() throws Exception {
+    describe("Test to see correct population of read and write operations in "
+        + "Abfs");
+
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path smallOperationsFile = new Path("testOneReadWriteOps");
+    Path largeOperationsFile = new Path("testLargeReadWriteOps");
+    FileSystem.Statistics statistics = fs.getFsStatistics();
+    String testReadWriteOps = "test this";
+    statistics.reset();
+
+    //Test for zero read and write operation
+    Assert.assertEquals("Mismatch in read operations", 0,
+        statistics.getReadOps());
+    Assert.assertEquals("Mismatch in write operations", 0,
+        statistics.getWriteOps());
+
+    FSDataOutputStream outForOneOperation = fs.create(smallOperationsFile);
 
 Review comment:
   close in a try/finally

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mukund-thakur commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r396228815
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStreamStatistics.java
 ##########
 @@ -101,6 +115,8 @@ public void testAbfsStreamOps() throws Exception {
         //Creating the String for content Validation
         largeOperationsValidationString.append(testReadWriteOps);
       }
+      LOG.info("Number of bytes of Large data written: {}",
 
 Review comment:
   Is this logging required ? @steveloughran 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mehakmeet commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r393184979
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
 ##########
 @@ -36,6 +36,7 @@
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 
+import org.apache.hadoop.fs.FileSystem.Statistics;
 
 Review comment:
   I thought ordering was right?
   java
   non org.apache
   org.apache
   static

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r394271627
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsTestWithTimeout.java
 ##########
 @@ -67,4 +77,46 @@ public void nameThread() {
   protected int getTestTimeoutMillis() {
     return TEST_TIMEOUT;
   }
+
+  /**
+   * Describe a test in the logs.
+   *
+   * @param text text to print
+   * @param args arguments to format in the printing
+   */
+  protected void describe(String text, Object... args) {
+    LOG.info("\n\n{}: {}\n",
+        methodName.getMethodName(),
+        String.format(text, args));
+  }
+
+  /**
+   * Validate Contents written on a file in Abfs.
+   *
+   * @param fs                AzureBlobFileSystem
+   * @param path              Path of the file
+   * @param originalByteArray original byte array
+   * @return
+   * @throws IOException
+   */
+  protected boolean validateContent(AzureBlobFileSystem fs, Path path,
+      byte[] originalByteArray)
+      throws IOException {
+    FSDataInputStream in = fs.open(path);
 
 Review comment:
   needs to be closed. you can use `try (FSDataInputStream ...) { }` to manage this automatically

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mehakmeet edited a comment on issue #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet edited a comment on issue #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#issuecomment-595650634
 
 
   Tested on Azure account : mehakmeetdata
   Azure container : mehakmeet-abfs
   Tested by : mvn -T 1C -Dparallel-tests=abfs clean verify

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r392963453
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
 ##########
 @@ -36,6 +36,7 @@
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 
+import org.apache.hadoop.fs.FileSystem.Statistics;
 
 Review comment:
   nit: ordering

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mehakmeet commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r389364668
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java
 ##########
 @@ -59,6 +62,59 @@ public void testEnsureFileCreatedImmediately() throws Exception {
     assertIsFile(fs, TEST_FILE_PATH);
   }
 
+  /**
+   * {@link AbfsOutputStream#incrementWriteOps()}
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testWriteOpsMetric() throws Exception {
 
 Review comment:
   I'll remove this test and add write operations tests in testAbfsInputStreamReadOps

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mehakmeet commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r390493936
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java
 ##########
 @@ -59,6 +62,59 @@ public void testEnsureFileCreatedImmediately() throws Exception {
     assertIsFile(fs, TEST_FILE_PATH);
   }
 
+  /**
+   * {@link AbfsOutputStream#incrementWriteOps()}
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testWriteOpsMetric() throws Exception {
+    describe("Test to see correct population of write operations in Abfs");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path smallFile = new Path("testOneCall");
+    Path largeFile = new Path("testLargeCalls");
+    String testWriteOps = "test";
+    FileSystem.Statistics statistics = fs.getFsStatistics();
+    statistics.reset();
+
+    //Test for zero write operation
+    Assert.assertEquals(0, statistics.getWriteOps());
+
+    FSDataOutputStream outForOneCall = fs.create(smallFile);
+    statistics.reset();
+    //Writing "test" 1 time
+    outForOneCall
+        .write(testWriteOps.getBytes(), 0, testWriteOps.getBytes().length);
+
+    //Test for one write operation
+    Assert.assertEquals(1, statistics.getWriteOps());
+
+    outForOneCall.close();
+    //validating Content of file
+    Assert.assertEquals(true, validateContent(fs, smallFile,
+        testWriteOps.getBytes()));
+
+    String largeFileValidationString = "";
+    FSDataOutputStream outForLargeCalls = fs.create(largeFile);
+    statistics.reset();
+    //Writing "test" 1000 times
+    for (int i = 0; i < 1000; i++) {
+      outForLargeCalls.write(testWriteOps.getBytes(), 0,
+          testWriteOps.getBytes().length);
+
+      //Creating Validation string of "test" 1000 times
+      largeFileValidationString += testWriteOps;
 
 Review comment:
   Will be using StringBuilder here and use append() istead of += for String concatenation.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r388857077
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStream.java
 ##########
 @@ -0,0 +1,81 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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 org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream;
+
+/**
+ * Test Abfs Input Stream.
+ */
+
+public class ITestAbfsInputStream extends AbstractAbfsIntegrationTest {
+  public ITestAbfsInputStream() throws Exception {
+  }
+
+  /***
+   * {@link AbfsInputStream#incrementReadOps()}
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testAbfsInputStreamReadOps() throws Exception {
+    describe("Test to see correct population of Read operations in Abfs");
+
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path smallFile = new Path("testOneReadCall");
+    Path largeFile = new Path("testLargeReadCalls");
+    FileSystem.Statistics statistics = fs.getFsStatistics();
+    String testReadOps = "test this";
+    statistics.reset();
+
+    //Test for zero read operation
+    Assert.assertEquals(0, statistics.getReadOps());
+
+    FSDataOutputStream outForOneCall = fs.create(smallFile);
+    statistics.reset();
+    outForOneCall.write(testReadOps.getBytes());
+    FSDataInputStream inForOneCall = fs.open(smallFile);
+    inForOneCall.read(testReadOps.getBytes(), 0, testReadOps.getBytes().length);
+
+    //Test for one read operation
+    Assert.assertEquals(1, statistics.getReadOps());
+
+    FSDataOutputStream outForLargeCalls = fs.create(largeFile);
 
 Review comment:
   does this really create large reads? it's still only small amounts of data

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mehakmeet commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r393214188
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStreamStatistics.java
 ##########
 @@ -0,0 +1,111 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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 org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream;
+
+/**
+ * Test Abfs Stream.
+ */
+
+public class ITestAbfsStreamStatistics extends AbstractAbfsIntegrationTest {
+  public ITestAbfsStreamStatistics() throws Exception {
+  }
+
+  /***
+   * {@link AbfsInputStream#incrementReadOps()}.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testAbfsStreamOps() throws Exception {
+    describe("Test to see correct population of read and write operations in "
+        + "Abfs");
+
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path smallOperationsFile = new Path("testOneReadWriteOps");
+    Path largeOperationsFile = new Path("testLargeReadWriteOps");
+    FileSystem.Statistics statistics = fs.getFsStatistics();
+    String testReadWriteOps = "test this";
+    statistics.reset();
+
+    //Test for zero read and write operation
+    Assert.assertEquals("Mismatch in read operations", 0,
+        statistics.getReadOps());
+    Assert.assertEquals("Mismatch in write operations", 0,
+        statistics.getWriteOps());
+
+    FSDataOutputStream outForOneOperation = fs.create(smallOperationsFile);
+    statistics.reset();
+    outForOneOperation.write(testReadWriteOps.getBytes());
+    FSDataInputStream inForOneCall = fs.open(smallOperationsFile);
+    inForOneCall.read(testReadWriteOps.getBytes(), 0,
+        testReadWriteOps.getBytes().length);
+
+    //Test for one read and write operation
 
 Review comment:
   what would be the use of "operation" parameter be here ? Is this for Assert message? If yes, wouldn't assert message of both Read and Write Ops be same?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] hadoop-yetus removed a comment on issue #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
hadoop-yetus removed a comment on issue #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#issuecomment-595688964
 
 
   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |  25m 39s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 4 new or modified test files.  |
   ||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  19m  4s |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 31s |  trunk passed  |
   | +1 :green_heart: |  checkstyle  |   0m 25s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 34s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  15m  4s |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 26s |  trunk passed  |
   | +0 :ok: |  spotbugs  |   0m 51s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 49s |  trunk passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 27s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 24s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 24s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 17s |  hadoop-tools/hadoop-azure: The patch generated 2 new + 1 unchanged - 0 fixed = 3 total (was 1)  |
   | +1 :green_heart: |  mvnsite  |   0m 26s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  13m 42s |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 23s |  the patch passed  |
   | +1 :green_heart: |  findbugs  |   0m 53s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 20s |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 32s |  The patch does not generate ASF License warnings.  |
   |  |   |  82m 33s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.7 Server=19.03.7 base: https://builds.apache.org/job/hadoop-multibranch/job/PR-1881/1/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/1881 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux 6db0065e9119 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | personality/hadoop.sh |
   | git revision | trunk / 004e955 |
   | Default Java | 1.8.0_242 |
   | checkstyle | https://builds.apache.org/job/hadoop-multibranch/job/PR-1881/1/artifact/out/diff-checkstyle-hadoop-tools_hadoop-azure.txt |
   |  Test Results | https://builds.apache.org/job/hadoop-multibranch/job/PR-1881/1/testReport/ |
   | Max. process+thread count | 422 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://builds.apache.org/job/hadoop-multibranch/job/PR-1881/1/console |
   | versions | git=2.7.4 maven=3.3.9 findbugs=3.1.0-RC1 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mehakmeet commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r388940913
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsTestWithTimeout.java
 ##########
 @@ -67,4 +77,42 @@ public void nameThread() {
   protected int getTestTimeoutMillis() {
     return TEST_TIMEOUT;
   }
+
+  /**
+   * Describe a test in the logs.
+   *
+   * @param text text to print
+   * @param args arguments to format in the printing
+   */
+  protected void describe(String text, Object... args) {
+    LOG.info("\n\n{}: {}\n",
+        methodName.getMethodName(),
+        String.format(text, args));
+  }
+
+  /**
+   * Validate Contents written on a file in Abfs
+   *
+   * @param fs AzureBlobFileSystem
+   * @param path Path of the file
+   * @param originalByteArray original byte array
+   * @return
+   * @throws IOException
+   */
+  protected boolean validateContent(AzureBlobFileSystem fs, Path path,
+      byte[] originalByteArray)
+      throws IOException {
+    FSDataInputStream in = fs.open(path);
+    byte[] contentByteArray = new byte[originalByteArray.length];
+    int seekPos = 0;
+    while (in.read() != -1) {
 
 Review comment:
   I will still need seekPos for indexing of contentByteArray..
   seek() is however an extra method call, I can compare byte by byte with originalByteArray ?
   Something like this ,maybe ?
   
   ` while (valueOfContentAtPos != -1 && pos < lenOfOriginalByteArray) {
         if (originalByteArray[pos] != valueOfContentAtPos)
           return false;
         valueOfContentAtPos = (byte) in.read();
         pos++;
       }
       if (valueOfContentAtPos != -1)
         return false;
       return true;`

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mehakmeet edited a comment on issue #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet edited a comment on issue #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#issuecomment-595684938
 
 
   test run : mvn -T 1C -Dparallel-tests=abfs clean verify
   Region : East US, West US
   
   Failed tests(Container configurations based) :
   ```
   [ERROR] Failures: [ERROR] ITestGetNameSpaceEnabled.testNonXNSAccount:59->Assert.assertFalse:64->Assert.assertTrue:41->Assert.fail:88 Expecting getIsNamespaceEnabled() return false
   [ERROR] Errors: [ERROR] ITestGetNameSpaceEnabled.testFailedRequestWhenCredentialsNotCorrect:91->AbstractAbfsIntegrationTest.getFileSystem:197 ? InvalidConfigurationValue

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mehakmeet closed pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet closed pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881
 
 
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mehakmeet commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r391786077
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStreamStatistics.java
 ##########
 @@ -0,0 +1,111 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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 org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream;
+
+/**
+ * Test Abfs Stream.
+ */
+
+public class ITestAbfsStreamStatistics extends AbstractAbfsIntegrationTest {
+  public ITestAbfsStreamStatistics() throws Exception {
+  }
+
+  /***
+   * {@link AbfsInputStream#incrementReadOps()}.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testAbfsStreamOps() throws Exception {
+    describe("Test to see correct population of read and write operations in "
+        + "Abfs");
+
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path smallOperaionsFile = new Path("testOneReadWriteOps");
 
 Review comment:
   just saw this typo. will fix it in the next commit.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r392956693
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsTestWithTimeout.java
 ##########
 @@ -67,4 +77,46 @@ public void nameThread() {
   protected int getTestTimeoutMillis() {
     return TEST_TIMEOUT;
   }
+
+  /**
+   * Describe a test in the logs.
+   *
+   * @param text text to print
+   * @param args arguments to format in the printing
+   */
+  protected void describe(String text, Object... args) {
+    LOG.info("\n\n{}: {}\n",
+        methodName.getMethodName(),
+        String.format(text, args));
+  }
+
+  /**
+   * Validate Contents written on a file in Abfs.
+   *
+   * @param fs                AzureBlobFileSystem
+   * @param path              Path of the file
+   * @param originalByteArray original byte array
+   * @return
 
 Review comment:
   nit: Say what you return

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] steveloughran commented on issue #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on issue #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#issuecomment-602601426
 
 
   +1, merged to trunk. thanks!

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mehakmeet commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r389364668
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java
 ##########
 @@ -59,6 +62,59 @@ public void testEnsureFileCreatedImmediately() throws Exception {
     assertIsFile(fs, TEST_FILE_PATH);
   }
 
+  /**
+   * {@link AbfsOutputStream#incrementWriteOps()}
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testWriteOpsMetric() throws Exception {
 
 Review comment:
   I'll remove this test and add write operations tests in testAbfsInputStreamReadOps, or should I keep this separate for write operations and validating ?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mehakmeet commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r389370089
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStream.java
 ##########
 @@ -0,0 +1,81 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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 org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream;
+
+/**
+ * Test Abfs Input Stream.
+ */
+
+public class ITestAbfsInputStream extends AbstractAbfsIntegrationTest {
+  public ITestAbfsInputStream() throws Exception {
+  }
+
+  /***
+   * {@link AbfsInputStream#incrementReadOps()}
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testAbfsInputStreamReadOps() throws Exception {
+    describe("Test to see correct population of Read operations in Abfs");
+
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path smallFile = new Path("testOneReadCall");
+    Path largeFile = new Path("testLargeReadCalls");
+    FileSystem.Statistics statistics = fs.getFsStatistics();
+    String testReadOps = "test this";
+    statistics.reset();
+
+    //Test for zero read operation
+    Assert.assertEquals(0, statistics.getReadOps());
+
+    FSDataOutputStream outForOneCall = fs.create(smallFile);
+    statistics.reset();
+    outForOneCall.write(testReadOps.getBytes());
+    FSDataInputStream inForOneCall = fs.open(smallFile);
+    inForOneCall.read(testReadOps.getBytes(), 0, testReadOps.getBytes().length);
+
+    //Test for one read operation
+    Assert.assertEquals(1, statistics.getReadOps());
+
+    FSDataOutputStream outForLargeCalls = fs.create(largeFile);
 
 Review comment:
   meant to show Large number of operations rather than large read I think. I'll change the name for better understanding. still 1000 ops does seem small. I'll increase to 10^6 read ops then ?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mehakmeet commented on issue #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on issue #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#issuecomment-595684938
 
 
   test run : mvn -T 1C -Dparallel-tests=abfs clean verify
   Region : East US, West US
   
   Failed tests(Container configurations based) :
   `[ERROR] Failures: [ERROR] ITestGetNameSpaceEnabled.testNonXNSAccount:59->Assert.assertFalse:64->Assert.assertTrue:41->Assert.fail:88 Expecting getIsNamespaceEnabled() return false
   [ERROR] Errors: [ERROR] ITestGetNameSpaceEnabled.testFailedRequestWhenCredentialsNotCorrect:91->AbstractAbfsIntegrationTest.getFileSystem:197 ? InvalidConfigurationValue`

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] aajisaka commented on issue #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
aajisaka commented on issue #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#issuecomment-603068418
 
 
   Hi @mehakmeet and @steveloughran 
   Failed to compile hadoop-azure after the merge. Would you fix the test as well?
   ```
   [ERROR] Failed to execute goal org.apache.maven.plugins:maven-compiler-plugin:3.1:testCompile (default-testCompile) on project hadoop-azure: Compilation failure: Compilation failure: 
   [ERROR] /Users/aajisaka/git/ghe.corp/hadoop-mirror/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java:[71,28] constructor AbfsOutputStream in class org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream cannot be applied to given types;
   [ERROR]   required: org.apache.hadoop.fs.azurebfs.services.AbfsClient,org.apache.hadoop.fs.FileSystem.Statistics,java.lang.String,long,int,boolean,boolean,boolean,boolean
   [ERROR]   found:    org.apache.hadoop.fs.azurebfs.services.AbfsClient,java.lang.String,int,int,boolean,boolean,boolean,boolean
   [ERROR]   reason: actual and formal argument lists differ in length
   [ERROR] /Users/aajisaka/git/ghe.corp/hadoop-mirror/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java:[121,28] constructor AbfsOutputStream in class org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream cannot be applied to given types;
   [ERROR]   required: org.apache.hadoop.fs.azurebfs.services.AbfsClient,org.apache.hadoop.fs.FileSystem.Statistics,java.lang.String,long,int,boolean,boolean,boolean,boolean
   [ERROR]   found:    org.apache.hadoop.fs.azurebfs.services.AbfsClient,java.lang.String,int,int,boolean,boolean,boolean,boolean
   [ERROR]   reason: actual and formal argument lists differ in length
   [ERROR] /Users/aajisaka/git/ghe.corp/hadoop-mirror/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java:[168,28] constructor AbfsOutputStream in class org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream cannot be applied to given types;
   [ERROR]   required: org.apache.hadoop.fs.azurebfs.services.AbfsClient,org.apache.hadoop.fs.FileSystem.Statistics,java.lang.String,long,int,boolean,boolean,boolean,boolean
   [ERROR]   found:    org.apache.hadoop.fs.azurebfs.services.AbfsClient,java.lang.String,int,int,boolean,boolean,boolean,boolean
   [ERROR]   reason: actual and formal argument lists differ in length
   [ERROR] /Users/aajisaka/git/ghe.corp/hadoop-mirror/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java:[226,28] constructor AbfsOutputStream in class org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream cannot be applied to given types;
   [ERROR]   required: org.apache.hadoop.fs.azurebfs.services.AbfsClient,org.apache.hadoop.fs.FileSystem.Statistics,java.lang.String,long,int,boolean,boolean,boolean,boolean
   [ERROR]   found:    org.apache.hadoop.fs.azurebfs.services.AbfsClient,java.lang.String,int,int,boolean,boolean,boolean,boolean
   [ERROR]   reason: actual and formal argument lists differ in length
   [ERROR] /Users/aajisaka/git/ghe.corp/hadoop-mirror/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java:[273,28] constructor AbfsOutputStream in class org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream cannot be applied to given types;
   [ERROR]   required: org.apache.hadoop.fs.azurebfs.services.AbfsClient,org.apache.hadoop.fs.FileSystem.Statistics,java.lang.String,long,int,boolean,boolean,boolean,boolean
   [ERROR]   found:    org.apache.hadoop.fs.azurebfs.services.AbfsClient,java.lang.String,int,int,boolean,boolean,boolean,boolean
   [ERROR]   reason: actual and formal argument lists differ in length
   [ERROR] /Users/aajisaka/git/ghe.corp/hadoop-mirror/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java:[320,28] constructor AbfsOutputStream in class org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream cannot be applied to given types;
   [ERROR]   required: org.apache.hadoop.fs.azurebfs.services.AbfsClient,org.apache.hadoop.fs.FileSystem.Statistics,java.lang.String,long,int,boolean,boolean,boolean,boolean
   [ERROR]   found:    org.apache.hadoop.fs.azurebfs.services.AbfsClient,java.lang.String,int,int,boolean,boolean,boolean,boolean
   [ERROR]   reason: actual and formal argument lists differ in length
   [ERROR] /Users/aajisaka/git/ghe.corp/hadoop-mirror/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java:[377,28] constructor AbfsOutputStream in class org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream cannot be applied to given types;
   [ERROR]   required: org.apache.hadoop.fs.azurebfs.services.AbfsClient,org.apache.hadoop.fs.FileSystem.Statistics,java.lang.String,long,int,boolean,boolean,boolean,boolean
   [ERROR]   found:    org.apache.hadoop.fs.azurebfs.services.AbfsClient,java.lang.String,int,int,boolean,boolean,boolean,boolean
   [ERROR]   reason: actual and formal argument lists differ in length
   ```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r392958482
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java
 ##########
 @@ -252,6 +255,13 @@ int readRemote(long position, byte[] b, int offset, int length) throws IOExcepti
     return (int) bytesRead;
   }
 
+  /**
+   * Increment Read Operations.
+   */
+  public void incrementReadOps() {
+    statistics.incrementReadOps(1);
 
 Review comment:
   skip if statistics == null

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mukund-thakur commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r394121291
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStreamStatistics.java
 ##########
 @@ -46,66 +46,89 @@ public void testAbfsStreamOps() throws Exception {
         + "Abfs");
 
     final AzureBlobFileSystem fs = getFileSystem();
-    Path smallOperaionsFile = new Path("testOneReadWriteOps");
+    Path smallOperationsFile = new Path("testOneReadWriteOps");
     Path largeOperationsFile = new Path("testLargeReadWriteOps");
     FileSystem.Statistics statistics = fs.getFsStatistics();
     String testReadWriteOps = "test this";
     statistics.reset();
 
     //Test for zero read and write operation
-    Assert.assertEquals("Mismatch in read operations", 0,
-        statistics.getReadOps());
-    Assert.assertEquals("Mismatch in write operations", 0,
-        statistics.getWriteOps());
-
-    FSDataOutputStream outForOneOperation = fs.create(smallOperaionsFile);
-    statistics.reset();
-    outForOneOperation.write(testReadWriteOps.getBytes());
-    FSDataInputStream inForOneCall = fs.open(smallOperaionsFile);
-    inForOneCall.read(testReadWriteOps.getBytes(), 0,
-        testReadWriteOps.getBytes().length);
-
-    //Test for one read and write operation
-    Assert.assertEquals("Mismatch in read operations", 1,
-        statistics.getReadOps());
-    Assert.assertEquals("Mismatch in write operations", 1,
-        statistics.getWriteOps());
-
-    outForOneOperation.close();
-    //Validating if Content is being written in the smallFile
-    Assert.assertEquals("Mismatch in content validation", true,
-        validateContent(fs, smallOperaionsFile,
+    assertReadWriteOps(0, statistics);
+
+    FSDataOutputStream outForOneOperation = null;
+    FSDataInputStream inForOneOperation = null;
+    try {
+      outForOneOperation = fs.create(smallOperationsFile);
+      statistics.reset();
+      outForOneOperation.write(testReadWriteOps.getBytes());
+      inForOneOperation = fs.open(smallOperationsFile);
+      inForOneOperation.read(testReadWriteOps.getBytes(), 0,
+          testReadWriteOps.getBytes().length);
+
+      //Test for one read and write operation
+      assertReadWriteOps(1, statistics);
+    } finally {
+      if (inForOneOperation != null) {
+        inForOneOperation.close();
+      }
+      if (outForOneOperation != null) {
+        outForOneOperation.close();
+      }
+    }
+    //Validating if content is being written in the smallOperationsFile
+    Assert.assertTrue("Mismatch in content validation",
+        validateContent(fs, smallOperationsFile,
             testReadWriteOps.getBytes()));
 
-    FSDataOutputStream outForLargeOperations = fs.create(largeOperationsFile);
-    statistics.reset();
-
+    FSDataOutputStream outForLargeOperations = null;
+    FSDataInputStream inForLargeOperations = null;
     StringBuilder largeOperationsValidationString = new StringBuilder();
-    for (int i = 0; i < 1000000; i++) {
-      outForLargeOperations.write(testReadWriteOps.getBytes());
-
-      //Creating the String for content Validation
-      largeOperationsValidationString.append(testReadWriteOps);
+    try {
+      outForLargeOperations = fs.create(largeOperationsFile);
+      statistics.reset();
+      int largeValue = 1000000;
+      for (int i = 0; i < largeValue; i++) {
+        outForLargeOperations.write(testReadWriteOps.getBytes());
+
+        //Creating the String for content Validation
+        largeOperationsValidationString.append(testReadWriteOps);
+      }
+
+      inForLargeOperations = fs.open(largeOperationsFile);
+      for (int i = 0; i < largeValue; i++)
+        inForLargeOperations
+            .read(testReadWriteOps.getBytes(), 0,
+                testReadWriteOps.getBytes().length);
+
+      //Test for one million read and write operations
+      assertReadWriteOps(largeValue, statistics);
+    } finally {
+      if (inForLargeOperations != null) {
+        inForLargeOperations.close();
+      }
+      if (outForLargeOperations != null) {
+        outForLargeOperations.close();
+      }
     }
 
-    FSDataInputStream inForLargeCalls = fs.open(largeOperationsFile);
+    //Validating if content is being written in largeOperationsFile
+    Assert.assertTrue("Mismatch in content validation",
+        validateContent(fs, largeOperationsFile,
+            largeOperationsValidationString.toString().getBytes()));
 
-    for (int i = 0; i < 1000000; i++)
-      inForLargeCalls
-          .read(testReadWriteOps.getBytes(), 0,
-              testReadWriteOps.getBytes().length);
+  }
 
-    //Test for one million read and write operations
-    Assert.assertEquals("Mismatch in read operations", 1000000,
+  /**
+   * Method for Read and Write Ops Assertion.
+   *
+   * @param expectedReadWriteOps Expected Value
+   * @param statistics           fs stats to get Actual Values
+   */
+  private void assertReadWriteOps(long expectedReadWriteOps,
 
 Review comment:
   This is wrong. How can you compare the expected value with both read and write value? What if read and write values are different?
    assertReadWriteOps(operation, expected, actual) is what we discussed right?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mehakmeet commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r389370089
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStream.java
 ##########
 @@ -0,0 +1,81 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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 org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream;
+
+/**
+ * Test Abfs Input Stream.
+ */
+
+public class ITestAbfsInputStream extends AbstractAbfsIntegrationTest {
+  public ITestAbfsInputStream() throws Exception {
+  }
+
+  /***
+   * {@link AbfsInputStream#incrementReadOps()}
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testAbfsInputStreamReadOps() throws Exception {
+    describe("Test to see correct population of Read operations in Abfs");
+
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path smallFile = new Path("testOneReadCall");
+    Path largeFile = new Path("testLargeReadCalls");
+    FileSystem.Statistics statistics = fs.getFsStatistics();
+    String testReadOps = "test this";
+    statistics.reset();
+
+    //Test for zero read operation
+    Assert.assertEquals(0, statistics.getReadOps());
+
+    FSDataOutputStream outForOneCall = fs.create(smallFile);
+    statistics.reset();
+    outForOneCall.write(testReadOps.getBytes());
+    FSDataInputStream inForOneCall = fs.open(smallFile);
+    inForOneCall.read(testReadOps.getBytes(), 0, testReadOps.getBytes().length);
+
+    //Test for one read operation
+    Assert.assertEquals(1, statistics.getReadOps());
+
+    FSDataOutputStream outForLargeCalls = fs.create(largeFile);
 
 Review comment:
   meant to show Large number of operations rather than large read I think. I'll change the name for better understanding. still 1000 ops does seem small. I'll increase to 10^6 read and write ops then ?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mehakmeet commented on issue #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on issue #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#issuecomment-595650634
 
 
   Tested on Azure account : mehakmeetdata
   Tested by : mvn -T 1C -Dparallel-tests=abfs clean verify

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mehakmeet opened a new pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet opened a new pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881
 
 
   - Write_ops
   - Read_ops
   - Bytes_written (already updated)
   - Bytes_Read (already updated)
   
   Change-Id: I77349fdd158babd66df665713201fa9c8606f191
   
   ## NOTICE
   
   Please create an issue in ASF JIRA before opening a pull request,
   and you need to set the title of the pull request which starts with
   the corresponding JIRA issue number. (e.g. HADOOP-XXXXX. Fix a typo in YYY.)
   For more details, please see https://cwiki.apache.org/confluence/display/HADOOP/How+To+Contribute
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mehakmeet commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r390494255
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java
 ##########
 @@ -59,6 +62,59 @@ public void testEnsureFileCreatedImmediately() throws Exception {
     assertIsFile(fs, TEST_FILE_PATH);
   }
 
+  /**
+   * {@link AbfsOutputStream#incrementWriteOps()}
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testWriteOpsMetric() throws Exception {
+    describe("Test to see correct population of write operations in Abfs");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path smallFile = new Path("testOneCall");
+    Path largeFile = new Path("testLargeCalls");
+    String testWriteOps = "test";
+    FileSystem.Statistics statistics = fs.getFsStatistics();
+    statistics.reset();
+
+    //Test for zero write operation
+    Assert.assertEquals(0, statistics.getWriteOps());
+
+    FSDataOutputStream outForOneCall = fs.create(smallFile);
+    statistics.reset();
+    //Writing "test" 1 time
+    outForOneCall
+        .write(testWriteOps.getBytes(), 0, testWriteOps.getBytes().length);
+
+    //Test for one write operation
+    Assert.assertEquals(1, statistics.getWriteOps());
 
 Review comment:
   Noted.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mukund-thakur commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r389612155
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsTestWithTimeout.java
 ##########
 @@ -67,4 +77,42 @@ public void nameThread() {
   protected int getTestTimeoutMillis() {
     return TEST_TIMEOUT;
   }
+
+  /**
+   * Describe a test in the logs.
+   *
+   * @param text text to print
+   * @param args arguments to format in the printing
+   */
+  protected void describe(String text, Object... args) {
+    LOG.info("\n\n{}: {}\n",
+        methodName.getMethodName(),
+        String.format(text, args));
+  }
+
+  /**
+   * Validate Contents written on a file in Abfs
+   *
+   * @param fs AzureBlobFileSystem
+   * @param path Path of the file
+   * @param originalByteArray original byte array
+   * @return
+   * @throws IOException
+   */
+  protected boolean validateContent(AzureBlobFileSystem fs, Path path,
+      byte[] originalByteArray)
+      throws IOException {
+    FSDataInputStream in = fs.open(path);
+    byte[] contentByteArray = new byte[originalByteArray.length];
+    int seekPos = 0;
+    while (in.read() != -1) {
 
 Review comment:
   Yes this will be a better approach as it will take less time and space. Also as Steve suggested you don't need seek() here as read() maintains its own pointer. Also seek() is a very slow operation. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r392957031
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStreamStatistics.java
 ##########
 @@ -0,0 +1,111 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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 org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream;
+
+/**
+ * Test Abfs Stream.
+ */
+
+public class ITestAbfsStreamStatistics extends AbstractAbfsIntegrationTest {
+  public ITestAbfsStreamStatistics() throws Exception {
+  }
+
+  /***
+   * {@link AbfsInputStream#incrementReadOps()}.
 
 Review comment:
   just makes @code 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] mukund-thakur commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r394122012
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsTestWithTimeout.java
 ##########
 @@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
 
 Review comment:
   remove

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r388855031
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsTestWithTimeout.java
 ##########
 @@ -67,4 +77,42 @@ public void nameThread() {
   protected int getTestTimeoutMillis() {
     return TEST_TIMEOUT;
   }
+
+  /**
+   * Describe a test in the logs.
+   *
+   * @param text text to print
+   * @param args arguments to format in the printing
+   */
+  protected void describe(String text, Object... args) {
+    LOG.info("\n\n{}: {}\n",
+        methodName.getMethodName(),
+        String.format(text, args));
+  }
+
+  /**
+   * Validate Contents written on a file in Abfs
 
 Review comment:
   nit: add a trailing . to keep javadoc happy

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [hadoop] steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1881: HADOOP-16910 Adding file system counters in ABFS
URL: https://github.com/apache/hadoop/pull/1881#discussion_r388854876
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsTestWithTimeout.java
 ##########
 @@ -17,12 +17,19 @@
  */
 package org.apache.hadoop.fs.azurebfs;
 
+import org.apache.hadoop.fs.FSDataInputStream;
 
 Review comment:
   nit: should be in its own block after the non org-apache-hadoop imports

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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