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/04/27 20:35:22 UTC

[GitHub] [hadoop] steveloughran opened a new pull request #1982: HADOOP-16830. IOStatistics API.

steveloughran opened a new pull request #1982:
URL: https://github.com/apache/hadoop/pull/1982


   
   Contributed by Steve Loughran.
   
   This patch adds to hadoop-common an API For querying IO classes (Especially
   input and output streams) for statistics.
   
   It includes a big rework of the S3A Statistics including
   
   * implementation of the IOStatistics APIs
   * and contract tests for those and any other streams which implement the same interfaces and	the same bytes read/written counters.
   * A split  of the existing S3AInstrumentation classes into interface/implementations.
   * Troubled attempt to wire up the AWSSDK metrics
   
   The AWS metric binding is breaking some of the S3 region handling code, so is
   disabled, we're still using the old "create client then set endpoint" logic rather
   than the builder API for constructing the S3 client. 
   
   Doing the public interface hand-in-hand with that implementation helps evolve
   the interface, but it makes for a bigger patch.
   
   There are contract tests for those and any other streams which implement
   the same interfaces and	the same bytes read/written counters.
   
   Proposed: once the reviewers are happy with the design we can split the two up
   into the hadoop-common changes (which can be used in ABFS) and the S3A FS
   changes.
   
   Writing up the package info class makes me think it is a bit overcomplicated
   right now and that maybe we should go for "dynamic always" statistics. If you
   want a snapshot, it can be wrapped with IOStatisticsSupport.takeSnapshot().
   This will simplify the client code and remove ambiguity in implementations as
   to what they should be doing. We either provide callbacks to evaluate values or
   references to AtomicLongs/AtomicIntegers which are probed on demand.
   


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



---------------------------------------------------------------------
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 #1982: HADOOP-16830. IOStatistics API.

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1982:
URL: https://github.com/apache/hadoop/pull/1982#discussion_r420812203



