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/17 11:38:48 UTC

[GitHub] [hadoop] mehakmeet opened a new pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

mehakmeet opened a new pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899
 
 
   Change-Id: Ie976f23c6c3e2f5cf9167794357cfc669a232c80
   
   ## 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] hadoop-yetus commented on issue #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on issue #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#issuecomment-613454620
 
 
   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 42s |  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 59s |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 32s |  trunk passed  |
   | +1 :green_heart: |  checkstyle  |   0m 24s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 34s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  15m  1s |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 27s |  trunk passed  |
   | +0 :ok: |  spotbugs  |   0m 53s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 51s |  trunk passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 27s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 23s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 23s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 16s |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 27s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  13m 50s |  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 22s |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 32s |  The patch does not generate ASF License warnings.  |
   |  |   |  58m 46s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.8 Server=19.03.8 base: https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/8/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/1899 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux 5a1f873cb36e 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 / aeeebc5 |
   | Default Java | 1.8.0_242 |
   |  Test Results | https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/8/testReport/ |
   | Max. process+thread count | 419 (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-1899/8/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 commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r405022610
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
 ##########
 @@ -279,6 +283,7 @@ public synchronized void close() throws IOException {
         threadExecutor.shutdownNow();
       }
     }
+    LOG.debug("Closing AbfsOutputStream ", toString());
 
 Review comment:
   let's guard this with a LOG.isDebugEnabled(), because that toString() operation is doing enough work. Or, use `this` as the argument and have SLF4J Call this.toString() only if it is printing the log entry

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r408178498
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsOutputStreamStatistics.java
 ##########
 @@ -0,0 +1,120 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+import java.util.Random;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl;
+
+/**
+ * Unit Tests for AbfsOutputStream Statistics.
+ */
+public class TestAbfsOutputStreamStatistics
+    extends AbstractAbfsIntegrationTest {
+
+  private static final int LOW_RANGE_FOR_RANDOM_VALUE = 49;
+  private static final int HIGH_RANGE_FOR_RANDOM_VALUE = 9999;
+
+  public TestAbfsOutputStreamStatistics() throws Exception {
+  }
+
+  /**
+   * Tests to check bytes failed to Upload in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamBytesFailed() {
+    describe("Testing Bytes Failed during uploading in AbfsOutputSteam");
+
+    AbfsOutputStreamStatisticsImpl abfsOutputStreamStatistics =
+        new AbfsOutputStreamStatisticsImpl();
+
+    //Test for zero bytes uploaded.
+    assertValues("number fo bytes failed to upload", 0,
+        abfsOutputStreamStatistics.getBytesUploadFailed());
+
+    //Populating small random value for bytesFailed.
+    int randomBytesFailed = new Random().nextInt(LOW_RANGE_FOR_RANDOM_VALUE);
+    abfsOutputStreamStatistics.uploadFailed(randomBytesFailed);
+    //Test for bytes failed to upload.
+    assertValues("number fo bytes failed to upload", randomBytesFailed,
+        abfsOutputStreamStatistics.getBytesUploadFailed());
+
+    //Initializing again to reset the statistics.
+    abfsOutputStreamStatistics = new AbfsOutputStreamStatisticsImpl();
+
+    //Populating large random values for bytesFailed.
+    randomBytesFailed = new Random().nextInt(HIGH_RANGE_FOR_RANDOM_VALUE);
+    abfsOutputStreamStatistics.uploadFailed(randomBytesFailed);
+    //Test for bytes failed to upload.
+    assertValues("number fo bytes failed to upload", randomBytesFailed,
+        abfsOutputStreamStatistics.getBytesUploadFailed());
+  }
+
+  /**
+   * Tests to check time spent on waiting for tasks to be complete on a
+   * blocking queue in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamTimeSpentOnWaitTask() {
+    describe("Testing Time Spend on Waiting for Task to be complete");
+
+    AbfsOutputStreamStatisticsImpl abfsOutputStreamStatistics =
+        new AbfsOutputStreamStatisticsImpl();
+
+    //Test for initial value of timeSpentWaitTask.
+    assertValues("Time spend on waiting for tasks to complete", 0,
+        abfsOutputStreamStatistics.getTimeSpendOnTaskWait());
+
+    int smallRandomStartTime =
+        new Random().nextInt(LOW_RANGE_FOR_RANDOM_VALUE);
+    int smallRandomEndTime =
+        new Random().nextInt(LOW_RANGE_FOR_RANDOM_VALUE)
+            + smallRandomStartTime;
+    int smallDiff = smallRandomEndTime - smallRandomStartTime;
+    abfsOutputStreamStatistics
+        .timeSpentTaskWait(smallRandomStartTime, smallRandomEndTime);
+    //Test for small random value of timeSpentWaitTask.
+    assertValues("Time spend on waiting for tasks to complete", smallDiff,
+        abfsOutputStreamStatistics.getTimeSpendOnTaskWait());
+
+    int largeRandomStartTime =
+        new Random().nextInt(HIGH_RANGE_FOR_RANDOM_VALUE);
+    int largeRandomEndTime = new Random().nextInt(HIGH_RANGE_FOR_RANDOM_VALUE)
+        + largeRandomStartTime;
+    int randomDiff = largeRandomEndTime - largeRandomStartTime;
+    abfsOutputStreamStatistics
+        .timeSpentTaskWait(largeRandomStartTime, largeRandomEndTime);
+      /*
+      Test for large random value of timeSpentWaitTask plus the time spent
+      waiting in previous test.
+       */
+    assertValues("Time spend on waiting for tasks to complete",
+        smallDiff + randomDiff,
+        abfsOutputStreamStatistics.getTimeSpendOnTaskWait());
+  }
+
+}
 
 Review comment:
   nit: newline

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r404604989
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStream.java
 ##########
 @@ -0,0 +1,294 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+import java.util.Random;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl;
