[ 
https://issues.apache.org/jira/browse/MAPREDUCE-7435?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17728468#comment-17728468
 ] 

ASF GitHub Bot commented on MAPREDUCE-7435:
-------------------------------------------

steveloughran commented on code in PR #5519:
URL: https://github.com/apache/hadoop/pull/5519#discussion_r1213556815


##########
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestIOStatisticsSetters.java:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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;
+
+import java.util.Arrays;
+import java.util.Collection;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.hadoop.fs.statistics.impl.ForwardingIOStatisticsStore;
+import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
+import org.apache.hadoop.test.AbstractHadoopTestBase;
+
+import static 
org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticCounter;
+import static 
org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticGauge;
+import static 
org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticMaximum;
+import static 
org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticMean;
+import static 
org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticMinimum;
+import static 
org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatisticsStore;
+
+/**
+ * Test the {@link IOStatisticsSetters} interface implementations through
+ * a parameterized run with each implementation.
+ * For each of the setters, the value is set, verified,
+ * updated, verified again.
+ * An option known to be undefined in all created IOStatisticsStore instances
+ * is set, to verify it is harmless.
+ */
+
+@RunWith(Parameterized.class)
+
+public class TestIOStatisticsSetters extends AbstractHadoopTestBase {
+
+  public static final String COUNTER = "counter";
+
+  public static final String GAUGE = "gauge";
+
+  public static final String MAXIMUM = "max";
+
+  public static final String MINIMUM = "min";
+
+  public static final String MEAN = "mean";
+
+  private final IOStatisticsSetters ioStatistics;
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> params() {
+    return Arrays.asList(new Object[][]{
+        {new IOStatisticsSnapshot()},
+        {createTestStore()},
+        {new ForwardingIOStatisticsStore(createTestStore())},
+    });
+  }
+
+  /**
+   * Create a test store with the stats used for testing set up.
+   * @return a set up store
+   */
+  private static IOStatisticsStore createTestStore() {
+    return iostatisticsStore()
+        .withCounters(COUNTER)
+        .withGauges(GAUGE)
+        .withMaximums(MAXIMUM)
+        .withMinimums(MINIMUM)
+        .withMeanStatistics(MEAN)
+        .build();
+  }
+
+  public TestIOStatisticsSetters(IOStatisticsSetters ioStatisticsSetters) {
+    this.ioStatistics = ioStatisticsSetters;
+  }
+
+  @Test
+  public void testCounter() throws Throwable {
+    // write
+    ioStatistics.setCounter(COUNTER, 1);
+    assertThatStatisticCounter(ioStatistics, COUNTER)
+        .isEqualTo(1);
+
+    // update
+    ioStatistics.setCounter(COUNTER, 2);
+    assertThatStatisticCounter(ioStatistics, COUNTER)
+        .isEqualTo(2);
+
+    // unknown value
+    ioStatistics.setCounter("c2", 3);

Review Comment:
   had to parameterize the test so we assert than on snapshots they do accrue, 
but on the other impls they don't





> ManifestCommitter OOM on azure job
> ----------------------------------
>
>                 Key: MAPREDUCE-7435
>                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-7435
>             Project: Hadoop Map/Reduce
>          Issue Type: Bug
>          Components: client
>    Affects Versions: 3.3.5
>            Reporter: Steve Loughran
>            Assignee: Steve Loughran
>            Priority: Major
>              Labels: pull-request-available
>
> I've got some reports of spark jobs OOM if the manifest committer is used 
> through abfs.
> either the manifests are using too much memory, or something is not working 
> with azure stream memory use (or both).
> before proposing a solution, first step should be to write a test to load 
> many, many manifests, each with lots of dirs and files to see what breaks.
> note: we did have OOM issues with the s3a committer, on teragen but those 
> structures have to include every etag of every block, so the manifest size is 
> O(blocks); the new committer is O(files + dirs).
> {code}
> java.lang.OutOfMemoryError: Java heap space
> at 
> org.apache.hadoop.fs.azurebfs.services.AbfsInputStream.readOneBlock(AbfsInputStream.java:314)
> at 
> org.apache.hadoop.fs.azurebfs.services.AbfsInputStream.read(AbfsInputStream.java:267)
> at java.io.DataInputStream.read(DataInputStream.java:149)
> at 
> com.fasterxml.jackson.core.json.ByteSourceJsonBootstrapper.ensureLoaded(ByteSourceJsonBootstrapper.java:539)
> at 
> com.fasterxml.jackson.core.json.ByteSourceJsonBootstrapper.detectEncoding(ByteSourceJsonBootstrapper.java:133)
> at 
> com.fasterxml.jackson.core.json.ByteSourceJsonBootstrapper.constructParser(ByteSourceJsonBootstrapper.java:256)
> at com.fasterxml.jackson.core.JsonFactory._createParser(JsonFactory.java:1656)
> at com.fasterxml.jackson.core.JsonFactory.createParser(JsonFactory.java:1085)
> at 
> com.fasterxml.jackson.databind.ObjectMapper.readValue(ObjectMapper.java:3585)
> at 
> org.apache.hadoop.util.JsonSerialization.fromJsonStream(JsonSerialization.java:164)
> at org.apache.hadoop.util.JsonSerialization.load(JsonSerialization.java:279)
> at 
> org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest.load(TaskManifest.java:361)
> at 
> org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestStoreOperationsThroughFileSystem.loadTaskManifest(ManifestStoreOperationsThroughFileSystem.java:133)
> at 
> org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.AbstractJobOrTaskStage.lambda$loadManifest$6(AbstractJobOrTaskStage.java:493)
> at 
> org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.AbstractJobOrTaskStage$$Lambda$231/1813048085.apply(Unknown
>  Source)
> at 
> org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.invokeTrackingDuration(IOStatisticsBinding.java:543)
> at 
> org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.lambda$trackDurationOfOperation$5(IOStatisticsBinding.java:524)
> at 
> org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding$$Lambda$217/489150849.apply(Unknown
>  Source)
> at 
> org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDuration(IOStatisticsBinding.java:445)
> at 
> org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.AbstractJobOrTaskStage.loadManifest(AbstractJobOrTaskStage.java:492)
> at 
> org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.LoadManifestsStage.fetchTaskManifest(LoadManifestsStage.java:170)
> at 
> org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.LoadManifestsStage.processOneManifest(LoadManifestsStage.java:138)
> at 
> org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.LoadManifestsStage$$Lambda$229/137752948.run(Unknown
>  Source)
> at 
> org.apache.hadoop.util.functional.TaskPool$Builder.lambda$runParallel$0(TaskPool.java:410)
> at 
> org.apache.hadoop.util.functional.TaskPool$Builder$$Lambda$230/467893357.run(Unknown
>  Source)
> at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> 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:750)
> {code}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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

Reply via email to