HADOOP-10353. Merging change r1572679 and r1572680 from trunk to branch-2.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1572682 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Chris Nauroth 2014-02-27 19:03:58 +00:00
parent e54f4dcb5e
commit 0eb2d3355a
4 changed files with 87 additions and 3 deletions

View File

@ -81,6 +81,9 @@ Release 2.4.0 - UNRELEASED
HADOOP-10368. InputStream is not closed in VersionInfo ctor. HADOOP-10368. InputStream is not closed in VersionInfo ctor.
(Tsuyoshi OZAWA via szetszwo) (Tsuyoshi OZAWA via szetszwo)
HADOOP-10353. FsUrlStreamHandlerFactory is not thread safe.
(Tudor Scurtu via cnauroth)
BREAKDOWN OF HADOOP-10184 SUBTASKS AND RELATED JIRAS BREAKDOWN OF HADOOP-10184 SUBTASKS AND RELATED JIRAS
HADOOP-10185. FileSystem API for ACLs. (cnauroth) HADOOP-10185. FileSystem API for ACLs. (cnauroth)

View File

@ -19,8 +19,8 @@ package org.apache.hadoop.fs;
import java.io.IOException; import java.io.IOException;
import java.net.URLStreamHandlerFactory; import java.net.URLStreamHandlerFactory;
import java.util.HashMap;
import java.util.Map; import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
@ -45,7 +45,8 @@ public class FsUrlStreamHandlerFactory implements
private Configuration conf; private Configuration conf;
// This map stores whether a protocol is know or not by FileSystem // This map stores whether a protocol is know or not by FileSystem
private Map<String, Boolean> protocols = new HashMap<String, Boolean>(); private Map<String, Boolean> protocols =
new ConcurrentHashMap<String, Boolean>();
// The URL Stream handler // The URL Stream handler
private java.net.URLStreamHandler handler; private java.net.URLStreamHandler handler;

View File

@ -35,7 +35,7 @@ import org.apache.hadoop.test.PathUtils;
import org.junit.Test; import org.junit.Test;
/** /**
* Test of the URL stream handler factory. * Test of the URL stream handler.
*/ */
public class TestUrlStreamHandler { public class TestUrlStreamHandler {

View File

@ -0,0 +1,80 @@
/**
* 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;
import org.junit.Test;
import java.util.ArrayList;
import java.util.Random;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
/**
* Test of the URL stream handler factory.
*/
public class TestUrlStreamHandlerFactory {
private static final int RUNS = 20;
private static final int THREADS = 10;
private static final int TASKS = 200;
private static final int TIMEOUT = 30;
@Test
public void testConcurrency() throws Exception {
for (int i = 0; i < RUNS; i++) {
singleRun();
}
}
private void singleRun() throws Exception {
final FsUrlStreamHandlerFactory factory = new FsUrlStreamHandlerFactory();
final Random random = new Random();
ExecutorService executor = Executors.newFixedThreadPool(THREADS);
ArrayList<Future<?>> futures = new ArrayList<Future<?>>(TASKS);
for (int i = 0; i < TASKS ; i++) {
final int aux = i;
futures.add(executor.submit(new Runnable() {
@Override
public void run() {
int rand = aux + random.nextInt(3);
factory.createURLStreamHandler(String.valueOf(rand));
}
}));
}
executor.shutdown();
try {
executor.awaitTermination(TIMEOUT, TimeUnit.SECONDS);
executor.shutdownNow();
} catch (InterruptedException e) {
// pass
}
// check for exceptions
for (Future future : futures) {
if (!future.isDone()) {
break; // timed out
}
future.get();
}
}
}