+import org.apache.hadoop.fs.permission.FsPermission;
+
+/**
+ * Test AbfsOutputStream statistics.
+ */
+public class ITestAbfsOutputStream extends AbstractAbfsIntegrationTest {
+  public ITestAbfsOutputStream() throws Exception {
+  }
+
+  private static final int LARGE_OPERATIONS = 10;
+  private static final int LOW_RANGE_FOR_RANDOM_VALUE = 49;
+  private static final int HIGH_RANGE_FOR_RANDOM_VALUE = 9999;
+
+  /**
+   * Tests to check bytes Uploading in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamUploadingBytes() throws IOException {
+    describe("Testing Bytes uploaded in AbfsOutputSteam");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path uploadBytesFilePath = new Path("AbfsOutputStreamStatsPath");
+    String testBytesToUpload = "bytes";
+
+    try (AbfsOutputStream outForSomeBytes = createAbfsOutputStream(fs,
+        uploadBytesFilePath)
+    ) {
+
+      //Test for zero bytes To upload
+      assertValues("bytes to upload", 0,
+          outForSomeBytes.getOutputStreamStatistics().getBytesToUpload());
+
+      outForSomeBytes.write(testBytesToUpload.getBytes());
+      outForSomeBytes.flush();
+      AbfsOutputStreamStatisticsImpl abfsOutputStreamStatistics =
+          outForSomeBytes.getOutputStreamStatistics();
+
+      //Test for bytes to upload
+      assertValues("bytes to upload", testBytesToUpload.getBytes().length,
+          abfsOutputStreamStatistics.getBytesToUpload());
+
+      //Test for successful bytes uploaded
+      assertValues("successful bytes uploaded",
+          testBytesToUpload.getBytes().length,
+          abfsOutputStreamStatistics.getBytesUploadSuccessful());
+
+      //Populating random value for bytesFailed
+      int randomBytesFailed = new Random().nextInt(LOW_RANGE_FOR_RANDOM_VALUE);
+      abfsOutputStreamStatistics.uploadFailed(randomBytesFailed);
+      //Test for bytes failed to upload
+      assertValues("number fo bytes failed to upload", randomBytesFailed,
+          abfsOutputStreamStatistics.getBytesUploadFailed());
+
+    }
+
+    try (AbfsOutputStream outForLargeBytes = createAbfsOutputStream(fs,
+        uploadBytesFilePath)) {
+
+      int largeValue = LARGE_OPERATIONS;
 
 Review comment:
   Use LARGE_OPERATIONS directly. Why to create a new variable.

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r408174898
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStreamStatistics.java
 ##########
 @@ -0,0 +1,233 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl;
+
+/**
+ * Test AbfsOutputStream statistics.
+ */
+public class ITestAbfsOutputStreamStatistics
+    extends AbstractAbfsIntegrationTest {
+  private static final int LARGE_OPERATIONS = 10;
+
+  public ITestAbfsOutputStreamStatistics() throws Exception {
+  }
+
+  /**
+   * Tests to check bytes Uploaded successfully in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamUploadingBytes() throws IOException {
+    describe("Testing Bytes uploaded successfully in AbfsOutputSteam");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path uploadBytesFilePath = path(getMethodName());
+    String testBytesToUpload = "bytes";
+
+    try (
+        AbfsOutputStream outForSomeBytes = createAbfsOutputStreamWithFlushEnabled(
+            fs,
+            uploadBytesFilePath)
+    ) {
 
 Review comment:
   nit: pull up to previous line

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r404563779
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStream.java
 ##########
 @@ -0,0 +1,294 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+import java.util.Random;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl;
+import org.apache.hadoop.fs.permission.FsPermission;
+
+/**
+ * Test AbfsOutputStream statistics.
+ */
+public class ITestAbfsOutputStream extends AbstractAbfsIntegrationTest {
+  public ITestAbfsOutputStream() throws Exception {
 
 Review comment:
   Change name to ITestAbfsOutputStreamStatictics.

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r402076584
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamStatisticsImpl.java
 ##########
 @@ -0,0 +1,148 @@
+/**
+ * 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.services;
+
+/**
+ * OutputStream Statistics Implementation for Abfs.
+ * timeSpendOnTaskWait - Time spend on waiting for tasks to be complete on
+ * Blocking Queue in AbfsOutputStream.
+ *
+ * queueShrink - Number of times Blocking Queue was shrunk after writing
+ * data.
+ *
+ * WriteCurrentBufferOperations - Number of times
+ * {@code writeCurrentBufferToService()} calls were made.
+ */
+public class AbfsOutputStreamStatisticsImpl
+    implements AbfsOutputStreamStatistics {
+  private volatile long bytesToUpload;
+  private volatile long bytesUploadSuccessful;
+  private volatile long bytesUploadFailed;
+  private volatile long timeSpendOnTaskWait;
+  private volatile long queueShrink;
+  private volatile long writeCurrentBufferOperations;
+
+  /**
+   * Number of bytes uploaded.
+   *
+   * @param bytes negative values are ignored
+   */
+  @Override
+  public void bytesToUpload(long bytes) {
+    if (bytes > 0) {
+      bytesToUpload += bytes;
+    }
+  }
+
+  /**
+   * Upload successful with the number of bytes.
+   * @param bytes number of bytes that were successfully uploaded
+   */
+  @Override
+  public void uploadSuccessful(long bytes) {
+    if (bytes > 0) {
+      bytesUploadSuccessful += bytes;
+    }
+  }
+
+  /**
+   * Upload failed and the number of bytes.
+   *
+   * @param bytes negative values are ignored
+   */
+  @Override
+  public void uploadFailed(long bytes) {
+    if (bytes > 0) {
+      bytesUploadFailed += bytes;
+    }
+  }
+
+  /**
+   * Time spent for waiting a task to be completed.
 
 Review comment:
   What is the task here? Please explain. This is for all javadocs. There is no harm in writing more lines :P

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r395753442
 
 

 ##########
 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",
 
 Review comment:
   once validateContent raises exceptions, you don't need to wrap in an assert

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r395751870
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStream.java
 ##########
 @@ -0,0 +1,278 @@
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
+import org.apache.hadoop.fs.permission.FsPermission;
+
+/**
+ * Test AbfsOutputStream statistics.
+ */
+public class ITestAbfsOutputStream extends AbstractAbfsIntegrationTest {
+
+  public ITestAbfsOutputStream() throws Exception {
+  }
+
+  /**
+   * Tests to check bytes Uploading in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamUploadingBytes() throws IOException {
+    describe("Testing Bytes uploaded in AbfsOutputSteam");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path uploadBytesFilePath = new Path("AbfsOutputStreamStatsPath");
+    AzureBlobFileSystemStore abfss = fs.getAbfsStore();
+    FileSystem.Statistics statistics = fs.getFsStatistics();
+    abfss.getAbfsConfiguration().setDisableOutputStreamFlush(false);
+    String testBytesToUpload = "bytes";
+
+    AbfsOutputStream outForSomeBytes = null;
+    try {
+      outForSomeBytes = (AbfsOutputStream) abfss.createFile(uploadBytesFilePath,
+          statistics,
+          true,
+          FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
+
+      //Test for zero bytes To upload
+      assertValues("bytes to upload", 0,
+          outForSomeBytes.getOutputStreamStatistics().bytesToUpload);
+
+      outForSomeBytes.write(testBytesToUpload.getBytes());
+      outForSomeBytes.flush();
+
+      //Test for some bytes to upload
+      assertValues("bytes to upload", testBytesToUpload.getBytes().length,
+          outForSomeBytes.getOutputStreamStatistics().bytesToUpload);
+
+      //Test for relation between bytesUploadSuccessful, bytesUploadFailed
+      // and bytesToUpload
+      assertValues("bytesUploadSuccessful equal to difference between "
+              + "bytesToUpload and bytesUploadFailed",
+          outForSomeBytes.getOutputStreamStatistics().bytesUploadSuccessful,
+          outForSomeBytes.getOutputStreamStatistics().bytesToUpload -
+              outForSomeBytes.getOutputStreamStatistics().bytesUploadFailed);
+
+    } finally {
+      if (outForSomeBytes != null) {
+        outForSomeBytes.close();
+      }
+    }
+
+    AbfsOutputStream outForLargeBytes = null;
+    try {
+      outForLargeBytes =
+          (AbfsOutputStream) abfss.createFile(uploadBytesFilePath,
+              statistics
+              , true, FsPermission.getDefault(),
+              FsPermission.getUMask(fs.getConf()));
+
+      int largeValue = 100000;
+      for (int i = 0; i < largeValue; i++) {
+        outForLargeBytes.write(testBytesToUpload.getBytes());
+      }
+      outForLargeBytes.flush();
+
+      //Test for large bytes to upload
+      assertValues("bytes to upload",
+          largeValue * (testBytesToUpload.getBytes().length),
+          outForLargeBytes.getOutputStreamStatistics().bytesToUpload);
+
+      //Test for relation between bytesUploadSuccessful, bytesUploadFailed
+      // and bytesToUpload
+      assertValues("bytesUploadSuccessful equal to difference between "
+              + "bytesToUpload and bytesUploadFailed",
+          outForSomeBytes.getOutputStreamStatistics().bytesUploadSuccessful,
+          outForSomeBytes.getOutputStreamStatistics().bytesToUpload -
+              outForSomeBytes.getOutputStreamStatistics().bytesUploadFailed);
+
+    } finally {
+      if (outForLargeBytes != null) {
+        outForLargeBytes.close();
+      }
+    }
+
+  }
+
+  /**
+   * Tests to check time spend on waiting for tasks to be complete on a
+   * blocking queue in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamTimeSpendOnWaitTask() throws IOException {
+    describe("Testing Time Spend on Waiting for Task to be complete");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path timeSpendFilePath = new Path("AbfsOutputStreamStatsPath");
+    AzureBlobFileSystemStore abfss = fs.getAbfsStore();
+    FileSystem.Statistics statistics = fs.getFsStatistics();
+
+    AbfsOutputStream out =
+        (AbfsOutputStream) abfss.createFile(timeSpendFilePath,
+            statistics, true,
+            FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
+
+  }
+
+  /**
+   * Tests to check number of {@codes shrinkWriteOperationQueue()}
+   * calls.
+   * After writing data, AbfsOutputStream doesn't upload the data until
+   * Flushed. Hence, flush() method is called after write() to test Queue
+   * shrink calls.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamQueueShrink() throws IOException {
+    describe("Testing Queue Shrink calls in AbfsOutputStream");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path queueShrinkFilePath = new Path("AbfsOutputStreamStatsPath");
+    AzureBlobFileSystemStore abfss = fs.getAbfsStore();
+    abfss.getAbfsConfiguration().setDisableOutputStreamFlush(false);
+    FileSystem.Statistics statistics = fs.getFsStatistics();
+    String testQueueShrink = "testQueue";
+
+    AbfsOutputStream outForOneOp = null;
+
+    try {
+      outForOneOp =
+          (AbfsOutputStream) abfss.createFile(queueShrinkFilePath, statistics,
+              true,
+              FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
+
+      //Test for shrinking Queue zero time
+      assertValues("number of queueShrink() Calls", 0,
+          outForOneOp.getOutputStreamStatistics().queueShrink);
+
+      outForOneOp.write(testQueueShrink.getBytes());
+      // Queue is shrunk 2 times when outStream is flushed
+      outForOneOp.flush();
+
+      //Test for shrinking Queue 2 times
+      assertValues("number of queueShrink() Calls", 2,
+          outForOneOp.getOutputStreamStatistics().queueShrink);
+
+    } finally {
+      if (outForOneOp != null) {
+        outForOneOp.close();
+      }
+    }
+
+    AbfsOutputStream outForLargeOps = null;
+
+    try {
+      outForLargeOps = (AbfsOutputStream) abfss.createFile(queueShrinkFilePath,
+          statistics, true,
+          FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
+
+      int largeValue = 1000;
+      //QueueShrink is called 2 times in 1 flush(), hence 1000 flushes must
+      // give 2000 QueueShrink calls
+      for (int i = 0; i < largeValue; i++) {
+        outForLargeOps.write(testQueueShrink.getBytes());
+        //Flush is quite expensive so 1000 calls only which takes 1 min+
+        outForLargeOps.flush();
+      }
+
+      //Test for 2000 queue shrink calls
+      assertValues("number of queueShrink() Calls",
+          2 * largeValue,
+          outForLargeOps.getOutputStreamStatistics().queueShrink);
+    } finally {
+      if (outForLargeOps != null) {
+        outForLargeOps.close();
+      }
+    }
+
+  }
+
+  /**
+   * Test to check number of {@codes writeCurrentBufferToService()}
+   * calls.
+   * After writing data, AbfsOutputStream doesn't upload data till flush() is
+   * called. Hence, flush() calls were made after write() to simulate the
+   * scenario.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamWriteBuffer() throws IOException {
+    describe("Testing writeCurrentBufferToService() calls");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path writeBufferFilePath = new Path("AbfsOutputStreamStatsPath");
+    AzureBlobFileSystemStore abfss = fs.getAbfsStore();
+    FileSystem.Statistics statistics = fs.getFsStatistics();
+    abfss.getAbfsConfiguration().setDisableOutputStreamFlush(false);
+    String testWriteBuffer = "Buffer";
+
+    AbfsOutputStream outForOneOp = null;
+
+    try {
+      outForOneOp =
+          (AbfsOutputStream) abfss.createFile(writeBufferFilePath, statistics,
+              true,
+              FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
+
+      //Test for zero time writing Buffer to service
+      assertValues("number writeCurrentBufferToService() calls", 0,
+          outForOneOp.getOutputStreamStatistics().writeCurrentBufferOperations);
+
+      outForOneOp.write(testWriteBuffer.getBytes());
+      outForOneOp.flush();
+
+      //Test for one time writeCurrentBuffer() call
+      assertValues("number writeCurrentBufferToService() calls", 1,
+          outForOneOp.getOutputStreamStatistics().writeCurrentBufferOperations);
+    } finally {
+      if (outForOneOp != null) {
+        outForOneOp.close();
+      }
+    }
+
+    AbfsOutputStream outForLargeOps = null;
+    try {
 
 Review comment:
   try-with-resources or IOUtils.closeQuietly

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r408178312
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsOutputStreamStatistics.java
 ##########
 @@ -0,0 +1,120 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+import java.util.Random;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl;
+
+/**
+ * Unit Tests for AbfsOutputStream Statistics.
+ */
+public class TestAbfsOutputStreamStatistics
+    extends AbstractAbfsIntegrationTest {
+
+  private static final int LOW_RANGE_FOR_RANDOM_VALUE = 49;
+  private static final int HIGH_RANGE_FOR_RANDOM_VALUE = 9999;
+
+  public TestAbfsOutputStreamStatistics() throws Exception {
+  }
+
+  /**
+   * Tests to check bytes failed to Upload in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamBytesFailed() {
+    describe("Testing Bytes Failed during uploading in AbfsOutputSteam");
+
+    AbfsOutputStreamStatisticsImpl abfsOutputStreamStatistics =
+        new AbfsOutputStreamStatisticsImpl();
+
+    //Test for zero bytes uploaded.
+    assertValues("number fo bytes failed to upload", 0,
 
 Review comment:
   typo

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r395754519
 
 

 ##########
 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(), even if the original code didn't. Always good to improve a test

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r394227894
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStream.java
 ##########
 @@ -28,28 +26,38 @@ public ITestAbfsOutputStream() throws Exception {
   public void testAbfsOutputStreamUploadingBytes() throws IOException {
 
 Review comment:
   Need help in simulating Bytes to fail to upload in this test to get some values for bytesUploadFailed counter.

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on issue #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#issuecomment-614591927
 
 
   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 32s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  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  |  18m 56s |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 31s |  trunk passed  |
   | +1 :green_heart: |  checkstyle  |   0m 24s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 35s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  14m 52s |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 27s |  trunk passed  |
   | +0 :ok: |  spotbugs  |   0m 52s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 50s |  trunk passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 28s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 23s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 23s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 16s |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 27s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  14m  5s |  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 54s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 22s |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 31s |  The patch does not generate ASF License warnings.  |
   |  |   |  57m 19s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.8 Server=19.03.8 base: https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/11/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/1899 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux 6ca25026889e 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 / cc5c1da |
   | Default Java | 1.8.0_242 |
   |  Test Results | https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/11/testReport/ |
   | Max. process+thread count | 400 (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-1899/11/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] mukund-thakur commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r404618650
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStream.java
 ##########
 @@ -0,0 +1,294 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+import java.util.Random;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl;
+import org.apache.hadoop.fs.permission.FsPermission;
+
+/**
+ * Test AbfsOutputStream statistics.
+ */
+public class ITestAbfsOutputStream extends AbstractAbfsIntegrationTest {
+  public ITestAbfsOutputStream() throws Exception {
+  }
+
+  private static final int LARGE_OPERATIONS = 10;
+  private static final int LOW_RANGE_FOR_RANDOM_VALUE = 49;
+  private static final int HIGH_RANGE_FOR_RANDOM_VALUE = 9999;
+
+  /**
+   * Tests to check bytes Uploading in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamUploadingBytes() throws IOException {
+    describe("Testing Bytes uploaded in AbfsOutputSteam");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path uploadBytesFilePath = new Path("AbfsOutputStreamStatsPath");
+    String testBytesToUpload = "bytes";
+
+    try (AbfsOutputStream outForSomeBytes = createAbfsOutputStream(fs,
+        uploadBytesFilePath)
+    ) {
+
+      //Test for zero bytes To upload
+      assertValues("bytes to upload", 0,
+          outForSomeBytes.getOutputStreamStatistics().getBytesToUpload());
+
+      outForSomeBytes.write(testBytesToUpload.getBytes());
+      outForSomeBytes.flush();
+      AbfsOutputStreamStatisticsImpl abfsOutputStreamStatistics =
+          outForSomeBytes.getOutputStreamStatistics();
+
+      //Test for bytes to upload
+      assertValues("bytes to upload", testBytesToUpload.getBytes().length,
+          abfsOutputStreamStatistics.getBytesToUpload());
+
+      //Test for successful bytes uploaded
+      assertValues("successful bytes uploaded",
+          testBytesToUpload.getBytes().length,
+          abfsOutputStreamStatistics.getBytesUploadSuccessful());
+
+      //Populating random value for bytesFailed
+      int randomBytesFailed = new Random().nextInt(LOW_RANGE_FOR_RANDOM_VALUE);
+      abfsOutputStreamStatistics.uploadFailed(randomBytesFailed);
+      //Test for bytes failed to upload
+      assertValues("number fo bytes failed to upload", randomBytesFailed,
+          abfsOutputStreamStatistics.getBytesUploadFailed());
+
+    }
+
+    try (AbfsOutputStream outForLargeBytes = createAbfsOutputStream(fs,
+        uploadBytesFilePath)) {
+
+      int largeValue = LARGE_OPERATIONS;
+      for (int i = 0; i < largeValue; i++) {
+        outForLargeBytes.write(testBytesToUpload.getBytes());
+      }
+      outForLargeBytes.flush();
+      AbfsOutputStreamStatisticsImpl abfsOutputStreamStatistics =
+          outForLargeBytes.getOutputStreamStatistics();
+
+      //Test for bytes to upload
+      assertValues("bytes to upload",
+          largeValue * (testBytesToUpload.getBytes().length),
+          abfsOutputStreamStatistics.getBytesToUpload());
+
+      //Test for successful bytes uploaded
+      assertValues("successful bytes uploaded",
+          largeValue * (testBytesToUpload.getBytes().length),
+          abfsOutputStreamStatistics.getBytesUploadSuccessful());
+
+      //Populating random values for bytesFailed
+      int randomBytesFailed = new Random().nextInt(HIGH_RANGE_FOR_RANDOM_VALUE);
+      abfsOutputStreamStatistics.uploadFailed(randomBytesFailed);
+      //Test for bytes failed to upload
+      assertValues("bytes failed to upload", randomBytesFailed,
+          abfsOutputStreamStatistics.getBytesUploadFailed());
+    }
+  }
+
+  /**
+   * Tests to check time spent on waiting for tasks to be complete on a
+   * blocking queue in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamTimeSpentOnWaitTask() throws IOException {
+    describe("Testing Time Spend on Waiting for Task to be complete");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path timeSpendFilePath = new Path("AbfsOutputStreamStatsPath");
+
+    try (AbfsOutputStream out = createAbfsOutputStream(fs, timeSpendFilePath)) {
+
+      AbfsOutputStreamStatisticsImpl abfsOutputStreamStatistics =
+          out.getOutputStreamStatistics();
+
+      //Test for initial value of timeSpentWaitTask
+      assertValues("Time spent on waiting tasks", 0,
+          abfsOutputStreamStatistics.getTimeSpendOnTaskWait());
+
+      int smallRandomStartTime =
+          new Random().nextInt(LOW_RANGE_FOR_RANDOM_VALUE);
+      int smallRandomEndTime =
+          new Random().nextInt(LOW_RANGE_FOR_RANDOM_VALUE)
+              + smallRandomStartTime;
+      int smallDiff = smallRandomEndTime - smallRandomStartTime;
+      abfsOutputStreamStatistics
+          .timeSpentTaskWait(smallRandomStartTime, smallRandomEndTime);
+      //Test for small random value of timeSpentWaitTask
+      assertValues("Time spent on waiting tasks", smallDiff,
+          abfsOutputStreamStatistics.getTimeSpendOnTaskWait());
+
+      int largeRandomStartTime =
+          new Random().nextInt(HIGH_RANGE_FOR_RANDOM_VALUE);
+      int largeRandomEndTime = new Random().nextInt(HIGH_RANGE_FOR_RANDOM_VALUE)
+          + largeRandomStartTime;
+      int randomDiff = largeRandomEndTime - largeRandomStartTime;
+      abfsOutputStreamStatistics
+          .timeSpentTaskWait(largeRandomStartTime, largeRandomEndTime);
+      /*
+      Test for large random value of timeSpentWaitTask plus the time spent
+      in previous test
+       */
+      assertValues("Time spent on waiting tasks", smallDiff + randomDiff,
+          abfsOutputStreamStatistics.getTimeSpendOnTaskWait());
+    }
+
+  }
+
+  /**
+   * Tests to check number of {@code shrinkWriteOperationQueue()}
+   * calls.
+   * After writing data, AbfsOutputStream doesn't upload the data until
+   * Flushed. Hence, flush() method is called after write() to test Queue
+   * shrink calls.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamQueueShrink() throws IOException {
+    describe("Testing Queue Shrink calls in AbfsOutputStream");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path queueShrinkFilePath = new Path("AbfsOutputStreamStatsPath");
+    String testQueueShrink = "testQueue";
+
+    try (AbfsOutputStream outForOneOp = createAbfsOutputStream(fs,
+        queueShrinkFilePath)) {
+
+      //Test for shrinking Queue zero time
+      assertValues("number of queueShrink() Calls", 0,
+          outForOneOp.getOutputStreamStatistics().getQueueShrink());
+
+      outForOneOp.write(testQueueShrink.getBytes());
+      // Queue is shrunk 2 times when outStream is flushed
+      outForOneOp.flush();
+
+      //Test for shrinking Queue 2 times
+      assertValues("number of queueShrink() Calls", 2,
+          outForOneOp.getOutputStreamStatistics().getQueueShrink());
+
+    }
+
 
 Review comment:
   I can see you are disabling flush in createAbfsOutputStream(). Then why are you expecting double the actual values?

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r396230156
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java
 ##########
 @@ -101,6 +101,7 @@ public synchronized int read(final byte[] b, final int off, final int len) throw
     int currentLen = len;
     int lastReadBytes;
     int totalReadBytes = 0;
+    incrementReadOps();
 
 Review comment:
   this is from the master PR(#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] steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r395753205
 
 

 ##########
 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) {
 
 Review comment:
   IOUtils.closeQuietly

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r408805468
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
 ##########
 @@ -384,6 +400,7 @@ private synchronized void flushWrittenBytesToServiceInternal(final long offset,
    * operation FIFO queue.
    */
   private synchronized void shrinkWriteOperationQueue() throws IOException {
+    outputStreamStatistics.queueShrunk();
 
 Review comment:
   actually, queue is not shrunk here. Rather it is inside the while loop. will change this.

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r396884860
 
 

 ##########
 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 {
 
 Review comment:
   I think constructor is needed to handle Exception ?

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r394227017
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStream.java
 ##########
 @@ -133,58 +133,60 @@ public void testAbfsOutputStreamTimeSpendOnWaitTask() throws IOException {
   public void testAbfsOutputStreamQueueShrink() throws IOException {
     describe("Testing Queue Shrink calls in AbfsOutputStream");
 
 Review comment:
   Need help in writing test for this counter.

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r408680852
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
 ##########
 @@ -436,4 +453,28 @@ private void waitForTaskToComplete() throws IOException {
   public synchronized void waitForPendingUploads() throws IOException {
     waitForTaskToComplete();
   }
+
+  /**
+   * Getter method for AbfsOutputStream Statistics.
+   *
+   * @return statistics for AbfsOutputStream.
+   */
+  @VisibleForTesting
+  public AbfsOutputStreamStatisticsImpl getOutputStreamStatistics() {
 
 Review comment:
   I could cast in the tests where this is called, but I thought it would easier to cast here than casting there as there would be many calls to this method.

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r395749023
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStream.java
 ##########
 @@ -0,0 +1,278 @@
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
+import org.apache.hadoop.fs.permission.FsPermission;
+
+/**
+ * Test AbfsOutputStream statistics.
+ */
+public class ITestAbfsOutputStream extends AbstractAbfsIntegrationTest {
+
+  public ITestAbfsOutputStream() throws Exception {
+  }
+
+  /**
+   * Tests to check bytes Uploading in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamUploadingBytes() throws IOException {
+    describe("Testing Bytes uploaded in AbfsOutputSteam");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path uploadBytesFilePath = new Path("AbfsOutputStreamStatsPath");
+    AzureBlobFileSystemStore abfss = fs.getAbfsStore();
+    FileSystem.Statistics statistics = fs.getFsStatistics();
+    abfss.getAbfsConfiguration().setDisableOutputStreamFlush(false);
+    String testBytesToUpload = "bytes";
+
+    AbfsOutputStream outForSomeBytes = null;
+    try {
+      outForSomeBytes = (AbfsOutputStream) abfss.createFile(uploadBytesFilePath,
+          statistics,
+          true,
+          FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
+
+      //Test for zero bytes To upload
+      assertValues("bytes to upload", 0,
+          outForSomeBytes.getOutputStreamStatistics().bytesToUpload);
+
+      outForSomeBytes.write(testBytesToUpload.getBytes());
+      outForSomeBytes.flush();
+
+      //Test for some bytes to upload
+      assertValues("bytes to upload", testBytesToUpload.getBytes().length,
+          outForSomeBytes.getOutputStreamStatistics().bytesToUpload);
+
+      //Test for relation between bytesUploadSuccessful, bytesUploadFailed
+      // and bytesToUpload
+      assertValues("bytesUploadSuccessful equal to difference between "
+              + "bytesToUpload and bytesUploadFailed",
+          outForSomeBytes.getOutputStreamStatistics().bytesUploadSuccessful,
+          outForSomeBytes.getOutputStreamStatistics().bytesToUpload -
+              outForSomeBytes.getOutputStreamStatistics().bytesUploadFailed);
+
+    } finally {
+      if (outForSomeBytes != null) {
 
 Review comment:
   IOUtils.closeQuietly(LOG, ...), or try-with-resources

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r395746252
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamStatisticsImpl.java
 ##########
 @@ -0,0 +1,102 @@
+package org.apache.hadoop.fs.azurebfs.services;
+
+/**
+ * OutputStream Statistics Implementation for Abfs.
+ * timeSpendOnTaskWait - Time spend on waiting for tasks to be complete on
+ * Blocking Queue in AbfsOutputStream.
+ *
+ * queueShrink - Number of times Blocking Queue was shrunk after writing
+ * data.
+ *
+ * WriteCurrentBufferOperations - Number of times
+ * {@codes writeCurrentBufferToService()} calls were made.
+ */
+public class AbfsOutputStreamStatisticsImpl
+    implements AbfsOutputStreamStatistics {
+  public volatile long bytesToUpload;
+  public volatile long bytesUploadSuccessful;
+  public volatile long bytesUploadFailed;
+  public volatile long timeSpendOnTaskWait;
+  public volatile long queueShrink;
+  public volatile long writeCurrentBufferOperations;
+
+  /**
+   * Number of bytes uploaded only when bytes passed are positive.
+   *
+   * @param bytes negative values are ignored
+   */
+  @Override
+  public void bytesToUpload(long bytes) {
+    if (bytes > 0) {
+      bytesToUpload += bytes;
+    }
+  }
+
+  @Override
+  public void bytesUploadedSuccessfully(long bytes) {
+    if (bytes > 0) {
+      bytesUploadSuccessful += bytes;
+    }
+  }
+
+  /**
+   * Number of bytes that weren't uploaded.
+   *
+   * @param bytes negative values are ignored
+   */
+  @Override
+  public void bytesFailed(long bytes) {
+    if (bytes > 0) {
+      bytesUploadFailed += bytes;
+    }
+  }
+
+  /**
+   * Time spend for waiting a task to be completed.
+   *
+   * @param startTime on calling {@link AbfsOutputStream#waitForTaskToComplete()}
+   * @param endTime   on method completing
+   */
+  @Override
+  public void timeSpendTaskWait(long startTime, long endTime) {
+    timeSpendOnTaskWait += endTime - startTime;
+  }
+
+  /**
+   * Number of calls to {@link AbfsOutputStream#shrinkWriteOperationQueue()}.
 
 Review comment:
   see above comment about javadocs

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r408175364
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStreamStatistics.java
 ##########
 @@ -0,0 +1,233 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl;
+
+/**
+ * Test AbfsOutputStream statistics.
+ */
+public class ITestAbfsOutputStreamStatistics
+    extends AbstractAbfsIntegrationTest {
+  private static final int LARGE_OPERATIONS = 10;
+
+  public ITestAbfsOutputStreamStatistics() throws Exception {
+  }
+
+  /**
+   * Tests to check bytes Uploaded successfully in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamUploadingBytes() throws IOException {
+    describe("Testing Bytes uploaded successfully in AbfsOutputSteam");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path uploadBytesFilePath = path(getMethodName());
+    String testBytesToUpload = "bytes";
+
+    try (
+        AbfsOutputStream outForSomeBytes = createAbfsOutputStreamWithFlushEnabled(
+            fs,
+            uploadBytesFilePath)
+    ) {
+
+      AbfsOutputStreamStatisticsImpl abfsOutputStreamStatisticsForUploadBytes =
+          outForSomeBytes.getOutputStreamStatistics();
+
+      //Test for zero bytes To upload.
+      assertValues("bytes to upload", 0,
+          abfsOutputStreamStatisticsForUploadBytes.getBytesToUpload());
+
+      outForSomeBytes.write(testBytesToUpload.getBytes());
+      outForSomeBytes.flush();
+      abfsOutputStreamStatisticsForUploadBytes =
+          outForSomeBytes.getOutputStreamStatistics();
+
+      //Test for bytes to upload.
+      assertValues("bytes to upload", testBytesToUpload.getBytes().length,
+          abfsOutputStreamStatisticsForUploadBytes.getBytesToUpload());
+
+      //Test for successful bytes uploaded.
+      assertValues("successful bytes uploaded",
+          testBytesToUpload.getBytes().length,
+          abfsOutputStreamStatisticsForUploadBytes.getBytesUploadSuccessful());
+
+    }
+
+    try (
+        AbfsOutputStream outForLargeBytes = createAbfsOutputStreamWithFlushEnabled(
+            fs,
+            uploadBytesFilePath)) {
+
+      for (int i = 0; i < LARGE_OPERATIONS; i++) {
+        outForLargeBytes.write(testBytesToUpload.getBytes());
+      }
+      outForLargeBytes.flush();
+      AbfsOutputStreamStatisticsImpl abfsOutputStreamStatistics =
+          outForLargeBytes.getOutputStreamStatistics();
+
+      //Test for bytes to upload.
+      assertValues("bytes to upload",
+          LARGE_OPERATIONS * (testBytesToUpload.getBytes().length),
+          abfsOutputStreamStatistics.getBytesToUpload());
+
+      //Test for successful bytes uploaded.
+      assertValues("successful bytes uploaded",
+          LARGE_OPERATIONS * (testBytesToUpload.getBytes().length),
+          abfsOutputStreamStatistics.getBytesUploadSuccessful());
+
+    }
+  }
+
+  /**
+   * Tests to check number of {@code
+   * AbfsOutputStream#shrinkWriteOperationQueue()} calls.
+   * After writing data, AbfsOutputStream doesn't upload the data until
+   * Flushed. Hence, flush() method is called after write() to test Queue
+   * shrink calls.
+   *
+   * @throws IOException
 
 Review comment:
   nit: just cut these lines from the javadoc. Nice to see the rest of the detail

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r404601085
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
 ##########
 @@ -384,6 +398,7 @@ private synchronized void flushWrittenBytesToServiceInternal(final long offset,
    * operation FIFO queue.
    */
   private synchronized void shrinkWriteOperationQueue() throws IOException {
+    outputStreamStatistics.queueShrinked();
 
 Review comment:
   I don't understand the benifits of capturing this metric?

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on issue #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#issuecomment-607205921
 
 
   getting timeout errors in abfs-tests(might be server issue):
   
   ```
   [ERROR] Errors:
   [ERROR]   ITestAzureBlobFileSystemDelete.testDeleteFirstLevelDirectory:127->lambda$testDeleteFirstLevelDirectory$1:128 » TestTimedOut
   [ERROR]   ITestAzureBlobFileSystemMainOperation>FSMainOperationsBaseTest.testDeleteEmptyDirectory:787 » AbfsRestOperation
   [ERROR]   ITestAzureBlobFileSystemMainOperation>FSMainOperationsBaseTest.testRenameDirectoryAsEmptyDirectory:1033->FSMainOperationsBaseTest.rename:1154 » AbfsRestOperation
   [ERROR]   ITestGetNameSpaceEnabled.testFailedRequestWhenCredentialsNotCorrect:91->AbstractAbfsIntegrationTest.getFileSystem:229 » InvalidConfigurationValue
   
   
   [ERROR] Errors:
   [ERROR]   ITestAbfsFileSystemContractDelete>AbstractContractDeleteTest.testDeleteEmptyDirNonRecursive:37->AbstractFSContractTestBase.assertDeleted:369 » TestTimedOut
   [ERROR]   ITestAbfsFileSystemContractDelete>AbstractContractDeleteTest.testDeleteNonEmptyDirNonRecursive:77 » TestTimedOut
   [ERROR]   ITestAbfsFileSystemContractMkdir>AbstractContractMkdirTest.testMkDirRmDir:46->AbstractFSContractTestBase.assertDeleted:369 » TestTimedOut
   [ERROR]   ITestAzureBlobFileSystemBasics>FileSystemContractBaseTest.testDeleteEmptyDirectory:441 » TestTimedOut
   [ERROR]   ITestAzureBlobFileSystemBasics>FileSystemContractBaseTest.testDeleteRecursively:421 » TestTimedOut
   [ERROR]   ITestAzureBlobFileSystemBasics>FileSystemContractBaseTest.testFilesystemIsCaseSensitive:651 » TestTimedOut
   [ERROR]   ITestAzureBlobFileSystemBasics.setUp:49 » TestTimedOut test timed out after 30...
   [ERROR]   ITestAzureBlobFileSystemBasics>FileSystemContractBaseTest.testRenameDirectoryAsExistingDirectory:548->FileSystemContractBaseTest.createFile:590 » TestTimedOut
   [ERROR]   ITestAzureBlobFileSystemBasics>FileSystemContractBaseTest.testRenameDirectoryMoveToExistingDirectory:515->FileSystemContractBaseTest.createFile:590 » TestTimedOut
   [ERROR]   ITestAzureBlobFileSystemBasics.tearDown:61 » TestTimedOut test timed out after...

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r402070920
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
 ##########
 @@ -80,6 +82,7 @@
           = new ElasticByteBufferPool();
 
   private final Statistics statistics;
+  private final AbfsOutputStreamStatisticsImpl outputStreamStatistics;
 
 Review comment:
   We should be defining here using the interface not concrete class and initialising it in constructor with concrete class implementation. That whole point of using interface is we can change add the new implementation in future and change the the implementation to be used in constructor using some configuration. Right @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] hadoop-yetus commented on issue #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on issue #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#issuecomment-610641314
 
 
   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 32s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  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 3 new or modified test files.  |
   ||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  19m 12s |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 30s |  trunk passed  |
   | +1 :green_heart: |  checkstyle  |   0m 24s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 35s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  15m 13s |  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  |
   | -0 :warning: |  patch  |   1m  8s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 28s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 21s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 21s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 14s |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 24s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  14m  5s |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 22s |  the patch passed  |
   | -1 :x: |  findbugs  |   0m 57s |  hadoop-tools/hadoop-azure generated 2 new + 0 unchanged - 0 fixed = 2 total (was 0)  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  |   1m 48s |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 33s |  The patch does not generate ASF License warnings.  |
   |  |   |  58m 21s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hadoop-tools/hadoop-azure |
   |  |  Increment of volatile field org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl.queueShrunkOps in org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl.queueShrunk()  At AbfsOutputStreamStatisticsImpl.java:in org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl.queueShrunk()  At AbfsOutputStreamStatisticsImpl.java:[line 118] |
   |  |  Increment of volatile field org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl.writeCurrentBufferOperations in org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl.writeCurrentBuffer()  At AbfsOutputStreamStatisticsImpl.java:in org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl.writeCurrentBuffer()  At AbfsOutputStreamStatisticsImpl.java:[line 129] |
   | Failed junit tests | hadoop.fs.azurebfs.services.TestAbfsClientThrottlingAnalyzer |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.8 Server=19.03.8 base: https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/5/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/1899 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux a734d6832366 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 / 20eec95 |
   | Default Java | 1.8.0_242 |
   | findbugs | https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/5/artifact/out/new-findbugs-hadoop-tools_hadoop-azure.html |
   | unit | https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/5/artifact/out/patch-unit-hadoop-tools_hadoop-azure.txt |
   |  Test Results | https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/5/testReport/ |
   | Max. process+thread count | 414 (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-1899/5/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 commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r395750967
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStream.java
 ##########
 @@ -0,0 +1,278 @@
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
+import org.apache.hadoop.fs.permission.FsPermission;
+
+/**
+ * Test AbfsOutputStream statistics.
+ */
+public class ITestAbfsOutputStream extends AbstractAbfsIntegrationTest {
+
+  public ITestAbfsOutputStream() throws Exception {
+  }
+
+  /**
+   * Tests to check bytes Uploading in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamUploadingBytes() throws IOException {
+    describe("Testing Bytes uploaded in AbfsOutputSteam");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path uploadBytesFilePath = new Path("AbfsOutputStreamStatsPath");
+    AzureBlobFileSystemStore abfss = fs.getAbfsStore();
+    FileSystem.Statistics statistics = fs.getFsStatistics();
+    abfss.getAbfsConfiguration().setDisableOutputStreamFlush(false);
+    String testBytesToUpload = "bytes";
+
+    AbfsOutputStream outForSomeBytes = null;
+    try {
+      outForSomeBytes = (AbfsOutputStream) abfss.createFile(uploadBytesFilePath,
+          statistics,
+          true,
+          FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
+
+      //Test for zero bytes To upload
+      assertValues("bytes to upload", 0,
+          outForSomeBytes.getOutputStreamStatistics().bytesToUpload);
+
+      outForSomeBytes.write(testBytesToUpload.getBytes());
+      outForSomeBytes.flush();
+
+      //Test for some bytes to upload
+      assertValues("bytes to upload", testBytesToUpload.getBytes().length,
+          outForSomeBytes.getOutputStreamStatistics().bytesToUpload);
+
+      //Test for relation between bytesUploadSuccessful, bytesUploadFailed
+      // and bytesToUpload
+      assertValues("bytesUploadSuccessful equal to difference between "
+              + "bytesToUpload and bytesUploadFailed",
+          outForSomeBytes.getOutputStreamStatistics().bytesUploadSuccessful,
+          outForSomeBytes.getOutputStreamStatistics().bytesToUpload -
+              outForSomeBytes.getOutputStreamStatistics().bytesUploadFailed);
+
+    } finally {
+      if (outForSomeBytes != null) {
+        outForSomeBytes.close();
+      }
+    }
+
+    AbfsOutputStream outForLargeBytes = null;
+    try {
+      outForLargeBytes =
+          (AbfsOutputStream) abfss.createFile(uploadBytesFilePath,
+              statistics
+              , true, FsPermission.getDefault(),
+              FsPermission.getUMask(fs.getConf()));
+
+      int largeValue = 100000;
+      for (int i = 0; i < largeValue; i++) {
+        outForLargeBytes.write(testBytesToUpload.getBytes());
+      }
+      outForLargeBytes.flush();
+
+      //Test for large bytes to upload
+      assertValues("bytes to upload",
+          largeValue * (testBytesToUpload.getBytes().length),
+          outForLargeBytes.getOutputStreamStatistics().bytesToUpload);
+
+      //Test for relation between bytesUploadSuccessful, bytesUploadFailed
+      // and bytesToUpload
+      assertValues("bytesUploadSuccessful equal to difference between "
+              + "bytesToUpload and bytesUploadFailed",
+          outForSomeBytes.getOutputStreamStatistics().bytesUploadSuccessful,
+          outForSomeBytes.getOutputStreamStatistics().bytesToUpload -
+              outForSomeBytes.getOutputStreamStatistics().bytesUploadFailed);
+
+    } finally {
+      if (outForLargeBytes != null) {
+        outForLargeBytes.close();
+      }
+    }
+
+  }
+
+  /**
+   * Tests to check time spend on waiting for tasks to be complete on a
+   * blocking queue in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamTimeSpendOnWaitTask() throws IOException {
+    describe("Testing Time Spend on Waiting for Task to be complete");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path timeSpendFilePath = new Path("AbfsOutputStreamStatsPath");
+    AzureBlobFileSystemStore abfss = fs.getAbfsStore();
+    FileSystem.Statistics statistics = fs.getFsStatistics();
+
+    AbfsOutputStream out =
+        (AbfsOutputStream) abfss.createFile(timeSpendFilePath,
+            statistics, true,
+            FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
+
+  }
+
+  /**
+   * Tests to check number of {@codes shrinkWriteOperationQueue()}
+   * calls.
+   * After writing data, AbfsOutputStream doesn't upload the data until
+   * Flushed. Hence, flush() method is called after write() to test Queue
+   * shrink calls.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamQueueShrink() throws IOException {
+    describe("Testing Queue Shrink calls in AbfsOutputStream");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path queueShrinkFilePath = new Path("AbfsOutputStreamStatsPath");
+    AzureBlobFileSystemStore abfss = fs.getAbfsStore();
+    abfss.getAbfsConfiguration().setDisableOutputStreamFlush(false);
+    FileSystem.Statistics statistics = fs.getFsStatistics();
+    String testQueueShrink = "testQueue";
+
+    AbfsOutputStream outForOneOp = null;
+
+    try {
+      outForOneOp =
+          (AbfsOutputStream) abfss.createFile(queueShrinkFilePath, statistics,
+              true,
+              FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
+
+      //Test for shrinking Queue zero time
+      assertValues("number of queueShrink() Calls", 0,
+          outForOneOp.getOutputStreamStatistics().queueShrink);
+
+      outForOneOp.write(testQueueShrink.getBytes());
+      // Queue is shrunk 2 times when outStream is flushed
+      outForOneOp.flush();
+
+      //Test for shrinking Queue 2 times
+      assertValues("number of queueShrink() Calls", 2,
+          outForOneOp.getOutputStreamStatistics().queueShrink);
+
+    } finally {
+      if (outForOneOp != null) {
+        outForOneOp.close();
+      }
+    }
+
+    AbfsOutputStream outForLargeOps = null;
+
+    try {
+      outForLargeOps = (AbfsOutputStream) abfss.createFile(queueShrinkFilePath,
 
 Review comment:
   you are calling createFile() enough in these tests it makes sense to factor out into it own method

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on issue #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#issuecomment-600024548
 
 
   test run : mvn -T 1C -Dparallel-tests=abfs clean verify
   Region : East US, West US
   
   ```
   [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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r395749213
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStream.java
 ##########
 @@ -0,0 +1,278 @@
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
+import org.apache.hadoop.fs.permission.FsPermission;
+
+/**
+ * Test AbfsOutputStream statistics.
+ */
+public class ITestAbfsOutputStream extends AbstractAbfsIntegrationTest {
+
+  public ITestAbfsOutputStream() throws Exception {
+  }
+
+  /**
+   * Tests to check bytes Uploading in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamUploadingBytes() throws IOException {
+    describe("Testing Bytes uploaded in AbfsOutputSteam");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path uploadBytesFilePath = new Path("AbfsOutputStreamStatsPath");
+    AzureBlobFileSystemStore abfss = fs.getAbfsStore();
+    FileSystem.Statistics statistics = fs.getFsStatistics();
+    abfss.getAbfsConfiguration().setDisableOutputStreamFlush(false);
+    String testBytesToUpload = "bytes";
+
+    AbfsOutputStream outForSomeBytes = null;
+    try {
+      outForSomeBytes = (AbfsOutputStream) abfss.createFile(uploadBytesFilePath,
+          statistics,
+          true,
+          FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
+
+      //Test for zero bytes To upload
+      assertValues("bytes to upload", 0,
+          outForSomeBytes.getOutputStreamStatistics().bytesToUpload);
+
+      outForSomeBytes.write(testBytesToUpload.getBytes());
+      outForSomeBytes.flush();
+
+      //Test for some bytes to upload
+      assertValues("bytes to upload", testBytesToUpload.getBytes().length,
+          outForSomeBytes.getOutputStreamStatistics().bytesToUpload);
+
+      //Test for relation between bytesUploadSuccessful, bytesUploadFailed
+      // and bytesToUpload
+      assertValues("bytesUploadSuccessful equal to difference between "
+              + "bytesToUpload and bytesUploadFailed",
+          outForSomeBytes.getOutputStreamStatistics().bytesUploadSuccessful,
+          outForSomeBytes.getOutputStreamStatistics().bytesToUpload -
+              outForSomeBytes.getOutputStreamStatistics().bytesUploadFailed);
+
+    } finally {
+      if (outForSomeBytes != null) {
+        outForSomeBytes.close();
+      }
+    }
+
+    AbfsOutputStream outForLargeBytes = null;
+    try {
 
 Review comment:
   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] steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r395745249
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamStatistics.java
 ##########
 @@ -0,0 +1,60 @@
+package org.apache.hadoop.fs.azurebfs.services;
+
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Interface for {@link AbfsOutputStream} statistics.
+ */
+@InterfaceStability.Unstable
+public interface AbfsOutputStreamStatistics {
+
+  /**
+   * Number of bytes to be uploaded.
+   *
+   * @param bytes number of bytes to upload
+   */
+  void bytesToUpload(long bytes);
+
+  /**
+   * Number of bytes uploaded Successfully.
+   *
+   * @param bytes number of bytes that were successfully uploaded
+   */
+  void bytesUploadedSuccessfully(long bytes);
+
+  /**
+   * Number of bytes failed to upload.
+   *
+   * @param bytes number of bytes that failed to upload
+   */
+  void bytesFailed(long bytes);
 
 Review comment:
   prefer a more detailed description like uploadFailed(long). It's recording that an upload failed and the number of bytes

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r404606007
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStream.java
 ##########
 @@ -0,0 +1,294 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+import java.util.Random;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl;
+import org.apache.hadoop.fs.permission.FsPermission;
+
+/**
+ * Test AbfsOutputStream statistics.
+ */
+public class ITestAbfsOutputStream extends AbstractAbfsIntegrationTest {
+  public ITestAbfsOutputStream() throws Exception {
+  }
+
+  private static final int LARGE_OPERATIONS = 10;
+  private static final int LOW_RANGE_FOR_RANDOM_VALUE = 49;
+  private static final int HIGH_RANGE_FOR_RANDOM_VALUE = 9999;
+
+  /**
+   * Tests to check bytes Uploading in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamUploadingBytes() throws IOException {
+    describe("Testing Bytes uploaded in AbfsOutputSteam");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path uploadBytesFilePath = new Path("AbfsOutputStreamStatsPath");
 
 Review comment:
   Create a new protected method in the base clase to get methode name using 
   methodName.getMethodName()
   

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r395754102
 
 

 ##########
 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) {
+        inForLargeOperations.close();
+      }
+      if (outForLargeOperations != null) {
+        outForLargeOperations.close();
+      }
+    }
+
+    //Validating if content is being written in largeOperationsFile
+    Assert.assertTrue("Mismatch in content validation",
 
 Review comment:
   again, superflous with validateContent raising exceptions

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r395756248
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStream.java
 ##########
 @@ -28,28 +26,38 @@ public ITestAbfsOutputStream() throws Exception {
   public void testAbfsOutputStreamUploadingBytes() throws IOException {
 
 Review comment:
   I can't think of any. Maybe just have a unit test to take an AbfsOutputStreamsImpl and verify that when the method is called, the counter is updated.
   
   (Actually, mocking could simulate failure, ...)

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r395746300
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamStatisticsImpl.java
 ##########
 @@ -0,0 +1,102 @@
+package org.apache.hadoop.fs.azurebfs.services;
+
+/**
+ * OutputStream Statistics Implementation for Abfs.
+ * timeSpendOnTaskWait - Time spend on waiting for tasks to be complete on
+ * Blocking Queue in AbfsOutputStream.
+ *
+ * queueShrink - Number of times Blocking Queue was shrunk after writing
+ * data.
+ *
+ * WriteCurrentBufferOperations - Number of times
+ * {@codes writeCurrentBufferToService()} calls were made.
+ */
+public class AbfsOutputStreamStatisticsImpl
+    implements AbfsOutputStreamStatistics {
+  public volatile long bytesToUpload;
+  public volatile long bytesUploadSuccessful;
+  public volatile long bytesUploadFailed;
+  public volatile long timeSpendOnTaskWait;
+  public volatile long queueShrink;
+  public volatile long writeCurrentBufferOperations;
+
+  /**
+   * Number of bytes uploaded only when bytes passed are positive.
+   *
+   * @param bytes negative values are ignored
+   */
+  @Override
+  public void bytesToUpload(long bytes) {
+    if (bytes > 0) {
+      bytesToUpload += bytes;
+    }
+  }
+
+  @Override
+  public void bytesUploadedSuccessfully(long bytes) {
+    if (bytes > 0) {
+      bytesUploadSuccessful += bytes;
+    }
+  }
+
+  /**
+   * Number of bytes that weren't uploaded.
+   *
+   * @param bytes negative values are ignored
+   */
+  @Override
+  public void bytesFailed(long bytes) {
+    if (bytes > 0) {
+      bytesUploadFailed += bytes;
+    }
+  }
+
+  /**
+   * Time spend for waiting a task to be completed.
+   *
+   * @param startTime on calling {@link AbfsOutputStream#waitForTaskToComplete()}
+   * @param endTime   on method completing
+   */
+  @Override
+  public void timeSpendTaskWait(long startTime, long endTime) {
+    timeSpendOnTaskWait += endTime - startTime;
+  }
+
+  /**
+   * Number of calls to {@link AbfsOutputStream#shrinkWriteOperationQueue()}.
+   */
+  @Override
+  public void queueShrinked() {
+    queueShrink++;
+  }
+
+  /**
+   * Number of calls to {@link AbfsOutputStream#writeCurrentBufferToService()}.
 
 Review comment:
   see above comment about javadocs

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r404807005
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStream.java
 ##########
 @@ -0,0 +1,294 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+import java.util.Random;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl;
+import org.apache.hadoop.fs.permission.FsPermission;
+
+/**
+ * Test AbfsOutputStream statistics.
+ */
+public class ITestAbfsOutputStream extends AbstractAbfsIntegrationTest {
+  public ITestAbfsOutputStream() throws Exception {
+  }
+
+  private static final int LARGE_OPERATIONS = 10;
+  private static final int LOW_RANGE_FOR_RANDOM_VALUE = 49;
+  private static final int HIGH_RANGE_FOR_RANDOM_VALUE = 9999;
+
+  /**
+   * Tests to check bytes Uploading in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamUploadingBytes() throws IOException {
+    describe("Testing Bytes uploaded in AbfsOutputSteam");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path uploadBytesFilePath = new Path("AbfsOutputStreamStatsPath");
+    String testBytesToUpload = "bytes";
+
+    try (AbfsOutputStream outForSomeBytes = createAbfsOutputStream(fs,
+        uploadBytesFilePath)
+    ) {
+
+      //Test for zero bytes To upload
+      assertValues("bytes to upload", 0,
+          outForSomeBytes.getOutputStreamStatistics().getBytesToUpload());
+
+      outForSomeBytes.write(testBytesToUpload.getBytes());
+      outForSomeBytes.flush();
+      AbfsOutputStreamStatisticsImpl abfsOutputStreamStatistics =
+          outForSomeBytes.getOutputStreamStatistics();
+
+      //Test for bytes to upload
+      assertValues("bytes to upload", testBytesToUpload.getBytes().length,
+          abfsOutputStreamStatistics.getBytesToUpload());
+
+      //Test for successful bytes uploaded
+      assertValues("successful bytes uploaded",
+          testBytesToUpload.getBytes().length,
+          abfsOutputStreamStatistics.getBytesUploadSuccessful());
+
+      //Populating random value for bytesFailed
+      int randomBytesFailed = new Random().nextInt(LOW_RANGE_FOR_RANDOM_VALUE);
+      abfsOutputStreamStatistics.uploadFailed(randomBytesFailed);
+      //Test for bytes failed to upload
+      assertValues("number fo bytes failed to upload", randomBytesFailed,
+          abfsOutputStreamStatistics.getBytesUploadFailed());
+
+    }
+
+    try (AbfsOutputStream outForLargeBytes = createAbfsOutputStream(fs,
+        uploadBytesFilePath)) {
+
+      int largeValue = LARGE_OPERATIONS;
+      for (int i = 0; i < largeValue; i++) {
+        outForLargeBytes.write(testBytesToUpload.getBytes());
+      }
+      outForLargeBytes.flush();
+      AbfsOutputStreamStatisticsImpl abfsOutputStreamStatistics =
+          outForLargeBytes.getOutputStreamStatistics();
+
+      //Test for bytes to upload
+      assertValues("bytes to upload",
+          largeValue * (testBytesToUpload.getBytes().length),
+          abfsOutputStreamStatistics.getBytesToUpload());
+
+      //Test for successful bytes uploaded
+      assertValues("successful bytes uploaded",
+          largeValue * (testBytesToUpload.getBytes().length),
+          abfsOutputStreamStatistics.getBytesUploadSuccessful());
+
+      //Populating random values for bytesFailed
+      int randomBytesFailed = new Random().nextInt(HIGH_RANGE_FOR_RANDOM_VALUE);
+      abfsOutputStreamStatistics.uploadFailed(randomBytesFailed);
+      //Test for bytes failed to upload
+      assertValues("bytes failed to upload", randomBytesFailed,
+          abfsOutputStreamStatistics.getBytesUploadFailed());
+    }
+  }
+
+  /**
+   * Tests to check time spent on waiting for tasks to be complete on a
+   * blocking queue in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamTimeSpentOnWaitTask() throws IOException {
+    describe("Testing Time Spend on Waiting for Task to be complete");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path timeSpendFilePath = new Path("AbfsOutputStreamStatsPath");
+
+    try (AbfsOutputStream out = createAbfsOutputStream(fs, timeSpendFilePath)) {
+
+      AbfsOutputStreamStatisticsImpl abfsOutputStreamStatistics =
+          out.getOutputStreamStatistics();
+
+      //Test for initial value of timeSpentWaitTask
+      assertValues("Time spent on waiting tasks", 0,
+          abfsOutputStreamStatistics.getTimeSpendOnTaskWait());
+
+      int smallRandomStartTime =
+          new Random().nextInt(LOW_RANGE_FOR_RANDOM_VALUE);
+      int smallRandomEndTime =
+          new Random().nextInt(LOW_RANGE_FOR_RANDOM_VALUE)
+              + smallRandomStartTime;
+      int smallDiff = smallRandomEndTime - smallRandomStartTime;
+      abfsOutputStreamStatistics
+          .timeSpentTaskWait(smallRandomStartTime, smallRandomEndTime);
+      //Test for small random value of timeSpentWaitTask
+      assertValues("Time spent on waiting tasks", smallDiff,
+          abfsOutputStreamStatistics.getTimeSpendOnTaskWait());
+
+      int largeRandomStartTime =
+          new Random().nextInt(HIGH_RANGE_FOR_RANDOM_VALUE);
+      int largeRandomEndTime = new Random().nextInt(HIGH_RANGE_FOR_RANDOM_VALUE)
+          + largeRandomStartTime;
+      int randomDiff = largeRandomEndTime - largeRandomStartTime;
+      abfsOutputStreamStatistics
+          .timeSpentTaskWait(largeRandomStartTime, largeRandomEndTime);
+      /*
+      Test for large random value of timeSpentWaitTask plus the time spent
+      in previous test
+       */
+      assertValues("Time spent on waiting tasks", smallDiff + randomDiff,
+          abfsOutputStreamStatistics.getTimeSpendOnTaskWait());
+    }
+
+  }
+
+  /**
+   * Tests to check number of {@code shrinkWriteOperationQueue()}
+   * calls.
+   * After writing data, AbfsOutputStream doesn't upload the data until
+   * Flushed. Hence, flush() method is called after write() to test Queue
+   * shrink calls.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamQueueShrink() throws IOException {
+    describe("Testing Queue Shrink calls in AbfsOutputStream");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path queueShrinkFilePath = new Path("AbfsOutputStreamStatsPath");
+    String testQueueShrink = "testQueue";
+
+    try (AbfsOutputStream outForOneOp = createAbfsOutputStream(fs,
+        queueShrinkFilePath)) {
+
+      //Test for shrinking Queue zero time
+      assertValues("number of queueShrink() Calls", 0,
+          outForOneOp.getOutputStreamStatistics().getQueueShrink());
+
+      outForOneOp.write(testQueueShrink.getBytes());
+      // Queue is shrunk 2 times when outStream is flushed
+      outForOneOp.flush();
+
+      //Test for shrinking Queue 2 times
+      assertValues("number of queueShrink() Calls", 2,
+          outForOneOp.getOutputStreamStatistics().getQueueShrink());
+
+    }
+
 
 Review comment:
   I am setting disableOutputStreamFlush to false, which implies flush is enabled.

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r404589011
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStream.java
 ##########
 @@ -0,0 +1,294 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+import java.util.Random;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl;
+import org.apache.hadoop.fs.permission.FsPermission;
+
+/**
+ * Test AbfsOutputStream statistics.
+ */
+public class ITestAbfsOutputStream extends AbstractAbfsIntegrationTest {
+  public ITestAbfsOutputStream() throws Exception {
+  }
+
+  private static final int LARGE_OPERATIONS = 10;
+  private static final int LOW_RANGE_FOR_RANDOM_VALUE = 49;
+  private static final int HIGH_RANGE_FOR_RANDOM_VALUE = 9999;
+
+  /**
+   * Tests to check bytes Uploading in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamUploadingBytes() throws IOException {
+    describe("Testing Bytes uploaded in AbfsOutputSteam");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path uploadBytesFilePath = new Path("AbfsOutputStreamStatsPath");
+    String testBytesToUpload = "bytes";
+
+    try (AbfsOutputStream outForSomeBytes = createAbfsOutputStream(fs,
+        uploadBytesFilePath)
+    ) {
+
+      //Test for zero bytes To upload
+      assertValues("bytes to upload", 0,
+          outForSomeBytes.getOutputStreamStatistics().getBytesToUpload());
+
+      outForSomeBytes.write(testBytesToUpload.getBytes());
+      outForSomeBytes.flush();
+      AbfsOutputStreamStatisticsImpl abfsOutputStreamStatistics =
+          outForSomeBytes.getOutputStreamStatistics();
+
+      //Test for bytes to upload
+      assertValues("bytes to upload", testBytesToUpload.getBytes().length,
+          abfsOutputStreamStatistics.getBytesToUpload());
+
+      //Test for successful bytes uploaded
+      assertValues("successful bytes uploaded",
+          testBytesToUpload.getBytes().length,
+          abfsOutputStreamStatistics.getBytesUploadSuccessful());
+
+      //Populating random value for bytesFailed
+      int randomBytesFailed = new Random().nextInt(LOW_RANGE_FOR_RANDOM_VALUE);
+      abfsOutputStreamStatistics.uploadFailed(randomBytesFailed);
+      //Test for bytes failed to upload
+      assertValues("number fo bytes failed to upload", randomBytesFailed,
+          abfsOutputStreamStatistics.getBytesUploadFailed());
+
+    }
+
+    try (AbfsOutputStream outForLargeBytes = createAbfsOutputStream(fs,
+        uploadBytesFilePath)) {
+
+      int largeValue = LARGE_OPERATIONS;
+      for (int i = 0; i < largeValue; i++) {
+        outForLargeBytes.write(testBytesToUpload.getBytes());
+      }
+      outForLargeBytes.flush();
+      AbfsOutputStreamStatisticsImpl abfsOutputStreamStatistics =
+          outForLargeBytes.getOutputStreamStatistics();
+
+      //Test for bytes to upload
+      assertValues("bytes to upload",
+          largeValue * (testBytesToUpload.getBytes().length),
+          abfsOutputStreamStatistics.getBytesToUpload());
+
+      //Test for successful bytes uploaded
+      assertValues("successful bytes uploaded",
+          largeValue * (testBytesToUpload.getBytes().length),
+          abfsOutputStreamStatistics.getBytesUploadSuccessful());
+
+      //Populating random values for bytesFailed
+      int randomBytesFailed = new Random().nextInt(HIGH_RANGE_FOR_RANDOM_VALUE);
+      abfsOutputStreamStatistics.uploadFailed(randomBytesFailed);
+      //Test for bytes failed to upload
+      assertValues("bytes failed to upload", randomBytesFailed,
+          abfsOutputStreamStatistics.getBytesUploadFailed());
+    }
+  }
+
+  /**
+   * Tests to check time spent on waiting for tasks to be complete on a
+   * blocking queue in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamTimeSpentOnWaitTask() throws IOException {
+    describe("Testing Time Spend on Waiting for Task to be complete");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path timeSpendFilePath = new Path("AbfsOutputStreamStatsPath");
+
+    try (AbfsOutputStream out = createAbfsOutputStream(fs, timeSpendFilePath)) {
+
+      AbfsOutputStreamStatisticsImpl abfsOutputStreamStatistics =
+          out.getOutputStreamStatistics();
+
+      //Test for initial value of timeSpentWaitTask
+      assertValues("Time spent on waiting tasks", 0,
+          abfsOutputStreamStatistics.getTimeSpendOnTaskWait());
+
+      int smallRandomStartTime =
+          new Random().nextInt(LOW_RANGE_FOR_RANDOM_VALUE);
+      int smallRandomEndTime =
+          new Random().nextInt(LOW_RANGE_FOR_RANDOM_VALUE)
+              + smallRandomStartTime;
+      int smallDiff = smallRandomEndTime - smallRandomStartTime;
+      abfsOutputStreamStatistics
+          .timeSpentTaskWait(smallRandomStartTime, smallRandomEndTime);
+      //Test for small random value of timeSpentWaitTask
+      assertValues("Time spent on waiting tasks", smallDiff,
+          abfsOutputStreamStatistics.getTimeSpendOnTaskWait());
+
+      int largeRandomStartTime =
+          new Random().nextInt(HIGH_RANGE_FOR_RANDOM_VALUE);
+      int largeRandomEndTime = new Random().nextInt(HIGH_RANGE_FOR_RANDOM_VALUE)
+          + largeRandomStartTime;
+      int randomDiff = largeRandomEndTime - largeRandomStartTime;
+      abfsOutputStreamStatistics
+          .timeSpentTaskWait(largeRandomStartTime, largeRandomEndTime);
+      /*
+      Test for large random value of timeSpentWaitTask plus the time spent
+      in previous test
+       */
+      assertValues("Time spent on waiting tasks", smallDiff + randomDiff,
+          abfsOutputStreamStatistics.getTimeSpendOnTaskWait());
+    }
+
+  }
+
+  /**
+   * Tests to check number of {@code shrinkWriteOperationQueue()}
+   * calls.
+   * After writing data, AbfsOutputStream doesn't upload the data until
+   * Flushed. Hence, flush() method is called after write() to test Queue
+   * shrink calls.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamQueueShrink() throws IOException {
+    describe("Testing Queue Shrink calls in AbfsOutputStream");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path queueShrinkFilePath = new Path("AbfsOutputStreamStatsPath");
+    String testQueueShrink = "testQueue";
+
+    try (AbfsOutputStream outForOneOp = createAbfsOutputStream(fs,
+        queueShrinkFilePath)) {
+
+      //Test for shrinking Queue zero time
+      assertValues("number of queueShrink() Calls", 0,
+          outForOneOp.getOutputStreamStatistics().getQueueShrink());
+
+      outForOneOp.write(testQueueShrink.getBytes());
+      // Queue is shrunk 2 times when outStream is flushed
+      outForOneOp.flush();
+
+      //Test for shrinking Queue 2 times
+      assertValues("number of queueShrink() Calls", 2,
+          outForOneOp.getOutputStreamStatistics().getQueueShrink());
+
+    }
+
+    try (AbfsOutputStream outForLargeOps = createAbfsOutputStream(fs,
+        queueShrinkFilePath)) {
+
+      int largeValue = LARGE_OPERATIONS;
+      for (int i = 0; i < largeValue; i++) {
+        outForLargeOps.write(testQueueShrink.getBytes());
+        outForLargeOps.flush();
+      }
+
+      //Test for 20 queue shrink calls
+      assertValues("number of queueShrink() Calls",
+          2 * largeValue,
+          outForLargeOps.getOutputStreamStatistics().getQueueShrink());
+    }
+
+  }
+
+  /**
+   * Test to check number of {@code writeCurrentBufferToService()}
+   * calls.
+   * After writing data, AbfsOutputStream doesn't upload data till flush() is
+   * called. Hence, flush() calls were made after write() to simulate the
+   * scenario.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamWriteBuffer() throws IOException {
+    describe("Testing writeCurrentBufferToService() calls");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path writeBufferFilePath = new Path("AbfsOutputStreamStatsPath");
+    String testWriteBuffer = "Buffer";
+
+    try (AbfsOutputStream outForOneOp = createAbfsOutputStream(fs,
+        writeBufferFilePath)) {
+
+      //Test for zero time writing Buffer to service
+      assertValues("number writeCurrentBufferToService() calls", 0,
+          outForOneOp.getOutputStreamStatistics()
+              .getWriteCurrentBufferOperations());
+
+      outForOneOp.write(testWriteBuffer.getBytes());
+      outForOneOp.flush();
+
+      //Test for one time writeCurrentBuffer() call
+      assertValues("number writeCurrentBufferToService() calls", 1,
+          outForOneOp.getOutputStreamStatistics()
+              .getWriteCurrentBufferOperations());
+    }
+
+    try (AbfsOutputStream outForLargeOps = createAbfsOutputStream(fs,
+        writeBufferFilePath)) {
+
+      int largeValue = LARGE_OPERATIONS;
+      for (int i = 0; i < largeValue; i++) {
+        outForLargeOps.write(testWriteBuffer.getBytes());
+        outForLargeOps.flush();
+      }
+      //Test for 10 writeBufferOperations
+      assertValues("number of writeCurrentBufferToService() calls", largeValue,
+          outForLargeOps
+              .getOutputStreamStatistics().getWriteCurrentBufferOperations());
+    }
+
+  }
+
+  /**
+   * Generic create File and setting OutputStreamFlush to false.
+   *
+   * @param fs   AzureBlobFileSystem that is initialised in the test
+   * @param path Path of the file to be created
+   * @return AbfsOutputStream for writing
+   * @throws AzureBlobFileSystemException
+   */
+  private AbfsOutputStream createAbfsOutputStream(AzureBlobFileSystem fs,
+      Path path) throws AzureBlobFileSystemException {
+    AzureBlobFileSystemStore abfss = fs.getAbfsStore();
+    abfss.getAbfsConfiguration().setDisableOutputStreamFlush(false);
+
+    return (AbfsOutputStream) abfss.createFile(path, fs.getFsStatistics(),
+        true, FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
+  }
+
+  /**
+   * Generic assert method.
+   *
+   * @param operation     operation being asserted
+   * @param expectedValue value that is expected
+   * @param actualValue   value that is actual
+   */
 
 Review comment:
   Same method was present in earlier patch. Move to base class and reuse.

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r408172102
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
 ##########
 @@ -279,6 +283,9 @@ public synchronized void close() throws IOException {
         threadExecutor.shutdownNow();
       }
     }
+    if (LOG.isDebugEnabled()) {
 
 Review comment:
   just create your own static LOG

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on issue #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#issuecomment-600547500
 
 
   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 28s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  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  |  21m 45s |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 27s |  trunk passed  |
   | +1 :green_heart: |  checkstyle  |   0m 20s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 30s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  16m 13s |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 23s |  trunk passed  |
   | +0 :ok: |  spotbugs  |   0m 49s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 48s |  trunk passed  |
   | -0 :warning: |  patch  |   1m  5s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 25s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 21s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 21s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 14s |  hadoop-tools/hadoop-azure: The patch generated 16 new + 1 unchanged - 0 fixed = 17 total (was 1)  |
   | +1 :green_heart: |  mvnsite  |   0m 24s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  15m 16s |  patch has no errors when building and testing our client artifacts.  |
   | -1 :x: |  javadoc  |   0m 19s |  hadoop-tools_hadoop-azure generated 2 new + 0 unchanged - 0 fixed = 2 total (was 0)  |
   | -1 :x: |  findbugs  |   0m 53s |  hadoop-tools/hadoop-azure generated 2 new + 0 unchanged - 0 fixed = 2 total (was 0)  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m  8s |  hadoop-azure in the patch passed.  |
   | -1 :x: |  asflicense  |   0m 27s |  The patch generated 3 ASF License warnings.  |
   |  |   |  61m 55s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hadoop-tools/hadoop-azure |
   |  |  Increment of volatile field org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl.queueShrink in org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl.queueShrinked()  At AbfsOutputStreamStatisticsImpl.java:in org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl.queueShrinked()  At AbfsOutputStreamStatisticsImpl.java:[line 70] |
   |  |  Increment of volatile field org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl.writeCurrentBufferOperations in org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl.writeCurrentBuffer()  At AbfsOutputStreamStatisticsImpl.java:in org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl.writeCurrentBuffer()  At AbfsOutputStreamStatisticsImpl.java:[line 78] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.8 Server=19.03.8 base: https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/2/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/1899 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux ff2694ac7a1e 4.15.0-74-generic #84-Ubuntu SMP Thu Dec 19 08:06:28 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | personality/hadoop.sh |
   | git revision | trunk / 8d63734 |
   | Default Java | 1.8.0_242 |
   | checkstyle | https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/2/artifact/out/diff-checkstyle-hadoop-tools_hadoop-azure.txt |
   | javadoc | https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/2/artifact/out/diff-javadoc-javadoc-hadoop-tools_hadoop-azure.txt |
   | findbugs | https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/2/artifact/out/new-findbugs-hadoop-tools_hadoop-azure.html |
   |  Test Results | https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/2/testReport/ |
   | asflicense | https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/2/artifact/out/patch-asflicense-problems.txt |
   | Max. process+thread count | 308 (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-1899/2/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 commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r395749601
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStream.java
 ##########
 @@ -0,0 +1,278 @@
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
+import org.apache.hadoop.fs.permission.FsPermission;
+
+/**
+ * Test AbfsOutputStream statistics.
+ */
+public class ITestAbfsOutputStream extends AbstractAbfsIntegrationTest {
+
+  public ITestAbfsOutputStream() throws Exception {
+  }
+
+  /**
+   * Tests to check bytes Uploading in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamUploadingBytes() throws IOException {
+    describe("Testing Bytes uploaded in AbfsOutputSteam");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path uploadBytesFilePath = new Path("AbfsOutputStreamStatsPath");
+    AzureBlobFileSystemStore abfss = fs.getAbfsStore();
+    FileSystem.Statistics statistics = fs.getFsStatistics();
+    abfss.getAbfsConfiguration().setDisableOutputStreamFlush(false);
+    String testBytesToUpload = "bytes";
+
+    AbfsOutputStream outForSomeBytes = null;
+    try {
+      outForSomeBytes = (AbfsOutputStream) abfss.createFile(uploadBytesFilePath,
+          statistics,
+          true,
+          FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
+
+      //Test for zero bytes To upload
+      assertValues("bytes to upload", 0,
+          outForSomeBytes.getOutputStreamStatistics().bytesToUpload);
+
+      outForSomeBytes.write(testBytesToUpload.getBytes());
+      outForSomeBytes.flush();
+
+      //Test for some bytes to upload
+      assertValues("bytes to upload", testBytesToUpload.getBytes().length,
+          outForSomeBytes.getOutputStreamStatistics().bytesToUpload);
+
+      //Test for relation between bytesUploadSuccessful, bytesUploadFailed
+      // and bytesToUpload
+      assertValues("bytesUploadSuccessful equal to difference between "
+              + "bytesToUpload and bytesUploadFailed",
+          outForSomeBytes.getOutputStreamStatistics().bytesUploadSuccessful,
+          outForSomeBytes.getOutputStreamStatistics().bytesToUpload -
+              outForSomeBytes.getOutputStreamStatistics().bytesUploadFailed);
+
+    } finally {
+      if (outForSomeBytes != null) {
+        outForSomeBytes.close();
+      }
+    }
+
+    AbfsOutputStream outForLargeBytes = null;
+    try {
+      outForLargeBytes =
+          (AbfsOutputStream) abfss.createFile(uploadBytesFilePath,
+              statistics
+              , true, FsPermission.getDefault(),
+              FsPermission.getUMask(fs.getConf()));
+
+      int largeValue = 100000;
+      for (int i = 0; i < largeValue; i++) {
+        outForLargeBytes.write(testBytesToUpload.getBytes());
+      }
+      outForLargeBytes.flush();
+
+      //Test for large bytes to upload
+      assertValues("bytes to upload",
+          largeValue * (testBytesToUpload.getBytes().length),
+          outForLargeBytes.getOutputStreamStatistics().bytesToUpload);
+
+      //Test for relation between bytesUploadSuccessful, bytesUploadFailed
+      // and bytesToUpload
+      assertValues("bytesUploadSuccessful equal to difference between "
+              + "bytesToUpload and bytesUploadFailed",
+          outForSomeBytes.getOutputStreamStatistics().bytesUploadSuccessful,
+          outForSomeBytes.getOutputStreamStatistics().bytesToUpload -
+              outForSomeBytes.getOutputStreamStatistics().bytesUploadFailed);
+
+    } finally {
+      if (outForLargeBytes != null) {
+        outForLargeBytes.close();
+      }
+    }
+
+  }
+
+  /**
+   * Tests to check time spend on waiting for tasks to be complete on a
+   * blocking queue in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamTimeSpendOnWaitTask() throws IOException {
 
 Review comment:
   I don't see any easy way except to assert that it is > 0

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r395751750
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStream.java
 ##########
 @@ -0,0 +1,278 @@
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
+import org.apache.hadoop.fs.permission.FsPermission;
+
+/**
+ * Test AbfsOutputStream statistics.
+ */
+public class ITestAbfsOutputStream extends AbstractAbfsIntegrationTest {
+
+  public ITestAbfsOutputStream() throws Exception {
+  }
+
+  /**
+   * Tests to check bytes Uploading in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamUploadingBytes() throws IOException {
+    describe("Testing Bytes uploaded in AbfsOutputSteam");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path uploadBytesFilePath = new Path("AbfsOutputStreamStatsPath");
+    AzureBlobFileSystemStore abfss = fs.getAbfsStore();
+    FileSystem.Statistics statistics = fs.getFsStatistics();
+    abfss.getAbfsConfiguration().setDisableOutputStreamFlush(false);
+    String testBytesToUpload = "bytes";
+
+    AbfsOutputStream outForSomeBytes = null;
+    try {
+      outForSomeBytes = (AbfsOutputStream) abfss.createFile(uploadBytesFilePath,
+          statistics,
+          true,
+          FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
+
+      //Test for zero bytes To upload
+      assertValues("bytes to upload", 0,
+          outForSomeBytes.getOutputStreamStatistics().bytesToUpload);
+
+      outForSomeBytes.write(testBytesToUpload.getBytes());
+      outForSomeBytes.flush();
+
+      //Test for some bytes to upload
+      assertValues("bytes to upload", testBytesToUpload.getBytes().length,
+          outForSomeBytes.getOutputStreamStatistics().bytesToUpload);
+
+      //Test for relation between bytesUploadSuccessful, bytesUploadFailed
+      // and bytesToUpload
+      assertValues("bytesUploadSuccessful equal to difference between "
+              + "bytesToUpload and bytesUploadFailed",
+          outForSomeBytes.getOutputStreamStatistics().bytesUploadSuccessful,
+          outForSomeBytes.getOutputStreamStatistics().bytesToUpload -
+              outForSomeBytes.getOutputStreamStatistics().bytesUploadFailed);
+
+    } finally {
+      if (outForSomeBytes != null) {
+        outForSomeBytes.close();
+      }
+    }
+
+    AbfsOutputStream outForLargeBytes = null;
+    try {
+      outForLargeBytes =
+          (AbfsOutputStream) abfss.createFile(uploadBytesFilePath,
+              statistics
+              , true, FsPermission.getDefault(),
+              FsPermission.getUMask(fs.getConf()));
+
+      int largeValue = 100000;
+      for (int i = 0; i < largeValue; i++) {
+        outForLargeBytes.write(testBytesToUpload.getBytes());
+      }
+      outForLargeBytes.flush();
+
+      //Test for large bytes to upload
+      assertValues("bytes to upload",
+          largeValue * (testBytesToUpload.getBytes().length),
+          outForLargeBytes.getOutputStreamStatistics().bytesToUpload);
+
+      //Test for relation between bytesUploadSuccessful, bytesUploadFailed
+      // and bytesToUpload
+      assertValues("bytesUploadSuccessful equal to difference between "
+              + "bytesToUpload and bytesUploadFailed",
+          outForSomeBytes.getOutputStreamStatistics().bytesUploadSuccessful,
+          outForSomeBytes.getOutputStreamStatistics().bytesToUpload -
+              outForSomeBytes.getOutputStreamStatistics().bytesUploadFailed);
+
+    } finally {
+      if (outForLargeBytes != null) {
+        outForLargeBytes.close();
+      }
+    }
+
+  }
+
+  /**
+   * Tests to check time spend on waiting for tasks to be complete on a
+   * blocking queue in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamTimeSpendOnWaitTask() throws IOException {
+    describe("Testing Time Spend on Waiting for Task to be complete");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path timeSpendFilePath = new Path("AbfsOutputStreamStatsPath");
+    AzureBlobFileSystemStore abfss = fs.getAbfsStore();
+    FileSystem.Statistics statistics = fs.getFsStatistics();
+
+    AbfsOutputStream out =
+        (AbfsOutputStream) abfss.createFile(timeSpendFilePath,
+            statistics, true,
+            FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
+
+  }
+
+  /**
+   * Tests to check number of {@codes shrinkWriteOperationQueue()}
+   * calls.
+   * After writing data, AbfsOutputStream doesn't upload the data until
+   * Flushed. Hence, flush() method is called after write() to test Queue
+   * shrink calls.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamQueueShrink() throws IOException {
+    describe("Testing Queue Shrink calls in AbfsOutputStream");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path queueShrinkFilePath = new Path("AbfsOutputStreamStatsPath");
+    AzureBlobFileSystemStore abfss = fs.getAbfsStore();
+    abfss.getAbfsConfiguration().setDisableOutputStreamFlush(false);
+    FileSystem.Statistics statistics = fs.getFsStatistics();
+    String testQueueShrink = "testQueue";
+
+    AbfsOutputStream outForOneOp = null;
+
+    try {
+      outForOneOp =
+          (AbfsOutputStream) abfss.createFile(queueShrinkFilePath, statistics,
+              true,
+              FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
+
+      //Test for shrinking Queue zero time
+      assertValues("number of queueShrink() Calls", 0,
+          outForOneOp.getOutputStreamStatistics().queueShrink);
+
+      outForOneOp.write(testQueueShrink.getBytes());
+      // Queue is shrunk 2 times when outStream is flushed
+      outForOneOp.flush();
+
+      //Test for shrinking Queue 2 times
+      assertValues("number of queueShrink() Calls", 2,
+          outForOneOp.getOutputStreamStatistics().queueShrink);
+
+    } finally {
+      if (outForOneOp != null) {
+        outForOneOp.close();
+      }
+    }
+
+    AbfsOutputStream outForLargeOps = null;
+
+    try {
+      outForLargeOps = (AbfsOutputStream) abfss.createFile(queueShrinkFilePath,
+          statistics, true,
+          FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
+
+      int largeValue = 1000;
+      //QueueShrink is called 2 times in 1 flush(), hence 1000 flushes must
+      // give 2000 QueueShrink calls
+      for (int i = 0; i < largeValue; i++) {
+        outForLargeOps.write(testQueueShrink.getBytes());
+        //Flush is quite expensive so 1000 calls only which takes 1 min+
+        outForLargeOps.flush();
+      }
+
+      //Test for 2000 queue shrink calls
+      assertValues("number of queueShrink() Calls",
+          2 * largeValue,
+          outForLargeOps.getOutputStreamStatistics().queueShrink);
+    } finally {
+      if (outForLargeOps != null) {
+        outForLargeOps.close();
+      }
+    }
+
+  }
+
+  /**
+   * Test to check number of {@codes writeCurrentBufferToService()}
+   * calls.
+   * After writing data, AbfsOutputStream doesn't upload data till flush() is
+   * called. Hence, flush() calls were made after write() to simulate the
+   * scenario.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamWriteBuffer() throws IOException {
+    describe("Testing writeCurrentBufferToService() calls");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path writeBufferFilePath = new Path("AbfsOutputStreamStatsPath");
+    AzureBlobFileSystemStore abfss = fs.getAbfsStore();
+    FileSystem.Statistics statistics = fs.getFsStatistics();
+    abfss.getAbfsConfiguration().setDisableOutputStreamFlush(false);
+    String testWriteBuffer = "Buffer";
+
+    AbfsOutputStream outForOneOp = null;
+
+    try {
 
 Review comment:
   try-with-resources or IOUtils.closeQuietly

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r408178198
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsOutputStreamStatistics.java
 ##########
 @@ -0,0 +1,120 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+import java.util.Random;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl;
+
+/**
+ * Unit Tests for AbfsOutputStream Statistics.
+ */
+public class TestAbfsOutputStreamStatistics
+    extends AbstractAbfsIntegrationTest {
+
+  private static final int LOW_RANGE_FOR_RANDOM_VALUE = 49;
+  private static final int HIGH_RANGE_FOR_RANDOM_VALUE = 9999;
+
+  public TestAbfsOutputStreamStatistics() throws Exception {
+  }
+
+  /**
+   * Tests to check bytes failed to Upload in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamBytesFailed() {
+    describe("Testing Bytes Failed during uploading in AbfsOutputSteam");
+
+    AbfsOutputStreamStatisticsImpl abfsOutputStreamStatistics =
+        new AbfsOutputStreamStatisticsImpl();
+
+    //Test for zero bytes uploaded.
+    assertValues("number fo bytes failed to upload", 0,
+        abfsOutputStreamStatistics.getBytesUploadFailed());
+
+    //Populating small random value for bytesFailed.
+    int randomBytesFailed = new Random().nextInt(LOW_RANGE_FOR_RANDOM_VALUE);
+    abfsOutputStreamStatistics.uploadFailed(randomBytesFailed);
+    //Test for bytes failed to upload.
+    assertValues("number fo bytes failed to upload", randomBytesFailed,
+        abfsOutputStreamStatistics.getBytesUploadFailed());
+
+    //Initializing again to reset the statistics.
+    abfsOutputStreamStatistics = new AbfsOutputStreamStatisticsImpl();
+
+    //Populating large random values for bytesFailed.
+    randomBytesFailed = new Random().nextInt(HIGH_RANGE_FOR_RANDOM_VALUE);
+    abfsOutputStreamStatistics.uploadFailed(randomBytesFailed);
+    //Test for bytes failed to upload.
+    assertValues("number fo bytes failed to upload", randomBytesFailed,
 
 Review comment:
   nit: typo

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r408173830
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamStatisticsImpl.java
 ##########
 @@ -0,0 +1,177 @@
+/**
+ * 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.services;
+
+/**
+ * OutputStream Statistics Implementation for Abfs.
+ */
+public class AbfsOutputStreamStatisticsImpl
+    implements AbfsOutputStreamStatistics {
+  private long bytesToUpload;
 
 Review comment:
   remember that discussion we had about volatile vs long and you concluded that we could shut yetus up by going non volatile?
   In #1820 I've moved s3a input stream stats to volatile so that the IOStatistics gets the latest values without blocking...and then turned off findbugs warnings (or at least, I'm trying to)

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r395743551
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java
 ##########
 @@ -101,6 +101,7 @@ public synchronized int read(final byte[] b, final int off, final int len) throw
     int currentLen = len;
     int lastReadBytes;
     int totalReadBytes = 0;
+    incrementReadOps();
 
 Review comment:
   this is input stream; presumably it's come in from somewhere else

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r408992189
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
 ##########
 @@ -384,6 +400,7 @@ private synchronized void flushWrittenBytesToServiceInternal(final long offset,
    * operation FIFO queue.
    */
   private synchronized void shrinkWriteOperationQueue() throws IOException {
+    outputStreamStatistics.queueShrunk();
 
 Review comment:
   Queue is not actually getting shrunk here, rather inside the while loop. I'll change this and the respective tests.

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r405022845
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
 ##########
 @@ -294,19 +299,23 @@ private synchronized void flushInternalAsync() throws IOException {
   }
 
   private synchronized void writeCurrentBufferToService() throws IOException {
+    outputStreamStatistics.writeCurrentBuffer();
 
 Review comment:
   yes, let's do that

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on issue #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#issuecomment-611419966
 
 
   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 47s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  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  |  22m 54s |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 35s |  trunk passed  |
   | +1 :green_heart: |  checkstyle  |   0m 24s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 32s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  16m 44s |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 24s |  trunk passed  |
   | +0 :ok: |  spotbugs  |   0m 52s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 51s |  trunk passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 27s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 23s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 23s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 14s |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 25s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  15m 36s |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 20s |  the patch passed  |
   | -1 :x: |  findbugs  |   0m 59s |  hadoop-tools/hadoop-azure generated 2 new + 0 unchanged - 0 fixed = 2 total (was 0)  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 20s |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 28s |  The patch does not generate ASF License warnings.  |
   |  |   |  65m 51s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hadoop-tools/hadoop-azure |
   |  |  Increment of volatile field org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl.queueShrunkOps in org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl.queueShrunk()  At AbfsOutputStreamStatisticsImpl.java:in org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl.queueShrunk()  At AbfsOutputStreamStatisticsImpl.java:[line 118] |
   |  |  Increment of volatile field org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl.writeCurrentBufferOperations in org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl.writeCurrentBuffer()  At AbfsOutputStreamStatisticsImpl.java:in org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl.writeCurrentBuffer()  At AbfsOutputStreamStatisticsImpl.java:[line 129] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.8 Server=19.03.8 base: https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/7/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/1899 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux cf949463ccb3 4.15.0-74-generic #84-Ubuntu SMP Thu Dec 19 08:06:28 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | personality/hadoop.sh |
   | git revision | trunk / e53d472 |
   | Default Java | 1.8.0_242 |
   | findbugs | https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/7/artifact/out/new-findbugs-hadoop-tools_hadoop-azure.html |
   |  Test Results | https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/7/testReport/ |
   | Max. process+thread count | 368 (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-1899/7/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 commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r404060217
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
 ##########
 @@ -80,6 +82,7 @@
           = new ElasticByteBufferPool();
 
   private final Statistics statistics;
+  private final AbfsOutputStreamStatisticsImpl outputStreamStatistics;
 
 Review comment:
   yes. Use the interface in reference/args and pass the instance in

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on issue #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#issuecomment-611144249
 
 
   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 33s |  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 5 new or modified test files.  |
   ||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  26m 39s |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 33s |  trunk passed  |
   | +1 :green_heart: |  checkstyle  |   0m 24s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 34s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  18m 47s |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 27s |  trunk passed  |
   | +0 :ok: |  spotbugs  |   1m  9s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   1m  8s |  trunk passed  |
   ||| _ Patch Compile Tests _ |
   | -1 :x: |  mvninstall  |   0m 26s |  hadoop-azure in the patch failed.  |
   | -1 :x: |  compile  |   0m 27s |  hadoop-azure in the patch failed.  |
   | -1 :x: |  javac  |   0m 27s |  hadoop-azure in the patch failed.  |
   | +1 :green_heart: |  checkstyle  |   0m 16s |  the patch passed  |
   | -1 :x: |  mvnsite  |   0m 29s |  hadoop-azure in the patch failed.  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  17m 36s |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 20s |  the patch passed  |
   | -1 :x: |  findbugs  |   0m 26s |  hadoop-azure in the patch failed.  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  |   0m 31s |  hadoop-azure in the patch failed.  |
   | +1 :green_heart: |  asflicense  |   0m 35s |  The patch does not generate ASF License warnings.  |
   |  |   |  71m 44s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.8 Server=19.03.8 base: https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/6/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/1899 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux 61f941a345b6 4.15.0-74-generic #84-Ubuntu SMP Thu Dec 19 08:06:28 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | personality/hadoop.sh |
   | git revision | trunk / e53d472 |
   | Default Java | 1.8.0_242 |
   | mvninstall | https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/6/artifact/out/patch-mvninstall-hadoop-tools_hadoop-azure.txt |
   | compile | https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/6/artifact/out/patch-compile-hadoop-tools_hadoop-azure.txt |
   | javac | https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/6/artifact/out/patch-compile-hadoop-tools_hadoop-azure.txt |
   | mvnsite | https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/6/artifact/out/patch-mvnsite-hadoop-tools_hadoop-azure.txt |
   | findbugs | https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/6/artifact/out/patch-findbugs-hadoop-tools_hadoop-azure.txt |
   | unit | https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/6/artifact/out/patch-unit-hadoop-tools_hadoop-azure.txt |
   |  Test Results | https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/6/testReport/ |
   | Max. process+thread count | 312 (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-1899/6/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] mukund-thakur commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r404611696
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStream.java
 ##########
 @@ -133,58 +133,60 @@ public void testAbfsOutputStreamTimeSpendOnWaitTask() throws IOException {
   public void testAbfsOutputStreamQueueShrink() throws IOException {
     describe("Testing Queue Shrink calls in AbfsOutputStream");
     final AzureBlobFileSystem fs = getFileSystem();
-    Path TEST_PATH = new Path("AbfsOutputStreamStatsPath");
+    Path queueShrinkFilePath = new Path("AbfsOutputStreamStatsPath");
     AzureBlobFileSystemStore abfss = fs.getAbfsStore();
     abfss.getAbfsConfiguration().setDisableOutputStreamFlush(false);
     FileSystem.Statistics statistics = fs.getFsStatistics();
     String testQueueShrink = "testQueue";
 
-
     AbfsOutputStream outForOneOp = null;
 
     try {
-      outForOneOp = (AbfsOutputStream) abfss.createFile(TEST_PATH, statistics,
-        true,
-          FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
+      outForOneOp =
+          (AbfsOutputStream) abfss.createFile(queueShrinkFilePath, statistics,
+              true,
+              FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
 
       //Test for shrinking Queue zero time
-      Assert.assertEquals("Mismatch in number of queueShrink() Calls", 0,
+      assertValues("number of queueShrink() Calls", 0,
           outForOneOp.getOutputStreamStatistics().queueShrink);
 
       outForOneOp.write(testQueueShrink.getBytes());
       // Queue is shrunk 2 times when outStream is flushed
       outForOneOp.flush();
 
       //Test for shrinking Queue 2 times
-      Assert.assertEquals("Mismatch in number of queueShrink() Calls", 2,
+      assertValues("number of queueShrink() Calls", 2,
           outForOneOp.getOutputStreamStatistics().queueShrink);
 
     } finally {
-      if(outForOneOp != null){
+      if (outForOneOp != null) {
         outForOneOp.close();
       }
     }
 
     AbfsOutputStream outForLargeOps = null;
 
     try {
-      outForLargeOps = (AbfsOutputStream) abfss.createFile(TEST_PATH,
+      outForLargeOps = (AbfsOutputStream) abfss.createFile(queueShrinkFilePath,
           statistics, true,
           FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
 
+      int largeValue = 1000;
       //QueueShrink is called 2 times in 1 flush(), hence 1000 flushes must
       // give 2000 QueueShrink calls
-      for (int i = 0; i < 1000; i++) {
+      for (int i = 0; i < largeValue; i++) {
         outForLargeOps.write(testQueueShrink.getBytes());
         //Flush is quite expensive so 1000 calls only which takes 1 min+
         outForLargeOps.flush();
 
 Review comment:
   You can call flush outside the for loop?

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r404586195
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStream.java
 ##########
 @@ -0,0 +1,294 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+import java.util.Random;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl;
+import org.apache.hadoop.fs.permission.FsPermission;
+
+/**
+ * Test AbfsOutputStream statistics.
+ */
+public class ITestAbfsOutputStream extends AbstractAbfsIntegrationTest {
+  public ITestAbfsOutputStream() throws Exception {
+  }
+
+  private static final int LARGE_OPERATIONS = 10;
+  private static final int LOW_RANGE_FOR_RANDOM_VALUE = 49;
+  private static final int HIGH_RANGE_FOR_RANDOM_VALUE = 9999;
+
+  /**
+   * Tests to check bytes Uploading in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamUploadingBytes() throws IOException {
+    describe("Testing Bytes uploaded in AbfsOutputSteam");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path uploadBytesFilePath = new Path("AbfsOutputStreamStatsPath");
 
 Review comment:
   use path(methodName.getMethodName());. The current code won't create your files under test directory so the cleanup might miss cleaning up them during teardown.

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on issue #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#issuecomment-607245767
 
 
   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |  36m 14s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  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 1 new or modified test files.  |
   ||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  20m 47s |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 32s |  trunk passed  |
   | +1 :green_heart: |  checkstyle  |   0m 23s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 34s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  15m  3s |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 26s |  trunk passed  |
   | +0 :ok: |  spotbugs  |   0m 53s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 52s |  trunk passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 27s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 23s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 23s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 16s |  hadoop-tools/hadoop-azure: The patch generated 6 new + 0 unchanged - 0 fixed = 6 total (was 0)  |
   | +1 :green_heart: |  mvnsite  |   0m 27s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  14m  0s |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 23s |  the patch passed  |
   | -1 :x: |  findbugs  |   0m 56s |  hadoop-tools/hadoop-azure generated 2 new + 0 unchanged - 0 fixed = 2 total (was 0)  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 22s |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 32s |  The patch does not generate ASF License warnings.  |
   |  |   |  95m 29s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hadoop-tools/hadoop-azure |
   |  |  Increment of volatile field org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl.queueShrink in org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl.queueShrinked()  At AbfsOutputStreamStatisticsImpl.java:in org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl.queueShrinked()  At AbfsOutputStreamStatisticsImpl.java:[line 92] |
   |  |  Increment of volatile field org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl.writeCurrentBufferOperations in org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl.writeCurrentBuffer()  At AbfsOutputStreamStatisticsImpl.java:in org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl.writeCurrentBuffer()  At AbfsOutputStreamStatisticsImpl.java:[line 100] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.8 Server=19.03.8 base: https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/3/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/1899 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux f3ca9902feab 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 / c162648 |
   | Default Java | 1.8.0_242 |
   | checkstyle | https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/3/artifact/out/diff-checkstyle-hadoop-tools_hadoop-azure.txt |
   | findbugs | https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/3/artifact/out/new-findbugs-hadoop-tools_hadoop-azure.html |
   |  Test Results | https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/3/testReport/ |
   | Max. process+thread count | 413 (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-1899/3/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 commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r395746011
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamStatisticsImpl.java
 ##########
 @@ -0,0 +1,102 @@
+package org.apache.hadoop.fs.azurebfs.services;
+
+/**
+ * OutputStream Statistics Implementation for Abfs.
+ * timeSpendOnTaskWait - Time spend on waiting for tasks to be complete on
+ * Blocking Queue in AbfsOutputStream.
+ *
+ * queueShrink - Number of times Blocking Queue was shrunk after writing
+ * data.
+ *
+ * WriteCurrentBufferOperations - Number of times
+ * {@codes writeCurrentBufferToService()} calls were made.
+ */
+public class AbfsOutputStreamStatisticsImpl
+    implements AbfsOutputStreamStatistics {
+  public volatile long bytesToUpload;
+  public volatile long bytesUploadSuccessful;
+  public volatile long bytesUploadFailed;
+  public volatile long timeSpendOnTaskWait;
+  public volatile long queueShrink;
+  public volatile long writeCurrentBufferOperations;
+
+  /**
+   * Number of bytes uploaded only when bytes passed are positive.
+   *
+   * @param bytes negative values are ignored
+   */
+  @Override
+  public void bytesToUpload(long bytes) {
+    if (bytes > 0) {
+      bytesToUpload += bytes;
+    }
+  }
+
+  @Override
+  public void bytesUploadedSuccessfully(long bytes) {
+    if (bytes > 0) {
+      bytesUploadSuccessful += bytes;
+    }
+  }
+
+  /**
+   * Number of bytes that weren't uploaded.
+   *
+   * @param bytes negative values are ignored
+   */
+  @Override
+  public void bytesFailed(long bytes) {
+    if (bytes > 0) {
+      bytesUploadFailed += bytes;
+    }
+  }
+
+  /**
+   * Time spend for waiting a task to be completed.
+   *
+   * @param startTime on calling {@link AbfsOutputStream#waitForTaskToComplete()}
 
 Review comment:
   MUST NOT use @link to private/package-private/protected methods. Javadoc will fail

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r408627634
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStreamStatistics.java
 ##########
 @@ -0,0 +1,233 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl;
+
+/**
+ * Test AbfsOutputStream statistics.
+ */
+public class ITestAbfsOutputStreamStatistics
+    extends AbstractAbfsIntegrationTest {
+  private static final int LARGE_OPERATIONS = 10;
+
+  public ITestAbfsOutputStreamStatistics() throws Exception {
+  }
+
+  /**
+   * Tests to check bytes Uploaded successfully in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamUploadingBytes() throws IOException {
+    describe("Testing Bytes uploaded successfully in AbfsOutputSteam");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path uploadBytesFilePath = path(getMethodName());
+    String testBytesToUpload = "bytes";
+
+    try (
+        AbfsOutputStream outForSomeBytes = createAbfsOutputStreamWithFlushEnabled(
+            fs,
+            uploadBytesFilePath)
+    ) {
+
+      AbfsOutputStreamStatisticsImpl abfsOutputStreamStatisticsForUploadBytes =
+          outForSomeBytes.getOutputStreamStatistics();
+
+      //Test for zero bytes To upload.
+      assertValues("bytes to upload", 0,
+          abfsOutputStreamStatisticsForUploadBytes.getBytesToUpload());
+
+      outForSomeBytes.write(testBytesToUpload.getBytes());
+      outForSomeBytes.flush();
+      abfsOutputStreamStatisticsForUploadBytes =
+          outForSomeBytes.getOutputStreamStatistics();
+
+      //Test for bytes to upload.
+      assertValues("bytes to upload", testBytesToUpload.getBytes().length,
+          abfsOutputStreamStatisticsForUploadBytes.getBytesToUpload());
+
+      //Test for successful bytes uploaded.
+      assertValues("successful bytes uploaded",
+          testBytesToUpload.getBytes().length,
+          abfsOutputStreamStatisticsForUploadBytes.getBytesUploadSuccessful());
+
+    }
+
+    try (
+        AbfsOutputStream outForLargeBytes = createAbfsOutputStreamWithFlushEnabled(
+            fs,
+            uploadBytesFilePath)) {
+
+      for (int i = 0; i < LARGE_OPERATIONS; i++) {
+        outForLargeBytes.write(testBytesToUpload.getBytes());
+      }
+      outForLargeBytes.flush();
+      AbfsOutputStreamStatisticsImpl abfsOutputStreamStatistics =
+          outForLargeBytes.getOutputStreamStatistics();
+
+      //Test for bytes to upload.
+      assertValues("bytes to upload",
+          LARGE_OPERATIONS * (testBytesToUpload.getBytes().length),
+          abfsOutputStreamStatistics.getBytesToUpload());
+
+      //Test for successful bytes uploaded.
+      assertValues("successful bytes uploaded",
+          LARGE_OPERATIONS * (testBytesToUpload.getBytes().length),
+          abfsOutputStreamStatistics.getBytesUploadSuccessful());
+
+    }
+  }
+
+  /**
+   * Tests to check number of {@code
+   * AbfsOutputStream#shrinkWriteOperationQueue()} calls.
+   * After writing data, AbfsOutputStream doesn't upload the data until
+   * Flushed. Hence, flush() method is called after write() to test Queue
+   * shrink calls.
+   *
+   * @throws IOException
 
 Review comment:
   So, the throws comments from all the javadoc in the tests, 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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r405059806
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStream.java
 ##########
 @@ -133,58 +133,60 @@ public void testAbfsOutputStreamTimeSpendOnWaitTask() throws IOException {
   public void testAbfsOutputStreamQueueShrink() throws IOException {
     describe("Testing Queue Shrink calls in AbfsOutputStream");
     final AzureBlobFileSystem fs = getFileSystem();
-    Path TEST_PATH = new Path("AbfsOutputStreamStatsPath");
+    Path queueShrinkFilePath = new Path("AbfsOutputStreamStatsPath");
     AzureBlobFileSystemStore abfss = fs.getAbfsStore();
     abfss.getAbfsConfiguration().setDisableOutputStreamFlush(false);
     FileSystem.Statistics statistics = fs.getFsStatistics();
     String testQueueShrink = "testQueue";
 
-
     AbfsOutputStream outForOneOp = null;
 
     try {
-      outForOneOp = (AbfsOutputStream) abfss.createFile(TEST_PATH, statistics,
-        true,
-          FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
+      outForOneOp =
+          (AbfsOutputStream) abfss.createFile(queueShrinkFilePath, statistics,
+              true,
+              FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
 
       //Test for shrinking Queue zero time
-      Assert.assertEquals("Mismatch in number of queueShrink() Calls", 0,
+      assertValues("number of queueShrink() Calls", 0,
           outForOneOp.getOutputStreamStatistics().queueShrink);
 
       outForOneOp.write(testQueueShrink.getBytes());
       // Queue is shrunk 2 times when outStream is flushed
       outForOneOp.flush();
 
       //Test for shrinking Queue 2 times
-      Assert.assertEquals("Mismatch in number of queueShrink() Calls", 2,
+      assertValues("number of queueShrink() Calls", 2,
           outForOneOp.getOutputStreamStatistics().queueShrink);
 
     } finally {
-      if(outForOneOp != null){
+      if (outForOneOp != null) {
         outForOneOp.close();
       }
     }
 
     AbfsOutputStream outForLargeOps = null;
 
     try {
-      outForLargeOps = (AbfsOutputStream) abfss.createFile(TEST_PATH,
+      outForLargeOps = (AbfsOutputStream) abfss.createFile(queueShrinkFilePath,
           statistics, true,
           FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
 
+      int largeValue = 1000;
       //QueueShrink is called 2 times in 1 flush(), hence 1000 flushes must
       // give 2000 QueueShrink calls
-      for (int i = 0; i < 1000; i++) {
+      for (int i = 0; i < largeValue; i++) {
         outForLargeOps.write(testQueueShrink.getBytes());
         //Flush is quite expensive so 1000 calls only which takes 1 min+
         outForLargeOps.flush();
 
 Review comment:
   No I can't, basically calling flush after I write means the write task is done. This would trigger the shrinkWriteOperationQueue() method and we need to do it after each read to get 20 operations.
   If I flush after the loop, it would take all the write calls as 1 write operation and only 1 time the shrinkWriteOperationQueue() method is triggered.

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r408179735
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamStatisticsImpl.java
 ##########
 @@ -0,0 +1,177 @@
+/**
+ * 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.services;
+
+/**
+ * OutputStream Statistics Implementation for Abfs.
+ */
+public class AbfsOutputStreamStatisticsImpl
+    implements AbfsOutputStreamStatistics {
+  private long bytesToUpload;
+  private long bytesUploadSuccessful;
+  private long bytesUploadFailed;
+  /**
+   * counter to get the total time spent while waiting for tasks to complete
+   * in the Blocking queue inside the thread executor.
+   */
+  private long timeSpendOnTaskWait;
+  /**
+   * counter to get the total number of queue shrink operations done{@code
+   * AbfsOutputStream#shrinkWriteOperationQueue()} by
+   * AbfsOutputStream to remove the write operations which were successfully
+   * done by AbfsOutputStream from the Blocking Queue.
+   */
+  private long queueShrunkOps;
+  /**
+   * counter to get the total number of times the current buffer is written
+   * to the service{@code AbfsOutputStream#writeCurrentBufferToService()} via
+   * AbfsClient and appended to the
+   * Data store by
+   * AbfsResOperation.
 
 Review comment:
   typo

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r405061359
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
 ##########
 @@ -384,6 +398,7 @@ private synchronized void flushWrittenBytesToServiceInternal(final long offset,
    * operation FIFO queue.
    */
   private synchronized void shrinkWriteOperationQueue() throws IOException {
+    outputStreamStatistics.queueShrinked();
 
 Review comment:
   It is an expensive method. So, I thought it would benefit in knowing how many times it's being called after some write operations. 
   Sorry for the spelling mistake.

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r405109621
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStream.java
 ##########
 @@ -0,0 +1,294 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+import java.util.Random;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl;
+import org.apache.hadoop.fs.permission.FsPermission;
+
+/**
+ * Test AbfsOutputStream statistics.
+ */
+public class ITestAbfsOutputStream extends AbstractAbfsIntegrationTest {
+  public ITestAbfsOutputStream() throws Exception {
+  }
+
+  private static final int LARGE_OPERATIONS = 10;
+  private static final int LOW_RANGE_FOR_RANDOM_VALUE = 49;
+  private static final int HIGH_RANGE_FOR_RANDOM_VALUE = 9999;
+
+  /**
+   * Tests to check bytes Uploading in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamUploadingBytes() throws IOException {
+    describe("Testing Bytes uploaded in AbfsOutputSteam");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path uploadBytesFilePath = new Path("AbfsOutputStreamStatsPath");
+    String testBytesToUpload = "bytes";
+
+    try (AbfsOutputStream outForSomeBytes = createAbfsOutputStream(fs,
+        uploadBytesFilePath)
+    ) {
+
+      //Test for zero bytes To upload
+      assertValues("bytes to upload", 0,
+          outForSomeBytes.getOutputStreamStatistics().getBytesToUpload());
+
+      outForSomeBytes.write(testBytesToUpload.getBytes());
+      outForSomeBytes.flush();
+      AbfsOutputStreamStatisticsImpl abfsOutputStreamStatistics =
+          outForSomeBytes.getOutputStreamStatistics();
+
+      //Test for bytes to upload
+      assertValues("bytes to upload", testBytesToUpload.getBytes().length,
+          abfsOutputStreamStatistics.getBytesToUpload());
+
+      //Test for successful bytes uploaded
+      assertValues("successful bytes uploaded",
+          testBytesToUpload.getBytes().length,
+          abfsOutputStreamStatistics.getBytesUploadSuccessful());
+
+      //Populating random value for bytesFailed
+      int randomBytesFailed = new Random().nextInt(LOW_RANGE_FOR_RANDOM_VALUE);
+      abfsOutputStreamStatistics.uploadFailed(randomBytesFailed);
+      //Test for bytes failed to upload
+      assertValues("number fo bytes failed to upload", randomBytesFailed,
+          abfsOutputStreamStatistics.getBytesUploadFailed());
+
+    }
+
+    try (AbfsOutputStream outForLargeBytes = createAbfsOutputStream(fs,
+        uploadBytesFilePath)) {
+
+      int largeValue = LARGE_OPERATIONS;
+      for (int i = 0; i < largeValue; i++) {
+        outForLargeBytes.write(testBytesToUpload.getBytes());
+      }
+      outForLargeBytes.flush();
+      AbfsOutputStreamStatisticsImpl abfsOutputStreamStatistics =
+          outForLargeBytes.getOutputStreamStatistics();
+
+      //Test for bytes to upload
+      assertValues("bytes to upload",
+          largeValue * (testBytesToUpload.getBytes().length),
+          abfsOutputStreamStatistics.getBytesToUpload());
+
+      //Test for successful bytes uploaded
+      assertValues("successful bytes uploaded",
+          largeValue * (testBytesToUpload.getBytes().length),
+          abfsOutputStreamStatistics.getBytesUploadSuccessful());
+
+      //Populating random values for bytesFailed
+      int randomBytesFailed = new Random().nextInt(HIGH_RANGE_FOR_RANDOM_VALUE);
+      abfsOutputStreamStatistics.uploadFailed(randomBytesFailed);
+      //Test for bytes failed to upload
+      assertValues("bytes failed to upload", randomBytesFailed,
+          abfsOutputStreamStatistics.getBytesUploadFailed());
+    }
+  }
+
+  /**
+   * Tests to check time spent on waiting for tasks to be complete on a
+   * blocking queue in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamTimeSpentOnWaitTask() throws IOException {
+    describe("Testing Time Spend on Waiting for Task to be complete");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path timeSpendFilePath = new Path("AbfsOutputStreamStatsPath");
+
+    try (AbfsOutputStream out = createAbfsOutputStream(fs, timeSpendFilePath)) {
+
+      AbfsOutputStreamStatisticsImpl abfsOutputStreamStatistics =
+          out.getOutputStreamStatistics();
+
+      //Test for initial value of timeSpentWaitTask
+      assertValues("Time spent on waiting tasks", 0,
+          abfsOutputStreamStatistics.getTimeSpendOnTaskWait());
+
+      int smallRandomStartTime =
+          new Random().nextInt(LOW_RANGE_FOR_RANDOM_VALUE);
+      int smallRandomEndTime =
+          new Random().nextInt(LOW_RANGE_FOR_RANDOM_VALUE)
+              + smallRandomStartTime;
+      int smallDiff = smallRandomEndTime - smallRandomStartTime;
+      abfsOutputStreamStatistics
+          .timeSpentTaskWait(smallRandomStartTime, smallRandomEndTime);
+      //Test for small random value of timeSpentWaitTask
+      assertValues("Time spent on waiting tasks", smallDiff,
+          abfsOutputStreamStatistics.getTimeSpendOnTaskWait());
+
+      int largeRandomStartTime =
+          new Random().nextInt(HIGH_RANGE_FOR_RANDOM_VALUE);
+      int largeRandomEndTime = new Random().nextInt(HIGH_RANGE_FOR_RANDOM_VALUE)
+          + largeRandomStartTime;
+      int randomDiff = largeRandomEndTime - largeRandomStartTime;
+      abfsOutputStreamStatistics
+          .timeSpentTaskWait(largeRandomStartTime, largeRandomEndTime);
+      /*
+      Test for large random value of timeSpentWaitTask plus the time spent
+      in previous test
+       */
+      assertValues("Time spent on waiting tasks", smallDiff + randomDiff,
+          abfsOutputStreamStatistics.getTimeSpendOnTaskWait());
+    }
+
+  }
+
+  /**
+   * Tests to check number of {@code shrinkWriteOperationQueue()}
+   * calls.
+   * After writing data, AbfsOutputStream doesn't upload the data until
+   * Flushed. Hence, flush() method is called after write() to test Queue
+   * shrink calls.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamQueueShrink() throws IOException {
+    describe("Testing Queue Shrink calls in AbfsOutputStream");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path queueShrinkFilePath = new Path("AbfsOutputStreamStatsPath");
+    String testQueueShrink = "testQueue";
+
+    try (AbfsOutputStream outForOneOp = createAbfsOutputStream(fs,
+        queueShrinkFilePath)) {
+
+      //Test for shrinking Queue zero time
+      assertValues("number of queueShrink() Calls", 0,
+          outForOneOp.getOutputStreamStatistics().getQueueShrink());
+
+      outForOneOp.write(testQueueShrink.getBytes());
+      // Queue is shrunk 2 times when outStream is flushed
+      outForOneOp.flush();
+
+      //Test for shrinking Queue 2 times
+      assertValues("number of queueShrink() Calls", 2,
+          outForOneOp.getOutputStreamStatistics().getQueueShrink());
+
+    }
+
 
 Review comment:
   I had a mistake in the comments of that method so, that is why you could've assumed it to be disabling flush. Sorry for that.

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r404591370
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamStatisticsImpl.java
 ##########
 @@ -0,0 +1,148 @@
+/**
+ * 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.services;
+
+/**
+ * OutputStream Statistics Implementation for Abfs.
+ * timeSpendOnTaskWait - Time spend on waiting for tasks to be complete on
+ * Blocking Queue in AbfsOutputStream.
+ *
+ * queueShrink - Number of times Blocking Queue was shrunk after writing
+ * data.
+ *
+ * WriteCurrentBufferOperations - Number of times
+ * {@code writeCurrentBufferToService()} calls were made.
+ */
+public class AbfsOutputStreamStatisticsImpl
+    implements AbfsOutputStreamStatistics {
+  private volatile long bytesToUpload;
+  private volatile long bytesUploadSuccessful;
+  private volatile long bytesUploadFailed;
+  private volatile long timeSpendOnTaskWait;
+  private volatile long queueShrink;
+  private volatile long writeCurrentBufferOperations;
+
+  /**
+   * Number of bytes uploaded.
+   *
+   * @param bytes negative values are ignored
+   */
+  @Override
+  public void bytesToUpload(long bytes) {
+    if (bytes > 0) {
+      bytesToUpload += bytes;
+    }
+  }
+
+  /**
+   * Upload successful with the number of bytes.
+   * @param bytes number of bytes that were successfully uploaded
+   */
+  @Override
+  public void uploadSuccessful(long bytes) {
+    if (bytes > 0) {
+      bytesUploadSuccessful += bytes;
+    }
+  }
+
+  /**
+   * Upload failed and the number of bytes.
+   *
+   * @param bytes negative values are ignored
+   */
+  @Override
+  public void uploadFailed(long bytes) {
+    if (bytes > 0) {
+      bytesUploadFailed += bytes;
+    }
+  }
+
+  /**
+   * Time spent for waiting a task to be completed.
+   *
+   * @param startTime on calling {@code waitForTaskToComplete()}
+   * @param endTime   on method completing
+   */
+  @Override
+  public void timeSpentTaskWait(long startTime, long endTime) {
+    timeSpendOnTaskWait += endTime - startTime;
+  }
+
+  /**
+   * Number of calls to {@code shrinkWriteOperationQueue()}.
+   */
+  @Override
+  public void queueShrinked() {
 
 Review comment:
   How is this metrics important??

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r395744474
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
 ##########
 @@ -36,20 +36,25 @@
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 
+import org.apache.hadoop.fs.FileSystem.Statistics;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
 import org.apache.hadoop.io.ElasticByteBufferPool;
 import org.apache.hadoop.fs.FSExceptionMessages;
 import org.apache.hadoop.fs.StreamCapabilities;
 import org.apache.hadoop.fs.Syncable;
 
+import static org.apache.hadoop.io.IOUtils.LOG;
 import static org.apache.hadoop.io.IOUtils.wrapException;
 
 /**
  * The BlobFsOutputStream for Rest AbfsClient.
  */
 public class AbfsOutputStream extends OutputStream implements Syncable, StreamCapabilities {
+
 
 Review comment:
   add both new fields at the bottom of the other fields, e.g Line 85, and keep togeher. 

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r395752609
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStream.java
 ##########
 @@ -0,0 +1,278 @@
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
+import org.apache.hadoop.fs.permission.FsPermission;
+
+/**
+ * Test AbfsOutputStream statistics.
+ */
+public class ITestAbfsOutputStream extends AbstractAbfsIntegrationTest {
 
 Review comment:
   This is sounds like a slow test.
   
   1. Use smaller values than 1000, e.g. "10"
   2. make the value a constant used across all tests. 

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r395753822
 
 

 ##########
 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.closeQuietly

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r404600592
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
 ##########
 @@ -294,19 +299,23 @@ private synchronized void flushInternalAsync() throws IOException {
   }
 
   private synchronized void writeCurrentBufferToService() throws IOException {
+    outputStreamStatistics.writeCurrentBuffer();
 
 Review comment:
   Should go down after if clause?

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r408179593
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamStatisticsImpl.java
 ##########
 @@ -0,0 +1,177 @@
+/**
+ * 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.services;
+
+/**
+ * OutputStream Statistics Implementation for Abfs.
+ */
+public class AbfsOutputStreamStatisticsImpl
+    implements AbfsOutputStreamStatistics {
+  private long bytesToUpload;
+  private long bytesUploadSuccessful;
+  private long bytesUploadFailed;
+  /**
+   * counter to get the total time spent while waiting for tasks to complete
+   * in the Blocking queue inside the thread executor.
+   */
+  private long timeSpendOnTaskWait;
 
 Review comment:
   timeSpent

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet edited a comment on issue #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#issuecomment-607205921
 
 
   getting timeout errors in abfs-tests(might be server issue):
   
   ```
   [ERROR] Errors:
   [ERROR]   ITestAzureBlobFileSystemDelete.testDeleteFirstLevelDirectory:127->lambda$testDeleteFirstLevelDirectory$1:128 » TestTimedOut
   [ERROR]   ITestAzureBlobFileSystemMainOperation>FSMainOperationsBaseTest.testDeleteEmptyDirectory:787 » AbfsRestOperation
   [ERROR]   ITestAzureBlobFileSystemMainOperation>FSMainOperationsBaseTest.testRenameDirectoryAsEmptyDirectory:1033->FSMainOperationsBaseTest.rename:1154 » AbfsRestOperation
   
   
   [ERROR] Errors:
   [ERROR]   ITestAbfsFileSystemContractDelete>AbstractContractDeleteTest.testDeleteEmptyDirNonRecursive:37->AbstractFSContractTestBase.assertDeleted:369 » TestTimedOut
   [ERROR]   ITestAbfsFileSystemContractDelete>AbstractContractDeleteTest.testDeleteNonEmptyDirNonRecursive:77 » TestTimedOut
   [ERROR]   ITestAbfsFileSystemContractMkdir>AbstractContractMkdirTest.testMkDirRmDir:46->AbstractFSContractTestBase.assertDeleted:369 » TestTimedOut
   [ERROR]   ITestAzureBlobFileSystemBasics>FileSystemContractBaseTest.testDeleteEmptyDirectory:441 » TestTimedOut
   [ERROR]   ITestAzureBlobFileSystemBasics>FileSystemContractBaseTest.testDeleteRecursively:421 » TestTimedOut
   [ERROR]   ITestAzureBlobFileSystemBasics>FileSystemContractBaseTest.testFilesystemIsCaseSensitive:651 » TestTimedOut
   [ERROR]   ITestAzureBlobFileSystemBasics.setUp:49 » TestTimedOut test timed out after 30...
   [ERROR]   ITestAzureBlobFileSystemBasics>FileSystemContractBaseTest.testRenameDirectoryAsExistingDirectory:548->FileSystemContractBaseTest.createFile:590 » TestTimedOut
   [ERROR]   ITestAzureBlobFileSystemBasics>FileSystemContractBaseTest.testRenameDirectoryMoveToExistingDirectory:515->FileSystemContractBaseTest.createFile:590 » TestTimedOut
   [ERROR]   ITestAzureBlobFileSystemBasics.tearDown:61 » TestTimedOut test timed out after...

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r408188275
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsOutputStreamStatistics.java
 ##########
 @@ -0,0 +1,120 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+import java.util.Random;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl;
+
+/**
+ * Unit Tests for AbfsOutputStream Statistics.
+ */
+public class TestAbfsOutputStreamStatistics
+    extends AbstractAbfsIntegrationTest {
+
+  private static final int LOW_RANGE_FOR_RANDOM_VALUE = 49;
+  private static final int HIGH_RANGE_FOR_RANDOM_VALUE = 9999;
+
+  public TestAbfsOutputStreamStatistics() throws Exception {
+  }
+
+  /**
+   * Tests to check bytes failed to Upload in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamBytesFailed() {
+    describe("Testing Bytes Failed during uploading in AbfsOutputSteam");
+
+    AbfsOutputStreamStatisticsImpl abfsOutputStreamStatistics =
+        new AbfsOutputStreamStatisticsImpl();
+
+    //Test for zero bytes uploaded.
+    assertValues("number fo bytes failed to upload", 0,
+        abfsOutputStreamStatistics.getBytesUploadFailed());
+
+    //Populating small random value for bytesFailed.
+    int randomBytesFailed = new Random().nextInt(LOW_RANGE_FOR_RANDOM_VALUE);
+    abfsOutputStreamStatistics.uploadFailed(randomBytesFailed);
+    //Test for bytes failed to upload.
+    assertValues("number fo bytes failed to upload", randomBytesFailed,
+        abfsOutputStreamStatistics.getBytesUploadFailed());
+
+    //Initializing again to reset the statistics.
+    abfsOutputStreamStatistics = new AbfsOutputStreamStatisticsImpl();
+
+    //Populating large random values for bytesFailed.
+    randomBytesFailed = new Random().nextInt(HIGH_RANGE_FOR_RANDOM_VALUE);
+    abfsOutputStreamStatistics.uploadFailed(randomBytesFailed);
+    //Test for bytes failed to upload.
+    assertValues("number fo bytes failed to upload", randomBytesFailed,
+        abfsOutputStreamStatistics.getBytesUploadFailed());
+  }
+
+  /**
+   * Tests to check time spent on waiting for tasks to be complete on a
+   * blocking queue in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamTimeSpentOnWaitTask() {
+    describe("Testing Time Spend on Waiting for Task to be complete");
+
+    AbfsOutputStreamStatisticsImpl abfsOutputStreamStatistics =
+        new AbfsOutputStreamStatisticsImpl();
+
+    //Test for initial value of timeSpentWaitTask.
+    assertValues("Time spend on waiting for tasks to complete", 0,
+        abfsOutputStreamStatistics.getTimeSpendOnTaskWait());
+
+    int smallRandomStartTime =
+        new Random().nextInt(LOW_RANGE_FOR_RANDOM_VALUE);
+    int smallRandomEndTime =
+        new Random().nextInt(LOW_RANGE_FOR_RANDOM_VALUE)
+            + smallRandomStartTime;
+    int smallDiff = smallRandomEndTime - smallRandomStartTime;
+    abfsOutputStreamStatistics
+        .timeSpentTaskWait(smallRandomStartTime, smallRandomEndTime);
+    //Test for small random value of timeSpentWaitTask.
+    assertValues("Time spend on waiting for tasks to complete", smallDiff,
+        abfsOutputStreamStatistics.getTimeSpendOnTaskWait());
+
+    int largeRandomStartTime =
+        new Random().nextInt(HIGH_RANGE_FOR_RANDOM_VALUE);
+    int largeRandomEndTime = new Random().nextInt(HIGH_RANGE_FOR_RANDOM_VALUE)
+        + largeRandomStartTime;
+    int randomDiff = largeRandomEndTime - largeRandomStartTime;
+    abfsOutputStreamStatistics
+        .timeSpentTaskWait(largeRandomStartTime, largeRandomEndTime);
+      /*
+      Test for large random value of timeSpentWaitTask plus the time spent
+      waiting in previous test.
+       */
+    assertValues("Time spend on waiting for tasks to complete",
+        smallDiff + randomDiff,
+        abfsOutputStreamStatistics.getTimeSpendOnTaskWait());
+  }
+
+}
 
 Review comment:
   No new line at the end. I think this will cause checkstyle issue.

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r404588417
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStream.java
 ##########
 @@ -0,0 +1,294 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+import java.util.Random;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl;
+import org.apache.hadoop.fs.permission.FsPermission;
+
+/**
+ * Test AbfsOutputStream statistics.
+ */
+public class ITestAbfsOutputStream extends AbstractAbfsIntegrationTest {
+  public ITestAbfsOutputStream() throws Exception {
+  }
+
+  private static final int LARGE_OPERATIONS = 10;
+  private static final int LOW_RANGE_FOR_RANDOM_VALUE = 49;
+  private static final int HIGH_RANGE_FOR_RANDOM_VALUE = 9999;
+
+  /**
+   * Tests to check bytes Uploading in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamUploadingBytes() throws IOException {
+    describe("Testing Bytes uploaded in AbfsOutputSteam");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path uploadBytesFilePath = new Path("AbfsOutputStreamStatsPath");
+    String testBytesToUpload = "bytes";
+
+    try (AbfsOutputStream outForSomeBytes = createAbfsOutputStream(fs,
+        uploadBytesFilePath)
+    ) {
+
+      //Test for zero bytes To upload
+      assertValues("bytes to upload", 0,
 
 Review comment:
   extract in variable utForSomeBytes.getOutputStreamStatistics(). like it is done at L65.

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r404605628
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStream.java
 ##########
 @@ -0,0 +1,294 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+import java.util.Random;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl;
+import org.apache.hadoop.fs.permission.FsPermission;
+
+/**
+ * Test AbfsOutputStream statistics.
+ */
+public class ITestAbfsOutputStream extends AbstractAbfsIntegrationTest {
+  public ITestAbfsOutputStream() throws Exception {
+  }
+
+  private static final int LARGE_OPERATIONS = 10;
+  private static final int LOW_RANGE_FOR_RANDOM_VALUE = 49;
+  private static final int HIGH_RANGE_FOR_RANDOM_VALUE = 9999;
+
+  /**
+   * Tests to check bytes Uploading in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamUploadingBytes() throws IOException {
+    describe("Testing Bytes uploaded in AbfsOutputSteam");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path uploadBytesFilePath = new Path("AbfsOutputStreamStatsPath");
+    String testBytesToUpload = "bytes";
+
+    try (AbfsOutputStream outForSomeBytes = createAbfsOutputStream(fs,
+        uploadBytesFilePath)
+    ) {
+
+      //Test for zero bytes To upload
+      assertValues("bytes to upload", 0,
+          outForSomeBytes.getOutputStreamStatistics().getBytesToUpload());
+
+      outForSomeBytes.write(testBytesToUpload.getBytes());
+      outForSomeBytes.flush();
+      AbfsOutputStreamStatisticsImpl abfsOutputStreamStatistics =
+          outForSomeBytes.getOutputStreamStatistics();
+
+      //Test for bytes to upload
+      assertValues("bytes to upload", testBytesToUpload.getBytes().length,
+          abfsOutputStreamStatistics.getBytesToUpload());
+
+      //Test for successful bytes uploaded
+      assertValues("successful bytes uploaded",
+          testBytesToUpload.getBytes().length,
+          abfsOutputStreamStatistics.getBytesUploadSuccessful());
+
+      //Populating random value for bytesFailed
+      int randomBytesFailed = new Random().nextInt(LOW_RANGE_FOR_RANDOM_VALUE);
+      abfsOutputStreamStatistics.uploadFailed(randomBytesFailed);
+      //Test for bytes failed to upload
+      assertValues("number fo bytes failed to upload", randomBytesFailed,
+          abfsOutputStreamStatistics.getBytesUploadFailed());
+
+    }
+
+    try (AbfsOutputStream outForLargeBytes = createAbfsOutputStream(fs,
+        uploadBytesFilePath)) {
+
+      int largeValue = LARGE_OPERATIONS;
+      for (int i = 0; i < largeValue; i++) {
+        outForLargeBytes.write(testBytesToUpload.getBytes());
+      }
+      outForLargeBytes.flush();
+      AbfsOutputStreamStatisticsImpl abfsOutputStreamStatistics =
+          outForLargeBytes.getOutputStreamStatistics();
+
+      //Test for bytes to upload
+      assertValues("bytes to upload",
+          largeValue * (testBytesToUpload.getBytes().length),
+          abfsOutputStreamStatistics.getBytesToUpload());
+
+      //Test for successful bytes uploaded
+      assertValues("successful bytes uploaded",
+          largeValue * (testBytesToUpload.getBytes().length),
+          abfsOutputStreamStatistics.getBytesUploadSuccessful());
+
+      //Populating random values for bytesFailed
+      int randomBytesFailed = new Random().nextInt(HIGH_RANGE_FOR_RANDOM_VALUE);
+      abfsOutputStreamStatistics.uploadFailed(randomBytesFailed);
+      //Test for bytes failed to upload
+      assertValues("bytes failed to upload", randomBytesFailed,
+          abfsOutputStreamStatistics.getBytesUploadFailed());
+    }
+  }
+
+  /**
+   * Tests to check time spent on waiting for tasks to be complete on a
+   * blocking queue in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamTimeSpentOnWaitTask() throws IOException {
+    describe("Testing Time Spend on Waiting for Task to be complete");
+    final AzureBlobFileSystem fs = getFileSystem();
 
 Review comment:
   same use path(getMethodName())

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r408178910
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsOutputStreamStatistics.java
 ##########
 @@ -0,0 +1,120 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+import java.util.Random;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl;
+
+/**
+ * Unit Tests for AbfsOutputStream Statistics.
+ */
+public class TestAbfsOutputStreamStatistics
+    extends AbstractAbfsIntegrationTest {
+
+  private static final int LOW_RANGE_FOR_RANDOM_VALUE = 49;
+  private static final int HIGH_RANGE_FOR_RANDOM_VALUE = 9999;
+
+  public TestAbfsOutputStreamStatistics() throws Exception {
+  }
+
+  /**
+   * Tests to check bytes failed to Upload in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamBytesFailed() {
+    describe("Testing Bytes Failed during uploading in AbfsOutputSteam");
+
+    AbfsOutputStreamStatisticsImpl abfsOutputStreamStatistics =
+        new AbfsOutputStreamStatisticsImpl();
+
+    //Test for zero bytes uploaded.
+    assertValues("number fo bytes failed to upload", 0,
+        abfsOutputStreamStatistics.getBytesUploadFailed());
+
+    //Populating small random value for bytesFailed.
+    int randomBytesFailed = new Random().nextInt(LOW_RANGE_FOR_RANDOM_VALUE);
+    abfsOutputStreamStatistics.uploadFailed(randomBytesFailed);
+    //Test for bytes failed to upload.
+    assertValues("number fo bytes failed to upload", randomBytesFailed,
+        abfsOutputStreamStatistics.getBytesUploadFailed());
+
+    //Initializing again to reset the statistics.
+    abfsOutputStreamStatistics = new AbfsOutputStreamStatisticsImpl();
+
+    //Populating large random values for bytesFailed.
+    randomBytesFailed = new Random().nextInt(HIGH_RANGE_FOR_RANDOM_VALUE);
+    abfsOutputStreamStatistics.uploadFailed(randomBytesFailed);
+    //Test for bytes failed to upload.
+    assertValues("number fo bytes failed to upload", randomBytesFailed,
+        abfsOutputStreamStatistics.getBytesUploadFailed());
+  }
+
+  /**
+   * Tests to check time spent on waiting for tasks to be complete on a
+   * blocking queue in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamTimeSpentOnWaitTask() {
+    describe("Testing Time Spend on Waiting for Task to be complete");
+
+    AbfsOutputStreamStatisticsImpl abfsOutputStreamStatistics =
+        new AbfsOutputStreamStatisticsImpl();
+
+    //Test for initial value of timeSpentWaitTask.
+    assertValues("Time spend on waiting for tasks to complete", 0,
+        abfsOutputStreamStatistics.getTimeSpendOnTaskWait());
+
+    int smallRandomStartTime =
+        new Random().nextInt(LOW_RANGE_FOR_RANDOM_VALUE);
+    int smallRandomEndTime =
+        new Random().nextInt(LOW_RANGE_FOR_RANDOM_VALUE)
+            + smallRandomStartTime;
+    int smallDiff = smallRandomEndTime - smallRandomStartTime;
+    abfsOutputStreamStatistics
+        .timeSpentTaskWait(smallRandomStartTime, smallRandomEndTime);
+    //Test for small random value of timeSpentWaitTask.
+    assertValues("Time spend on waiting for tasks to complete", smallDiff,
+        abfsOutputStreamStatistics.getTimeSpendOnTaskWait());
+
+    int largeRandomStartTime =
+        new Random().nextInt(HIGH_RANGE_FOR_RANDOM_VALUE);
+    int largeRandomEndTime = new Random().nextInt(HIGH_RANGE_FOR_RANDOM_VALUE)
+        + largeRandomStartTime;
+    int randomDiff = largeRandomEndTime - largeRandomStartTime;
+    abfsOutputStreamStatistics
+        .timeSpentTaskWait(largeRandomStartTime, largeRandomEndTime);
+      /*
 
 Review comment:
   Indentation.

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r404608299
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStream.java
 ##########
 @@ -0,0 +1,294 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+import java.util.Random;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl;
+import org.apache.hadoop.fs.permission.FsPermission;
+
+/**
+ * Test AbfsOutputStream statistics.
+ */
+public class ITestAbfsOutputStream extends AbstractAbfsIntegrationTest {
+  public ITestAbfsOutputStream() throws Exception {
+  }
+
+  private static final int LARGE_OPERATIONS = 10;
+  private static final int LOW_RANGE_FOR_RANDOM_VALUE = 49;
+  private static final int HIGH_RANGE_FOR_RANDOM_VALUE = 9999;
+
+  /**
+   * Tests to check bytes Uploading in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamUploadingBytes() throws IOException {
+    describe("Testing Bytes uploaded in AbfsOutputSteam");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path uploadBytesFilePath = new Path("AbfsOutputStreamStatsPath");
+    String testBytesToUpload = "bytes";
+
+    try (AbfsOutputStream outForSomeBytes = createAbfsOutputStream(fs,
+        uploadBytesFilePath)
+    ) {
+
+      //Test for zero bytes To upload
+      assertValues("bytes to upload", 0,
+          outForSomeBytes.getOutputStreamStatistics().getBytesToUpload());
+
+      outForSomeBytes.write(testBytesToUpload.getBytes());
+      outForSomeBytes.flush();
+      AbfsOutputStreamStatisticsImpl abfsOutputStreamStatistics =
+          outForSomeBytes.getOutputStreamStatistics();
+
+      //Test for bytes to upload
+      assertValues("bytes to upload", testBytesToUpload.getBytes().length,
+          abfsOutputStreamStatistics.getBytesToUpload());
+
+      //Test for successful bytes uploaded
+      assertValues("successful bytes uploaded",
+          testBytesToUpload.getBytes().length,
+          abfsOutputStreamStatistics.getBytesUploadSuccessful());
+
+      //Populating random value for bytesFailed
+      int randomBytesFailed = new Random().nextInt(LOW_RANGE_FOR_RANDOM_VALUE);
+      abfsOutputStreamStatistics.uploadFailed(randomBytesFailed);
+      //Test for bytes failed to upload
+      assertValues("number fo bytes failed to upload", randomBytesFailed,
+          abfsOutputStreamStatistics.getBytesUploadFailed());
+
+    }
+
+    try (AbfsOutputStream outForLargeBytes = createAbfsOutputStream(fs,
+        uploadBytesFilePath)) {
+
+      int largeValue = LARGE_OPERATIONS;
+      for (int i = 0; i < largeValue; i++) {
+        outForLargeBytes.write(testBytesToUpload.getBytes());
+      }
+      outForLargeBytes.flush();
+      AbfsOutputStreamStatisticsImpl abfsOutputStreamStatistics =
+          outForLargeBytes.getOutputStreamStatistics();
+
+      //Test for bytes to upload
+      assertValues("bytes to upload",
+          largeValue * (testBytesToUpload.getBytes().length),
+          abfsOutputStreamStatistics.getBytesToUpload());
+
+      //Test for successful bytes uploaded
+      assertValues("successful bytes uploaded",
+          largeValue * (testBytesToUpload.getBytes().length),
+          abfsOutputStreamStatistics.getBytesUploadSuccessful());
+
+      //Populating random values for bytesFailed
+      int randomBytesFailed = new Random().nextInt(HIGH_RANGE_FOR_RANDOM_VALUE);
+      abfsOutputStreamStatistics.uploadFailed(randomBytesFailed);
+      //Test for bytes failed to upload
+      assertValues("bytes failed to upload", randomBytesFailed,
+          abfsOutputStreamStatistics.getBytesUploadFailed());
+    }
+  }
+
+  /**
+   * Tests to check time spent on waiting for tasks to be complete on a
+   * blocking queue in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamTimeSpentOnWaitTask() throws IOException {
+    describe("Testing Time Spend on Waiting for Task to be complete");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path timeSpendFilePath = new Path("AbfsOutputStreamStatsPath");
+
+    try (AbfsOutputStream out = createAbfsOutputStream(fs, timeSpendFilePath)) {
+
+      AbfsOutputStreamStatisticsImpl abfsOutputStreamStatistics =
+          out.getOutputStreamStatistics();
+
+      //Test for initial value of timeSpentWaitTask
+      assertValues("Time spent on waiting tasks", 0,
+          abfsOutputStreamStatistics.getTimeSpendOnTaskWait());
+
+      int smallRandomStartTime =
+          new Random().nextInt(LOW_RANGE_FOR_RANDOM_VALUE);
+      int smallRandomEndTime =
+          new Random().nextInt(LOW_RANGE_FOR_RANDOM_VALUE)
+              + smallRandomStartTime;
+      int smallDiff = smallRandomEndTime - smallRandomStartTime;
+      abfsOutputStreamStatistics
+          .timeSpentTaskWait(smallRandomStartTime, smallRandomEndTime);
+      //Test for small random value of timeSpentWaitTask
 
 Review comment:
   Looks like this an UT. Should move under UT folder. Create a new class TestAbfsOutputStreamStatictics and write all UT's there. You don't even have to create a file there. What do you say @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] mukund-thakur commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r404589778
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamStatisticsImpl.java
 ##########
 @@ -0,0 +1,171 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+/**
+ * OutputStream Statistics Implementation for Abfs.
+ */
+public class AbfsOutputStreamStatisticsImpl
+    implements AbfsOutputStreamStatistics {
+
+  /**
+   * timeSpentOnTaskWait - Time spent on waiting for tasks to be complete on
+   * Blocking Queue in AbfsOutputStream.
+   *
+   * queueShrink - Number of times Blocking Queue was shrunk after writing
+   * data.
+   *
+   * WriteCurrentBufferOperations - Number of times the current buffer which
+   * was written has been forwarded to the service.
+   */
 
 Review comment:
   Move the java doc to corresponding variables.

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r408176558
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
 ##########
 @@ -436,4 +453,28 @@ private void waitForTaskToComplete() throws IOException {
   public synchronized void waitForPendingUploads() throws IOException {
     waitForTaskToComplete();
   }
+
+  /**
+   * Getter method for AbfsOutputStream Statistics.
+   *
+   * @return statistics for AbfsOutputStream.
+   */
+  @VisibleForTesting
+  public AbfsOutputStreamStatisticsImpl getOutputStreamStatistics() {
 
 Review comment:
   why is this being cast rather than returned as is?

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r395743922
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
 ##########
 @@ -36,20 +36,25 @@
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 
+import org.apache.hadoop.fs.FileSystem.Statistics;
 
 Review comment:
   move down to under ElasticByteBufferPool

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on issue #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#issuecomment-614531091
 
 
   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 19s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  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  |  22m 43s |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 29s |  trunk passed  |
   | +1 :green_heart: |  checkstyle  |   0m 22s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 32s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  16m 26s |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 22s |  trunk passed  |
   | +0 :ok: |  spotbugs  |   0m 53s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 50s |  trunk passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 26s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 23s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 23s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 15s |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 24s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  15m 50s |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 19s |  the patch passed  |
   | +1 :green_heart: |  findbugs  |   0m 53s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 19s |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 29s |  The patch does not generate ASF License warnings.  |
   |  |   |  64m 44s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.8 Server=19.03.8 base: https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/9/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/1899 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux 0fa3cc1b14bf 4.15.0-74-generic #84-Ubuntu SMP Thu Dec 19 08:06:28 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | personality/hadoop.sh |
   | git revision | trunk / cc5c1da |
   | Default Java | 1.8.0_242 |
   |  Test Results | https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/9/testReport/ |
   | Max. process+thread count | 308 (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-1899/9/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] hadoop-yetus commented on issue #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on issue #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#issuecomment-600068083
 
 
   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |  21m 24s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  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 1 new or modified test files.  |
   ||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  19m 13s |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 32s |  trunk passed  |
   | +1 :green_heart: |  checkstyle  |   0m 24s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 34s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  15m  1s |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 27s |  trunk passed  |
   | +0 :ok: |  spotbugs  |   0m 53s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 52s |  trunk passed  |
   ||| _ Patch Compile Tests _ |
   | -1 :x: |  mvninstall  |   0m 22s |  hadoop-azure in the patch failed.  |
   | -1 :x: |  compile  |   0m 23s |  hadoop-azure in the patch failed.  |
   | -1 :x: |  javac  |   0m 23s |  hadoop-azure in the patch failed.  |
   | -0 :warning: |  checkstyle  |   0m 17s |  hadoop-tools/hadoop-azure: The patch generated 21 new + 0 unchanged - 0 fixed = 21 total (was 0)  |
   | -1 :x: |  mvnsite  |   0m 24s |  hadoop-azure in the patch failed.  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  13m 54s |  patch has no errors when building and testing our client artifacts.  |
   | -1 :x: |  javadoc  |   0m 24s |  hadoop-tools_hadoop-azure generated 2 new + 0 unchanged - 0 fixed = 2 total (was 0)  |
   | -1 :x: |  findbugs  |   0m 27s |  hadoop-azure in the patch failed.  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  |   0m 26s |  hadoop-azure in the patch failed.  |
   | -1 :x: |  asflicense  |   0m 31s |  The patch generated 3 ASF License warnings.  |
   |  |   |  76m 53s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.8 Server=19.03.8 base: https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/1/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/1899 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux 3847b4d4a6fb 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | personality/hadoop.sh |
   | git revision | trunk / 1975479 |
   | Default Java | 1.8.0_242 |
   | mvninstall | https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/1/artifact/out/patch-mvninstall-hadoop-tools_hadoop-azure.txt |
   | compile | https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/1/artifact/out/patch-compile-hadoop-tools_hadoop-azure.txt |
   | javac | https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/1/artifact/out/patch-compile-hadoop-tools_hadoop-azure.txt |
   | checkstyle | https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/1/artifact/out/diff-checkstyle-hadoop-tools_hadoop-azure.txt |
   | mvnsite | https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/1/artifact/out/patch-mvnsite-hadoop-tools_hadoop-azure.txt |
   | javadoc | https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/1/artifact/out/diff-javadoc-javadoc-hadoop-tools_hadoop-azure.txt |
   | findbugs | https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/1/artifact/out/patch-findbugs-hadoop-tools_hadoop-azure.txt |
   | unit | https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/1/artifact/out/patch-unit-hadoop-tools_hadoop-azure.txt |
   |  Test Results | https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/1/testReport/ |
   | asflicense | https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/1/artifact/out/patch-asflicense-problems.txt |
   | Max. process+thread count | 414 (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-1899/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 commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r408177932
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java
 ##########
 @@ -383,4 +391,34 @@ protected AbfsDelegationTokenManager getDelegationTokenManager()
       throws IOException {
     return getFileSystem().getDelegationTokenManager();
   }
+
+  /**
+   * Generic assert method.
+   *
+   * @param operation     operation being asserted.
+   * @param expectedValue value that is expected.
+   * @param actualValue   value that is actual.
+   */
+  protected void assertValues(String operation, long expectedValue,
 
 Review comment:
   too vague a name and no obvious difference with assertEquals. 
   
   Propose:
   *  leave existing test alone, 
   and then one of
   * use Junit assertEquals()
   * or Assertions.assertThat(object).describedAs().equals().
   

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r408496049
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsOutputStreamStatistics.java
 ##########
 @@ -0,0 +1,120 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+import java.util.Random;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl;
+
+/**
+ * Unit Tests for AbfsOutputStream Statistics.
+ */
+public class TestAbfsOutputStreamStatistics
+    extends AbstractAbfsIntegrationTest {
+
+  private static final int LOW_RANGE_FOR_RANDOM_VALUE = 49;
+  private static final int HIGH_RANGE_FOR_RANDOM_VALUE = 9999;
+
+  public TestAbfsOutputStreamStatistics() throws Exception {
+  }
+
+  /**
+   * Tests to check bytes failed to Upload in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamBytesFailed() {
+    describe("Testing Bytes Failed during uploading in AbfsOutputSteam");
+
+    AbfsOutputStreamStatisticsImpl abfsOutputStreamStatistics =
+        new AbfsOutputStreamStatisticsImpl();
+
+    //Test for zero bytes uploaded.
+    assertValues("number fo bytes failed to upload", 0,
+        abfsOutputStreamStatistics.getBytesUploadFailed());
+
+    //Populating small random value for bytesFailed.
+    int randomBytesFailed = new Random().nextInt(LOW_RANGE_FOR_RANDOM_VALUE);
+    abfsOutputStreamStatistics.uploadFailed(randomBytesFailed);
+    //Test for bytes failed to upload.
+    assertValues("number fo bytes failed to upload", randomBytesFailed,
+        abfsOutputStreamStatistics.getBytesUploadFailed());
+
+    //Initializing again to reset the statistics.
+    abfsOutputStreamStatistics = new AbfsOutputStreamStatisticsImpl();
+
+    //Populating large random values for bytesFailed.
+    randomBytesFailed = new Random().nextInt(HIGH_RANGE_FOR_RANDOM_VALUE);
 
 Review comment:
   I can use the previous value(L61) and see if it's being summed by removing the reset?

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r408179091
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamStatisticsImpl.java
 ##########
 @@ -0,0 +1,177 @@
+/**
+ * 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.services;
+
+/**
+ * OutputStream Statistics Implementation for Abfs.
+ */
+public class AbfsOutputStreamStatisticsImpl
+    implements AbfsOutputStreamStatistics {
+  private long bytesToUpload;
+  private long bytesUploadSuccessful;
+  private long bytesUploadFailed;
+  /**
+   * counter to get the total time spent while waiting for tasks to complete
+   * in the Blocking queue inside the thread executor.
+   */
+  private long timeSpendOnTaskWait;
+  /**
+   * counter to get the total number of queue shrink operations done{@code
+   * AbfsOutputStream#shrinkWriteOperationQueue()} by
+   * AbfsOutputStream to remove the write operations which were successfully
+   * done by AbfsOutputStream from the Blocking Queue.
+   */
+  private long queueShrunkOps;
+  /**
+   * counter to get the total number of times the current buffer is written
+   * to the service{@code AbfsOutputStream#writeCurrentBufferToService()} via
+   * AbfsClient and appended to the
+   * Data store by
+   * AbfsResOperation.
+   */
+  private long writeCurrentBufferOperations;
+
+  /**
+   * Records the need to upload bytes and increments the total bytes that
+   * needs to be uploaded.
+   *
+   * @param bytes Total bytes to upload. Negative bytes are ignored.
+   */
+  @Override
+  public void bytesToUpload(long bytes) {
+    if (bytes > 0) {
+      bytesToUpload += bytes;
+    }
+  }
+
+  /**
+   * Records the total bytes successfully uploaded through AbfsOutputStream.
+   *
+   * @param bytes number of bytes that were successfully uploaded. Negative
+   *              bytes are ignored.
+   */
+  @Override
+  public void uploadSuccessful(long bytes) {
+    if (bytes > 0) {
+      bytesUploadSuccessful += bytes;
+    }
+  }
+
+  /**
+   * Records the total bytes failed to upload through AbfsOutputStream.
+   *
+   * @param bytes number of bytes failed to upload. Negative bytes are ignored.
+   */
+  @Override
+  public void uploadFailed(long bytes) {
+    if (bytes > 0) {
+      bytesUploadFailed += bytes;
+    }
+  }
+
+  /**
+   * {@inheritDoc}
+   *
+   * Records the total time spent waiting for a task.
+   * When the thread executor has a task
+   * queue{@link java.util.concurrent.BlockingQueue} of size greater than or equal to 2
+   * times the maxConcurrentRequestCounts then, it waits for a task in that
+   * queue to finish, then do the next task in the queue.
+   *
+   * This time spent while waiting for the task to be completed is being
+   * recorded in this counter.
+   *
+   * @param startTime time(in milliseconds) before the wait for task to be
+   *                  completed is begin.
+   * @param endTime   time(in milliseconds) after the wait for the task to be
+   *                  completed is done.
+   */
+  @Override
+  public void timeSpentTaskWait(long startTime, long endTime) {
+    timeSpendOnTaskWait += endTime - startTime;
+  }
+
+  /**
+   * {@inheritDoc}
+   *
+   * Records the number of times AbfsOutputStream try to remove the completed
+   * write operations from the beginning of write operation FIFO queue.
+   */
+  @Override
+  public void queueShrunk() {
+    queueShrunkOps++;
+  }
+
+  /**
+   * {@inheritDoc}
+   *
+   * Records the number of times AbfsOutputStream writes the buffer to the
+   * service via the AbfsClient and appends the buffer to the service.
+   */
+  @Override
+  public void writeCurrentBuffer() {
+    writeCurrentBufferOperations++;
+  }
+
+  public long getBytesToUpload() {
+    return bytesToUpload;
+  }
+
+  public long getBytesUploadSuccessful() {
+    return bytesUploadSuccessful;
+  }
+
+  public long getBytesUploadFailed() {
+    return bytesUploadFailed;
+  }
+
+  public long getTimeSpendOnTaskWait() {
+    return timeSpendOnTaskWait;
+  }
+
+  public long getQueueShrunkOps() {
+    return queueShrunkOps;
+  }
+
+  public long getWriteCurrentBufferOperations() {
+    return writeCurrentBufferOperations;
+  }
+
+  /**
+   * String to show AbfsOutputStream statistics values in AbfsOutputStream.
+   *
+   * @return String with AbfsOutputStream statistics.
+   */
+  @Override public String toString() {
+    final StringBuilder outputStreamStats = new StringBuilder(
+        "OutputStream Statistics{");
+    outputStreamStats.append(", BYTES_UPLOAD=").append(bytesToUpload);
 
 Review comment:
   don't think we need capitals 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 a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r408179333
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsOutputStreamStatistics.java
 ##########
 @@ -0,0 +1,120 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+import java.util.Random;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl;
+
+/**
+ * Unit Tests for AbfsOutputStream Statistics.
+ */
+public class TestAbfsOutputStreamStatistics
+    extends AbstractAbfsIntegrationTest {
+
+  private static final int LOW_RANGE_FOR_RANDOM_VALUE = 49;
+  private static final int HIGH_RANGE_FOR_RANDOM_VALUE = 9999;
+
+  public TestAbfsOutputStreamStatistics() throws Exception {
+  }
+
+  /**
+   * Tests to check bytes failed to Upload in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamBytesFailed() {
+    describe("Testing Bytes Failed during uploading in AbfsOutputSteam");
+
+    AbfsOutputStreamStatisticsImpl abfsOutputStreamStatistics =
+        new AbfsOutputStreamStatisticsImpl();
+
+    //Test for zero bytes uploaded.
+    assertValues("number fo bytes failed to upload", 0,
+        abfsOutputStreamStatistics.getBytesUploadFailed());
+
+    //Populating small random value for bytesFailed.
+    int randomBytesFailed = new Random().nextInt(LOW_RANGE_FOR_RANDOM_VALUE);
+    abfsOutputStreamStatistics.uploadFailed(randomBytesFailed);
+    //Test for bytes failed to upload.
+    assertValues("number fo bytes failed to upload", randomBytesFailed,
+        abfsOutputStreamStatistics.getBytesUploadFailed());
+
+    //Initializing again to reset the statistics.
+    abfsOutputStreamStatistics = new AbfsOutputStreamStatisticsImpl();
+
+    //Populating large random values for bytesFailed.
+    randomBytesFailed = new Random().nextInt(HIGH_RANGE_FOR_RANDOM_VALUE);
+    abfsOutputStreamStatistics.uploadFailed(randomBytesFailed);
+    //Test for bytes failed to upload.
+    assertValues("number fo bytes failed to upload", randomBytesFailed,
+        abfsOutputStreamStatistics.getBytesUploadFailed());
+  }
+
+  /**
+   * Tests to check time spent on waiting for tasks to be complete on a
+   * blocking queue in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamTimeSpentOnWaitTask() {
+    describe("Testing Time Spend on Waiting for Task to be complete");
+
+    AbfsOutputStreamStatisticsImpl abfsOutputStreamStatistics =
+        new AbfsOutputStreamStatisticsImpl();
+
+    //Test for initial value of timeSpentWaitTask.
+    assertValues("Time spend on waiting for tasks to complete", 0,
+        abfsOutputStreamStatistics.getTimeSpendOnTaskWait());
+
+    int smallRandomStartTime =
+        new Random().nextInt(LOW_RANGE_FOR_RANDOM_VALUE);
+    int smallRandomEndTime =
+        new Random().nextInt(LOW_RANGE_FOR_RANDOM_VALUE)
+            + smallRandomStartTime;
+    int smallDiff = smallRandomEndTime - smallRandomStartTime;
+    abfsOutputStreamStatistics
+        .timeSpentTaskWait(smallRandomStartTime, smallRandomEndTime);
+    //Test for small random value of timeSpentWaitTask.
+    assertValues("Time spend on waiting for tasks to complete", smallDiff,
+        abfsOutputStreamStatistics.getTimeSpendOnTaskWait());
+
+    int largeRandomStartTime =
+        new Random().nextInt(HIGH_RANGE_FOR_RANDOM_VALUE);
+    int largeRandomEndTime = new Random().nextInt(HIGH_RANGE_FOR_RANDOM_VALUE)
+        + largeRandomStartTime;
+    int randomDiff = largeRandomEndTime - largeRandomStartTime;
+    abfsOutputStreamStatistics
 
 Review comment:
   same as comment on L68.

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r408181009
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamStatisticsImpl.java
 ##########
 @@ -0,0 +1,177 @@
+/**
+ * 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.services;
+
+/**
+ * OutputStream Statistics Implementation for Abfs.
+ */
+public class AbfsOutputStreamStatisticsImpl
+    implements AbfsOutputStreamStatistics {
+  private long bytesToUpload;
+  private long bytesUploadSuccessful;
+  private long bytesUploadFailed;
+  /**
+   * counter to get the total time spent while waiting for tasks to complete
+   * in the Blocking queue inside the thread executor.
+   */
+  private long timeSpendOnTaskWait;
+  /**
+   * counter to get the total number of queue shrink operations done{@code
+   * AbfsOutputStream#shrinkWriteOperationQueue()} by
+   * AbfsOutputStream to remove the write operations which were successfully
+   * done by AbfsOutputStream from the Blocking Queue.
+   */
+  private long queueShrunkOps;
+  /**
+   * counter to get the total number of times the current buffer is written
+   * to the service{@code AbfsOutputStream#writeCurrentBufferToService()} via
+   * AbfsClient and appended to the
+   * Data store by
+   * AbfsResOperation.
+   */
+  private long writeCurrentBufferOperations;
+
+  /**
+   * Records the need to upload bytes and increments the total bytes that
+   * needs to be uploaded.
+   *
+   * @param bytes Total bytes to upload. Negative bytes are ignored.
 
 Review comment:
   remove bytes.

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r395748307
 
 

 ##########
 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 if content is validated true else, false
+   * @throws IOException
+   */
+  protected boolean validateContent(AzureBlobFileSystem fs, Path path,
+      byte[] originalByteArray)
+      throws IOException {
+    FSDataInputStream in = fs.open(path);
+
+    int pos = 0;
+    int lenOfOriginalByteArray = originalByteArray.length;
+    byte valueOfContentAtPos = (byte) in.read();
+
+    while (valueOfContentAtPos != -1 && pos < lenOfOriginalByteArray) {
 
 Review comment:
   1. MUST use { } in all if () clauses.
   2. If there's a mismatch, use AssertEquals and include the pos where the problem occurred
   
   Imagine: "A remote test run failed -what information should be in the test report to begin debugging this?"

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r394227017
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStream.java
 ##########
 @@ -133,58 +133,60 @@ public void testAbfsOutputStreamTimeSpendOnWaitTask() throws IOException {
   public void testAbfsOutputStreamQueueShrink() throws IOException {
     describe("Testing Queue Shrink calls in AbfsOutputStream");
 
 Review comment:
   Need help in writing test for this counter.

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r402095177
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamStatistics.java
 ##########
 @@ -0,0 +1,78 @@
+/**
+ * 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.services;
+
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Interface for {@link AbfsOutputStream} statistics.
+ */
+@InterfaceStability.Unstable
+public interface AbfsOutputStreamStatistics {
+
+  /**
+   * Number of bytes to be uploaded.
+   *
+   * @param bytes number of bytes to upload
+   */
+  void bytesToUpload(long bytes);
+
+  /**
+   * Records a successful upload and the number of bytes uploaded.
+   *
+   * @param bytes number of bytes that were successfully uploaded
+   */
+  void uploadSuccessful(long bytes);
+
+  /**
+   * Records that upload is failed and the number of bytes.
+   *
+   * @param bytes number of bytes that failed to upload
+   */
+  void uploadFailed(long bytes);
+
+  /**
+   * Time spent in waiting for tasks to be completed in the blocking Queue.
 
 Review comment:
   explain more

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on issue #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#issuecomment-609805651
 
 
   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |  24m 47s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  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 1 new or modified test files.  |
   ||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  19m  7s |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 30s |  trunk passed  |
   | +1 :green_heart: |  checkstyle  |   0m 24s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 35s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  15m 19s |  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 50s |  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  |
   | +1 :green_heart: |  checkstyle  |   0m 17s |  the patch passed  |
   | +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 57s |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 23s |  the patch passed  |
   | -1 :x: |  findbugs  |   0m 55s |  hadoop-tools/hadoop-azure generated 2 new + 0 unchanged - 0 fixed = 2 total (was 0)  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 23s |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 33s |  The patch does not generate ASF License warnings.  |
   |  |   |  82m  2s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hadoop-tools/hadoop-azure |
   |  |  Increment of volatile field org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl.queueShrink in org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl.queueShrinked()  At AbfsOutputStreamStatisticsImpl.java:in org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl.queueShrinked()  At AbfsOutputStreamStatisticsImpl.java:[line 112] |
   |  |  Increment of volatile field org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl.writeCurrentBufferOperations in org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl.writeCurrentBuffer()  At AbfsOutputStreamStatisticsImpl.java:in org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl.writeCurrentBuffer()  At AbfsOutputStreamStatisticsImpl.java:[line 123] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.8 Server=19.03.8 base: https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/4/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/1899 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux 6588c0c15f15 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 / ab7495d |
   | Default Java | 1.8.0_242 |
   | findbugs | https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/4/artifact/out/new-findbugs-hadoop-tools_hadoop-azure.html |
   |  Test Results | https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/4/testReport/ |
   | Max. process+thread count | 413 (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-1899/4/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 commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r395752947
 
 

 ##########
 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 {
 
 Review comment:
   not needed

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r405059806
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStream.java
 ##########
 @@ -133,58 +133,60 @@ public void testAbfsOutputStreamTimeSpendOnWaitTask() throws IOException {
   public void testAbfsOutputStreamQueueShrink() throws IOException {
     describe("Testing Queue Shrink calls in AbfsOutputStream");
     final AzureBlobFileSystem fs = getFileSystem();
-    Path TEST_PATH = new Path("AbfsOutputStreamStatsPath");
+    Path queueShrinkFilePath = new Path("AbfsOutputStreamStatsPath");
     AzureBlobFileSystemStore abfss = fs.getAbfsStore();
     abfss.getAbfsConfiguration().setDisableOutputStreamFlush(false);
     FileSystem.Statistics statistics = fs.getFsStatistics();
     String testQueueShrink = "testQueue";
 
-
     AbfsOutputStream outForOneOp = null;
 
     try {
-      outForOneOp = (AbfsOutputStream) abfss.createFile(TEST_PATH, statistics,
-        true,
-          FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
+      outForOneOp =
+          (AbfsOutputStream) abfss.createFile(queueShrinkFilePath, statistics,
+              true,
+              FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
 
       //Test for shrinking Queue zero time
-      Assert.assertEquals("Mismatch in number of queueShrink() Calls", 0,
+      assertValues("number of queueShrink() Calls", 0,
           outForOneOp.getOutputStreamStatistics().queueShrink);
 
       outForOneOp.write(testQueueShrink.getBytes());
       // Queue is shrunk 2 times when outStream is flushed
       outForOneOp.flush();
 
       //Test for shrinking Queue 2 times
-      Assert.assertEquals("Mismatch in number of queueShrink() Calls", 2,
+      assertValues("number of queueShrink() Calls", 2,
           outForOneOp.getOutputStreamStatistics().queueShrink);
 
     } finally {
-      if(outForOneOp != null){
+      if (outForOneOp != null) {
         outForOneOp.close();
       }
     }
 
     AbfsOutputStream outForLargeOps = null;
 
     try {
-      outForLargeOps = (AbfsOutputStream) abfss.createFile(TEST_PATH,
+      outForLargeOps = (AbfsOutputStream) abfss.createFile(queueShrinkFilePath,
           statistics, true,
           FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
 
+      int largeValue = 1000;
       //QueueShrink is called 2 times in 1 flush(), hence 1000 flushes must
       // give 2000 QueueShrink calls
-      for (int i = 0; i < 1000; i++) {
+      for (int i = 0; i < largeValue; i++) {
         outForLargeOps.write(testQueueShrink.getBytes());
         //Flush is quite expensive so 1000 calls only which takes 1 min+
         outForLargeOps.flush();
 
 Review comment:
   No I can't, basically calling flush after I write means the write task is done. This would trigger the shrinkWriteOperationQueue() method and we need to do it after each write to get 10 calls to shrinkWriteOperationQueue() .
   If I flush after the loop, it would take all the write calls as 1 write operation and only 1 time the shrinkWriteOperationQueue() method is triggered.

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r395750011
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStream.java
 ##########
 @@ -0,0 +1,278 @@
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
+import org.apache.hadoop.fs.permission.FsPermission;
+
+/**
+ * Test AbfsOutputStream statistics.
+ */
+public class ITestAbfsOutputStream extends AbstractAbfsIntegrationTest {
+
+  public ITestAbfsOutputStream() throws Exception {
+  }
+
+  /**
+   * Tests to check bytes Uploading in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamUploadingBytes() throws IOException {
+    describe("Testing Bytes uploaded in AbfsOutputSteam");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path uploadBytesFilePath = new Path("AbfsOutputStreamStatsPath");
+    AzureBlobFileSystemStore abfss = fs.getAbfsStore();
+    FileSystem.Statistics statistics = fs.getFsStatistics();
+    abfss.getAbfsConfiguration().setDisableOutputStreamFlush(false);
+    String testBytesToUpload = "bytes";
+
+    AbfsOutputStream outForSomeBytes = null;
+    try {
+      outForSomeBytes = (AbfsOutputStream) abfss.createFile(uploadBytesFilePath,
+          statistics,
+          true,
+          FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
+
+      //Test for zero bytes To upload
+      assertValues("bytes to upload", 0,
+          outForSomeBytes.getOutputStreamStatistics().bytesToUpload);
+
+      outForSomeBytes.write(testBytesToUpload.getBytes());
+      outForSomeBytes.flush();
+
+      //Test for some bytes to upload
+      assertValues("bytes to upload", testBytesToUpload.getBytes().length,
+          outForSomeBytes.getOutputStreamStatistics().bytesToUpload);
+
+      //Test for relation between bytesUploadSuccessful, bytesUploadFailed
+      // and bytesToUpload
+      assertValues("bytesUploadSuccessful equal to difference between "
+              + "bytesToUpload and bytesUploadFailed",
+          outForSomeBytes.getOutputStreamStatistics().bytesUploadSuccessful,
+          outForSomeBytes.getOutputStreamStatistics().bytesToUpload -
+              outForSomeBytes.getOutputStreamStatistics().bytesUploadFailed);
+
+    } finally {
+      if (outForSomeBytes != null) {
+        outForSomeBytes.close();
+      }
+    }
+
+    AbfsOutputStream outForLargeBytes = null;
+    try {
+      outForLargeBytes =
+          (AbfsOutputStream) abfss.createFile(uploadBytesFilePath,
+              statistics
+              , true, FsPermission.getDefault(),
+              FsPermission.getUMask(fs.getConf()));
+
+      int largeValue = 100000;
+      for (int i = 0; i < largeValue; i++) {
+        outForLargeBytes.write(testBytesToUpload.getBytes());
+      }
+      outForLargeBytes.flush();
+
+      //Test for large bytes to upload
+      assertValues("bytes to upload",
+          largeValue * (testBytesToUpload.getBytes().length),
+          outForLargeBytes.getOutputStreamStatistics().bytesToUpload);
+
+      //Test for relation between bytesUploadSuccessful, bytesUploadFailed
+      // and bytesToUpload
+      assertValues("bytesUploadSuccessful equal to difference between "
+              + "bytesToUpload and bytesUploadFailed",
+          outForSomeBytes.getOutputStreamStatistics().bytesUploadSuccessful,
+          outForSomeBytes.getOutputStreamStatistics().bytesToUpload -
+              outForSomeBytes.getOutputStreamStatistics().bytesUploadFailed);
+
+    } finally {
+      if (outForLargeBytes != null) {
+        outForLargeBytes.close();
+      }
+    }
+
+  }
+
+  /**
+   * Tests to check time spend on waiting for tasks to be complete on a
+   * blocking queue in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamTimeSpendOnWaitTask() throws IOException {
+    describe("Testing Time Spend on Waiting for Task to be complete");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path timeSpendFilePath = new Path("AbfsOutputStreamStatsPath");
+    AzureBlobFileSystemStore abfss = fs.getAbfsStore();
+    FileSystem.Statistics statistics = fs.getFsStatistics();
+
+    AbfsOutputStream out =
 
 Review comment:
   will need to be closed

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r408487715
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStreamStatistics.java
 ##########
 @@ -0,0 +1,233 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl;
+
+/**
+ * Test AbfsOutputStream statistics.
+ */
+public class ITestAbfsOutputStreamStatistics
+    extends AbstractAbfsIntegrationTest {
+  private static final int LARGE_OPERATIONS = 10;
+
+  public ITestAbfsOutputStreamStatistics() throws Exception {
 
 Review comment:
   Think I need to handle Exception for this Test, and don't know any way other than making a constructor and doing so.

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r402075785
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamStatisticsImpl.java
 ##########
 @@ -0,0 +1,148 @@
+/**
+ * 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.services;
+
+/**
+ * OutputStream Statistics Implementation for Abfs.
+ * timeSpendOnTaskWait - Time spend on waiting for tasks to be complete on
+ * Blocking Queue in AbfsOutputStream.
+ *
+ * queueShrink - Number of times Blocking Queue was shrunk after writing
+ * data.
+ *
+ * WriteCurrentBufferOperations - Number of times
+ * {@code writeCurrentBufferToService()} calls were made.
+ */
+public class AbfsOutputStreamStatisticsImpl
+    implements AbfsOutputStreamStatistics {
+  private volatile long bytesToUpload;
+  private volatile long bytesUploadSuccessful;
+  private volatile long bytesUploadFailed;
+  private volatile long timeSpendOnTaskWait;
+  private volatile long queueShrink;
+  private volatile long writeCurrentBufferOperations;
+
+  /**
+   * Number of bytes uploaded.
+   *
+   * @param bytes negative values are ignored
+   */
+  @Override
+  public void bytesToUpload(long bytes) {
+    if (bytes > 0) {
+      bytesToUpload += bytes;
+    }
+  }
+
+  /**
+   * Upload successful with the number of bytes.
+   * @param bytes number of bytes that were successfully uploaded
+   */
+  @Override
+  public void uploadSuccessful(long bytes) {
+    if (bytes > 0) {
+      bytesUploadSuccessful += bytes;
+    }
+  }
+
+  /**
+   * Upload failed and the number of bytes.
+   *
+   * @param bytes negative values are ignored
+   */
+  @Override
+  public void uploadFailed(long bytes) {
+    if (bytes > 0) {
+      bytesUploadFailed += bytes;
+    }
+  }
+
+  /**
+   * Time spent for waiting a task to be completed.
+   *
+   * @param startTime on calling {@code waitForTaskToComplete()}
 
 Review comment:
   Write full method in javadoc like AbfsOutputStream#waitForTaskToComplete() otherwise people will have to figure out where this method actually is.

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r394230246
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStream.java
 ##########
 @@ -0,0 +1,278 @@
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
+import org.apache.hadoop.fs.permission.FsPermission;
+
+/**
+ * Test AbfsOutputStream statistics.
+ */
+public class ITestAbfsOutputStream extends AbstractAbfsIntegrationTest {
+
+  public ITestAbfsOutputStream() throws Exception {
+  }
+
+  /**
+   * Tests to check bytes Uploading in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamUploadingBytes() throws IOException {
+    describe("Testing Bytes uploaded in AbfsOutputSteam");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path uploadBytesFilePath = new Path("AbfsOutputStreamStatsPath");
+    AzureBlobFileSystemStore abfss = fs.getAbfsStore();
+    FileSystem.Statistics statistics = fs.getFsStatistics();
+    abfss.getAbfsConfiguration().setDisableOutputStreamFlush(false);
+    String testBytesToUpload = "bytes";
+
+    AbfsOutputStream outForSomeBytes = null;
+    try {
+      outForSomeBytes = (AbfsOutputStream) abfss.createFile(uploadBytesFilePath,
+          statistics,
+          true,
+          FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
+
+      //Test for zero bytes To upload
+      assertValues("bytes to upload", 0,
+          outForSomeBytes.getOutputStreamStatistics().bytesToUpload);
+
+      outForSomeBytes.write(testBytesToUpload.getBytes());
+      outForSomeBytes.flush();
+
+      //Test for some bytes to upload
+      assertValues("bytes to upload", testBytesToUpload.getBytes().length,
+          outForSomeBytes.getOutputStreamStatistics().bytesToUpload);
+
+      //Test for relation between bytesUploadSuccessful, bytesUploadFailed
+      // and bytesToUpload
+      assertValues("bytesUploadSuccessful equal to difference between "
+              + "bytesToUpload and bytesUploadFailed",
+          outForSomeBytes.getOutputStreamStatistics().bytesUploadSuccessful,
+          outForSomeBytes.getOutputStreamStatistics().bytesToUpload -
+              outForSomeBytes.getOutputStreamStatistics().bytesUploadFailed);
+
+    } finally {
+      if (outForSomeBytes != null) {
+        outForSomeBytes.close();
+      }
+    }
+
+    AbfsOutputStream outForLargeBytes = null;
+    try {
+      outForLargeBytes =
+          (AbfsOutputStream) abfss.createFile(uploadBytesFilePath,
+              statistics
+              , true, FsPermission.getDefault(),
+              FsPermission.getUMask(fs.getConf()));
+
+      int largeValue = 100000;
+      for (int i = 0; i < largeValue; i++) {
+        outForLargeBytes.write(testBytesToUpload.getBytes());
+      }
+      outForLargeBytes.flush();
+
+      //Test for large bytes to upload
+      assertValues("bytes to upload",
+          largeValue * (testBytesToUpload.getBytes().length),
+          outForLargeBytes.getOutputStreamStatistics().bytesToUpload);
+
+      //Test for relation between bytesUploadSuccessful, bytesUploadFailed
+      // and bytesToUpload
+      assertValues("bytesUploadSuccessful equal to difference between "
+              + "bytesToUpload and bytesUploadFailed",
+          outForSomeBytes.getOutputStreamStatistics().bytesUploadSuccessful,
+          outForSomeBytes.getOutputStreamStatistics().bytesToUpload -
+              outForSomeBytes.getOutputStreamStatistics().bytesUploadFailed);
+
+    } finally {
+      if (outForLargeBytes != null) {
+        outForLargeBytes.close();
+      }
+    }
+
+  }
+
+  /**
+   * Tests to check time spend on waiting for tasks to be complete on a
+   * blocking queue in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamTimeSpendOnWaitTask() throws IOException {
 
 Review comment:
   Need help on how to write tests for this counter.

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r404587067
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStream.java
 ##########
 @@ -0,0 +1,294 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+import java.util.Random;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl;
+import org.apache.hadoop.fs.permission.FsPermission;
+
+/**
+ * Test AbfsOutputStream statistics.
+ */
+public class ITestAbfsOutputStream extends AbstractAbfsIntegrationTest {
+  public ITestAbfsOutputStream() throws Exception {
 
 Review comment:
   Move the constructor down after the class variables.

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r402095417
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamStatisticsImpl.java
 ##########
 @@ -0,0 +1,148 @@
+/**
+ * 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.services;
+
+/**
+ * OutputStream Statistics Implementation for Abfs.
+ * timeSpendOnTaskWait - Time spend on waiting for tasks to be complete on
+ * Blocking Queue in AbfsOutputStream.
+ *
+ * queueShrink - Number of times Blocking Queue was shrunk after writing
+ * data.
+ *
+ * WriteCurrentBufferOperations - Number of times
+ * {@code writeCurrentBufferToService()} calls were made.
+ */
 
 Review comment:
   this java doc above variable name

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r405106456
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStreamStatistics.java
 ##########
 @@ -0,0 +1,296 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+import java.util.Random;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl;
+import org.apache.hadoop.fs.permission.FsPermission;
+
+/**
+ * Test AbfsOutputStream statistics.
+ */
+public class ITestAbfsOutputStreamStatistics
+    extends AbstractAbfsIntegrationTest {
+  private static final int LARGE_OPERATIONS = 10;
+  private static final int LOW_RANGE_FOR_RANDOM_VALUE = 49;
+  private static final int HIGH_RANGE_FOR_RANDOM_VALUE = 9999;
+
+  public ITestAbfsOutputStreamStatistics() throws Exception {
+  }
+
+  /**
+   * Tests to check bytes Uploading in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamUploadingBytes() throws IOException {
+    describe("Testing Bytes uploaded in AbfsOutputSteam");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path uploadBytesFilePath = path(getMethodName());
+    String testBytesToUpload = "bytes";
+
+    try (AbfsOutputStream outForSomeBytes = createAbfsOutputStream(fs,
+        uploadBytesFilePath)
+    ) {
+
+      AbfsOutputStreamStatisticsImpl abfsOutputStreamStatisticsForUploadBytes =
+          outForSomeBytes.getOutputStreamStatistics();
+
+      //Test for zero bytes To upload.
+      assertValues("bytes to upload", 0,
+          abfsOutputStreamStatisticsForUploadBytes.getBytesToUpload());
+
+      outForSomeBytes.write(testBytesToUpload.getBytes());
+      outForSomeBytes.flush();
+      abfsOutputStreamStatisticsForUploadBytes =
+          outForSomeBytes.getOutputStreamStatistics();
+
+      //Test for bytes to upload.
+      assertValues("bytes to upload", testBytesToUpload.getBytes().length,
+          abfsOutputStreamStatisticsForUploadBytes.getBytesToUpload());
+
+      //Test for successful bytes uploaded.
+      assertValues("successful bytes uploaded",
+          testBytesToUpload.getBytes().length,
+          abfsOutputStreamStatisticsForUploadBytes.getBytesUploadSuccessful());
+
+      //Populating random value for bytesFailed.
+      int randomBytesFailed = new Random().nextInt(LOW_RANGE_FOR_RANDOM_VALUE);
+      abfsOutputStreamStatisticsForUploadBytes.uploadFailed(randomBytesFailed);
+      //Test for bytes failed to upload.
+      assertValues("number fo bytes failed to upload", randomBytesFailed,
+          abfsOutputStreamStatisticsForUploadBytes.getBytesUploadFailed());
+
+    }
+
+    try (AbfsOutputStream outForLargeBytes = createAbfsOutputStream(fs,
+        uploadBytesFilePath)) {
+
+      for (int i = 0; i < LARGE_OPERATIONS; i++) {
+        outForLargeBytes.write(testBytesToUpload.getBytes());
+      }
+      outForLargeBytes.flush();
+      AbfsOutputStreamStatisticsImpl abfsOutputStreamStatistics =
+          outForLargeBytes.getOutputStreamStatistics();
+
+      //Test for bytes to upload.
+      assertValues("bytes to upload",
+          LARGE_OPERATIONS * (testBytesToUpload.getBytes().length),
+          abfsOutputStreamStatistics.getBytesToUpload());
+
+      //Test for successful bytes uploaded.
+      assertValues("successful bytes uploaded",
+          LARGE_OPERATIONS * (testBytesToUpload.getBytes().length),
+          abfsOutputStreamStatistics.getBytesUploadSuccessful());
+
+      //Populating random values for bytesFailed.
+      int randomBytesFailed = new Random().nextInt(HIGH_RANGE_FOR_RANDOM_VALUE);
+      abfsOutputStreamStatistics.uploadFailed(randomBytesFailed);
+      //Test for bytes failed to upload.
+      assertValues("bytes failed to upload", randomBytesFailed,
+          abfsOutputStreamStatistics.getBytesUploadFailed());
+    }
+  }
+
+  /**
+   * Tests to check time spent on waiting for tasks to be complete on a
+   * blocking queue in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamTimeSpentOnWaitTask() throws IOException {
+    describe("Testing Time Spend on Waiting for Task to be complete");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path timeSpendFilePath = path(getMethodName());
+
+    try (AbfsOutputStream out = createAbfsOutputStream(fs, timeSpendFilePath)) {
+
+      AbfsOutputStreamStatisticsImpl abfsOutputStreamStatistics =
+          out.getOutputStreamStatistics();
+
+      //Test for initial value of timeSpentWaitTask.
+      assertValues("Time spent on waiting tasks", 0,
+          abfsOutputStreamStatistics.getTimeSpendOnTaskWait());
+
+      int smallRandomStartTime =
+          new Random().nextInt(LOW_RANGE_FOR_RANDOM_VALUE);
+      int smallRandomEndTime =
+          new Random().nextInt(LOW_RANGE_FOR_RANDOM_VALUE)
+              + smallRandomStartTime;
+      int smallDiff = smallRandomEndTime - smallRandomStartTime;
+      abfsOutputStreamStatistics
+          .timeSpentTaskWait(smallRandomStartTime, smallRandomEndTime);
+      //Test for small random value of timeSpentWaitTask.
+      assertValues("Time spend on waiting for tasks to complete", smallDiff,
+          abfsOutputStreamStatistics.getTimeSpendOnTaskWait());
+
+      int largeRandomStartTime =
+          new Random().nextInt(HIGH_RANGE_FOR_RANDOM_VALUE);
+      int largeRandomEndTime = new Random().nextInt(HIGH_RANGE_FOR_RANDOM_VALUE)
+          + largeRandomStartTime;
+      int randomDiff = largeRandomEndTime - largeRandomStartTime;
+      abfsOutputStreamStatistics
+          .timeSpentTaskWait(largeRandomStartTime, largeRandomEndTime);
+      /*
+      Test for large random value of timeSpentWaitTask plus the time spent
+      in previous test.
+       */
+      assertValues("Time spend on waiting for tasks to complete",
+          smallDiff + randomDiff,
+          abfsOutputStreamStatistics.getTimeSpendOnTaskWait());
+    }
+
+  }
+
+  /**
+   * Tests to check number of {@code
+   * AbfsOutputStream#shrinkWriteOperationQueue()} calls.
+   * After writing data, AbfsOutputStream doesn't upload the data until
+   * Flushed. Hence, flush() method is called after write() to test Queue
+   * shrink calls.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamQueueShrink() throws IOException {
+    describe("Testing Queue Shrink calls in AbfsOutputStream");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path queueShrinkFilePath = path(getMethodName());
+    String testQueueShrink = "testQueue";
+
+    try (AbfsOutputStream outForOneOp = createAbfsOutputStream(fs,
+        queueShrinkFilePath)) {
+
+      AbfsOutputStreamStatisticsImpl abfsOutputStreamStatistics =
+          outForOneOp.getOutputStreamStatistics();
+
+      //Test for shrinking Queue zero time.
+      assertValues("Queue shrunk operations", 0,
+          abfsOutputStreamStatistics.getQueueShrunkOps());
+
+      outForOneOp.write(testQueueShrink.getBytes());
+      // Queue is shrunk 2 times when outputStream is flushed.
+      outForOneOp.flush();
+
+      abfsOutputStreamStatistics = outForOneOp.getOutputStreamStatistics();
+
+      //Test for shrinking Queue 2 times.
+      assertValues("Queue shrunk operations", 2,
+          abfsOutputStreamStatistics.getQueueShrunkOps());
+
+    }
+
+    try (AbfsOutputStream outForLargeOps = createAbfsOutputStream(fs,
+        queueShrinkFilePath)) {
+      for (int i = 0; i < LARGE_OPERATIONS; i++) {
+        outForLargeOps.write(testQueueShrink.getBytes());
+        outForLargeOps.flush();
+      }
+
+      AbfsOutputStreamStatisticsImpl abfsOutputStreamStatistics =
+          outForLargeOps.getOutputStreamStatistics();
+
+      //Test for 20 queue shrink operations.
+      assertValues("Queue shrunk operations",
+          2 * LARGE_OPERATIONS,
+          abfsOutputStreamStatistics.getQueueShrunkOps());
+    }
+
+  }
+
+  /**
+   * Test to check number of {@code
+   * AbfsOutputStream#writeCurrentBufferToService()} calls.
+   * After writing data, AbfsOutputStream doesn't upload data till flush() is
+   * called. Hence, flush() calls were made after write() to simulate the
+   * scenario.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamWriteBuffer() throws IOException {
+    describe("Testing writeCurrentBufferToService() calls");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path writeBufferFilePath = path(getMethodName());
+    String testWriteBuffer = "Buffer";
+
+    try (AbfsOutputStream outForOneOp = createAbfsOutputStream(fs,
+        writeBufferFilePath)) {
+
+      AbfsOutputStreamStatisticsImpl abfsOutputStreamStatistics =
+          outForOneOp.getOutputStreamStatistics();
+
+      //Test for zero time writing Buffer to service.
+      assertValues("number writeCurrentBufferToService() calls", 0,
+          abfsOutputStreamStatistics.getWriteCurrentBufferOperations());
+
+      outForOneOp.write(testWriteBuffer.getBytes());
+      outForOneOp.flush();
+
+      abfsOutputStreamStatistics = outForOneOp.getOutputStreamStatistics();
+
+      //Test for one time writeCurrentBuffer() call.
+      assertValues("number writeCurrentBufferToService() calls", 1,
+          abfsOutputStreamStatistics.getWriteCurrentBufferOperations());
+    }
+
+    try (AbfsOutputStream outForLargeOps = createAbfsOutputStream(fs,
+        writeBufferFilePath)) {
+
+      for (int i = 0; i < LARGE_OPERATIONS; i++) {
+        outForLargeOps.write(testWriteBuffer.getBytes());
+        outForLargeOps.flush();
+      }
+      AbfsOutputStreamStatisticsImpl abfsOutputStreamStatistics =
+          outForLargeOps.getOutputStreamStatistics();
+      //Test for 10 writeBufferOperations.
+      assertValues("number of writeCurrentBufferToService() calls",
+          LARGE_OPERATIONS,
+          abfsOutputStreamStatistics.getWriteCurrentBufferOperations());
+    }
+
+  }
+
+  /**
+   * Generic create File and setting OutputStreamFlush to false.
 
 Review comment:
   *disableOutputStreamFlush

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r408181568
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamStatisticsImpl.java
 ##########
 @@ -0,0 +1,177 @@
+/**
+ * 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.services;
+
+/**
+ * OutputStream Statistics Implementation for Abfs.
+ */
+public class AbfsOutputStreamStatisticsImpl
+    implements AbfsOutputStreamStatistics {
+  private long bytesToUpload;
+  private long bytesUploadSuccessful;
+  private long bytesUploadFailed;
+  /**
+   * counter to get the total time spent while waiting for tasks to complete
+   * in the Blocking queue inside the thread executor.
+   */
+  private long timeSpendOnTaskWait;
+  /**
+   * counter to get the total number of queue shrink operations done{@code
+   * AbfsOutputStream#shrinkWriteOperationQueue()} by
+   * AbfsOutputStream to remove the write operations which were successfully
+   * done by AbfsOutputStream from the Blocking Queue.
+   */
+  private long queueShrunkOps;
+  /**
+   * counter to get the total number of times the current buffer is written
+   * to the service{@code AbfsOutputStream#writeCurrentBufferToService()} via
+   * AbfsClient and appended to the
+   * Data store by
+   * AbfsResOperation.
+   */
+  private long writeCurrentBufferOperations;
+
+  /**
+   * Records the need to upload bytes and increments the total bytes that
+   * needs to be uploaded.
+   *
+   * @param bytes Total bytes to upload. Negative bytes are ignored.
+   */
+  @Override
+  public void bytesToUpload(long bytes) {
+    if (bytes > 0) {
+      bytesToUpload += bytes;
+    }
+  }
+
+  /**
+   * Records the total bytes successfully uploaded through AbfsOutputStream.
+   *
+   * @param bytes number of bytes that were successfully uploaded. Negative
+   *              bytes are ignored.
+   */
+  @Override
+  public void uploadSuccessful(long bytes) {
+    if (bytes > 0) {
+      bytesUploadSuccessful += bytes;
+    }
+  }
+
+  /**
+   * Records the total bytes failed to upload through AbfsOutputStream.
+   *
+   * @param bytes number of bytes failed to upload. Negative bytes are ignored.
 
 Review comment:
   Remove bytes.

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r405059806
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStream.java
 ##########
 @@ -133,58 +133,60 @@ public void testAbfsOutputStreamTimeSpendOnWaitTask() throws IOException {
   public void testAbfsOutputStreamQueueShrink() throws IOException {
     describe("Testing Queue Shrink calls in AbfsOutputStream");
     final AzureBlobFileSystem fs = getFileSystem();
-    Path TEST_PATH = new Path("AbfsOutputStreamStatsPath");
+    Path queueShrinkFilePath = new Path("AbfsOutputStreamStatsPath");
     AzureBlobFileSystemStore abfss = fs.getAbfsStore();
     abfss.getAbfsConfiguration().setDisableOutputStreamFlush(false);
     FileSystem.Statistics statistics = fs.getFsStatistics();
     String testQueueShrink = "testQueue";
 
-
     AbfsOutputStream outForOneOp = null;
 
     try {
-      outForOneOp = (AbfsOutputStream) abfss.createFile(TEST_PATH, statistics,
-        true,
-          FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
+      outForOneOp =
+          (AbfsOutputStream) abfss.createFile(queueShrinkFilePath, statistics,
+              true,
+              FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
 
       //Test for shrinking Queue zero time
-      Assert.assertEquals("Mismatch in number of queueShrink() Calls", 0,
+      assertValues("number of queueShrink() Calls", 0,
           outForOneOp.getOutputStreamStatistics().queueShrink);
 
       outForOneOp.write(testQueueShrink.getBytes());
       // Queue is shrunk 2 times when outStream is flushed
       outForOneOp.flush();
 
       //Test for shrinking Queue 2 times
-      Assert.assertEquals("Mismatch in number of queueShrink() Calls", 2,
+      assertValues("number of queueShrink() Calls", 2,
           outForOneOp.getOutputStreamStatistics().queueShrink);
 
     } finally {
-      if(outForOneOp != null){
+      if (outForOneOp != null) {
         outForOneOp.close();
       }
     }
 
     AbfsOutputStream outForLargeOps = null;
 
     try {
-      outForLargeOps = (AbfsOutputStream) abfss.createFile(TEST_PATH,
+      outForLargeOps = (AbfsOutputStream) abfss.createFile(queueShrinkFilePath,
           statistics, true,
           FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
 
+      int largeValue = 1000;
       //QueueShrink is called 2 times in 1 flush(), hence 1000 flushes must
       // give 2000 QueueShrink calls
-      for (int i = 0; i < 1000; i++) {
+      for (int i = 0; i < largeValue; i++) {
         outForLargeOps.write(testQueueShrink.getBytes());
         //Flush is quite expensive so 1000 calls only which takes 1 min+
         outForLargeOps.flush();
 
 Review comment:
   No I can't, basically calling flush after I write means the write task is done. This would trigger the shrinkWriteOperationQueue() method and we need to do it after each write to get 20 operations.
   If I flush after the loop, it would take all the write calls as 1 write operation and only 1 time the shrinkWriteOperationQueue() method is triggered.

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r408178404
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsOutputStreamStatistics.java
 ##########
 @@ -0,0 +1,120 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+import java.util.Random;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl;
+
+/**
+ * Unit Tests for AbfsOutputStream Statistics.
+ */
+public class TestAbfsOutputStreamStatistics
+    extends AbstractAbfsIntegrationTest {
+
+  private static final int LOW_RANGE_FOR_RANDOM_VALUE = 49;
+  private static final int HIGH_RANGE_FOR_RANDOM_VALUE = 9999;
+
+  public TestAbfsOutputStreamStatistics() throws Exception {
+  }
+
+  /**
+   * Tests to check bytes failed to Upload in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamBytesFailed() {
+    describe("Testing Bytes Failed during uploading in AbfsOutputSteam");
+
+    AbfsOutputStreamStatisticsImpl abfsOutputStreamStatistics =
+        new AbfsOutputStreamStatisticsImpl();
+
+    //Test for zero bytes uploaded.
+    assertValues("number fo bytes failed to upload", 0,
+        abfsOutputStreamStatistics.getBytesUploadFailed());
+
+    //Populating small random value for bytesFailed.
+    int randomBytesFailed = new Random().nextInt(LOW_RANGE_FOR_RANDOM_VALUE);
+    abfsOutputStreamStatistics.uploadFailed(randomBytesFailed);
+    //Test for bytes failed to upload.
+    assertValues("number fo bytes failed to upload", randomBytesFailed,
+        abfsOutputStreamStatistics.getBytesUploadFailed());
+
+    //Initializing again to reset the statistics.
+    abfsOutputStreamStatistics = new AbfsOutputStreamStatisticsImpl();
+
+    //Populating large random values for bytesFailed.
+    randomBytesFailed = new Random().nextInt(HIGH_RANGE_FOR_RANDOM_VALUE);
 
 Review comment:
   A better test would to be call multiple times and see if the summation is working fine rather that resetting the stats?

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r408154919
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamStatistics.java
 ##########
 @@ -0,0 +1,79 @@
+/**
+ * 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.services;
+
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Interface for {@link AbfsOutputStream} statistics.
+ */
+@InterfaceStability.Unstable
+public interface AbfsOutputStreamStatistics {
+
+  /**
+   * Number of bytes to be uploaded.
+   *
+   * @param bytes number of bytes to upload.
+   */
+  void bytesToUpload(long bytes);
+
+  /**
+   * Records a successful upload and the number of bytes uploaded.
+   *
+   * @param bytes number of bytes that were successfully uploaded.
 
 Review comment:
   typo.. Remove first word bytes.

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r394228752
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStream.java
 ##########
 @@ -133,58 +133,60 @@ public void testAbfsOutputStreamTimeSpendOnWaitTask() throws IOException {
   public void testAbfsOutputStreamQueueShrink() throws IOException {
     describe("Testing Queue Shrink calls in AbfsOutputStream");
     final AzureBlobFileSystem fs = getFileSystem();
-    Path TEST_PATH = new Path("AbfsOutputStreamStatsPath");
+    Path queueShrinkFilePath = new Path("AbfsOutputStreamStatsPath");
     AzureBlobFileSystemStore abfss = fs.getAbfsStore();
     abfss.getAbfsConfiguration().setDisableOutputStreamFlush(false);
     FileSystem.Statistics statistics = fs.getFsStatistics();
     String testQueueShrink = "testQueue";
 
-
     AbfsOutputStream outForOneOp = null;
 
     try {
-      outForOneOp = (AbfsOutputStream) abfss.createFile(TEST_PATH, statistics,
-        true,
-          FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
+      outForOneOp =
+          (AbfsOutputStream) abfss.createFile(queueShrinkFilePath, statistics,
+              true,
+              FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
 
       //Test for shrinking Queue zero time
-      Assert.assertEquals("Mismatch in number of queueShrink() Calls", 0,
+      assertValues("number of queueShrink() Calls", 0,
           outForOneOp.getOutputStreamStatistics().queueShrink);
 
       outForOneOp.write(testQueueShrink.getBytes());
       // Queue is shrunk 2 times when outStream is flushed
       outForOneOp.flush();
 
       //Test for shrinking Queue 2 times
-      Assert.assertEquals("Mismatch in number of queueShrink() Calls", 2,
+      assertValues("number of queueShrink() Calls", 2,
           outForOneOp.getOutputStreamStatistics().queueShrink);
 
     } finally {
-      if(outForOneOp != null){
+      if (outForOneOp != null) {
         outForOneOp.close();
       }
     }
 
     AbfsOutputStream outForLargeOps = null;
 
     try {
-      outForLargeOps = (AbfsOutputStream) abfss.createFile(TEST_PATH,
+      outForLargeOps = (AbfsOutputStream) abfss.createFile(queueShrinkFilePath,
           statistics, true,
           FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
 
+      int largeValue = 1000;
       //QueueShrink is called 2 times in 1 flush(), hence 1000 flushes must
       // give 2000 QueueShrink calls
-      for (int i = 0; i < 1000; i++) {
+      for (int i = 0; i < largeValue; i++) {
         outForLargeOps.write(testQueueShrink.getBytes());
         //Flush is quite expensive so 1000 calls only which takes 1 min+
         outForLargeOps.flush();
 
 Review comment:
   any way around flush() to get queueShrink() calls after writing ?
   flush() is quite expensive as it takes some time even at 1000 calls to test.

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r408492995
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
 ##########
 @@ -436,4 +453,28 @@ private void waitForTaskToComplete() throws IOException {
   public synchronized void waitForPendingUploads() throws IOException {
     waitForTaskToComplete();
   }
+
+  /**
+   * Getter method for AbfsOutputStream Statistics.
+   *
+   * @return statistics for AbfsOutputStream.
+   */
+  @VisibleForTesting
+  public AbfsOutputStreamStatisticsImpl getOutputStreamStatistics() {
 
 Review comment:
   I think it expects to return AbfsOutputStreamStatisticsImpl, and if we return just AbfsOutputStreamStatistics, it would give rise to incompatible types error.

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r395749763
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStream.java
 ##########
 @@ -0,0 +1,278 @@
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
+import org.apache.hadoop.fs.permission.FsPermission;
+
+/**
+ * Test AbfsOutputStream statistics.
+ */
+public class ITestAbfsOutputStream extends AbstractAbfsIntegrationTest {
+
+  public ITestAbfsOutputStream() throws Exception {
+  }
+
+  /**
+   * Tests to check bytes Uploading in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamUploadingBytes() throws IOException {
+    describe("Testing Bytes uploaded in AbfsOutputSteam");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path uploadBytesFilePath = new Path("AbfsOutputStreamStatsPath");
+    AzureBlobFileSystemStore abfss = fs.getAbfsStore();
+    FileSystem.Statistics statistics = fs.getFsStatistics();
+    abfss.getAbfsConfiguration().setDisableOutputStreamFlush(false);
+    String testBytesToUpload = "bytes";
+
+    AbfsOutputStream outForSomeBytes = null;
+    try {
+      outForSomeBytes = (AbfsOutputStream) abfss.createFile(uploadBytesFilePath,
+          statistics,
+          true,
+          FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
+
+      //Test for zero bytes To upload
+      assertValues("bytes to upload", 0,
+          outForSomeBytes.getOutputStreamStatistics().bytesToUpload);
+
+      outForSomeBytes.write(testBytesToUpload.getBytes());
+      outForSomeBytes.flush();
+
+      //Test for some bytes to upload
+      assertValues("bytes to upload", testBytesToUpload.getBytes().length,
+          outForSomeBytes.getOutputStreamStatistics().bytesToUpload);
+
+      //Test for relation between bytesUploadSuccessful, bytesUploadFailed
+      // and bytesToUpload
+      assertValues("bytesUploadSuccessful equal to difference between "
+              + "bytesToUpload and bytesUploadFailed",
+          outForSomeBytes.getOutputStreamStatistics().bytesUploadSuccessful,
+          outForSomeBytes.getOutputStreamStatistics().bytesToUpload -
+              outForSomeBytes.getOutputStreamStatistics().bytesUploadFailed);
+
+    } finally {
+      if (outForSomeBytes != null) {
+        outForSomeBytes.close();
+      }
+    }
+
+    AbfsOutputStream outForLargeBytes = null;
+    try {
+      outForLargeBytes =
+          (AbfsOutputStream) abfss.createFile(uploadBytesFilePath,
+              statistics
+              , true, FsPermission.getDefault(),
+              FsPermission.getUMask(fs.getConf()));
+
+      int largeValue = 100000;
+      for (int i = 0; i < largeValue; i++) {
+        outForLargeBytes.write(testBytesToUpload.getBytes());
+      }
+      outForLargeBytes.flush();
+
+      //Test for large bytes to upload
+      assertValues("bytes to upload",
+          largeValue * (testBytesToUpload.getBytes().length),
+          outForLargeBytes.getOutputStreamStatistics().bytesToUpload);
+
+      //Test for relation between bytesUploadSuccessful, bytesUploadFailed
+      // and bytesToUpload
+      assertValues("bytesUploadSuccessful equal to difference between "
+              + "bytesToUpload and bytesUploadFailed",
+          outForSomeBytes.getOutputStreamStatistics().bytesUploadSuccessful,
+          outForSomeBytes.getOutputStreamStatistics().bytesToUpload -
+              outForSomeBytes.getOutputStreamStatistics().bytesUploadFailed);
+
+    } finally {
+      if (outForLargeBytes != null) {
+        outForLargeBytes.close();
+      }
+    }
+
+  }
+
+  /**
+   * Tests to check time spend on waiting for tasks to be complete on a
+   * blocking queue in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamTimeSpendOnWaitTask() throws IOException {
 
 Review comment:
   Also, "time spent"

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r408174325
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStreamStatistics.java
 ##########
 @@ -0,0 +1,233 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl;
+
+/**
+ * Test AbfsOutputStream statistics.
+ */
+public class ITestAbfsOutputStreamStatistics
+    extends AbstractAbfsIntegrationTest {
+  private static final int LARGE_OPERATIONS = 10;
+
+  public ITestAbfsOutputStreamStatistics() throws Exception {
 
 Review comment:
   not needed; just cut 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] mukund-thakur commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r404591627
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamStatisticsImpl.java
 ##########
 @@ -0,0 +1,148 @@
+/**
+ * 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.services;
+
+/**
+ * OutputStream Statistics Implementation for Abfs.
+ * timeSpendOnTaskWait - Time spend on waiting for tasks to be complete on
+ * Blocking Queue in AbfsOutputStream.
+ *
+ * queueShrink - Number of times Blocking Queue was shrunk after writing
+ * data.
+ *
+ * WriteCurrentBufferOperations - Number of times
+ * {@code writeCurrentBufferToService()} calls were made.
+ */
+public class AbfsOutputStreamStatisticsImpl
+    implements AbfsOutputStreamStatistics {
+  private volatile long bytesToUpload;
+  private volatile long bytesUploadSuccessful;
+  private volatile long bytesUploadFailed;
+  private volatile long timeSpendOnTaskWait;
+  private volatile long queueShrink;
+  private volatile long writeCurrentBufferOperations;
+
+  /**
+   * Number of bytes uploaded.
+   *
+   * @param bytes negative values are ignored
+   */
+  @Override
+  public void bytesToUpload(long bytes) {
+    if (bytes > 0) {
+      bytesToUpload += bytes;
+    }
+  }
+
+  /**
+   * Upload successful with the number of bytes.
+   * @param bytes number of bytes that were successfully uploaded
+   */
+  @Override
+  public void uploadSuccessful(long bytes) {
+    if (bytes > 0) {
+      bytesUploadSuccessful += bytes;
+    }
+  }
+
+  /**
+   * Upload failed and the number of bytes.
+   *
+   * @param bytes negative values are ignored
+   */
+  @Override
+  public void uploadFailed(long bytes) {
+    if (bytes > 0) {
+      bytesUploadFailed += bytes;
+    }
+  }
+
+  /**
+   * Time spent for waiting a task to be completed.
+   *
+   * @param startTime on calling {@code waitForTaskToComplete()}
+   * @param endTime   on method completing
+   */
+  @Override
+  public void timeSpentTaskWait(long startTime, long endTime) {
+    timeSpendOnTaskWait += endTime - startTime;
+  }
+
+  /**
+   * Number of calls to {@code shrinkWriteOperationQueue()}.
+   */
+  @Override
+  public void queueShrinked() {
 
 Review comment:
   typo in times, small t in try.

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r408486756
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsOutputStreamStatistics.java
 ##########
 @@ -0,0 +1,120 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+import java.util.Random;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl;
+
+/**
+ * Unit Tests for AbfsOutputStream Statistics.
+ */
+public class TestAbfsOutputStreamStatistics
+    extends AbstractAbfsIntegrationTest {
+
+  private static final int LOW_RANGE_FOR_RANDOM_VALUE = 49;
+  private static final int HIGH_RANGE_FOR_RANDOM_VALUE = 9999;
+
+  public TestAbfsOutputStreamStatistics() throws Exception {
+  }
+
+  /**
+   * Tests to check bytes failed to Upload in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamBytesFailed() {
+    describe("Testing Bytes Failed during uploading in AbfsOutputSteam");
+
+    AbfsOutputStreamStatisticsImpl abfsOutputStreamStatistics =
+        new AbfsOutputStreamStatisticsImpl();
+
+    //Test for zero bytes uploaded.
+    assertValues("number fo bytes failed to upload", 0,
+        abfsOutputStreamStatistics.getBytesUploadFailed());
+
+    //Populating small random value for bytesFailed.
+    int randomBytesFailed = new Random().nextInt(LOW_RANGE_FOR_RANDOM_VALUE);
+    abfsOutputStreamStatistics.uploadFailed(randomBytesFailed);
+    //Test for bytes failed to upload.
+    assertValues("number fo bytes failed to upload", randomBytesFailed,
+        abfsOutputStreamStatistics.getBytesUploadFailed());
+
+    //Initializing again to reset the statistics.
+    abfsOutputStreamStatistics = new AbfsOutputStreamStatisticsImpl();
+
+    //Populating large random values for bytesFailed.
+    randomBytesFailed = new Random().nextInt(HIGH_RANGE_FOR_RANDOM_VALUE);
+    abfsOutputStreamStatistics.uploadFailed(randomBytesFailed);
+    //Test for bytes failed to upload.
+    assertValues("number fo bytes failed to upload", randomBytesFailed,
+        abfsOutputStreamStatistics.getBytesUploadFailed());
+  }
+
+  /**
+   * Tests to check time spent on waiting for tasks to be complete on a
+   * blocking queue in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamTimeSpentOnWaitTask() {
+    describe("Testing Time Spend on Waiting for Task to be complete");
+
+    AbfsOutputStreamStatisticsImpl abfsOutputStreamStatistics =
+        new AbfsOutputStreamStatisticsImpl();
+
+    //Test for initial value of timeSpentWaitTask.
+    assertValues("Time spend on waiting for tasks to complete", 0,
+        abfsOutputStreamStatistics.getTimeSpendOnTaskWait());
+
+    int smallRandomStartTime =
+        new Random().nextInt(LOW_RANGE_FOR_RANDOM_VALUE);
+    int smallRandomEndTime =
+        new Random().nextInt(LOW_RANGE_FOR_RANDOM_VALUE)
+            + smallRandomStartTime;
+    int smallDiff = smallRandomEndTime - smallRandomStartTime;
+    abfsOutputStreamStatistics
+        .timeSpentTaskWait(smallRandomStartTime, smallRandomEndTime);
+    //Test for small random value of timeSpentWaitTask.
+    assertValues("Time spend on waiting for tasks to complete", smallDiff,
+        abfsOutputStreamStatistics.getTimeSpendOnTaskWait());
+
+    int largeRandomStartTime =
+        new Random().nextInt(HIGH_RANGE_FOR_RANDOM_VALUE);
+    int largeRandomEndTime = new Random().nextInt(HIGH_RANGE_FOR_RANDOM_VALUE)
+        + largeRandomStartTime;
+    int randomDiff = largeRandomEndTime - largeRandomStartTime;
+    abfsOutputStreamStatistics
 
 Review comment:
   actually in this test I am testing the summation, by expecting a summed value of test at L101 in test at L115. Should I do something similar for above tests(L68) ? or should I do multiple summations using loops ?

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r402076899
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamStatisticsImpl.java
 ##########
 @@ -0,0 +1,148 @@
+/**
+ * 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.services;
+
+/**
+ * OutputStream Statistics Implementation for Abfs.
+ * timeSpendOnTaskWait - Time spend on waiting for tasks to be complete on
+ * Blocking Queue in AbfsOutputStream.
+ *
+ * queueShrink - Number of times Blocking Queue was shrunk after writing
+ * data.
+ *
+ * WriteCurrentBufferOperations - Number of times
+ * {@code writeCurrentBufferToService()} calls were made.
+ */
+public class AbfsOutputStreamStatisticsImpl
+    implements AbfsOutputStreamStatistics {
+  private volatile long bytesToUpload;
+  private volatile long bytesUploadSuccessful;
+  private volatile long bytesUploadFailed;
+  private volatile long timeSpendOnTaskWait;
+  private volatile long queueShrink;
+  private volatile long writeCurrentBufferOperations;
+
+  /**
+   * Number of bytes uploaded.
+   *
+   * @param bytes negative values are ignored
+   */
+  @Override
+  public void bytesToUpload(long bytes) {
+    if (bytes > 0) {
+      bytesToUpload += bytes;
+    }
+  }
+
+  /**
+   * Upload successful with the number of bytes.
+   * @param bytes number of bytes that were successfully uploaded
+   */
+  @Override
+  public void uploadSuccessful(long bytes) {
+    if (bytes > 0) {
+      bytesUploadSuccessful += bytes;
+    }
+  }
+
+  /**
+   * Upload failed and the number of bytes.
+   *
+   * @param bytes negative values are ignored
+   */
+  @Override
+  public void uploadFailed(long bytes) {
+    if (bytes > 0) {
+      bytesUploadFailed += bytes;
+    }
+  }
+
+  /**
+   * Time spent for waiting a task to be completed.
+   *
+   * @param startTime on calling {@code waitForTaskToComplete()}
+   * @param endTime   on method completing
+   */
+  @Override
+  public void timeSpentTaskWait(long startTime, long endTime) {
+    timeSpendOnTaskWait += endTime - startTime;
+  }
+
+  /**
+   * Number of calls to {@code shrinkWriteOperationQueue()}.
+   */
+  @Override
+  public void queueShrinked() {
 
 Review comment:
   Once again which queue. How is this metrics important??

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r402073163
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamStatistics.java
 ##########
 @@ -0,0 +1,78 @@
+/**
+ * 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.services;
+
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Interface for {@link AbfsOutputStream} statistics.
+ */
+@InterfaceStability.Unstable
+public interface AbfsOutputStreamStatistics {
+
+  /**
+   * Number of bytes to be uploaded.
+   *
+   * @param bytes number of bytes to upload
 
 Review comment:
   Add trailing . in the end for all java docs.

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r408681962
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamStatisticsImpl.java
 ##########
 @@ -0,0 +1,177 @@
+/**
+ * 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.services;
+
+/**
+ * OutputStream Statistics Implementation for Abfs.
+ */
+public class AbfsOutputStreamStatisticsImpl
+    implements AbfsOutputStreamStatistics {
+  private long bytesToUpload;
 
 Review comment:
   I thought we concluded that AbfsInputStream and AbfsOutputStream uses synchronized for thread safety and we don't need either volatile or AtomicLong for the counters.
   Should I use volatile and suppress the warnings ?

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on issue #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#issuecomment-614559419
 
 
   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 36s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  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 17s |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 31s |  trunk passed  |
   | +1 :green_heart: |  checkstyle  |   0m 24s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 35s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  14m 59s |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 27s |  trunk passed  |
   | +0 :ok: |  spotbugs  |   0m 54s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 51s |  trunk passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 28s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 23s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 23s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 17s |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 27s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  13m 52s |  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 54s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 22s |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 33s |  The patch does not generate ASF License warnings.  |
   |  |   |  57m 41s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.8 Server=19.03.8 base: https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/10/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/1899 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux 18c821b9bceb 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 / cc5c1da |
   | Default Java | 1.8.0_242 |
   |  Test Results | https://builds.apache.org/job/hadoop-multibranch/job/PR-1899/10/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-1899/10/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 issue #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on issue #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#issuecomment-600028014
 
 
   Please suggest how to write tests for "timeSpendOnTaskWait" counter.
   Please suggest how to simulate bytes failed to upload for "bytesUploadFailed" counter to be populated.
   flush() is an expensive call but need to call it to see counters to populate, hence only 1000 operation is called "Large"(takes 1-3 minutes per test if 1000 flush() are called). Should I leave it as it is, or is there another way ?

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r395751202
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStream.java
 ##########
 @@ -133,58 +133,60 @@ public void testAbfsOutputStreamTimeSpendOnWaitTask() throws IOException {
   public void testAbfsOutputStreamQueueShrink() throws IOException {
     describe("Testing Queue Shrink calls in AbfsOutputStream");
     final AzureBlobFileSystem fs = getFileSystem();
-    Path TEST_PATH = new Path("AbfsOutputStreamStatsPath");
+    Path queueShrinkFilePath = new Path("AbfsOutputStreamStatsPath");
     AzureBlobFileSystemStore abfss = fs.getAbfsStore();
     abfss.getAbfsConfiguration().setDisableOutputStreamFlush(false);
     FileSystem.Statistics statistics = fs.getFsStatistics();
     String testQueueShrink = "testQueue";
 
-
     AbfsOutputStream outForOneOp = null;
 
     try {
-      outForOneOp = (AbfsOutputStream) abfss.createFile(TEST_PATH, statistics,
-        true,
-          FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
+      outForOneOp =
+          (AbfsOutputStream) abfss.createFile(queueShrinkFilePath, statistics,
+              true,
+              FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
 
       //Test for shrinking Queue zero time
-      Assert.assertEquals("Mismatch in number of queueShrink() Calls", 0,
+      assertValues("number of queueShrink() Calls", 0,
           outForOneOp.getOutputStreamStatistics().queueShrink);
 
       outForOneOp.write(testQueueShrink.getBytes());
       // Queue is shrunk 2 times when outStream is flushed
       outForOneOp.flush();
 
       //Test for shrinking Queue 2 times
-      Assert.assertEquals("Mismatch in number of queueShrink() Calls", 2,
+      assertValues("number of queueShrink() Calls", 2,
           outForOneOp.getOutputStreamStatistics().queueShrink);
 
     } finally {
-      if(outForOneOp != null){
+      if (outForOneOp != null) {
         outForOneOp.close();
       }
     }
 
     AbfsOutputStream outForLargeOps = null;
 
     try {
-      outForLargeOps = (AbfsOutputStream) abfss.createFile(TEST_PATH,
+      outForLargeOps = (AbfsOutputStream) abfss.createFile(queueShrinkFilePath,
           statistics, true,
           FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
 
+      int largeValue = 1000;
       //QueueShrink is called 2 times in 1 flush(), hence 1000 flushes must
       // give 2000 QueueShrink calls
-      for (int i = 0; i < 1000; i++) {
+      for (int i = 0; i < largeValue; i++) {
         outForLargeOps.write(testQueueShrink.getBytes());
         //Flush is quite expensive so 1000 calls only which takes 1 min+
         outForLargeOps.flush();
 
 Review comment:
   do you have to call it so many times?

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r405023294
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
 ##########
 @@ -384,6 +398,7 @@ private synchronized void flushWrittenBytesToServiceInternal(final long offset,
    * operation FIFO queue.
    */
   private synchronized void shrinkWriteOperationQueue() throws IOException {
+    outputStreamStatistics.queueShrinked();
 
 Review comment:
   it would also be "queueShrunk" as a name

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r408485372
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamStatistics.java
 ##########
 @@ -0,0 +1,79 @@
+/**
+ * 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.services;
+
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Interface for {@link AbfsOutputStream} statistics.
+ */
+@InterfaceStability.Unstable
+public interface AbfsOutputStreamStatistics {
+
+  /**
+   * Number of bytes to be uploaded.
+   *
+   * @param bytes number of bytes to upload.
+   */
+  void bytesToUpload(long bytes);
+
+  /**
+   * Records a successful upload and the number of bytes uploaded.
+   *
+   * @param bytes number of bytes that were successfully uploaded.
+   */
+  void uploadSuccessful(long bytes);
+
+  /**
+   * Records that upload is failed and the number of bytes.
+   *
+   * @param bytes number of bytes that failed to upload.
 
 Review comment:
   that is the name of the param, description follows. It would be color coded in an IDE.

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r395750400
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStream.java
 ##########
 @@ -0,0 +1,278 @@
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
+import org.apache.hadoop.fs.permission.FsPermission;
+
+/**
+ * Test AbfsOutputStream statistics.
+ */
+public class ITestAbfsOutputStream extends AbstractAbfsIntegrationTest {
+
+  public ITestAbfsOutputStream() throws Exception {
+  }
+
+  /**
+   * Tests to check bytes Uploading in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamUploadingBytes() throws IOException {
+    describe("Testing Bytes uploaded in AbfsOutputSteam");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path uploadBytesFilePath = new Path("AbfsOutputStreamStatsPath");
+    AzureBlobFileSystemStore abfss = fs.getAbfsStore();
+    FileSystem.Statistics statistics = fs.getFsStatistics();
+    abfss.getAbfsConfiguration().setDisableOutputStreamFlush(false);
+    String testBytesToUpload = "bytes";
+
+    AbfsOutputStream outForSomeBytes = null;
+    try {
+      outForSomeBytes = (AbfsOutputStream) abfss.createFile(uploadBytesFilePath,
+          statistics,
+          true,
+          FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
+
+      //Test for zero bytes To upload
+      assertValues("bytes to upload", 0,
+          outForSomeBytes.getOutputStreamStatistics().bytesToUpload);
+
+      outForSomeBytes.write(testBytesToUpload.getBytes());
+      outForSomeBytes.flush();
+
+      //Test for some bytes to upload
+      assertValues("bytes to upload", testBytesToUpload.getBytes().length,
+          outForSomeBytes.getOutputStreamStatistics().bytesToUpload);
+
+      //Test for relation between bytesUploadSuccessful, bytesUploadFailed
+      // and bytesToUpload
+      assertValues("bytesUploadSuccessful equal to difference between "
+              + "bytesToUpload and bytesUploadFailed",
+          outForSomeBytes.getOutputStreamStatistics().bytesUploadSuccessful,
+          outForSomeBytes.getOutputStreamStatistics().bytesToUpload -
+              outForSomeBytes.getOutputStreamStatistics().bytesUploadFailed);
+
+    } finally {
+      if (outForSomeBytes != null) {
+        outForSomeBytes.close();
+      }
+    }
+
+    AbfsOutputStream outForLargeBytes = null;
+    try {
+      outForLargeBytes =
+          (AbfsOutputStream) abfss.createFile(uploadBytesFilePath,
+              statistics
+              , true, FsPermission.getDefault(),
+              FsPermission.getUMask(fs.getConf()));
+
+      int largeValue = 100000;
+      for (int i = 0; i < largeValue; i++) {
+        outForLargeBytes.write(testBytesToUpload.getBytes());
+      }
+      outForLargeBytes.flush();
+
+      //Test for large bytes to upload
+      assertValues("bytes to upload",
+          largeValue * (testBytesToUpload.getBytes().length),
+          outForLargeBytes.getOutputStreamStatistics().bytesToUpload);
+
+      //Test for relation between bytesUploadSuccessful, bytesUploadFailed
+      // and bytesToUpload
+      assertValues("bytesUploadSuccessful equal to difference between "
+              + "bytesToUpload and bytesUploadFailed",
+          outForSomeBytes.getOutputStreamStatistics().bytesUploadSuccessful,
+          outForSomeBytes.getOutputStreamStatistics().bytesToUpload -
+              outForSomeBytes.getOutputStreamStatistics().bytesUploadFailed);
+
+    } finally {
+      if (outForLargeBytes != null) {
+        outForLargeBytes.close();
+      }
+    }
+
+  }
+
+  /**
+   * Tests to check time spend on waiting for tasks to be complete on a
+   * blocking queue in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamTimeSpendOnWaitTask() throws IOException {
+    describe("Testing Time Spend on Waiting for Task to be complete");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path timeSpendFilePath = new Path("AbfsOutputStreamStatsPath");
+    AzureBlobFileSystemStore abfss = fs.getAbfsStore();
+    FileSystem.Statistics statistics = fs.getFsStatistics();
+
+    AbfsOutputStream out =
+        (AbfsOutputStream) abfss.createFile(timeSpendFilePath,
+            statistics, true,
+            FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
+
+  }
+
+  /**
+   * Tests to check number of {@codes shrinkWriteOperationQueue()}
+   * calls.
+   * After writing data, AbfsOutputStream doesn't upload the data until
+   * Flushed. Hence, flush() method is called after write() to test Queue
+   * shrink calls.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamQueueShrink() throws IOException {
+    describe("Testing Queue Shrink calls in AbfsOutputStream");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path queueShrinkFilePath = new Path("AbfsOutputStreamStatsPath");
+    AzureBlobFileSystemStore abfss = fs.getAbfsStore();
+    abfss.getAbfsConfiguration().setDisableOutputStreamFlush(false);
+    FileSystem.Statistics statistics = fs.getFsStatistics();
+    String testQueueShrink = "testQueue";
+
+    AbfsOutputStream outForOneOp = null;
+
+    try {
+      outForOneOp =
+          (AbfsOutputStream) abfss.createFile(queueShrinkFilePath, statistics,
+              true,
+              FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
+
+      //Test for shrinking Queue zero time
+      assertValues("number of queueShrink() Calls", 0,
+          outForOneOp.getOutputStreamStatistics().queueShrink);
+
+      outForOneOp.write(testQueueShrink.getBytes());
+      // Queue is shrunk 2 times when outStream is flushed
+      outForOneOp.flush();
+
+      //Test for shrinking Queue 2 times
+      assertValues("number of queueShrink() Calls", 2,
+          outForOneOp.getOutputStreamStatistics().queueShrink);
+
+    } finally {
+      if (outForOneOp != null) {
 
 Review comment:
   try-with-resources or IOUtils.closeQuietly

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r395755381
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamStatisticsImpl.java
 ##########
 @@ -0,0 +1,102 @@
+package org.apache.hadoop.fs.azurebfs.services;
+
+/**
+ * OutputStream Statistics Implementation for Abfs.
+ * timeSpendOnTaskWait - Time spend on waiting for tasks to be complete on
+ * Blocking Queue in AbfsOutputStream.
+ *
+ * queueShrink - Number of times Blocking Queue was shrunk after writing
+ * data.
+ *
+ * WriteCurrentBufferOperations - Number of times
+ * {@codes writeCurrentBufferToService()} calls were made.
+ */
+public class AbfsOutputStreamStatisticsImpl
+    implements AbfsOutputStreamStatistics {
+  public volatile long bytesToUpload;
 
 Review comment:
   let's make these private and have getters

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet edited a comment on issue #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#issuecomment-600024548
 
 
   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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r395751470
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStream.java
 ##########
 @@ -0,0 +1,278 @@
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
+import org.apache.hadoop.fs.permission.FsPermission;
+
+/**
+ * Test AbfsOutputStream statistics.
+ */
+public class ITestAbfsOutputStream extends AbstractAbfsIntegrationTest {
+
+  public ITestAbfsOutputStream() throws Exception {
+  }
+
+  /**
+   * Tests to check bytes Uploading in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamUploadingBytes() throws IOException {
+    describe("Testing Bytes uploaded in AbfsOutputSteam");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path uploadBytesFilePath = new Path("AbfsOutputStreamStatsPath");
+    AzureBlobFileSystemStore abfss = fs.getAbfsStore();
+    FileSystem.Statistics statistics = fs.getFsStatistics();
+    abfss.getAbfsConfiguration().setDisableOutputStreamFlush(false);
+    String testBytesToUpload = "bytes";
+
+    AbfsOutputStream outForSomeBytes = null;
+    try {
+      outForSomeBytes = (AbfsOutputStream) abfss.createFile(uploadBytesFilePath,
+          statistics,
+          true,
+          FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
+
+      //Test for zero bytes To upload
+      assertValues("bytes to upload", 0,
+          outForSomeBytes.getOutputStreamStatistics().bytesToUpload);
+
+      outForSomeBytes.write(testBytesToUpload.getBytes());
+      outForSomeBytes.flush();
+
+      //Test for some bytes to upload
+      assertValues("bytes to upload", testBytesToUpload.getBytes().length,
+          outForSomeBytes.getOutputStreamStatistics().bytesToUpload);
+
+      //Test for relation between bytesUploadSuccessful, bytesUploadFailed
+      // and bytesToUpload
+      assertValues("bytesUploadSuccessful equal to difference between "
+              + "bytesToUpload and bytesUploadFailed",
+          outForSomeBytes.getOutputStreamStatistics().bytesUploadSuccessful,
+          outForSomeBytes.getOutputStreamStatistics().bytesToUpload -
+              outForSomeBytes.getOutputStreamStatistics().bytesUploadFailed);
+
+    } finally {
+      if (outForSomeBytes != null) {
+        outForSomeBytes.close();
+      }
+    }
+
+    AbfsOutputStream outForLargeBytes = null;
+    try {
+      outForLargeBytes =
+          (AbfsOutputStream) abfss.createFile(uploadBytesFilePath,
+              statistics
+              , true, FsPermission.getDefault(),
+              FsPermission.getUMask(fs.getConf()));
+
+      int largeValue = 100000;
+      for (int i = 0; i < largeValue; i++) {
+        outForLargeBytes.write(testBytesToUpload.getBytes());
+      }
+      outForLargeBytes.flush();
+
+      //Test for large bytes to upload
+      assertValues("bytes to upload",
+          largeValue * (testBytesToUpload.getBytes().length),
+          outForLargeBytes.getOutputStreamStatistics().bytesToUpload);
+
+      //Test for relation between bytesUploadSuccessful, bytesUploadFailed
+      // and bytesToUpload
+      assertValues("bytesUploadSuccessful equal to difference between "
+              + "bytesToUpload and bytesUploadFailed",
+          outForSomeBytes.getOutputStreamStatistics().bytesUploadSuccessful,
+          outForSomeBytes.getOutputStreamStatistics().bytesToUpload -
+              outForSomeBytes.getOutputStreamStatistics().bytesUploadFailed);
+
+    } finally {
+      if (outForLargeBytes != null) {
+        outForLargeBytes.close();
+      }
+    }
+
+  }
+
+  /**
+   * Tests to check time spend on waiting for tasks to be complete on a
+   * blocking queue in {@link AbfsOutputStream}.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamTimeSpendOnWaitTask() throws IOException {
+    describe("Testing Time Spend on Waiting for Task to be complete");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path timeSpendFilePath = new Path("AbfsOutputStreamStatsPath");
+    AzureBlobFileSystemStore abfss = fs.getAbfsStore();
+    FileSystem.Statistics statistics = fs.getFsStatistics();
+
+    AbfsOutputStream out =
+        (AbfsOutputStream) abfss.createFile(timeSpendFilePath,
+            statistics, true,
+            FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
+
+  }
+
+  /**
+   * Tests to check number of {@codes shrinkWriteOperationQueue()}
+   * calls.
+   * After writing data, AbfsOutputStream doesn't upload the data until
+   * Flushed. Hence, flush() method is called after write() to test Queue
+   * shrink calls.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testAbfsOutputStreamQueueShrink() throws IOException {
+    describe("Testing Queue Shrink calls in AbfsOutputStream");
+    final AzureBlobFileSystem fs = getFileSystem();
+    Path queueShrinkFilePath = new Path("AbfsOutputStreamStatsPath");
+    AzureBlobFileSystemStore abfss = fs.getAbfsStore();
+    abfss.getAbfsConfiguration().setDisableOutputStreamFlush(false);
+    FileSystem.Statistics statistics = fs.getFsStatistics();
+    String testQueueShrink = "testQueue";
+
+    AbfsOutputStream outForOneOp = null;
+
+    try {
+      outForOneOp =
+          (AbfsOutputStream) abfss.createFile(queueShrinkFilePath, statistics,
+              true,
+              FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
+
+      //Test for shrinking Queue zero time
+      assertValues("number of queueShrink() Calls", 0,
+          outForOneOp.getOutputStreamStatistics().queueShrink);
+
+      outForOneOp.write(testQueueShrink.getBytes());
+      // Queue is shrunk 2 times when outStream is flushed
+      outForOneOp.flush();
+
+      //Test for shrinking Queue 2 times
+      assertValues("number of queueShrink() Calls", 2,
+          outForOneOp.getOutputStreamStatistics().queueShrink);
+
+    } finally {
+      if (outForOneOp != null) {
+        outForOneOp.close();
+      }
+    }
+
+    AbfsOutputStream outForLargeOps = null;
+
+    try {
+      outForLargeOps = (AbfsOutputStream) abfss.createFile(queueShrinkFilePath,
+          statistics, true,
+          FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()));
+
+      int largeValue = 1000;
+      //QueueShrink is called 2 times in 1 flush(), hence 1000 flushes must
+      // give 2000 QueueShrink calls
+      for (int i = 0; i < largeValue; i++) {
+        outForLargeOps.write(testQueueShrink.getBytes());
+        //Flush is quite expensive so 1000 calls only which takes 1 min+
+        outForLargeOps.flush();
+      }
+
+      //Test for 2000 queue shrink calls
+      assertValues("number of queueShrink() Calls",
+          2 * largeValue,
+          outForLargeOps.getOutputStreamStatistics().queueShrink);
+    } finally {
+      if (outForLargeOps != null) {
+        outForLargeOps.close();
+      }
+    }
+
+  }
+
+  /**
+   * Test to check number of {@codes writeCurrentBufferToService()}
 
 Review comment:
   @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] mehakmeet commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r408805468
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
 ##########
 @@ -384,6 +400,7 @@ private synchronized void flushWrittenBytesToServiceInternal(final long offset,
    * operation FIFO queue.
    */
   private synchronized void shrinkWriteOperationQueue() throws IOException {
+    outputStreamStatistics.queueShrunk();
 
 Review comment:
   actually, queue is not shrunk here. Rather it is inside the while loop. will change this.

----------------------------------------------------------------
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 #1899: HADOOP-16914 Adding Output Stream Counters in ABFS

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #1899: HADOOP-16914 Adding Output Stream Counters in ABFS
URL: https://github.com/apache/hadoop/pull/1899#discussion_r408154819
 
 

 ##########
 File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamStatistics.java
 ##########
 @@ -0,0 +1,79 @@
+/**
+ * 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.services;
+
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Interface for {@link AbfsOutputStream} statistics.
+ */
+@InterfaceStability.Unstable
+public interface AbfsOutputStreamStatistics {
+
+  /**
+   * Number of bytes to be uploaded.
+   *
+   * @param bytes number of bytes to upload.
+   */
+  void bytesToUpload(long bytes);
+
+  /**
+   * Records a successful upload and the number of bytes uploaded.
+   *
+   * @param bytes number of bytes that were successfully uploaded.
+   */
+  void uploadSuccessful(long bytes);
+
+  /**
+   * Records that upload is failed and the number of bytes.
+   *
+   * @param bytes number of bytes that failed to upload.
 
 Review comment:
   typo.. Remove first word bytes.

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