You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by st...@apache.org on 2020/11/23 16:34:20 UTC

[hadoop] branch branch-3.3 updated: MAPREDUCE-7307. Potential thread leak in LocatedFileStatusFetcher. (#2469)

This is an automated email from the ASF dual-hosted git repository.

stevel pushed a commit to branch branch-3.3
in repository https://gitbox.apache.org/repos/asf/hadoop.git


The following commit(s) were added to refs/heads/branch-3.3 by this push:
     new abc87ae  MAPREDUCE-7307. Potential thread leak in LocatedFileStatusFetcher. (#2469)
abc87ae is described below

commit abc87aef1895ff92993ab650f84270b8be8d45c6
Author: dengzh <de...@gmail.com>
AuthorDate: Mon Nov 23 23:40:22 2020 +0800

    MAPREDUCE-7307. Potential thread leak in LocatedFileStatusFetcher. (#2469)
    
    Contributed by Zhihua Deng.
    
    Change-Id: Iee62539d02bd8f8a928171d8258e640487050a05
---
 .../hadoop/mapred/LocatedFileStatusFetcher.java    |  18 +++-
 .../mapred/TestLocatedFileStatusFetcher.java       | 120 +++++++++++++++++++++
 2 files changed, 133 insertions(+), 5 deletions(-)

diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/LocatedFileStatusFetcher.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/LocatedFileStatusFetcher.java
index a248f14..275fb64 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/LocatedFileStatusFetcher.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/LocatedFileStatusFetcher.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Iterables;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
@@ -151,12 +152,13 @@ public class LocatedFileStatusFetcher {
       }
     } finally {
       lock.unlock();
+      // either the scan completed or an error was raised.
+      // in the case of an error shutting down the executor will interrupt all
+      // active threads, which can add noise to the logs.
+      LOG.debug("Scan complete: shutting down");
+      this.exec.shutdownNow();
     }
-    // either the scan completed or an error was raised.
-    // in the case of an error shutting down the executor will interrupt all
-    // active threads, which can add noise to the logs.
-    LOG.debug("Scan complete: shutting down");
-    this.exec.shutdownNow();
+
     if (this.unknownError != null) {
       LOG.debug("Scan failed", this.unknownError);
       if (this.unknownError instanceof Error) {
@@ -401,4 +403,10 @@ public class LocatedFileStatusFetcher {
       registerError(t);
     }
   }
+
+  @VisibleForTesting
+  ListeningExecutorService getListeningExecutorService() {
+    return exec;
+  }
+
 }
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestLocatedFileStatusFetcher.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestLocatedFileStatusFetcher.java
new file mode 100644
index 0000000..c818a25
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestLocatedFileStatusFetcher.java
@@ -0,0 +1,120 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.mapred;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.concurrent.CountDownLatch;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.LocalFileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.test.AbstractHadoopTestBase;
+import org.apache.hadoop.test.GenericTestUtils;
+
+/**
+ *  Test that the executor service has been shut down
+ *  when the LocatedFileStatusFetcher is interrupted.
+ */
+public class TestLocatedFileStatusFetcher extends AbstractHadoopTestBase {
+
+  private Configuration conf;
+  private FileSystem fileSys;
+  private boolean mkdirs;
+  private File dir = GenericTestUtils.getTestDir("test-lfs-fetcher");
+  private static final CountDownLatch LATCH = new CountDownLatch(1);
+
+  @Before
+  public void setup() throws Exception {
+    conf = new Configuration(false);
+    conf.set("fs.file.impl", MockFileSystem.class.getName());
+    fileSys = FileSystem.getLocal(conf);
+  }
+
+  @After
+  public void after() {
+    if (mkdirs) {
+      FileUtil.fullyDelete(dir);
+    }
+  }
+
+  @Test
+  public void testExecutorsShutDown() throws Exception {
+    Path scanPath = new Path(dir.getAbsolutePath());
+    mkdirs = fileSys.mkdirs(scanPath);
+    Path[] dirs = new Path[] {scanPath};
+    final LocatedFileStatusFetcher fetcher = new LocatedFileStatusFetcher(conf,
+        dirs, true, new PathFilter() {
+          @Override
+          public boolean accept(Path path) {
+            return true;
+          }
+        }, true);
+
+    Thread t = new Thread() {
+      @Override
+      public void run() {
+        try {
+          fetcher.getFileStatuses();
+        } catch (Exception e) {
+          // This should interrupt condition.await()
+          Assert.assertTrue(e instanceof InterruptedException);
+        }
+      }
+    };
+
+    t.start();
+    LATCH.await();
+
+    t.interrupt();
+    t.join();
+    // Check the status for executor service
+    Assert.assertTrue("The executor service should have been shut down",
+        fetcher.getListeningExecutorService().isShutdown());
+  }
+
+  static class MockFileSystem extends LocalFileSystem {
+    @Override
+    public FileStatus[] globStatus(Path pathPattern, PathFilter filter)
+        throws IOException {
+      // The executor service now is running tasks
+      LATCH.countDown();
+      try {
+        // Try to sleep some time to
+        // let LocatedFileStatusFetcher#getFileStatuses be interrupted before
+        // the getting file info task finishes.
+        Thread.sleep(5000);
+      } catch (InterruptedException e) {
+        // Ignore this exception
+      }
+      return super.globStatus(pathPattern, filter);
+    }
+
+  }
+
+}


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