Skip to content

Commit abc87ae

Browse files
dengzhhu653steveloughran
authored andcommitted
MAPREDUCE-7307. Potential thread leak in LocatedFileStatusFetcher. (#2469)
Contributed by Zhihua Deng. Change-Id: Iee62539d02bd8f8a928171d8258e640487050a05
1 parent e4bc64c commit abc87ae

File tree

2 files changed

+133
-5
lines changed

2 files changed

+133
-5
lines changed

hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/LocatedFileStatusFetcher.java

Lines changed: 13 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -39,6 +39,7 @@
3939
import org.apache.hadoop.fs.RemoteIterator;
4040
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
4141

42+
import com.google.common.annotations.VisibleForTesting;
4243
import com.google.common.collect.Iterables;
4344
import com.google.common.util.concurrent.FutureCallback;
4445
import com.google.common.util.concurrent.Futures;
@@ -151,12 +152,13 @@ public Iterable<FileStatus> getFileStatuses() throws InterruptedException,
151152
}
152153
} finally {
153154
lock.unlock();
155+
// either the scan completed or an error was raised.
156+
// in the case of an error shutting down the executor will interrupt all
157+
// active threads, which can add noise to the logs.
158+
LOG.debug("Scan complete: shutting down");
159+
this.exec.shutdownNow();
154160
}
155-
// either the scan completed or an error was raised.
156-
// in the case of an error shutting down the executor will interrupt all
157-
// active threads, which can add noise to the logs.
158-
LOG.debug("Scan complete: shutting down");
159-
this.exec.shutdownNow();
161+
160162
if (this.unknownError != null) {
161163
LOG.debug("Scan failed", this.unknownError);
162164
if (this.unknownError instanceof Error) {
@@ -401,4 +403,10 @@ public void onFailure(Throwable t) {
401403
registerError(t);
402404
}
403405
}
406+
407+
@VisibleForTesting
408+
ListeningExecutorService getListeningExecutorService() {
409+
return exec;
410+
}
411+
404412
}
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,120 @@
1+
/**
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
19+
package org.apache.hadoop.mapred;
20+
21+
import java.io.File;
22+
import java.io.IOException;
23+
import java.util.concurrent.CountDownLatch;
24+
25+
import org.junit.After;
26+
import org.junit.Assert;
27+
import org.junit.Before;
28+
import org.junit.Test;
29+
30+
import org.apache.hadoop.conf.Configuration;
31+
import org.apache.hadoop.fs.FileStatus;
32+
import org.apache.hadoop.fs.FileSystem;
33+
import org.apache.hadoop.fs.FileUtil;
34+
import org.apache.hadoop.fs.LocalFileSystem;
35+
import org.apache.hadoop.fs.Path;
36+
import org.apache.hadoop.fs.PathFilter;
37+
import org.apache.hadoop.test.AbstractHadoopTestBase;
38+
import org.apache.hadoop.test.GenericTestUtils;
39+
40+
/**
41+
* Test that the executor service has been shut down
42+
* when the LocatedFileStatusFetcher is interrupted.
43+
*/
44+
public class TestLocatedFileStatusFetcher extends AbstractHadoopTestBase {
45+
46+
private Configuration conf;
47+
private FileSystem fileSys;
48+
private boolean mkdirs;
49+
private File dir = GenericTestUtils.getTestDir("test-lfs-fetcher");
50+
private static final CountDownLatch LATCH = new CountDownLatch(1);
51+
52+
@Before
53+
public void setup() throws Exception {
54+
conf = new Configuration(false);
55+
conf.set("fs.file.impl", MockFileSystem.class.getName());
56+
fileSys = FileSystem.getLocal(conf);
57+
}
58+
59+
@After
60+
public void after() {
61+
if (mkdirs) {
62+
FileUtil.fullyDelete(dir);
63+
}
64+
}
65+
66+
@Test
67+
public void testExecutorsShutDown() throws Exception {
68+
Path scanPath = new Path(dir.getAbsolutePath());
69+
mkdirs = fileSys.mkdirs(scanPath);
70+
Path[] dirs = new Path[] {scanPath};
71+
final LocatedFileStatusFetcher fetcher = new LocatedFileStatusFetcher(conf,
72+
dirs, true, new PathFilter() {
73+
@Override
74+
public boolean accept(Path path) {
75+
return true;
76+
}
77+
}, true);
78+
79+
Thread t = new Thread() {
80+
@Override
81+
public void run() {
82+
try {
83+
fetcher.getFileStatuses();
84+
} catch (Exception e) {
85+
// This should interrupt condition.await()
86+
Assert.assertTrue(e instanceof InterruptedException);
87+
}
88+
}
89+
};
90+
91+
t.start();
92+
LATCH.await();
93+
94+
t.interrupt();
95+
t.join();
96+
// Check the status for executor service
97+
Assert.assertTrue("The executor service should have been shut down",
98+
fetcher.getListeningExecutorService().isShutdown());
99+
}
100+
101+
static class MockFileSystem extends LocalFileSystem {
102+
@Override
103+
public FileStatus[] globStatus(Path pathPattern, PathFilter filter)
104+
throws IOException {
105+
// The executor service now is running tasks
106+
LATCH.countDown();
107+
try {
108+
// Try to sleep some time to
109+
// let LocatedFileStatusFetcher#getFileStatuses be interrupted before
110+
// the getting file info task finishes.
111+
Thread.sleep(5000);
112+
} catch (InterruptedException e) {
113+
// Ignore this exception
114+
}
115+
return super.globStatus(pathPattern, filter);
116+
}
117+
118+
}
119+
120+
}

0 commit comments

Comments
 (0)