##########
File path: hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractStreamIOStatisticsTest.java
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.contract;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
+import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.extractStatistics;
+import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticValue;
+import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_BYTES;
+import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_WRITE_BYTES;
+
+/**
+ * Tests {@link IOStatistics} support in input streams.
+ * Requires both the input and output streams to offer statistics.
+ */
+public abstract class AbstractContractStreamIOStatisticsTest
+    extends AbstractFSContractTestBase {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(AbstractContractStreamIOStatisticsTest.class);
+
+  @Test
+  public void testOutputStreamStatisticKeys() throws Throwable {
+    describe("Look at the statistic keys of an output stream");
+    Path path = methodPath();
+    FileSystem fs = getFileSystem();
+    fs.mkdirs(path.getParent());
+    try (FSDataOutputStream out = fs.create(path, true)) {
+      IOStatistics statistics = extractStatistics(out);
+      final List<String> keys = outputStreamStatisticKeys();
+      Assertions.assertThat(statistics.keys())
+          .describedAs("statistic keys of %s", statistics)
+          .containsAll(keys);
+      Assertions.assertThat(keys)
+          .describedAs("Statistics supported by the stream %s", out)
+          .contains(STREAM_WRITE_BYTES);
+    } finally {
+      fs.delete(path, false);
+    }
+  }
+
+  @Test
+  public void testWriteSingleByte() throws Throwable {
+    describe("Write a byte to a file and verify"
+        + " the stream statistics are updated");
+    Path path = methodPath();
+    FileSystem fs = getFileSystem();
+    fs.mkdirs(path.getParent());
+    try (FSDataOutputStream out = fs.create(path, true)) {
+      IOStatistics statistics = extractStatistics(out);
+      // before a write, no bytes
+      verifyStatisticValue(statistics, STREAM_WRITE_BYTES, 0);
+      out.write('0');
+      verifyStatisticValue(statistics, STREAM_WRITE_BYTES, 1);
+      // close the stream
+      out.close();
+      // statistics are still valid after the close
+      // always call the output stream to check that behavior
+      statistics = extractStatistics(out);
+      final String strVal = statistics.toString();
+      LOG.info("Statistics = {}", strVal);
+      verifyStatisticValue(statistics, STREAM_WRITE_BYTES, 1);
+    } finally {
+      fs.delete(path, false);

Review comment:
       just being thorough




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



---------------------------------------------------------------------
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 pull request #1982: HADOOP-16830. IOStatistics API.

Posted by GitBox <gi...@apache.org>.
steveloughran commented on pull request #1982:
URL: https://github.com/apache/hadoop/pull/1982#issuecomment-642142997


   aah, i need to rebase. @mehakmeet -this is going to complicate your life. sorry. I'll do a whole new PR.
   
   having written the new extensible design, I've decided I don't like it. It is too complex as I'm trying to support arbitrary arity tuples of any kind of statistic.it makes iterating/parsing this stuff way too complext
   
   here's a better idea: we only support a limited set; 
   
   * counter: long
   * min; long
   * max: long
   * mean (double, long)
   * gauge; long
   
   1. all but gauge have simple aggregation, for gauge i'll add stuff up too, on the assumption that they will be positive values (e.g 'number of active reads')
   1. and every set will have its own iterator.
   
   what do people think?


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



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


[GitHub] [hadoop] hadoop-yetus removed a comment on pull request #1982: HADOOP-16830. IOStatistics API.

Posted by GitBox <gi...@apache.org>.
hadoop-yetus removed a comment on pull request #1982:
URL: https://github.com/apache/hadoop/pull/1982#issuecomment-624880535


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |  26m 22s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  2s |  No case conflicting files found.  |
   | +0 :ok: |  markdownlint  |   0m  0s |  markdownlint was not available.  |
   | +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 22 new or modified test files.  |
   ||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 56s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  24m  7s |  trunk passed  |
   | +1 :green_heart: |  compile  |  20m 47s |  trunk passed  |
   | +1 :green_heart: |  checkstyle  |   3m 12s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   2m 23s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  22m 51s |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 38s |  trunk passed  |
   | +0 :ok: |  spotbugs  |   1m 18s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   3m 32s |  trunk passed  |
   | -0 :warning: |  patch  |   1m 37s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 27s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   1m 33s |  the patch passed  |
   | +1 :green_heart: |  compile  |  19m 53s |  the patch passed  |
   | -1 :x: |  javac  |  19m 53s |  root generated 1 new + 1870 unchanged - 1 fixed = 1871 total (was 1871)  |
   | -0 :warning: |  checkstyle  |   3m  7s |  root: The patch generated 66 new + 100 unchanged - 19 fixed = 166 total (was 119)  |
   | +1 :green_heart: |  mvnsite  |   2m 31s |  the patch passed  |
   | -1 :x: |  whitespace  |   0m  1s |  The patch has 7 line(s) that end in whitespace. Use git apply --whitespace=fix <<patch_file>>. Refer https://git-scm.com/docs/git-apply  |
   | +1 :green_heart: |  xml  |   0m  1s |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  shadedclient  |  16m 36s |  patch has no errors when building and testing our client artifacts.  |
   | -1 :x: |  javadoc  |   0m 40s |  hadoop-tools_hadoop-aws generated 1 new + 4 unchanged - 0 fixed = 5 total (was 4)  |
   | +1 :green_heart: |  findbugs  |   3m 59s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  |  10m 34s |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m  2s |  hadoop-aws in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 59s |  The patch does not generate ASF License warnings.  |
   |  |   | 167m 51s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.io.compress.snappy.TestSnappyCompressorDecompressor |
   |   | hadoop.metrics2.source.TestJvmMetrics |
   |   | hadoop.io.compress.TestCompressorDecompressor |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.40 ServerAPI=1.40 base: https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/4/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/1982 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle markdownlint xml |
   | uname | Linux 25486c8c3ba2 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 / 92e3ebb4019 |
   | Default Java | Private Build-1.8.0_252-8u252-b09-1~18.04-b09 |
   | javac | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/4/artifact/out/diff-compile-javac-root.txt |
   | checkstyle | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/4/artifact/out/diff-checkstyle-root.txt |
   | whitespace | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/4/artifact/out/whitespace-eol.txt |
   | javadoc | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/4/artifact/out/diff-javadoc-javadoc-hadoop-tools_hadoop-aws.txt |
   | unit | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/4/artifact/out/patch-unit-hadoop-common-project_hadoop-common.txt |
   |  Test Results | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/4/testReport/ |
   | Max. process+thread count | 2533 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common hadoop-tools/hadoop-aws U: . |
   | Console output | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/4/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=3.1.0-RC1 |
   | Powered by | Apache Yetus 0.12.0 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



---------------------------------------------------------------------
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 pull request #1982: HADOOP-16830. IOStatistics API.

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #1982:
URL: https://github.com/apache/hadoop/pull/1982#issuecomment-620272003


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 42s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  2s |  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 21 new or modified test files.  |
   ||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 26s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  19m 13s |  trunk passed  |
   | +1 :green_heart: |  compile  |  18m 23s |  trunk passed  |
   | +1 :green_heart: |  checkstyle  |   2m 37s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   2m 16s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  20m 19s |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 42s |  trunk passed  |
   | +0 :ok: |  spotbugs  |   1m 12s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   3m 24s |  trunk passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 27s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   1m 28s |  the patch passed  |
   | +1 :green_heart: |  compile  |  17m  6s |  the patch passed  |
   | -1 :x: |  javac  |  17m  6s |  root generated 1 new + 1870 unchanged - 1 fixed = 1871 total (was 1871)  |
   | -0 :warning: |  checkstyle  |   2m 46s |  root: The patch generated 69 new + 100 unchanged - 19 fixed = 169 total (was 119)  |
   | -1 :x: |  mvnsite  |   0m 36s |  hadoop-aws in the patch failed.  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  xml  |   0m  1s |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  shadedclient  |  14m 10s |  patch has no errors when building and testing our client artifacts.  |
   | -1 :x: |  javadoc  |   0m 57s |  hadoop-common-project_hadoop-common generated 1 new + 101 unchanged - 0 fixed = 102 total (was 101)  |
   | -1 :x: |  javadoc  |   0m 45s |  hadoop-tools_hadoop-aws generated 35 new + 4 unchanged - 0 fixed = 39 total (was 4)  |
   | -1 :x: |  findbugs  |   0m 32s |  hadoop-aws in the patch failed.  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  |   9m 13s |  hadoop-common in the patch passed.  |
   | -1 :x: |  unit  |   0m 36s |  hadoop-aws in the patch failed.  |
   | +1 :green_heart: |  asflicense  |   0m 49s |  The patch does not generate ASF License warnings.  |
   |  |   | 121m 37s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.fs.ftp.TestFTPFileSystem |
   |   | hadoop.metrics2.source.TestJvmMetrics |
   |   | hadoop.io.compress.snappy.TestSnappyCompressorDecompressor |
   |   | hadoop.io.compress.TestCompressorDecompressor |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.40 ServerAPI=1.40 base: https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/1/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/1982 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle xml |
   | uname | Linux 9ec374c75c78 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 / e2322e1117a |
   | Default Java | Private Build-1.8.0_252-8u252-b09-1~18.04-b09 |
   | javac | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/1/artifact/out/diff-compile-javac-root.txt |
   | checkstyle | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/1/artifact/out/diff-checkstyle-root.txt |
   | mvnsite | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/1/artifact/out/patch-mvnsite-hadoop-tools_hadoop-aws.txt |
   | javadoc | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/1/artifact/out/diff-javadoc-javadoc-hadoop-common-project_hadoop-common.txt |
   | javadoc | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/1/artifact/out/diff-javadoc-javadoc-hadoop-tools_hadoop-aws.txt |
   | findbugs | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/1/artifact/out/patch-findbugs-hadoop-tools_hadoop-aws.txt |
   | unit | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/1/artifact/out/patch-unit-hadoop-common-project_hadoop-common.txt |
   | unit | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/1/artifact/out/patch-unit-hadoop-tools_hadoop-aws.txt |
   |  Test Results | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/1/testReport/ |
   | Max. process+thread count | 1605 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common hadoop-tools/hadoop-aws U: . |
   | Console output | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/1/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=3.1.0-RC1 |
   | Powered by | Apache Yetus 0.12.0 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



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


[GitHub] [hadoop] steveloughran closed pull request #1982: HADOOP-16830. IOStatistics API.

Posted by GitBox <gi...@apache.org>.
steveloughran closed pull request #1982:
URL: https://github.com/apache/hadoop/pull/1982


   


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



---------------------------------------------------------------------
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 pull request #1982: HADOOP-16830. IOStatistics API.

Posted by GitBox <gi...@apache.org>.
steveloughran commented on pull request #1982:
URL: https://github.com/apache/hadoop/pull/1982#issuecomment-625240548


   failure on a test run, which is related to stats gathering. But I don't see this when run from the IDE. Race condition?
   
   ```
   [ERROR]   ITestCommitOperations.testBulkCommitFiles:650->Assert.assertEquals:645->Assert.failNotEquals:834->Assert.fail:88 Number of records written after commit #2; first commit had 4; first commit ancestors CommitContext{operationState=AncestorState{operation=Commitid=55; dest=s3a://stevel-london/fork-0006/test/DELAY_LISTING_ME/testBulkCommitFiles/out; size=6; paths={s3a://stevel-london/fork-0006/test/DELAY_LISTING_ME/testBulkCommitFiles/out/file1 s3a://stevel-london/fork-0006 s3a://stevel-london/fork-0006/test/DELAY_LISTING_ME/testBulkCommitFiles s3a://stevel-london/fork-0006/test/DELAY_LISTING_ME/testBulkCommitFiles/out s3a://stevel-london/fork-0006/test s3a://stevel-london/fork-0006/test/DELAY_LISTING_ME}}}; second commit ancestors: CommitContext{operationState=AncestorState{operation=Commitid=55; dest=s3a://stevel-london/fork-0006/test/DELAY_LISTING_ME/testBulkCommitFiles/out; size=8; paths={s3a://stevel-london/fork-0006/test/DELAY_LISTING_ME/testBulkCommitFiles/out/file1 s3a://stevel-london/fork-0006/test/DELAY_LISTING_ME/testBulkCommitFiles/out/subdir/file2 s3a://stevel-london/fork-0006 s3a://stevel-london/fork-0006/test/DELAY_LISTING_ME/testBulkCommitFiles s3a://stevel-london/fork-0006/test/DELAY_LISTING_ME/testBulkCommitFiles/out s3a://stevel-london/fork-0006/test s3a://stevel-london/fork-0006/test/DELAY_LISTING_ME/testBulkCommitFiles/out/subdir s3a://stevel-london/fork-0006/test/DELAY_LISTING_ME}}}: s3guard_metadatastore_record_writes expected:<2> but was:<3>
   [ERROR]   ITestS3ACommitterMRJob.test_200_execute:304->Assert.fail:88 Job job_1588787902323_0003 failed in state FAILED with cause Job commit failed: java.util.concurrent.RejectedExecutionException: Task java.util.concurrent.FutureTask@2104a338 rejected from org.apache.hadoop.util.concurrent.HadoopThreadPoolExecutor@7272786a[Terminated, pool size = 0, active threads = 0, queued tasks = 0, completed tasks = 10]
   	at java.util.concurrent.ThreadPoolExecutor$AbortPolicy.rejectedExecution(ThreadPoolExecutor.java:2063)
   	at java.util.concurrent.ThreadPoolExecutor.reject(ThreadPoolExecutor.java:830)
   	at java.util.concurrent.ThreadPoolExecutor.execute(ThreadPoolExecutor.java:1379)
   	at java.util.concurrent.AbstractExecutorService.submit(AbstractExecutorService.java:112)
   	at org.apache.hadoop.fs.s3a.commit.Tasks$Builder.runParallel(Tasks.java:313)
   	at org.apache.hadoop.fs.s3a.commit.Tasks$Builder.run(Tasks.java:148)
   	at org.apache.hadoop.fs.s3a.commit.AbstractS3ACommitter.commitPendingUploads(AbstractS3ACommitter.java:480)
   	at org.apache.hadoop.fs.s3a.commit.AbstractS3ACommitter.commitJobInternal(AbstractS3ACommitter.java:620)
   	at org.apache.hadoop.fs.s3a.commit.AbstractS3ACommitter.commitJob(AbstractS3ACommitter.java:722)
   	at org.apache.hadoop.mapreduce.v2.app.commit.CommitterEventHandler$EventProcessor.handleJobCommit(CommitterEventHandler.java:286)
   	at org.apache.hadoop.mapreduce.v2.app.commit.CommitterEventHandler$EventProcessor.run(CommitterEventHandler.java:238)
   	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
   	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
   	at java.lang.Thread.run(Thread.java:748)
   ```
   


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



---------------------------------------------------------------------
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 pull request #1982: HADOOP-16830. IOStatistics API.

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #1982:
URL: https://github.com/apache/hadoop/pull/1982#issuecomment-621412721


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 40s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  2s |  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 22 new or modified test files.  |
   ||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 57s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  19m 17s |  trunk passed  |
   | +1 :green_heart: |  compile  |  17m 19s |  trunk passed  |
   | +1 :green_heart: |  checkstyle  |   2m 42s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   2m 26s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  20m 16s |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 47s |  trunk passed  |
   | +0 :ok: |  spotbugs  |   1m 15s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   3m 22s |  trunk passed  |
   | -0 :warning: |  patch  |   1m 37s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 26s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   1m 24s |  the patch passed  |
   | +1 :green_heart: |  compile  |  16m 47s |  the patch passed  |
   | -1 :x: |  javac  |  16m 47s |  root generated 1 new + 1870 unchanged - 1 fixed = 1871 total (was 1871)  |
   | -0 :warning: |  checkstyle  |   2m 48s |  root: The patch generated 67 new + 100 unchanged - 19 fixed = 167 total (was 119)  |
   | +1 :green_heart: |  mvnsite  |   2m 24s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  xml  |   0m  2s |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  shadedclient  |  14m  3s |  patch has no errors when building and testing our client artifacts.  |
   | -1 :x: |  javadoc  |   0m 43s |  hadoop-tools_hadoop-aws generated 1 new + 4 unchanged - 0 fixed = 5 total (was 4)  |
   | +1 :green_heart: |  findbugs  |   3m 40s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  |   9m 22s |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 40s |  hadoop-aws in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 56s |  The patch does not generate ASF License warnings.  |
   |  |   | 123m 38s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.fs.ftp.TestFTPFileSystem |
   |   | hadoop.metrics2.source.TestJvmMetrics |
   |   | hadoop.io.compress.snappy.TestSnappyCompressorDecompressor |
   |   | hadoop.io.compress.TestCompressorDecompressor |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.40 ServerAPI=1.40 base: https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/3/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/1982 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle xml |
   | uname | Linux 32ae6d51ba7b 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 / 9ca6298a9ac |
   | Default Java | Private Build-1.8.0_252-8u252-b09-1~18.04-b09 |
   | javac | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/3/artifact/out/diff-compile-javac-root.txt |
   | checkstyle | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/3/artifact/out/diff-checkstyle-root.txt |
   | javadoc | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/3/artifact/out/diff-javadoc-javadoc-hadoop-tools_hadoop-aws.txt |
   | unit | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/3/artifact/out/patch-unit-hadoop-common-project_hadoop-common.txt |
   |  Test Results | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/3/testReport/ |
   | Max. process+thread count | 1605 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common hadoop-tools/hadoop-aws U: . |
   | Console output | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/3/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=3.1.0-RC1 |
   | Powered by | Apache Yetus 0.12.0 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



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


[GitHub] [hadoop] hadoop-yetus removed a comment on pull request #1982: HADOOP-16830. IOStatistics API.

Posted by GitBox <gi...@apache.org>.
hadoop-yetus removed a comment on pull request #1982:
URL: https://github.com/apache/hadoop/pull/1982#issuecomment-621412721


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 40s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  2s |  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 22 new or modified test files.  |
   ||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 57s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  19m 17s |  trunk passed  |
   | +1 :green_heart: |  compile  |  17m 19s |  trunk passed  |
   | +1 :green_heart: |  checkstyle  |   2m 42s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   2m 26s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  20m 16s |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 47s |  trunk passed  |
   | +0 :ok: |  spotbugs  |   1m 15s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   3m 22s |  trunk passed  |
   | -0 :warning: |  patch  |   1m 37s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 26s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   1m 24s |  the patch passed  |
   | +1 :green_heart: |  compile  |  16m 47s |  the patch passed  |
   | -1 :x: |  javac  |  16m 47s |  root generated 1 new + 1870 unchanged - 1 fixed = 1871 total (was 1871)  |
   | -0 :warning: |  checkstyle  |   2m 48s |  root: The patch generated 67 new + 100 unchanged - 19 fixed = 167 total (was 119)  |
   | +1 :green_heart: |  mvnsite  |   2m 24s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  xml  |   0m  2s |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  shadedclient  |  14m  3s |  patch has no errors when building and testing our client artifacts.  |
   | -1 :x: |  javadoc  |   0m 43s |  hadoop-tools_hadoop-aws generated 1 new + 4 unchanged - 0 fixed = 5 total (was 4)  |
   | +1 :green_heart: |  findbugs  |   3m 40s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  |   9m 22s |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 40s |  hadoop-aws in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 56s |  The patch does not generate ASF License warnings.  |
   |  |   | 123m 38s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.fs.ftp.TestFTPFileSystem |
   |   | hadoop.metrics2.source.TestJvmMetrics |
   |   | hadoop.io.compress.snappy.TestSnappyCompressorDecompressor |
   |   | hadoop.io.compress.TestCompressorDecompressor |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.40 ServerAPI=1.40 base: https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/3/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/1982 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle xml |
   | uname | Linux 32ae6d51ba7b 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 / 9ca6298a9ac |
   | Default Java | Private Build-1.8.0_252-8u252-b09-1~18.04-b09 |
   | javac | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/3/artifact/out/diff-compile-javac-root.txt |
   | checkstyle | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/3/artifact/out/diff-checkstyle-root.txt |
   | javadoc | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/3/artifact/out/diff-javadoc-javadoc-hadoop-tools_hadoop-aws.txt |
   | unit | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/3/artifact/out/patch-unit-hadoop-common-project_hadoop-common.txt |
   |  Test Results | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/3/testReport/ |
   | Max. process+thread count | 1605 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common hadoop-tools/hadoop-aws U: . |
   | Console output | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/3/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=3.1.0-RC1 |
   | Powered by | Apache Yetus 0.12.0 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



---------------------------------------------------------------------
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 pull request #1982: HADOOP-16830. IOStatistics API.

Posted by GitBox <gi...@apache.org>.
steveloughran commented on pull request #1982:
URL: https://github.com/apache/hadoop/pull/1982#issuecomment-625397876


   cannot repeat this was an eight thread parallel run, maybe there was some retry/timeout


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



---------------------------------------------------------------------
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 #1982: HADOOP-16830. IOStatistics API.

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1982:
URL: https://github.com/apache/hadoop/pull/1982#discussion_r419143786



##########
File path: hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractStreamIOStatisticsTest.java
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.contract;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
+import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.extractStatistics;
+import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticValue;
+import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_BYTES;
+import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_WRITE_BYTES;
+
+/**
+ * Tests {@link IOStatistics} support in input streams.
+ * Requires both the input and output streams to offer statistics.
+ */
+public abstract class AbstractContractStreamIOStatisticsTest
+    extends AbstractFSContractTestBase {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(AbstractContractStreamIOStatisticsTest.class);
+
+  @Test
+  public void testOutputStreamStatisticKeys() throws Throwable {
+    describe("Look at the statistic keys of an output stream");
+    Path path = methodPath();
+    FileSystem fs = getFileSystem();
+    fs.mkdirs(path.getParent());
+    try (FSDataOutputStream out = fs.create(path, true)) {
+      IOStatistics statistics = extractStatistics(out);
+      final List<String> keys = outputStreamStatisticKeys();
+      Assertions.assertThat(statistics.keys())
+          .describedAs("statistic keys of %s", statistics)
+          .containsAll(keys);
+      Assertions.assertThat(keys)
+          .describedAs("Statistics supported by the stream %s", out)
+          .contains(STREAM_WRITE_BYTES);
+    } finally {
+      fs.delete(path, false);
+    }
+  }
+
+  @Test
+  public void testWriteSingleByte() throws Throwable {
+    describe("Write a byte to a file and verify"
+        + " the stream statistics are updated");
+    Path path = methodPath();
+    FileSystem fs = getFileSystem();
+    fs.mkdirs(path.getParent());
+    try (FSDataOutputStream out = fs.create(path, true)) {
+      IOStatistics statistics = extractStatistics(out);
+      // before a write, no bytes
+      verifyStatisticValue(statistics, STREAM_WRITE_BYTES, 0);
+      out.write('0');
+      verifyStatisticValue(statistics, STREAM_WRITE_BYTES, 1);
+      // close the stream
+      out.close();
+      // statistics are still valid after the close
+      // always call the output stream to check that behavior
+      statistics = extractStatistics(out);
+      final String strVal = statistics.toString();
+      LOG.info("Statistics = {}", strVal);
+      verifyStatisticValue(statistics, STREAM_WRITE_BYTES, 1);
+    } finally {
+      fs.delete(path, false);

Review comment:
       Is fs.delete() required in these tests? Won't teardown() take care of 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



---------------------------------------------------------------------
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 pull request #1982: HADOOP-16830. IOStatistics API.

Posted by GitBox <gi...@apache.org>.
steveloughran commented on pull request #1982:
URL: https://github.com/apache/hadoop/pull/1982#issuecomment-641182474


   checkstyle. I intend to ignore those about _1, _2 and _3 methods as they match scala's; I plan to soon add tuple/triple classes with these to hadoop utils


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



---------------------------------------------------------------------
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 #1982: HADOOP-16830. IOStatistics API.

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1982:
URL: https://github.com/apache/hadoop/pull/1982#discussion_r419139789



##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
##########
@@ -634,46 +655,56 @@ public void close() {
 
   /**
    * Statistics updated by an input stream during its actual operation.
-   * These counters not thread-safe and are for use in a single instance
-   * of a stream.
+   * These counters are marked as volatile so that IOStatistics on the stream

Review comment:
       I think this Javadoc isn't updated to the new change.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



---------------------------------------------------------------------
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 pull request #1982: HADOOP-16830. IOStatistics API.

Posted by GitBox <gi...@apache.org>.
steveloughran commented on pull request #1982:
URL: https://github.com/apache/hadoop/pull/1982#issuecomment-620220105


   Successor to #1820. 
   
   regarding AWS metric failures, when enabled code which goes near landat and common crawl buckets are failing, even when the default endpoint is being used
   
   ```
   
   org.apache.hadoop.fs.s3a.AWSRedirectException: HEAD on landsat-pds: com.amazonaws.services.s3.model.AmazonS3Exception: The bucket is in this region: us-west-2. Please use this region to retry the request (Service: Amazon S3; Status Code: 301; Error Code: 301 Moved Permanently; Request ID: A783303EE9485EA1; S3 Extended Request ID: AT6EVbOELJpaqbsFDDAgH8FRHBv4WGkP4Cssk6N9ANLYIYFbeVqQllVf0/dKCZgKSV6MrPHMTzI=), S3 Extended Request ID: AT6EVbOELJpaqbsFDDAgH8FRHBv4WGkP4Cssk6N9ANLYIYFbeVqQllVf0/dKCZgKSV6MrPHMTzI=:301 Moved Permanently: The bucket is in this region: us-west-2. Please use this region to retry the request (Service: Amazon S3; Status Code: 301; Error Code: 301 Moved Permanently; Request ID: A783303EE9485EA1; S3 Extended Request ID: AT6EVbOELJpaqbsFDDAgH8FRHBv4WGkP4Cssk6N9ANLYIYFbeVqQllVf0/dKCZgKSV6MrPHMTzI=)
   
   	at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:234)
   	at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:112)
   	at org.apache.hadoop.fs.s3a.auth.delegation.ITestSessionDelegationInFileystem.readLandsatMetadata(ITestSessionDelegationInFileystem.java:574)
   	at org.apache.hadoop.fs.s3a.auth.delegation.ITestSessionDelegationInFileystem.testDelegatedFileSystem(ITestSessionDelegationInFileystem.java:312)
   	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
   	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   	at java.lang.reflect.Method.invoke(Method.java:498)
   	at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
   	at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
   	at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
   	at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
   	at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
   	at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
   	at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55)
   	at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:298)
   	at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:292)
   	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
   	at java.lang.Thread.run(Thread.java:748)
   Caused by: com.amazonaws.services.s3.model.AmazonS3Exception: The bucket is in this region: us-west-2. Please use this region to retry the request (Service: Amazon S3; Status Code: 301; Error Code: 301 Moved Permanently; Request ID: A783303EE9485EA1; S3 Extended Request ID: AT6EVbOELJpaqbsFDDAgH8FRHBv4WGkP4Cssk6N9ANLYIYFbeVqQllVf0/dKCZgKSV6MrPHMTzI=), S3 Extended Request ID: AT6EVbOELJpaqbsFDDAgH8FRHBv4WGkP4Cssk6N9ANLYIYFbeVqQllVf0/dKCZgKSV6MrPHMTzI=
   	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1712)
   	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1367)
   	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1113)
   	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:770)
   	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:744)
   	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:726)
   	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:686)
   	at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:668)
   	at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:532)
   	at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:512)
   	at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4920)
   	at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4866)
   	at com.amazonaws.services.s3.AmazonS3Client.getObjectMetadata(AmazonS3Client.java:1320)
   	at com.amazonaws.services.s3.AmazonS3Client.getObjectMetadata(AmazonS3Client.java:1294)
   	at org.apache.hadoop.fs.s3a.auth.delegation.ITestSessionDelegationInFileystem.lambda$readLandsatMetadata$2(ITestSessionDelegationInFileystem.java:575)
   	at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:110)
   	... 17 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



---------------------------------------------------------------------
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 pull request #1982: HADOOP-16830. IOStatistics API.

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #1982:
URL: https://github.com/apache/hadoop/pull/1982#issuecomment-632829256


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 44s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files found.  |
   | +0 :ok: |  markdownlint  |   0m  0s |  markdownlint was not available.  |
   | +1 :green_heart: |  @author  |   0m  1s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 23 new or modified test files.  |
   ||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |   2m  0s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  20m 23s |  trunk passed  |
   | +1 :green_heart: |  compile  |  20m 15s |  trunk passed  |
   | +1 :green_heart: |  checkstyle  |   3m 40s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   2m 38s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  25m  6s |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 44s |  trunk passed  |
   | +0 :ok: |  spotbugs  |   1m 18s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   3m 47s |  trunk passed  |
   | -0 :warning: |  patch  |   1m 42s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 28s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   1m 49s |  the patch passed  |
   | +1 :green_heart: |  compile  |  22m 28s |  the patch passed  |
   | -1 :x: |  javac  |  22m 28s |  root generated 1 new + 1862 unchanged - 1 fixed = 1863 total (was 1863)  |
   | -0 :warning: |  checkstyle  |   3m 39s |  root: The patch generated 30 new + 160 unchanged - 22 fixed = 190 total (was 182)  |
   | +1 :green_heart: |  mvnsite  |   2m 31s |  the patch passed  |
   | -1 :x: |  whitespace  |   0m  0s |  The patch has 8 line(s) that end in whitespace. Use git apply --whitespace=fix <<patch_file>>. Refer https://git-scm.com/docs/git-apply  |
   | +1 :green_heart: |  xml  |   0m  1s |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  shadedclient  |  17m 52s |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 42s |  the patch passed  |
   | +1 :green_heart: |  findbugs  |   4m 13s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |  10m 47s |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 30s |  hadoop-aws in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 52s |  The patch does not generate ASF License warnings.  |
   |  |   | 146m 20s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.40 ServerAPI=1.40 base: https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/9/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/1982 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle markdownlint xml |
   | uname | Linux 84e4bea4eded 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 / 96853146337 |
   | Default Java | Private Build-1.8.0_252-8u252-b09-1~18.04-b09 |
   | javac | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/9/artifact/out/diff-compile-javac-root.txt |
   | checkstyle | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/9/artifact/out/diff-checkstyle-root.txt |
   | whitespace | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/9/artifact/out/whitespace-eol.txt |
   |  Test Results | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/9/testReport/ |
   | Max. process+thread count | 1547 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common hadoop-tools/hadoop-aws U: . |
   | Console output | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/9/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=3.1.0-RC1 |
   | Powered by | Apache Yetus 0.12.0 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



---------------------------------------------------------------------
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 pull request #1982: HADOOP-16830. IOStatistics API.

Posted by GitBox <gi...@apache.org>.
steveloughran commented on pull request #1982:
URL: https://github.com/apache/hadoop/pull/1982#issuecomment-624658465


   > How can I add a counter of type **long** through the builder?  
   
   it takes any lambda expression, so you can go add(key, (key) ->this.getLongval()). 
   
   Having some method add(key, long) wouldn't work, as that's not an evaluator, only a static value.
   


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



---------------------------------------------------------------------
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 pull request #1982: HADOOP-16830. IOStatistics API.

Posted by GitBox <gi...@apache.org>.
steveloughran commented on pull request #1982:
URL: https://github.com/apache/hadoop/pull/1982#issuecomment-630284411


   This patch adds LocalFS/RawLocalFS/ChecksummedFileSystem statistics passthrough and collection.
   
   Getting everything passed through is actually the harder part of the process...
   I ended up having to debug things just work out what is going on there.
   
   The fact that the raw local streams are buffered complicates testing.
   The write tests expect the counter to not update until the stream
   has closed; I need to expand the read tests this way too.
   
   Although it makes for a bigger patch, it means that we get unit tests in
   hadoop-common and that passthrough is all correct. It will also permit
   applications to collect IO statistics on local storage operations.
   
   new interface/impl to make it easy to instrument a class;
   a map of key -> atomic long is built up as well as the stats mapping,
   all the stream needs to add is a varags list of counters
   
   ```
       private final CounterIOStatistics ioStatistics = counterIOStatistics(
           STREAM_READ_BYTES,
           STREAM_READ_EXCEPTIONS,
           STREAM_READ_SEEK_OPERATIONS,
           STREAM_READ_SKIP_OPERATIONS,
           STREAM_READ_SKIP_BYTES);
   ```
   
   which can then be set or incremented
   
   ```
             ioStatistics.increment(STREAM_READ_BYTES, 1);
             
   ```
   
   


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



---------------------------------------------------------------------
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 pull request #1982: HADOOP-16830. IOStatistics API.

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #1982:
URL: https://github.com/apache/hadoop/pull/1982#issuecomment-624880535


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |  26m 22s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  2s |  No case conflicting files found.  |
   | +0 :ok: |  markdownlint  |   0m  0s |  markdownlint was not available.  |
   | +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 22 new or modified test files.  |
   ||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 56s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  24m  7s |  trunk passed  |
   | +1 :green_heart: |  compile  |  20m 47s |  trunk passed  |
   | +1 :green_heart: |  checkstyle  |   3m 12s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   2m 23s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  22m 51s |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 38s |  trunk passed  |
   | +0 :ok: |  spotbugs  |   1m 18s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   3m 32s |  trunk passed  |
   | -0 :warning: |  patch  |   1m 37s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 27s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   1m 33s |  the patch passed  |
   | +1 :green_heart: |  compile  |  19m 53s |  the patch passed  |
   | -1 :x: |  javac  |  19m 53s |  root generated 1 new + 1870 unchanged - 1 fixed = 1871 total (was 1871)  |
   | -0 :warning: |  checkstyle  |   3m  7s |  root: The patch generated 66 new + 100 unchanged - 19 fixed = 166 total (was 119)  |
   | +1 :green_heart: |  mvnsite  |   2m 31s |  the patch passed  |
   | -1 :x: |  whitespace  |   0m  1s |  The patch has 7 line(s) that end in whitespace. Use git apply --whitespace=fix <<patch_file>>. Refer https://git-scm.com/docs/git-apply  |
   | +1 :green_heart: |  xml  |   0m  1s |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  shadedclient  |  16m 36s |  patch has no errors when building and testing our client artifacts.  |
   | -1 :x: |  javadoc  |   0m 40s |  hadoop-tools_hadoop-aws generated 1 new + 4 unchanged - 0 fixed = 5 total (was 4)  |
   | +1 :green_heart: |  findbugs  |   3m 59s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  |  10m 34s |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m  2s |  hadoop-aws in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 59s |  The patch does not generate ASF License warnings.  |
   |  |   | 167m 51s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.io.compress.snappy.TestSnappyCompressorDecompressor |
   |   | hadoop.metrics2.source.TestJvmMetrics |
   |   | hadoop.io.compress.TestCompressorDecompressor |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.40 ServerAPI=1.40 base: https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/4/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/1982 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle markdownlint xml |
   | uname | Linux 25486c8c3ba2 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 / 92e3ebb4019 |
   | Default Java | Private Build-1.8.0_252-8u252-b09-1~18.04-b09 |
   | javac | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/4/artifact/out/diff-compile-javac-root.txt |
   | checkstyle | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/4/artifact/out/diff-checkstyle-root.txt |
   | whitespace | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/4/artifact/out/whitespace-eol.txt |
   | javadoc | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/4/artifact/out/diff-javadoc-javadoc-hadoop-tools_hadoop-aws.txt |
   | unit | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/4/artifact/out/patch-unit-hadoop-common-project_hadoop-common.txt |
   |  Test Results | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/4/testReport/ |
   | Max. process+thread count | 2533 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common hadoop-tools/hadoop-aws U: . |
   | Console output | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/4/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=3.1.0-RC1 |
   | Powered by | Apache Yetus 0.12.0 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



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


[GitHub] [hadoop] hadoop-yetus removed a comment on pull request #1982: HADOOP-16830. IOStatistics API.

Posted by GitBox <gi...@apache.org>.
hadoop-yetus removed a comment on pull request #1982:
URL: https://github.com/apache/hadoop/pull/1982#issuecomment-620889592


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 24s |  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 22 new or modified test files.  |
   ||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for branch  |
   | -1 :x: |  mvninstall  |   0m 20s |  root in trunk failed.  |
   | -1 :x: |  compile  |   0m 23s |  root in trunk failed.  |
   | -0 :warning: |  checkstyle  |   0m 22s |  The patch fails to run checkstyle in root  |
   | -1 :x: |  mvnsite  |   0m 23s |  hadoop-common in trunk failed.  |
   | -1 :x: |  mvnsite  |   0m 22s |  hadoop-aws in trunk failed.  |
   | -1 :x: |  shadedclient  |   6m 25s |  branch has errors when building and testing our client artifacts.  |
   | -1 :x: |  javadoc  |   0m 24s |  hadoop-aws in trunk failed.  |
   | +0 :ok: |  spotbugs  |   8m 15s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | -1 :x: |  findbugs  |   0m 24s |  hadoop-common in trunk failed.  |
   | -1 :x: |  findbugs  |   0m 11s |  hadoop-aws in trunk failed.  |
   | -0 :warning: |  patch  |   8m 40s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 42s |  Maven dependency ordering for patch  |
   | -1 :x: |  mvninstall  |   0m 19s |  hadoop-aws in the patch failed.  |
   | -1 :x: |  compile  |   0m 23s |  root in the patch failed.  |
   | -1 :x: |  javac  |   0m 23s |  root in the patch failed.  |
   | -0 :warning: |  checkstyle  |   3m 12s |  root: The patch generated 167 new + 0 unchanged - 0 fixed = 167 total (was 0)  |
   | -1 :x: |  mvnsite  |   0m 23s |  hadoop-common in the patch failed.  |
   | -1 :x: |  mvnsite  |   0m 43s |  hadoop-aws in the patch failed.  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  xml  |   0m  2s |  The patch has no ill-formed XML file.  |
   | -1 :x: |  shadedclient  |   4m 50s |  patch has errors when building and testing our client artifacts.  |
   | -1 :x: |  javadoc  |   0m 28s |  hadoop-tools_hadoop-aws generated 5 new + 0 unchanged - 0 fixed = 5 total (was 0)  |
   | +1 :green_heart: |  findbugs  |   3m 51s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  |  19m 26s |  hadoop-common in the patch passed.  |
   | -1 :x: |  unit  |   0m 37s |  hadoop-aws in the patch failed.  |
   | +0 :ok: |  asflicense  |   0m 26s |  ASF License check generated no output?  |
   |  |   |  51m 11s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.metrics2.source.TestJvmMetrics |
   |   | hadoop.http.TestHttpServer |
   |   | hadoop.fs.TestStat |
   |   | hadoop.fs.contract.localfs.TestLocalFSContractMultipartUploader |
   |   | hadoop.ha.TestZKFailoverControllerStress |
   |   | hadoop.crypto.TestCryptoStreams |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.40 ServerAPI=1.40 base: https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/2/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/1982 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle xml |
   | uname | Linux 68a5126a219a 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 / ab364295597 |
   | Default Java | Private Build-1.8.0_252-8u252-b09-1~18.04-b09 |
   | mvninstall | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/2/artifact/out/branch-mvninstall-root.txt |
   | compile | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/2/artifact/out/branch-compile-root.txt |
   | checkstyle | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/2/artifact/out/buildtool-branch-checkstyle-root.txt |
   | mvnsite | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/2/artifact/out/branch-mvnsite-hadoop-common-project_hadoop-common.txt |
   | mvnsite | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/2/artifact/out/branch-mvnsite-hadoop-tools_hadoop-aws.txt |
   | javadoc | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/2/artifact/out/branch-javadoc-hadoop-tools_hadoop-aws.txt |
   | findbugs | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/2/artifact/out/branch-findbugs-hadoop-common-project_hadoop-common.txt |
   | findbugs | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/2/artifact/out/branch-findbugs-hadoop-tools_hadoop-aws.txt |
   | mvninstall | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/2/artifact/out/patch-mvninstall-hadoop-tools_hadoop-aws.txt |
   | compile | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/2/artifact/out/patch-compile-root.txt |
   | javac | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/2/artifact/out/patch-compile-root.txt |
   | checkstyle | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/2/artifact/out/diff-checkstyle-root.txt |
   | mvnsite | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/2/artifact/out/patch-mvnsite-hadoop-common-project_hadoop-common.txt |
   | mvnsite | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/2/artifact/out/patch-mvnsite-hadoop-tools_hadoop-aws.txt |
   | javadoc | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/2/artifact/out/diff-javadoc-javadoc-hadoop-tools_hadoop-aws.txt |
   | unit | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/2/artifact/out/patch-unit-hadoop-common-project_hadoop-common.txt |
   | unit | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/2/artifact/out/patch-unit-hadoop-tools_hadoop-aws.txt |
   |  Test Results | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/2/testReport/ |
   | Max. process+thread count | 549 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common hadoop-tools/hadoop-aws U: . |
   | Console output | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/2/console |
   | versions | git=2.17.1 maven=3.6.0 |
   | Powered by | Apache Yetus 0.12.0 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



---------------------------------------------------------------------
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 #1982: HADOOP-16830. IOStatistics API.

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1982:
URL: https://github.com/apache/hadoop/pull/1982#discussion_r419144314



##########
File path: hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractStreamIOStatisticsTest.java
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.contract;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
+import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.extractStatistics;
+import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticValue;
+import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_BYTES;
+import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_WRITE_BYTES;
+
+/**
+ * Tests {@link IOStatistics} support in input streams.
+ * Requires both the input and output streams to offer statistics.
+ */
+public abstract class AbstractContractStreamIOStatisticsTest
+    extends AbstractFSContractTestBase {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(AbstractContractStreamIOStatisticsTest.class);
+
+  @Test
+  public void testOutputStreamStatisticKeys() throws Throwable {
+    describe("Look at the statistic keys of an output stream");
+    Path path = methodPath();
+    FileSystem fs = getFileSystem();
+    fs.mkdirs(path.getParent());
+    try (FSDataOutputStream out = fs.create(path, true)) {
+      IOStatistics statistics = extractStatistics(out);
+      final List<String> keys = outputStreamStatisticKeys();
+      Assertions.assertThat(statistics.keys())
+          .describedAs("statistic keys of %s", statistics)
+          .containsAll(keys);
+      Assertions.assertThat(keys)
+          .describedAs("Statistics supported by the stream %s", out)
+          .contains(STREAM_WRITE_BYTES);
+    } finally {
+      fs.delete(path, false);
+    }
+  }
+
+  @Test
+  public void testWriteSingleByte() throws Throwable {
+    describe("Write a byte to a file and verify"
+        + " the stream statistics are updated");
+    Path path = methodPath();
+    FileSystem fs = getFileSystem();
+    fs.mkdirs(path.getParent());
+    try (FSDataOutputStream out = fs.create(path, true)) {
+      IOStatistics statistics = extractStatistics(out);
+      // before a write, no bytes
+      verifyStatisticValue(statistics, STREAM_WRITE_BYTES, 0);
+      out.write('0');
+      verifyStatisticValue(statistics, STREAM_WRITE_BYTES, 1);
+      // close the stream
+      out.close();
+      // statistics are still valid after the close
+      // always call the output stream to check that behavior
+      statistics = extractStatistics(out);
+      final String strVal = statistics.toString();
+      LOG.info("Statistics = {}", strVal);
+      verifyStatisticValue(statistics, STREAM_WRITE_BYTES, 1);
+    } finally {
+      fs.delete(path, false);
+    }
+  }
+
+  @Test
+  public void testWriteByteArrays() throws Throwable {
+    describe("Write byte arrays to a file and verify"
+        + " the stream statistics are updated");
+    Path path = methodPath();
+    FileSystem fs = getFileSystem();
+    fs.mkdirs(path.getParent());
+    try (FSDataOutputStream out = fs.create(path, true)) {
+      // before a write, no bytes
+      final byte[] bytes = ContractTestUtils.toAsciiByteArray(
+          "statistically-speaking");
+      final int len = bytes.length;
+      out.write(bytes);
+      IOStatistics statistics = extractStatistics(out);
+      verifyStatisticValue(statistics, STREAM_WRITE_BYTES, len);
+      out.write(bytes);
+      verifyStatisticValue(statistics, STREAM_WRITE_BYTES, len * 2);
+      // close the stream
+      out.close();
+      // statistics are still valid after the close
+      // always call the output stream to check that behavior
+      statistics = extractStatistics(out);
+      verifyStatisticValue(statistics, STREAM_WRITE_BYTES, len * 2);
+    } finally {
+      fs.delete(path, false);
+    }
+  }
+
+  @Test
+  public void testInputStreamStatisticKeys() throws Throwable {
+    describe("Look at the statistic keys of an input stream");
+    Path path = methodPath();
+    FileSystem fs = getFileSystem();
+    ContractTestUtils.touch(fs, path);
+    try (FSDataInputStream in = fs.open(path)) {
+      IOStatistics statistics = extractStatistics(in);
+      final List<String> keys = inputStreamStatisticKeys();
+      Assertions.assertThat(statistics.keys())
+          .describedAs("statistic keys of %s", statistics)
+          .containsAll(keys);
+      Assertions.assertThat(keys)
+          .describedAs("Statistics supported by the stream %s", in)
+          .contains(STREAM_READ_BYTES);
+      verifyStatisticValue(statistics, STREAM_READ_BYTES, 0);
+    } finally {
+      fs.delete(path, false);
+    }
+  }
+
+  @Test
+  public void testInputStreamStatisticRead() throws Throwable {
+    describe("Read Data from an input stream");
+    Path path = methodPath();
+    FileSystem fs = getFileSystem();
+    final int fileLen = 1024;
+    final byte[] ds = dataset(fileLen, 'a', 26);
+    ContractTestUtils.writeDataset(fs, path, ds, fileLen, 8_000, true);
+
+    try (FSDataInputStream in = fs.open(path)) {
+      long current = 0;
+      IOStatistics statistics = extractStatistics(in);
+      verifyStatisticValue(statistics, STREAM_READ_BYTES, 0);
+      Assertions.assertThat(in.read()).isEqualTo('a');
+      current = verifyStatisticValue(statistics, STREAM_READ_BYTES, 1);
+      final int bufferLen = 128;
+      byte[] buf128 = new byte[bufferLen];
+      in.read(buf128);
+      current = verifyStatisticValue(statistics, STREAM_READ_BYTES, current +
+          +bufferLen);
+      in.readFully(buf128);
+      current = verifyStatisticValue(statistics, STREAM_READ_BYTES, current
+          + bufferLen);
+      in.readFully(0, buf128);
+      current = verifyStatisticValue(statistics, STREAM_READ_BYTES, current
+          + bufferLen);
+      // seek must not increment the read counter
+      in.seek(256);
+      verifyStatisticValue(statistics, STREAM_READ_BYTES, current);
+
+      // if a stream implements lazy-seek the seek operation
+      // may be postponed until the read
+      final int sublen = 32;
+      Assertions.assertThat(in.read(buf128, 0, sublen))
+          .isEqualTo(sublen);
+      current = verifyStatisticValue(statistics, STREAM_READ_BYTES,
+          current + sublen);
+
+      // perform some read operations near the end of the file such that
+      // the buffer will not be completely read.
+      final int pos = fileLen - sublen;
+      in.seek(pos);
+      Assertions.assertThat(in.read(buf128))
+          .describedAs("Read overlapping EOF")
+          .isEqualTo(sublen);
+      current = verifyStatisticValue(statistics, STREAM_READ_BYTES,
+          current + sublen);
+      Assertions.assertThat(in.read(pos, buf128, 0, bufferLen))
+          .describedAs("Read(buffer) overlapping EOF")
+          .isEqualTo(sublen);
+      current = verifyStatisticValue(statistics, STREAM_READ_BYTES,
+          current + sublen);
+    } finally {
+      fs.delete(path, false);
+    }
+  }
+
+  /**
+   * Keys which the output stream must support.
+   * @return a list of keys
+   */
+  public List<String> outputStreamStatisticKeys() {
+    return Collections.singletonList(STREAM_WRITE_BYTES);
+  }
+
+  /**
+   * Keys which the output stream must support.

Review comment:
       ~output~ input




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



---------------------------------------------------------------------
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 pull request #1982: HADOOP-16830. IOStatistics API.

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #1982:
URL: https://github.com/apache/hadoop/pull/1982#issuecomment-631616268


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 33s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  2s |  No case conflicting files found.  |
   | +0 :ok: |  markdownlint  |   0m  0s |  markdownlint was not available.  |
   | +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 23 new or modified test files.  |
   ||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 49s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  19m  8s |  trunk passed  |
   | +1 :green_heart: |  compile  |  17m  8s |  trunk passed  |
   | +1 :green_heart: |  checkstyle  |   2m 43s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   2m 19s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  19m 38s |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 48s |  trunk passed  |
   | +0 :ok: |  spotbugs  |   1m 13s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   3m 16s |  trunk passed  |
   | -0 :warning: |  patch  |   1m 34s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 27s |  Maven dependency ordering for patch  |
   | -1 :x: |  mvninstall  |   0m 27s |  hadoop-aws in the patch failed.  |
   | -1 :x: |  compile  |  15m 33s |  root in the patch failed.  |
   | -1 :x: |  javac  |  15m 33s |  root in the patch failed.  |
   | -0 :warning: |  checkstyle  |   2m 43s |  root: The patch generated 15 new + 160 unchanged - 22 fixed = 175 total (was 182)  |
   | -1 :x: |  mvnsite  |   0m 49s |  hadoop-aws in the patch failed.  |
   | -1 :x: |  whitespace  |   0m  0s |  The patch has 6 line(s) that end in whitespace. Use git apply --whitespace=fix <<patch_file>>. Refer https://git-scm.com/docs/git-apply  |
   | +1 :green_heart: |  xml  |   0m  1s |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  shadedclient  |  14m 12s |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 47s |  the patch passed  |
   | -1 :x: |  findbugs  |   0m 48s |  hadoop-aws in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   9m 17s |  hadoop-common in the patch passed.  |
   | -1 :x: |  unit  |   0m 47s |  hadoop-aws in the patch failed.  |
   | +1 :green_heart: |  asflicense  |   0m 54s |  The patch does not generate ASF License warnings.  |
   |  |   | 118m 58s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.40 ServerAPI=1.40 base: https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/7/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/1982 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle markdownlint xml |
   | uname | Linux e76e345154f8 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 / 29b19cd5924 |
   | Default Java | Private Build-1.8.0_252-8u252-b09-1~18.04-b09 |
   | mvninstall | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/7/artifact/out/patch-mvninstall-hadoop-tools_hadoop-aws.txt |
   | compile | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/7/artifact/out/patch-compile-root.txt |
   | javac | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/7/artifact/out/patch-compile-root.txt |
   | checkstyle | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/7/artifact/out/diff-checkstyle-root.txt |
   | mvnsite | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/7/artifact/out/patch-mvnsite-hadoop-tools_hadoop-aws.txt |
   | whitespace | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/7/artifact/out/whitespace-eol.txt |
   | findbugs | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/7/artifact/out/patch-findbugs-hadoop-tools_hadoop-aws.txt |
   | unit | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/7/artifact/out/patch-unit-hadoop-tools_hadoop-aws.txt |
   |  Test Results | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/7/testReport/ |
   | Max. process+thread count | 3417 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common hadoop-tools/hadoop-aws U: . |
   | Console output | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/7/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=3.1.0-RC1 |
   | Powered by | Apache Yetus 0.12.0 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



---------------------------------------------------------------------
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 pull request #1982: HADOOP-16830. IOStatistics API.

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #1982:
URL: https://github.com/apache/hadoop/pull/1982#issuecomment-641510854


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m  0s |  Docker mode activated.  |
   | -1 :x: |  patch  |   0m  6s |  https://github.com/apache/hadoop/pull/1982 does not apply to trunk. Rebase required? Wrong Branch? See https://wiki.apache.org/hadoop/HowToContribute for help.  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | GITHUB PR | https://github.com/apache/hadoop/pull/1982 |
   | Console output | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/11/console |
   | versions | git=2.17.1 |
   | Powered by | Apache Yetus 0.12.0 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



---------------------------------------------------------------------
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 #1982: HADOOP-16830. IOStatistics API.

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1982:
URL: https://github.com/apache/hadoop/pull/1982#discussion_r419143786



##########
File path: hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractStreamIOStatisticsTest.java
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.contract;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
+import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.extractStatistics;
+import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticValue;
+import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_BYTES;
+import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_WRITE_BYTES;
+
+/**
+ * Tests {@link IOStatistics} support in input streams.
+ * Requires both the input and output streams to offer statistics.
+ */
+public abstract class AbstractContractStreamIOStatisticsTest
+    extends AbstractFSContractTestBase {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(AbstractContractStreamIOStatisticsTest.class);
+
+  @Test
+  public void testOutputStreamStatisticKeys() throws Throwable {
+    describe("Look at the statistic keys of an output stream");
+    Path path = methodPath();
+    FileSystem fs = getFileSystem();
+    fs.mkdirs(path.getParent());
+    try (FSDataOutputStream out = fs.create(path, true)) {
+      IOStatistics statistics = extractStatistics(out);
+      final List<String> keys = outputStreamStatisticKeys();
+      Assertions.assertThat(statistics.keys())
+          .describedAs("statistic keys of %s", statistics)
+          .containsAll(keys);
+      Assertions.assertThat(keys)
+          .describedAs("Statistics supported by the stream %s", out)
+          .contains(STREAM_WRITE_BYTES);
+    } finally {
+      fs.delete(path, false);
+    }
+  }
+
+  @Test
+  public void testWriteSingleByte() throws Throwable {
+    describe("Write a byte to a file and verify"
+        + " the stream statistics are updated");
+    Path path = methodPath();
+    FileSystem fs = getFileSystem();
+    fs.mkdirs(path.getParent());
+    try (FSDataOutputStream out = fs.create(path, true)) {
+      IOStatistics statistics = extractStatistics(out);
+      // before a write, no bytes
+      verifyStatisticValue(statistics, STREAM_WRITE_BYTES, 0);
+      out.write('0');
+      verifyStatisticValue(statistics, STREAM_WRITE_BYTES, 1);
+      // close the stream
+      out.close();
+      // statistics are still valid after the close
+      // always call the output stream to check that behavior
+      statistics = extractStatistics(out);
+      final String strVal = statistics.toString();
+      LOG.info("Statistics = {}", strVal);
+      verifyStatisticValue(statistics, STREAM_WRITE_BYTES, 1);
+    } finally {
+      fs.delete(path, false);

Review comment:
       Is fs.delete() require in these tests? Won't teardown() take care of 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



---------------------------------------------------------------------
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 pull request #1982: HADOOP-16830. IOStatistics API.

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #1982:
URL: https://github.com/apache/hadoop/pull/1982#issuecomment-640902616


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 34s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files found.  |
   | +0 :ok: |  markdownlint  |   0m  1s |  markdownlint was not available.  |
   | +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 23 new or modified test files.  |
   ||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 21s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  21m 37s |  trunk passed  |
   | +1 :green_heart: |  compile  |  18m 18s |  trunk passed  |
   | +1 :green_heart: |  checkstyle  |   2m 58s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   2m  9s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  21m 23s |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 32s |  trunk passed  |
   | +0 :ok: |  spotbugs  |   1m  8s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   3m 18s |  trunk passed  |
   | -0 :warning: |  patch  |   1m 26s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 27s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   1m 34s |  the patch passed  |
   | +1 :green_heart: |  compile  |  18m  1s |  the patch passed  |
   | -1 :x: |  javac  |  18m  1s |  root generated 1 new + 1862 unchanged - 1 fixed = 1863 total (was 1863)  |
   | -0 :warning: |  checkstyle  |   2m 57s |  root: The patch generated 20 new + 160 unchanged - 22 fixed = 180 total (was 182)  |
   | +1 :green_heart: |  mvnsite  |   2m  7s |  the patch passed  |
   | -1 :x: |  whitespace  |   0m  0s |  The patch has 8 line(s) that end in whitespace. Use git apply --whitespace=fix <<patch_file>>. Refer https://git-scm.com/docs/git-apply  |
   | +1 :green_heart: |  xml  |   0m  1s |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  shadedclient  |  15m 40s |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 30s |  the patch passed  |
   | +1 :green_heart: |  findbugs  |   3m 33s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   9m 50s |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 26s |  hadoop-aws in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 45s |  The patch does not generate ASF License warnings.  |
   |  |   | 129m 50s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.40 ServerAPI=1.40 base: https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/10/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/1982 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle markdownlint xml |
   | uname | Linux 039c4101d198 4.15.0-101-generic #102-Ubuntu SMP Mon May 11 10:07:26 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | personality/hadoop.sh |
   | git revision | trunk / 0c25131ca43 |
   | Default Java | Private Build-1.8.0_252-8u252-b09-1~18.04-b09 |
   | javac | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/10/artifact/out/diff-compile-javac-root.txt |
   | checkstyle | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/10/artifact/out/diff-checkstyle-root.txt |
   | whitespace | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/10/artifact/out/whitespace-eol.txt |
   |  Test Results | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/10/testReport/ |
   | Max. process+thread count | 3242 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common hadoop-tools/hadoop-aws U: . |
   | Console output | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/10/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=3.1.0-RC1 |
   | Powered by | Apache Yetus 0.12.0 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



---------------------------------------------------------------------
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 #1982: HADOOP-16830. IOStatistics API.

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1982:
URL: https://github.com/apache/hadoop/pull/1982#discussion_r422987566



##########
File path: hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/DynamicIOStatisticsBuilder.java
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.statistics.impl;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.ToLongFunction;
+
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+
+import static com.google.common.base.Preconditions.checkState;
+
+/**
+ * Builder of Dynamic IO Statistics.
+ * Instantiate through
+ * {@link IOStatisticsBinding#dynamicIOStatistics()}.
+ */
+public class DynamicIOStatisticsBuilder {
+
+  /**
+   * the instance being built up. Will be null after the (single)
+   * call to {@link #build()}.
+   */
+  private DynamicIOStatistics instance = new DynamicIOStatistics();
+
+  /**
+   * Add a new evaluator to the statistics being built up.
+   * @param key key of this statistic
+   * @param eval evaluator for the statistic
+   * @return the builder.
+   */
+  public DynamicIOStatisticsBuilder add(String key,

Review comment:
       via a lambda expression




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



---------------------------------------------------------------------
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 pull request #1982: HADOOP-16830. IOStatistics API.

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #1982:
URL: https://github.com/apache/hadoop/pull/1982#issuecomment-625458870


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 40s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files found.  |
   | +0 :ok: |  markdownlint  |   0m  1s |  markdownlint was not available.  |
   | +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 22 new or modified test files.  |
   ||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 55s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  20m 44s |  trunk passed  |
   | +1 :green_heart: |  compile  |  18m 39s |  trunk passed  |
   | +1 :green_heart: |  checkstyle  |   2m 47s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   2m 19s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  19m 49s |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 50s |  trunk passed  |
   | +0 :ok: |  spotbugs  |   1m 15s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   3m 20s |  trunk passed  |
   | -0 :warning: |  patch  |   1m 36s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 26s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   1m 24s |  the patch passed  |
   | +1 :green_heart: |  compile  |  16m 37s |  the patch passed  |
   | -1 :x: |  javac  |  16m 37s |  root generated 1 new + 1870 unchanged - 1 fixed = 1871 total (was 1871)  |
   | -0 :warning: |  checkstyle  |   2m 49s |  root: The patch generated 10 new + 81 unchanged - 19 fixed = 91 total (was 100)  |
   | +1 :green_heart: |  mvnsite  |   2m 25s |  the patch passed  |
   | -1 :x: |  whitespace  |   0m  0s |  The patch has 6 line(s) that end in whitespace. Use git apply --whitespace=fix <<patch_file>>. Refer https://git-scm.com/docs/git-apply  |
   | +1 :green_heart: |  xml  |   0m  2s |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  shadedclient  |  13m 57s |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 47s |  the patch passed  |
   | +1 :green_heart: |  findbugs  |   3m 35s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  |   9m 15s |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 34s |  hadoop-aws in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 55s |  The patch does not generate ASF License warnings.  |
   |  |   | 125m 24s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.fs.ftp.TestFTPFileSystem |
   |   | hadoop.metrics2.source.TestJvmMetrics |
   |   | hadoop.io.compress.snappy.TestSnappyCompressorDecompressor |
   |   | hadoop.io.compress.TestCompressorDecompressor |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.40 ServerAPI=1.40 base: https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/5/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/1982 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle markdownlint xml |
   | uname | Linux e2723b83da03 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 / 192cad9ee24 |
   | Default Java | Private Build-1.8.0_252-8u252-b09-1~18.04-b09 |
   | javac | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/5/artifact/out/diff-compile-javac-root.txt |
   | checkstyle | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/5/artifact/out/diff-checkstyle-root.txt |
   | whitespace | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/5/artifact/out/whitespace-eol.txt |
   | unit | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/5/artifact/out/patch-unit-hadoop-common-project_hadoop-common.txt |
   |  Test Results | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/5/testReport/ |
   | Max. process+thread count | 1541 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common hadoop-tools/hadoop-aws U: . |
   | Console output | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/5/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=3.1.0-RC1 |
   | Powered by | Apache Yetus 0.12.0 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



---------------------------------------------------------------------
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 #1982: HADOOP-16830. IOStatistics API.

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #1982:
URL: https://github.com/apache/hadoop/pull/1982#discussion_r419138855



##########
File path: hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/DynamicIOStatisticsBuilder.java
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.statistics.impl;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.ToLongFunction;
+
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+
+import static com.google.common.base.Preconditions.checkState;
+
+/**
+ * Builder of Dynamic IO Statistics.
+ * Instantiate through
+ * {@link IOStatisticsBinding#dynamicIOStatistics()}.
+ */
+public class DynamicIOStatisticsBuilder {
+
+  /**
+   * the instance being built up. Will be null after the (single)
+   * call to {@link #build()}.
+   */
+  private DynamicIOStatistics instance = new DynamicIOStatistics();
+
+  /**
+   * Add a new evaluator to the statistics being built up.
+   * @param key key of this statistic
+   * @param eval evaluator for the statistic
+   * @return the builder.
+   */
+  public DynamicIOStatisticsBuilder add(String key,

Review comment:
       How can I add a counter of type **long** through the builder? 




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



---------------------------------------------------------------------
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 pull request #1982: HADOOP-16830. IOStatistics API.

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #1982:
URL: https://github.com/apache/hadoop/pull/1982#issuecomment-630963353


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |  22m 37s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  2s |  No case conflicting files found.  |
   | +0 :ok: |  markdownlint  |   0m  0s |  markdownlint was not available.  |
   | +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 23 new or modified test files.  |
   ||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 50s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  24m 32s |  trunk passed  |
   | +1 :green_heart: |  compile  |  21m 38s |  trunk passed  |
   | +1 :green_heart: |  checkstyle  |   3m 34s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   2m 43s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  24m 38s |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 47s |  trunk passed  |
   | +0 :ok: |  spotbugs  |   1m 20s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   4m  4s |  trunk passed  |
   | -0 :warning: |  patch  |   1m 40s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 28s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   1m 42s |  the patch passed  |
   | +1 :green_heart: |  compile  |  21m 25s |  the patch passed  |
   | -1 :x: |  javac  |  21m 24s |  root generated 1 new + 1862 unchanged - 1 fixed = 1863 total (was 1863)  |
   | -0 :warning: |  checkstyle  |   3m 10s |  root: The patch generated 15 new + 160 unchanged - 22 fixed = 175 total (was 182)  |
   | +1 :green_heart: |  mvnsite  |   2m 30s |  the patch passed  |
   | -1 :x: |  whitespace  |   0m  0s |  The patch has 6 line(s) that end in whitespace. Use git apply --whitespace=fix <<patch_file>>. Refer https://git-scm.com/docs/git-apply  |
   | +1 :green_heart: |  xml  |   0m  2s |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  shadedclient  |  17m 24s |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 47s |  the patch passed  |
   | +1 :green_heart: |  findbugs  |   4m 13s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  |   9m 57s |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 34s |  hadoop-aws in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 54s |  The patch does not generate ASF License warnings.  |
   |  |   | 169m 39s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.metrics2.source.TestJvmMetrics |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.40 ServerAPI=1.40 base: https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/6/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/1982 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle markdownlint xml |
   | uname | Linux e829422f0463 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 / d4e36409d40 |
   | Default Java | Private Build-1.8.0_252-8u252-b09-1~18.04-b09 |
   | javac | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/6/artifact/out/diff-compile-javac-root.txt |
   | checkstyle | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/6/artifact/out/diff-checkstyle-root.txt |
   | whitespace | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/6/artifact/out/whitespace-eol.txt |
   | unit | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/6/artifact/out/patch-unit-hadoop-common-project_hadoop-common.txt |
   |  Test Results | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/6/testReport/ |
   | Max. process+thread count | 1599 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common hadoop-tools/hadoop-aws U: . |
   | Console output | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/6/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=3.1.0-RC1 |
   | Powered by | Apache Yetus 0.12.0 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



---------------------------------------------------------------------
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 pull request #1982: HADOOP-16830. IOStatistics API.

Posted by GitBox <gi...@apache.org>.
steveloughran commented on pull request #1982:
URL: https://github.com/apache/hadoop/pull/1982#issuecomment-628616574


   add methods to add two iostatistics instances, and to subtract one from the other. This will help merging stats from multiple file read/writes, and for when we add the API to a long-life instance (e.g the s3a and abfs connectors), we can isolate IO better (though not across threads)


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



---------------------------------------------------------------------
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 pull request #1982: HADOOP-16830. IOStatistics API.

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #1982:
URL: https://github.com/apache/hadoop/pull/1982#issuecomment-620889592


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 24s |  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 22 new or modified test files.  |
   ||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for branch  |
   | -1 :x: |  mvninstall  |   0m 20s |  root in trunk failed.  |
   | -1 :x: |  compile  |   0m 23s |  root in trunk failed.  |
   | -0 :warning: |  checkstyle  |   0m 22s |  The patch fails to run checkstyle in root  |
   | -1 :x: |  mvnsite  |   0m 23s |  hadoop-common in trunk failed.  |
   | -1 :x: |  mvnsite  |   0m 22s |  hadoop-aws in trunk failed.  |
   | -1 :x: |  shadedclient  |   6m 25s |  branch has errors when building and testing our client artifacts.  |
   | -1 :x: |  javadoc  |   0m 24s |  hadoop-aws in trunk failed.  |
   | +0 :ok: |  spotbugs  |   8m 15s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | -1 :x: |  findbugs  |   0m 24s |  hadoop-common in trunk failed.  |
   | -1 :x: |  findbugs  |   0m 11s |  hadoop-aws in trunk failed.  |
   | -0 :warning: |  patch  |   8m 40s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 42s |  Maven dependency ordering for patch  |
   | -1 :x: |  mvninstall  |   0m 19s |  hadoop-aws in the patch failed.  |
   | -1 :x: |  compile  |   0m 23s |  root in the patch failed.  |
   | -1 :x: |  javac  |   0m 23s |  root in the patch failed.  |
   | -0 :warning: |  checkstyle  |   3m 12s |  root: The patch generated 167 new + 0 unchanged - 0 fixed = 167 total (was 0)  |
   | -1 :x: |  mvnsite  |   0m 23s |  hadoop-common in the patch failed.  |
   | -1 :x: |  mvnsite  |   0m 43s |  hadoop-aws in the patch failed.  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  xml  |   0m  2s |  The patch has no ill-formed XML file.  |
   | -1 :x: |  shadedclient  |   4m 50s |  patch has errors when building and testing our client artifacts.  |
   | -1 :x: |  javadoc  |   0m 28s |  hadoop-tools_hadoop-aws generated 5 new + 0 unchanged - 0 fixed = 5 total (was 0)  |
   | +1 :green_heart: |  findbugs  |   3m 51s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  |  19m 26s |  hadoop-common in the patch passed.  |
   | -1 :x: |  unit  |   0m 37s |  hadoop-aws in the patch failed.  |
   | +0 :ok: |  asflicense  |   0m 26s |  ASF License check generated no output?  |
   |  |   |  51m 11s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.metrics2.source.TestJvmMetrics |
   |   | hadoop.http.TestHttpServer |
   |   | hadoop.fs.TestStat |
   |   | hadoop.fs.contract.localfs.TestLocalFSContractMultipartUploader |
   |   | hadoop.ha.TestZKFailoverControllerStress |
   |   | hadoop.crypto.TestCryptoStreams |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.40 ServerAPI=1.40 base: https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/2/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/1982 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle xml |
   | uname | Linux 68a5126a219a 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 / ab364295597 |
   | Default Java | Private Build-1.8.0_252-8u252-b09-1~18.04-b09 |
   | mvninstall | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/2/artifact/out/branch-mvninstall-root.txt |
   | compile | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/2/artifact/out/branch-compile-root.txt |
   | checkstyle | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/2/artifact/out/buildtool-branch-checkstyle-root.txt |
   | mvnsite | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/2/artifact/out/branch-mvnsite-hadoop-common-project_hadoop-common.txt |
   | mvnsite | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/2/artifact/out/branch-mvnsite-hadoop-tools_hadoop-aws.txt |
   | javadoc | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/2/artifact/out/branch-javadoc-hadoop-tools_hadoop-aws.txt |
   | findbugs | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/2/artifact/out/branch-findbugs-hadoop-common-project_hadoop-common.txt |
   | findbugs | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/2/artifact/out/branch-findbugs-hadoop-tools_hadoop-aws.txt |
   | mvninstall | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/2/artifact/out/patch-mvninstall-hadoop-tools_hadoop-aws.txt |
   | compile | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/2/artifact/out/patch-compile-root.txt |
   | javac | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/2/artifact/out/patch-compile-root.txt |
   | checkstyle | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/2/artifact/out/diff-checkstyle-root.txt |
   | mvnsite | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/2/artifact/out/patch-mvnsite-hadoop-common-project_hadoop-common.txt |
   | mvnsite | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/2/artifact/out/patch-mvnsite-hadoop-tools_hadoop-aws.txt |
   | javadoc | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/2/artifact/out/diff-javadoc-javadoc-hadoop-tools_hadoop-aws.txt |
   | unit | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/2/artifact/out/patch-unit-hadoop-common-project_hadoop-common.txt |
   | unit | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/2/artifact/out/patch-unit-hadoop-tools_hadoop-aws.txt |
   |  Test Results | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/2/testReport/ |
   | Max. process+thread count | 549 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common hadoop-tools/hadoop-aws U: . |
   | Console output | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/2/console |
   | versions | git=2.17.1 maven=3.6.0 |
   | Powered by | Apache Yetus 0.12.0 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



---------------------------------------------------------------------
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 #1982: HADOOP-16830. IOStatistics API.

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1982:
URL: https://github.com/apache/hadoop/pull/1982#discussion_r422988550



##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
##########
@@ -634,46 +655,56 @@ public void close() {
 
   /**
    * Statistics updated by an input stream during its actual operation.
-   * These counters not thread-safe and are for use in a single instance
-   * of a stream.
+   * These counters are marked as volatile so that IOStatistics on the stream

Review comment:
       fixed
   




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



---------------------------------------------------------------------
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 pull request #1982: HADOOP-16830. IOStatistics API.

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #1982:
URL: https://github.com/apache/hadoop/pull/1982#issuecomment-632039724


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 39s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  2s |  No case conflicting files found.  |
   | +0 :ok: |  markdownlint  |   0m  0s |  markdownlint was not available.  |
   | +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 23 new or modified test files.  |
   ||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 46s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  19m 45s |  trunk passed  |
   | +1 :green_heart: |  compile  |  17m 24s |  trunk passed  |
   | +1 :green_heart: |  checkstyle  |   2m 45s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   2m 20s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  19m 30s |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 46s |  trunk passed  |
   | +0 :ok: |  spotbugs  |   1m 10s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   3m 16s |  trunk passed  |
   | -0 :warning: |  patch  |   1m 32s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 25s |  Maven dependency ordering for patch  |
   | -1 :x: |  mvninstall  |   0m 27s |  hadoop-aws in the patch failed.  |
   | -1 :x: |  compile  |  15m 27s |  root in the patch failed.  |
   | -1 :x: |  javac  |  15m 27s |  root in the patch failed.  |
   | -0 :warning: |  checkstyle  |   2m 43s |  root: The patch generated 15 new + 160 unchanged - 22 fixed = 175 total (was 182)  |
   | -1 :x: |  mvnsite  |   0m 49s |  hadoop-aws in the patch failed.  |
   | -1 :x: |  whitespace  |   0m  0s |  The patch has 6 line(s) that end in whitespace. Use git apply --whitespace=fix <<patch_file>>. Refer https://git-scm.com/docs/git-apply  |
   | +1 :green_heart: |  xml  |   0m  1s |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  shadedclient  |  14m 17s |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 47s |  the patch passed  |
   | -1 :x: |  findbugs  |   0m 47s |  hadoop-aws in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   9m 18s |  hadoop-common in the patch passed.  |
   | -1 :x: |  unit  |   0m 46s |  hadoop-aws in the patch failed.  |
   | +1 :green_heart: |  asflicense  |   0m 53s |  The patch does not generate ASF License warnings.  |
   |  |   | 119m 40s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.40 ServerAPI=1.40 base: https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/8/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/1982 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle markdownlint xml |
   | uname | Linux c72bcbdd95a1 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 / 1a3c6bb33b6 |
   | Default Java | Private Build-1.8.0_252-8u252-b09-1~18.04-b09 |
   | mvninstall | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/8/artifact/out/patch-mvninstall-hadoop-tools_hadoop-aws.txt |
   | compile | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/8/artifact/out/patch-compile-root.txt |
   | javac | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/8/artifact/out/patch-compile-root.txt |
   | checkstyle | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/8/artifact/out/diff-checkstyle-root.txt |
   | mvnsite | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/8/artifact/out/patch-mvnsite-hadoop-tools_hadoop-aws.txt |
   | whitespace | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/8/artifact/out/whitespace-eol.txt |
   | findbugs | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/8/artifact/out/patch-findbugs-hadoop-tools_hadoop-aws.txt |
   | unit | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/8/artifact/out/patch-unit-hadoop-tools_hadoop-aws.txt |
   |  Test Results | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/8/testReport/ |
   | Max. process+thread count | 2702 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common hadoop-tools/hadoop-aws U: . |
   | Console output | https://builds.apache.org/job/hadoop-multibranch/job/PR-1982/8/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=3.1.0-RC1 |
   | Powered by | Apache Yetus 0.12.0 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



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