diff --git hadoop-common-project/hadoop-common/pom.xml hadoop-common-project/hadoop-common/pom.xml index 6f7fa0f..af5a7f4 100644 --- hadoop-common-project/hadoop-common/pom.xml +++ hadoop-common-project/hadoop-common/pom.xml @@ -780,7 +780,9 @@ org.apache.maven.plugins maven-surefire-plugin - ${testsThreadCount} + perthread + ${testsThreadCount} + classes -Xmx1024m -XX:+HeapDumpOnOutOfMemoryError -DminiClusterDedicatedDirs=true diff --git hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/LossyRetryInvocationHandler.java hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/LossyRetryInvocationHandler.java deleted file mode 100644 index df58955..0000000 --- hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/LossyRetryInvocationHandler.java +++ /dev/null @@ -1,62 +0,0 @@ -/** - * 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.io.retry; - -import java.lang.reflect.Method; -import java.net.UnknownHostException; - -import org.apache.hadoop.classification.InterfaceAudience; - -/** - * A dummy invocation handler extending RetryInvocationHandler. It drops the - * first N number of responses. This invocation handler is only used for testing. - */ -@InterfaceAudience.Private -public class LossyRetryInvocationHandler extends RetryInvocationHandler { - private final int numToDrop; - private static final ThreadLocal RetryCount = - new ThreadLocal(); - - public LossyRetryInvocationHandler(int numToDrop, - FailoverProxyProvider proxyProvider, RetryPolicy retryPolicy) { - super(proxyProvider, retryPolicy); - this.numToDrop = numToDrop; - } - - @Override - public Object invoke(Object proxy, Method method, Object[] args) - throws Throwable { - RetryCount.set(0); - return super.invoke(proxy, method, args); - } - - @Override - protected Object invokeMethod(Method method, Object[] args) throws Throwable { - Object result = super.invokeMethod(method, args); - int retryCount = RetryCount.get(); - if (retryCount < this.numToDrop) { - RetryCount.set(++retryCount); - LOG.info("Drop the response. Current retryCount == " + retryCount); - throw new UnknownHostException("Fake Exception"); - } else { - LOG.info("retryCount == " + retryCount - + ". It's time to normally process the response"); - return result; - } - } -} diff --git hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java index 51dd46a..974bac9 100644 --- hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java +++ hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java @@ -63,7 +63,7 @@ protected RetryInvocationHandler(FailoverProxyProvider proxyProvider, this(proxyProvider, retryPolicy, Collections.emptyMap()); } - protected RetryInvocationHandler(FailoverProxyProvider proxyProvider, + RetryInvocationHandler(FailoverProxyProvider proxyProvider, RetryPolicy defaultPolicy, Map methodNameToPolicyMap) { this.proxyProvider = proxyProvider; diff --git hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index dd86e0e..293de2d 100644 --- hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt @@ -244,9 +244,6 @@ Release 2.3.0 - UNRELEASED NEW FEATURES - HDFS-5118. Provide testing support for DFSClient to drop RPC responses. - (jing9) - IMPROVEMENTS HDFS-4657. Limit the number of blocks logged by the NN after a block @@ -270,8 +267,6 @@ Release 2.3.0 - UNRELEASED HDFS-5144. Document time unit to NameNodeMetrics. (Akira Ajisaka via suresh) - HDFS-4491. Parallel testing HDFS. (Andrey Klochkov via cnauroth) - OPTIMIZATIONS BUG FIXES @@ -286,9 +281,6 @@ Release 2.3.0 - UNRELEASED HDFS-4816. transitionToActive blocks if the SBN is doing checkpoint image transfer. (Andrew Wang) - HDFS-5164. deleteSnapshot should check if OperationCategory.WRITE is - possible before taking write lock. (Colin Patrick McCabe) - Release 2.1.1-beta - UNRELEASED INCOMPATIBLE CHANGES diff --git hadoop-hdfs-project/hadoop-hdfs/pom.xml hadoop-hdfs-project/hadoop-hdfs/pom.xml index 2160d83..59abffa 100644 --- hadoop-hdfs-project/hadoop-hdfs/pom.xml +++ hadoop-hdfs-project/hadoop-hdfs/pom.xml @@ -700,44 +700,5 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd"> - - parallel-tests - - - - maven-antrun-plugin - - - create-parallel-tests-dirs - test-compile - - - - - - - - - - run - - - - - - org.apache.maven.plugins - maven-surefire-plugin - - ${testsThreadCount} - -Xmx1024m -XX:+HeapDumpOnOutOfMemoryError -DminiClusterDedicatedDirs=true - - ${test.build.data}/${surefire.forkNumber} - ${hadoop.tmp.dir}/${surefire.forkNumber} - - - - - - diff --git hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java index d1cc784..511df17 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java +++ hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java @@ -27,9 +27,6 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_RETRIES_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CACHED_CONN_RETRY_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CACHED_CONN_RETRY_KEY; -import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CACHE_DROP_BEHIND_READS; -import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CACHE_DROP_BEHIND_WRITES; -import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CACHE_READAHEAD; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_DEFAULT; @@ -47,6 +44,9 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_USE_DN_HOSTNAME; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_USE_DN_HOSTNAME_DEFAULT; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CACHE_DROP_BEHIND_READS; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CACHE_DROP_BEHIND_WRITES; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CACHE_READAHEAD; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT; @@ -100,7 +100,6 @@ import org.apache.hadoop.fs.Options; import org.apache.hadoop.fs.Options.ChecksumOpt; import org.apache.hadoop.fs.ParentNotDirectoryException; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.UnresolvedLinkException; import org.apache.hadoop.fs.VolumeId; import org.apache.hadoop.fs.permission.FsPermission; @@ -114,13 +113,13 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata; import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException; import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType; import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlocks; import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException; -import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException; import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport; import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus; import org.apache.hadoop.hdfs.protocol.UnresolvedPathException; @@ -145,7 +144,6 @@ import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.MD5Hash; import org.apache.hadoop.io.Text; -import org.apache.hadoop.io.retry.LossyRetryInvocationHandler; import org.apache.hadoop.ipc.Client; import org.apache.hadoop.ipc.RPC; import org.apache.hadoop.ipc.RemoteException; @@ -453,11 +451,7 @@ public DFSClient(URI nameNodeUri, Configuration conf, /** * Create a new DFSClient connected to the given nameNodeUri or rpcNamenode. - * If HA is enabled and a positive value is set for - * {@link DFSConfigKeys#DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY} in the - * configuration, the DFSClient will use {@link LossyRetryInvocationHandler} - * as its RetryInvocationHandler. Otherwise one of nameNodeUri or rpcNamenode - * must be null. + * Exactly one of nameNodeUri or rpcNamenode must be null. */ @VisibleForTesting public DFSClient(URI nameNodeUri, ClientProtocol rpcNamenode, @@ -481,20 +475,7 @@ public DFSClient(URI nameNodeUri, ClientProtocol rpcNamenode, this.clientName = "DFSClient_" + dfsClientConf.taskId + "_" + DFSUtil.getRandom().nextInt() + "_" + Thread.currentThread().getId(); - int numResponseToDrop = conf.getInt( - DFSConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY, - DFSConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_DEFAULT); - if (numResponseToDrop > 0) { - // This case is used for testing. - LOG.warn(DFSConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY - + " is set to " + numResponseToDrop - + ", this hacked client will proactively drop responses"); - NameNodeProxies.ProxyAndInfo proxyInfo = NameNodeProxies - .createProxyWithLossyRetryHandler(conf, nameNodeUri, - ClientProtocol.class, numResponseToDrop); - this.dtService = proxyInfo.getDelegationTokenService(); - this.namenode = proxyInfo.getProxy(); - } else if (rpcNamenode != null) { + if (rpcNamenode != null) { // This case is used for testing. Preconditions.checkArgument(nameNodeUri == null); this.namenode = rpcNamenode; @@ -533,7 +514,7 @@ public DFSClient(URI nameNodeUri, ClientProtocol rpcNamenode, this.defaultWriteCachingStrategy = new CachingStrategy(writeDropBehind, readahead); } - + /** * Return the socket addresses to use with each configured * local interface. Local interfaces may be specified by IP diff --git hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java index b4d67ca..a66ec93 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java +++ hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java @@ -497,12 +497,6 @@ public static final long DFS_NAMENODE_RETRY_CACHE_EXPIRYTIME_MILLIS_DEFAULT = 600000; // 10 minutes public static final String DFS_NAMENODE_RETRY_CACHE_HEAP_PERCENT_KEY = "dfs.namenode.retrycache.heap.percent"; public static final float DFS_NAMENODE_RETRY_CACHE_HEAP_PERCENT_DEFAULT = 0.03f; - - // The number of NN response dropped by client proactively in each RPC call. - // For testing NN retry cache, we can set this property with positive value. - public static final String DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY = "dfs.client.test.drop.namenode.response.number"; - public static final int DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_DEFAULT = 0; - // Hidden configuration undocumented in hdfs-site. xml // Timeout to wait for block receiver and responder thread to stop diff --git hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HftpFileSystem.java hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HftpFileSystem.java index dd5e9c6..9d9cde4 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HftpFileSystem.java +++ hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HftpFileSystem.java @@ -51,7 +51,7 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector; import org.apache.hadoop.hdfs.server.common.JspHelper; import org.apache.hadoop.hdfs.tools.DelegationTokenFetcher; -import org.apache.hadoop.hdfs.web.URLConnectionFactory; +import org.apache.hadoop.hdfs.web.URLUtils; import org.apache.hadoop.io.Text; import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.net.NetUtils; @@ -86,8 +86,6 @@ HttpURLConnection.setFollowRedirects(true); } - URLConnectionFactory connectionFactory = URLConnectionFactory.DEFAULT_CONNECTION_FACTORY; - public static final Text TOKEN_KIND = new Text("HFTP delegation"); protected UserGroupInformation ugi; @@ -333,8 +331,8 @@ protected HttpURLConnection openConnection(String path, String query) throws IOException { query = addDelegationTokenParam(query); final URL url = getNamenodeURL(path, query); - final HttpURLConnection connection; - connection = (HttpURLConnection)connectionFactory.openConnection(url); + final HttpURLConnection connection = + (HttpURLConnection)URLUtils.openConnection(url); connection.setRequestMethod("GET"); connection.connect(); return connection; @@ -354,14 +352,12 @@ protected String addDelegationTokenParam(String query) throws IOException { } static class RangeHeaderUrlOpener extends ByteRangeInputStream.URLOpener { - URLConnectionFactory connectionFactory = URLConnectionFactory.DEFAULT_CONNECTION_FACTORY; - RangeHeaderUrlOpener(final URL url) { super(url); } protected HttpURLConnection openConnection() throws IOException { - return (HttpURLConnection)connectionFactory.openConnection(url); + return (HttpURLConnection)URLUtils.openConnection(url); } /** Use HTTP Range header for specifying offset. */ diff --git hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HsftpFileSystem.java hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HsftpFileSystem.java index 6a3bdba..438d56e 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HsftpFileSystem.java +++ hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HsftpFileSystem.java @@ -40,6 +40,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hdfs.web.URLUtils; import org.apache.hadoop.util.Time; /** @@ -153,8 +154,7 @@ protected HttpURLConnection openConnection(String path, String query) query = addDelegationTokenParam(query); final URL url = new URL("https", nnUri.getHost(), nnUri.getPort(), path + '?' + query); - HttpsURLConnection conn; - conn = (HttpsURLConnection)connectionFactory.openConnection(url); + HttpsURLConnection conn = (HttpsURLConnection)URLUtils.openConnection(url); // bypass hostname verification conn.setHostnameVerifier(new DummyHostnameVerifier()); conn.setRequestMethod("GET"); diff --git hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java index 41dac6a..eb745b8 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java +++ hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java @@ -17,18 +17,10 @@ */ package org.apache.hadoop.hdfs; -import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_DEFAULT; -import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX; -import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_DEFAULT; -import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_KEY; -import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_DEFAULT; -import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_KEY; import java.io.IOException; import java.lang.reflect.Constructor; -import java.lang.reflect.InvocationHandler; -import java.lang.reflect.Proxy; import java.net.InetSocketAddress; import java.net.URI; import java.util.HashMap; @@ -56,7 +48,6 @@ import org.apache.hadoop.io.Text; import org.apache.hadoop.io.retry.DefaultFailoverProxyProvider; import org.apache.hadoop.io.retry.FailoverProxyProvider; -import org.apache.hadoop.io.retry.LossyRetryInvocationHandler; import org.apache.hadoop.io.retry.RetryPolicies; import org.apache.hadoop.io.retry.RetryPolicy; import org.apache.hadoop.io.retry.RetryProxy; @@ -153,61 +144,6 @@ public Text getDelegationTokenService() { return new ProxyAndInfo(proxy, dtService); } } - - /** - * Generate a dummy namenode proxy instance that utilizes our hacked - * {@link LossyRetryInvocationHandler}. Proxy instance generated using this - * method will proactively drop RPC responses. Currently this method only - * support HA setup. IllegalStateException will be thrown if the given - * configuration is not for HA. - * - * @param config the configuration containing the required IPC - * properties, client failover configurations, etc. - * @param nameNodeUri the URI pointing either to a specific NameNode - * or to a logical nameservice. - * @param xface the IPC interface which should be created - * @param numResponseToDrop The number of responses to drop for each RPC call - * @return an object containing both the proxy and the associated - * delegation token service it corresponds to - * @throws IOException if there is an error creating the proxy - */ - @SuppressWarnings("unchecked") - public static ProxyAndInfo createProxyWithLossyRetryHandler( - Configuration config, URI nameNodeUri, Class xface, - int numResponseToDrop) throws IOException { - Preconditions.checkArgument(numResponseToDrop > 0); - Class> failoverProxyProviderClass = - getFailoverProxyProviderClass(config, nameNodeUri, xface); - if (failoverProxyProviderClass != null) { // HA case - FailoverProxyProvider failoverProxyProvider = - createFailoverProxyProvider(config, failoverProxyProviderClass, - xface, nameNodeUri); - int delay = config.getInt( - DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_KEY, - DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_DEFAULT); - int maxCap = config.getInt( - DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_KEY, - DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_DEFAULT); - int maxFailoverAttempts = config.getInt( - DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_KEY, - DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_DEFAULT); - InvocationHandler dummyHandler = new LossyRetryInvocationHandler( - numResponseToDrop, failoverProxyProvider, - RetryPolicies.failoverOnNetworkException( - RetryPolicies.TRY_ONCE_THEN_FAIL, - Math.max(numResponseToDrop + 1, maxFailoverAttempts), delay, - maxCap)); - - T proxy = (T) Proxy.newProxyInstance( - failoverProxyProvider.getInterface().getClassLoader(), - new Class[] { xface }, dummyHandler); - Text dtService = HAUtil.buildTokenServiceForLogicalUri(nameNodeUri); - return new ProxyAndInfo(proxy, dtService); - } else { - throw new IllegalStateException("Currently creating proxy using " + - "LossyRetryInvocationHandler requires NN HA setup"); - } - } /** * Creates an explicitly non-HA-enabled proxy object. Most of the time you diff --git hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java index 42965ed..0ed6632 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java +++ hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java @@ -6767,7 +6767,6 @@ void deleteSnapshot(String snapshotRoot, String snapshotName) return; // Return previous response } boolean success = false; - checkOperation(OperationCategory.WRITE); writeLock(); try { checkOperation(OperationCategory.WRITE); diff --git hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java deleted file mode 100644 index 54aab04..0000000 --- hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java +++ /dev/null @@ -1,68 +0,0 @@ -/** - * 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.hdfs.web; - -import java.io.IOException; -import java.net.URL; -import java.net.URLConnection; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; - -/** - * Utilities for handling URLs - */ -@InterfaceAudience.LimitedPrivate({"HDFS"}) -@InterfaceStability.Unstable -public class URLConnectionFactory { - /** - * Timeout for socket connects and reads - */ - public final static int DEFAULT_SOCKET_TIMEOUT = 1*60*1000; // 1 minute - - public static final URLConnectionFactory DEFAULT_CONNECTION_FACTORY = new URLConnectionFactory(DEFAULT_SOCKET_TIMEOUT); - - private int socketTimeout; - - public URLConnectionFactory(int socketTimeout) { - this.socketTimeout = socketTimeout; - } - - /** - * Opens a url with read and connect timeouts - * @param url to open - * @return URLConnection - * @throws IOException - */ - public URLConnection openConnection(URL url) throws IOException { - URLConnection connection = url.openConnection(); - setTimeouts(connection); - return connection; - } - - /** - * Sets timeout parameters on the given URLConnection. - * - * @param connection URLConnection to set - */ - public void setTimeouts(URLConnection connection) { - connection.setConnectTimeout(socketTimeout); - connection.setReadTimeout(socketTimeout); - } -} diff --git hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/URLUtils.java hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/URLUtils.java new file mode 100644 index 0000000..09feaf5 --- /dev/null +++ hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/URLUtils.java @@ -0,0 +1,60 @@ +/** + * 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.hdfs.web; + +import java.io.IOException; +import java.net.URL; +import java.net.URLConnection; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Utilities for handling URLs + */ +@InterfaceAudience.LimitedPrivate({"HDFS"}) +@InterfaceStability.Unstable +public class URLUtils { + /** + * Timeout for socket connects and reads + */ + public static int SOCKET_TIMEOUT = 1*60*1000; // 1 minute + + /** + * Opens a url with read and connect timeouts + * @param url to open + * @return URLConnection + * @throws IOException + */ + public static URLConnection openConnection(URL url) throws IOException { + URLConnection connection = url.openConnection(); + setTimeouts(connection); + return connection; + } + + /** + * Sets timeout parameters on the given URLConnection. + * + * @param connection URLConnection to set + */ + static void setTimeouts(URLConnection connection) { + connection.setConnectTimeout(SOCKET_TIMEOUT); + connection.setReadTimeout(SOCKET_TIMEOUT); + } +} diff --git hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java index 8f1c589..9dbb01b 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java +++ hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java @@ -119,15 +119,13 @@ /** SPNEGO authenticator */ private static final KerberosUgiAuthenticator AUTH = new KerberosUgiAuthenticator(); - /** Default connection factory may be overriden in tests to use smaller timeout values */ - URLConnectionFactory connectionFactory = URLConnectionFactory.DEFAULT_CONNECTION_FACTORY; /** Configures connections for AuthenticatedURL */ - private final ConnectionConfigurator CONN_CONFIGURATOR = + private static final ConnectionConfigurator CONN_CONFIGURATOR = new ConnectionConfigurator() { @Override public HttpURLConnection configure(HttpURLConnection conn) throws IOException { - connectionFactory.setTimeouts(conn); + URLUtils.setTimeouts(conn); return conn; } }; @@ -481,9 +479,10 @@ private HttpURLConnection openHttpUrlConnection(final URL url) final AuthenticatedURL.Token authToken = new AuthenticatedURL.Token(); conn = new AuthenticatedURL(AUTH, CONN_CONFIGURATOR).openConnection( url, authToken); + URLUtils.setTimeouts(conn); } else { LOG.debug("open URL connection"); - conn = (HttpURLConnection)connectionFactory.openConnection(url); + conn = (HttpURLConnection)URLUtils.openConnection(url); } } catch (AuthenticationException e) { throw new IOException(e); @@ -578,7 +577,7 @@ HttpURLConnection twoStepWrite() throws IOException { checkRetry = false; //Step 2) Submit another Http request with the URL from the Location header with data. - conn = (HttpURLConnection)connectionFactory.openConnection(new URL(redirect)); + conn = (HttpURLConnection)URLUtils.openConnection(new URL(redirect)); conn.setRequestProperty("Content-Type", MediaType.APPLICATION_OCTET_STREAM); conn.setChunkedStreamingMode(32 << 10); //32kB-chunk connect(); @@ -601,7 +600,7 @@ void getResponse(boolean getJsonAndDisconnect) throws IOException { disconnect(); checkRetry = false; - conn = (HttpURLConnection)connectionFactory.openConnection(new URL(redirect)); + conn = (HttpURLConnection)URLUtils.openConnection(new URL(redirect)); connect(); } diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/fs/TestFiRename.java hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/fs/TestFiRename.java index 5ad3930..794c057 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/fs/TestFiRename.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/fs/TestFiRename.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.fs; -import java.io.File; import java.io.IOException; import java.util.EnumSet; @@ -28,7 +27,6 @@ import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; -import org.apache.hadoop.test.PathUtils; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -58,7 +56,8 @@ private static String addChild = ""; private static byte[] data = { 0 }; - private static String TEST_ROOT_DIR = PathUtils.getTestDirName(TestFiRename.class); + private static String TEST_ROOT_DIR = + System.getProperty("test.build.data", "/tmp") + "/test"; private static Configuration CONF = new Configuration(); static { diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestResolveHdfsSymlink.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestResolveHdfsSymlink.java index cfd4a8d..c48759e 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestResolveHdfsSymlink.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestResolveHdfsSymlink.java @@ -18,7 +18,6 @@ package org.apache.hadoop.fs; -import java.io.File; import static org.junit.Assert.fail; import java.io.FileNotFoundException; @@ -37,7 +36,6 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier; -import org.apache.hadoop.test.PathUtils; import org.apache.hadoop.test.GenericTestUtils; import org.junit.AfterClass; import org.junit.Assert; @@ -50,7 +48,6 @@ * underlying file system as Hdfs. */ public class TestResolveHdfsSymlink { - private static File TEST_ROOT_DIR = PathUtils.getTestDir(TestResolveHdfsSymlink.class); private static MiniDFSCluster cluster = null; @BeforeClass @@ -83,12 +80,12 @@ public void testFcResolveAfs() throws IOException, InterruptedException { .getUri()); Path alphaLocalPath = new Path(fcLocal.getDefaultFileSystem().getUri() - .toString(), new File(TEST_ROOT_DIR, "alpha").getAbsolutePath()); + .toString(), "/tmp/alpha"); DFSTestUtil.createFile(FileSystem.getLocal(conf), alphaLocalPath, 16, (short) 1, 2); Path linkTarget = new Path(fcLocal.getDefaultFileSystem().getUri() - .toString(), TEST_ROOT_DIR.getAbsolutePath()); + .toString(), "/tmp"); Path hdfsLink = new Path(fcHdfs.getDefaultFileSystem().getUri().toString(), "/tmp/link"); fcHdfs.createSymlink(linkTarget, hdfsLink, true); diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestUrlStreamHandler.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestUrlStreamHandler.java index 845eb63..d28736c 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestUrlStreamHandler.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestUrlStreamHandler.java @@ -31,7 +31,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.MiniDFSCluster; -import org.apache.hadoop.test.PathUtils; import org.junit.Test; /** @@ -39,8 +38,6 @@ */ public class TestUrlStreamHandler { - private static File TEST_ROOT_DIR = PathUtils.getTestDir(TestUrlStreamHandler.class); - /** * Test opening and reading from an InputStream through a hdfs:// URL. *

@@ -114,12 +111,13 @@ public void testFileUrls() throws IOException, URISyntaxException { Configuration conf = new HdfsConfiguration(); // Locate the test temporary directory. - if (!TEST_ROOT_DIR.exists()) { - if (!TEST_ROOT_DIR.mkdirs()) - throw new IOException("Cannot create temporary directory: " + TEST_ROOT_DIR); + File tmpDir = new File(conf.get("hadoop.tmp.dir")); + if (!tmpDir.exists()) { + if (!tmpDir.mkdirs()) + throw new IOException("Cannot create temporary directory: " + tmpDir); } - File tmpFile = new File(TEST_ROOT_DIR, "thefile"); + File tmpFile = new File(tmpDir, "thefile"); URI uri = tmpFile.toURI(); FileSystem fs = FileSystem.get(uri, conf); diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/loadGenerator/TestLoadGenerator.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/loadGenerator/TestLoadGenerator.java index 6c7bac3..e9e14ce 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/loadGenerator/TestLoadGenerator.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/loadGenerator/TestLoadGenerator.java @@ -29,7 +29,6 @@ import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.MiniDFSCluster; -import org.apache.hadoop.test.PathUtils; import org.apache.hadoop.util.Time; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; @@ -40,7 +39,8 @@ public class TestLoadGenerator extends Configured implements Tool { private static final Configuration CONF = new HdfsConfiguration(); private static final int DEFAULT_BLOCK_SIZE = 10; - private static final File OUT_DIR = PathUtils.getTestDir(TestLoadGenerator.class); + private static final String OUT_DIR = + System.getProperty("test.build.data","build/test/data"); private static final File DIR_STRUCTURE_FILE = new File(OUT_DIR, StructureGenerator.DIR_STRUCTURE_FILE_NAME); private static final File FILE_STRUCTURE_FILE = @@ -65,7 +65,7 @@ public void testStructureGenerator() throws Exception { StructureGenerator sg = new StructureGenerator(); String[] args = new String[]{"-maxDepth", "2", "-minWidth", "1", "-maxWidth", "2", "-numOfFiles", "2", - "-avgFileSize", "1", "-outDir", OUT_DIR.getAbsolutePath(), "-seed", "1"}; + "-avgFileSize", "1", "-outDir", OUT_DIR, "-seed", "1"}; final int MAX_DEPTH = 1; final int MIN_WIDTH = 3; @@ -133,7 +133,8 @@ public void testStructureGenerator() throws Exception { public void testLoadGenerator() throws Exception { final String TEST_SPACE_ROOT = "/test"; - final String SCRIPT_TEST_DIR = OUT_DIR.getAbsolutePath(); + final String SCRIPT_TEST_DIR = new File(System.getProperty("test.build.data", + "/tmp")).getAbsolutePath(); String script = SCRIPT_TEST_DIR + "/" + "loadgenscript"; String script2 = SCRIPT_TEST_DIR + "/" + "loadgenscript2"; File scriptFile1 = new File(script); @@ -155,7 +156,7 @@ public void testLoadGenerator() throws Exception { try { DataGenerator dg = new DataGenerator(); dg.setConf(CONF); - String [] args = new String[] {"-inDir", OUT_DIR.getAbsolutePath(), "-root", TEST_SPACE_ROOT}; + String [] args = new String[] {"-inDir", OUT_DIR, "-root", TEST_SPACE_ROOT}; assertEquals(0, dg.run(args)); final int READ_PROBABILITY = 1; diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java index 222a9b7..7090d49 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java @@ -1407,13 +1407,6 @@ public int getNameNodeServicePort(int nnIndex) { * Shutdown all the nodes in the cluster. */ public void shutdown() { - shutdown(false); - } - - /** - * Shutdown all the nodes in the cluster. - */ - public void shutdown(boolean deleteDfsDir) { LOG.info("Shutting down the Mini HDFS Cluster"); if (checkExitOnShutdown) { if (ExitUtil.terminateCalled()) { @@ -1433,11 +1426,6 @@ public void shutdown(boolean deleteDfsDir) { nameNode = null; } } - if (deleteDfsDir) { - base_dir.delete(); - } else { - base_dir.deleteOnExit(); - } } /** @@ -2130,7 +2118,7 @@ public File getInstanceStorageDir(int dnIndex, int dirIndex) { *

  • /data/data<2*dnIndex + 1>
  • *
  • /data/data<2*dnIndex + 2>
  • * - * + * * @param dnIndex datanode index (starts from 0) * @param dirIndex directory index (0 or 1). Index 0 provides access to the * first storage directory. Index 1 provides access to the second @@ -2161,7 +2149,7 @@ private static String getStorageDirPath(int dnIndex, int dirIndex) { public static String getDNCurrentDir(File storageDir) { return storageDir + "/" + Storage.STORAGE_DIR_CURRENT + "/"; } - + /** * Get directory corresponding to block pool directory in the datanode * @param storageDir the storage directory of a datanode. @@ -2267,7 +2255,7 @@ public static File getBlockFile(int dnIndex, ExtendedBlock block) { } return null; } - + /** * Get the block metadata file for a block from a given datanode * @@ -2355,17 +2343,14 @@ protected void setupDatanodeAddress(Configuration conf, boolean setupHostsFile, } else { if (checkDataNodeAddrConfig) { conf.setIfUnset(DFS_DATANODE_ADDRESS_KEY, "127.0.0.1:0"); + conf.setIfUnset(DFS_DATANODE_HTTP_ADDRESS_KEY, "127.0.0.1:0"); + conf.setIfUnset(DFS_DATANODE_IPC_ADDRESS_KEY, "127.0.0.1:0"); } else { conf.set(DFS_DATANODE_ADDRESS_KEY, "127.0.0.1:0"); + conf.set(DFS_DATANODE_HTTP_ADDRESS_KEY, "127.0.0.1:0"); + conf.set(DFS_DATANODE_IPC_ADDRESS_KEY, "127.0.0.1:0"); } } - if (checkDataNodeAddrConfig) { - conf.setIfUnset(DFS_DATANODE_HTTP_ADDRESS_KEY, "127.0.0.1:0"); - conf.setIfUnset(DFS_DATANODE_IPC_ADDRESS_KEY, "127.0.0.1:0"); - } else { - conf.set(DFS_DATANODE_HTTP_ADDRESS_KEY, "127.0.0.1:0"); - conf.set(DFS_DATANODE_IPC_ADDRESS_KEY, "127.0.0.1:0"); - } } private void addToFile(String p, String address) throws IOException { diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientReportBadBlock.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientReportBadBlock.java index 74c763d..51fab66 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientReportBadBlock.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientReportBadBlock.java @@ -71,6 +71,10 @@ @Before public void startUpCluster() throws IOException { + if (System.getProperty("test.build.data") == null) { // to allow test to be + // run outside of Ant + System.setProperty("test.build.data", "build/test/data"); + } // disable block scanner conf.setInt(DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, -1); diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java index fbcce39..e54e277 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java @@ -20,6 +20,9 @@ import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType.DATA_NODE; import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType.NAME_NODE; import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.fail; import java.io.File; import java.io.IOException; diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java index 4287378..e7d8268 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java @@ -44,7 +44,6 @@ import org.apache.hadoop.io.compress.BZip2Codec; import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.test.PathUtils; import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.ToolRunner; @@ -62,7 +61,9 @@ private static final Log LOG = LogFactory.getLog(TestDFSShell.class); private static AtomicInteger counter = new AtomicInteger(); - static final String TEST_ROOT_DIR = PathUtils.getTestDirName(TestDFSShell.class); + static final String TEST_ROOT_DIR = + new Path(System.getProperty("test.build.data","/tmp")) + .toString().replace(' ', '+'); static Path writeFile(FileSystem fs, Path f) throws IOException { DataOutputStream out = fs.create(f); @@ -481,11 +482,12 @@ public void testURIPaths() throws Exception { Configuration dstConf = new HdfsConfiguration(); MiniDFSCluster srcCluster = null; MiniDFSCluster dstCluster = null; - File bak = new File(PathUtils.getTestDir(getClass()), "dfs_tmp_uri"); - bak.mkdirs(); + String bak = System.getProperty("test.build.data"); try{ srcCluster = new MiniDFSCluster.Builder(srcConf).numDataNodes(2).build(); - dstConf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, bak.getAbsolutePath()); + File nameDir = new File(new File(bak), "dfs_tmp_uri/"); + nameDir.mkdirs(); + System.setProperty("test.build.data", nameDir.toString()); dstCluster = new MiniDFSCluster.Builder(dstConf).numDataNodes(2).build(); FileSystem srcFs = srcCluster.getFileSystem(); FileSystem dstFs = dstCluster.getFileSystem(); @@ -557,6 +559,7 @@ public void testURIPaths() throws Exception { ret = ToolRunner.run(shell, argv); assertEquals("default works for rm/rmr", 0, ret); } finally { + System.setProperty("test.build.data", bak); if (null != srcCluster) { srcCluster.shutdown(); } diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java index 9c2e038..05d2296 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java @@ -45,7 +45,6 @@ import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter; -import org.apache.hadoop.test.PathUtils; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -76,7 +75,7 @@ public void setup() throws IOException { // Set up the hosts/exclude files. localFileSys = FileSystem.getLocal(conf); Path workingDir = localFileSys.getWorkingDirectory(); - Path dir = new Path(workingDir, PathUtils.getTestDirName(getClass()) + "/work-dir/decommission"); + Path dir = new Path(workingDir, System.getProperty("test.build.data", "target/test/data") + "/work-dir/decommission"); hostsFile = new Path(dir, "hosts"); excludeFile = new Path(dir, "exclude"); diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java index c0093d2..9b4f313 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java @@ -25,8 +25,6 @@ import static org.mockito.Matchers.eq; import static org.mockito.Mockito.inOrder; import static org.mockito.Mockito.mock; - -import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; import java.net.URI; @@ -75,9 +73,6 @@ private HdfsConfiguration getTestConfiguration() { HdfsConfiguration conf; if (noXmlDefaults) { conf = new HdfsConfiguration(false); - String namenodeDir = new File(MiniDFSCluster.getBaseDirectory(), "name").getAbsolutePath(); - conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, namenodeDir); - conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY, namenodeDir); } else { conf = new HdfsConfiguration(); } diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFSInputChecker.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFSInputChecker.java index 5fc567a..2f2c9a4 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFSInputChecker.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFSInputChecker.java @@ -39,7 +39,6 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.io.IOUtils; -import org.apache.hadoop.test.PathUtils; import org.junit.Test; /** @@ -261,7 +260,7 @@ private void testFileCorruption(LocalFileSystem fileSys) throws IOException { // create a file and verify that checksum corruption results in // a checksum exception on LocalFS - String dir = PathUtils.getTestDirName(getClass()); + String dir = System.getProperty("test.build.data", "."); Path file = new Path(dir + "/corruption-test.dat"); Path crcFile = new Path(dir + "/.corruption-test.dat.crc"); diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppendRestart.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppendRestart.java index 570b19f..d086c77 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppendRestart.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppendRestart.java @@ -37,7 +37,6 @@ import org.apache.hadoop.hdfs.util.Holder; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.test.GenericTestUtils; -import org.apache.hadoop.test.PathUtils; import org.junit.Test; /** @@ -146,7 +145,7 @@ public void testLoadLogsFromBuggyEarlierVersions() throws IOException { String tarFile = System.getProperty("test.cache.data", "build/test/cache") + "/" + HADOOP_23_BROKEN_APPEND_TGZ; - String testDir = PathUtils.getTestDirName(getClass()); + String testDir = System.getProperty("test.build.data", "build/test/data"); File dfsDir = new File(testDir, "image-with-buggy-append"); if (dfsDir.exists() && !FileUtil.fullyDelete(dfsDir)) { throw new IOException("Could not delete dfs directory '" + dfsDir + "'"); diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCorruption.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCorruption.java index 6531fe7..458880a 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCorruption.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCorruption.java @@ -43,7 +43,6 @@ import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration; -import org.apache.hadoop.test.PathUtils; import org.apache.log4j.Level; import org.junit.Test; @@ -96,7 +95,7 @@ public void testFileCorruption() throws Exception { @Test public void testLocalFileCorruption() throws Exception { Configuration conf = new HdfsConfiguration(); - Path file = new Path(PathUtils.getTestDirName(getClass()), "corruptFile"); + Path file = new Path(System.getProperty("test.build.data"), "corruptFile"); FileSystem fs = FileSystem.getLocal(conf); DataOutputStream dos = fs.create(file); dos.writeBytes("original bytes"); diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSServerPorts.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSServerPorts.java index 59d1615..036252d 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSServerPorts.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSServerPorts.java @@ -36,7 +36,6 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; import org.apache.hadoop.net.DNS; -import org.apache.hadoop.test.PathUtils; import org.junit.Test; /** @@ -54,9 +53,6 @@ // reset default 0.0.0.0 addresses in order to avoid IPv6 problem static final String THIS_HOST = getFullHostName() + ":0"; - - private static final File TEST_DATA_DIR = PathUtils.getTestDir(TestHDFSServerPorts.class); - static { DefaultMetricsSystem.setMiniClusterMode(true); } @@ -85,6 +81,13 @@ public static String getFullHostName() { } } + /** + * Get base directory these tests should run in. + */ + private String getTestingDir() { + return System.getProperty("test.build.data", "build/test/data"); + } + public NameNode startNameNode() throws IOException { return startNameNode(false); } @@ -92,7 +95,8 @@ public NameNode startNameNode() throws IOException { * Start the namenode. */ public NameNode startNameNode(boolean withService) throws IOException { - hdfsDir = new File(TEST_DATA_DIR, "dfs"); + String dataDir = getTestingDir(); + hdfsDir = new File(dataDir, "dfs"); if ( hdfsDir.exists() && !FileUtil.fullyDelete(hdfsDir) ) { throw new IOException("Could not delete hdfs directory '" + hdfsDir + "'"); } @@ -115,8 +119,9 @@ public NameNode startNameNode(boolean withService) throws IOException { * Start the BackupNode */ public BackupNode startBackupNode(Configuration conf) throws IOException { + String dataDir = getTestingDir(); // Set up testing environment directories - hdfsDir = new File(TEST_DATA_DIR, "backupNode"); + hdfsDir = new File(dataDir, "backupNode"); if ( hdfsDir.exists() && !FileUtil.fullyDelete(hdfsDir) ) { throw new IOException("Could not delete hdfs directory '" + hdfsDir + "'"); } @@ -145,7 +150,8 @@ public BackupNode startBackupNode(Configuration conf) throws IOException { */ public DataNode startDataNode(int index, Configuration config) throws IOException { - File dataNodeDir = new File(TEST_DATA_DIR, "data-" + index); + String dataDir = getTestingDir(); + File dataNodeDir = new File(dataDir, "data-" + index); config.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, dataNodeDir.getPath()); String[] args = new String[] {}; diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHftpURLTimeouts.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHftpURLTimeouts.java index 56bd21e..d9a22c1 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHftpURLTimeouts.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHftpURLTimeouts.java @@ -33,11 +33,16 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.hdfs.web.URLConnectionFactory; +import org.apache.hadoop.hdfs.web.URLUtils; +import org.junit.BeforeClass; import org.junit.Test; public class TestHftpURLTimeouts { - + @BeforeClass + public static void setup() { + URLUtils.SOCKET_TIMEOUT = 5; + } + @Test public void testHftpSocketTimeout() throws Exception { Configuration conf = new Configuration(); @@ -46,11 +51,9 @@ public void testHftpSocketTimeout() throws Exception { InetAddress.getByName(null).getHostAddress(), socket.getLocalPort(), null, null, null); + boolean timedout = false; HftpFileSystem fs = (HftpFileSystem)FileSystem.get(uri, conf); - fs.connectionFactory = new URLConnectionFactory(5); - - boolean timedout = false; try { HttpURLConnection conn = fs.openConnection("/", ""); timedout = false; @@ -66,7 +69,6 @@ public void testHftpSocketTimeout() throws Exception { assertTrue("read timedout", timedout); assertTrue("connect timedout", checkConnectTimeout(fs, false)); } finally { - fs.connectionFactory = URLConnectionFactory.DEFAULT_CONNECTION_FACTORY; fs.close(); } } @@ -82,8 +84,6 @@ public void testHsftpSocketTimeout() throws Exception { boolean timedout = false; HsftpFileSystem fs = (HsftpFileSystem)FileSystem.get(uri, conf); - fs.connectionFactory = new URLConnectionFactory(5); - try { HttpURLConnection conn = null; timedout = false; @@ -100,7 +100,6 @@ public void testHsftpSocketTimeout() throws Exception { assertTrue("ssl read connect timedout", timedout); assertTrue("connect timedout", checkConnectTimeout(fs, true)); } finally { - fs.connectionFactory = URLConnectionFactory.DEFAULT_CONNECTION_FACTORY; fs.close(); } } @@ -122,7 +121,7 @@ private boolean checkConnectTimeout(HftpFileSystem fs, boolean ignoreReadTimeout // https will get a read timeout due to SSL negotiation, but // a normal http will not, so need to ignore SSL read timeouts // until a connect timeout occurs - if (!(ignoreReadTimeout && "Read timed out".equals(message))) { + if (!(ignoreReadTimeout && message.equals("Read timed out"))) { timedout = true; assertEquals("connect timed out", message); } diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMiniDFSCluster.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMiniDFSCluster.java index 84678da..1400f07 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMiniDFSCluster.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMiniDFSCluster.java @@ -25,8 +25,8 @@ import java.io.File; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hdfs.protocol.HdfsConstants; -import org.apache.hadoop.test.PathUtils; +import org.apache.hadoop.hdfs.protocol.FSConstants; +import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -43,10 +43,20 @@ private static final String CLUSTER_3 = "cluster3"; private static final String CLUSTER_4 = "cluster4"; private static final String CLUSTER_5 = "cluster5"; - protected File testDataPath; + protected String testDataPath; + protected File testDataDir; @Before public void setUp() { - testDataPath = new File(PathUtils.getTestDir(getClass()), "miniclusters"); + testDataPath = System.getProperty(MiniDFSCluster.PROP_TEST_BUILD_DATA, + "build/test/data"); + testDataDir = new File(new File(testDataPath).getParentFile(), + "miniclusters"); + + + } + @After + public void tearDown() { + System.setProperty(MiniDFSCluster.PROP_TEST_BUILD_DATA, testDataPath); } /** @@ -110,7 +120,7 @@ public void testIsClusterUpAfterShutdown() throws Throwable { MiniDFSCluster cluster4 = new MiniDFSCluster.Builder(conf).build(); try { DistributedFileSystem dfs = (DistributedFileSystem) cluster4.getFileSystem(); - dfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER); + dfs.setSafeMode(FSConstants.SafeModeAction.SAFEMODE_ENTER); cluster4.shutdown(); } finally { while(cluster4.isClusterUp()){ diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPersistBlocks.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPersistBlocks.java index 424cc77..5077a6d 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPersistBlocks.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPersistBlocks.java @@ -43,7 +43,6 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.test.GenericTestUtils; -import org.apache.hadoop.test.PathUtils; import org.apache.log4j.Level; import org.junit.Test; @@ -323,7 +322,7 @@ public void testEarlierVersionEditLog() throws Exception { String tarFile = System.getProperty("test.cache.data", "build/test/cache") + "/" + HADOOP_1_0_MULTIBLOCK_TGZ; - String testDir = PathUtils.getTestDirName(getClass()); + String testDir = System.getProperty("test.build.data", "build/test/data"); File dfsDir = new File(testDir, "image-1.0"); if (dfsDir.exists() && !FileUtil.fullyDelete(dfsDir)) { throw new IOException("Could not delete dfs directory '" + dfsDir + "'"); diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestNNWithQJM.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestNNWithQJM.java index a7eed46..cfe463c 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestNNWithQJM.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestNNWithQJM.java @@ -174,7 +174,6 @@ public void testNewNamenodeTakesOverWriter() throws Exception { public void testMismatchedNNIsRejected() throws Exception { conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, MiniDFSCluster.getBaseDirectory() + "/TestNNWithQJM/image"); - String defaultEditsDir = conf.get(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY); conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY, mjc.getQuorumJournalURI("myjournal").toString()); @@ -188,7 +187,7 @@ public void testMismatchedNNIsRejected() throws Exception { // Reformat just the on-disk portion Configuration onDiskOnly = new Configuration(conf); - onDiskOnly.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY, defaultEditsDir); + onDiskOnly.unset(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY); NameNode.format(onDiskOnly); // Start the NN - should fail because the JNs are still formatted diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java index 79ca6ca..e6e1404 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java @@ -46,7 +46,6 @@ import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.MetricsAsserts; -import org.apache.hadoop.test.PathUtils; import org.apache.hadoop.util.Shell; import org.junit.After; import org.junit.Before; @@ -62,13 +61,13 @@ private static final NamespaceInfo FAKE_NSINFO = new NamespaceInfo( 12345, "mycluster", "my-bp", 0L); - private static File TEST_BUILD_DATA = PathUtils.getTestDir(TestJournalNode.class); - private JournalNode jn; private Journal journal; private Configuration conf = new Configuration(); private IPCLoggerChannel ch; private String journalId; + private File TEST_BUILD_DATA = + new File(System.getProperty("test.build.data", "build/test/data")); static { // Avoid an error when we double-initialize JvmMetrics diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java index 972a785..7978596 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java @@ -22,6 +22,7 @@ import static org.junit.Assert.assertTrue; import java.io.File; +import java.io.IOException; import java.util.Collection; import org.apache.hadoop.conf.Configuration; diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java index 5f15d0e..ba6c373 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java @@ -53,7 +53,6 @@ import org.apache.hadoop.hdfs.server.namenode.Namesystem; import org.apache.hadoop.net.NetworkTopology; import org.apache.hadoop.net.Node; -import org.apache.hadoop.test.PathUtils; import org.apache.hadoop.util.Time; import org.apache.log4j.Level; import org.apache.log4j.Logger; @@ -93,7 +92,8 @@ public static void setupCluster() throws Exception { FileSystem.setDefaultUri(conf, "hdfs://localhost:0"); conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "0.0.0.0:0"); - File baseDir = PathUtils.getTestDir(TestReplicationPolicy.class); + File baseDir = new File(System.getProperty( + "test.build.data", "build/test/data"), "dfs/"); conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, new File(baseDir, "name").getPath()); diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicyWithNodeGroup.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicyWithNodeGroup.java index c453f198..032c2c0 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicyWithNodeGroup.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicyWithNodeGroup.java @@ -22,6 +22,7 @@ import static org.junit.Assert.assertTrue; import java.io.File; +import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; @@ -40,9 +41,6 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.net.NetworkTopology; import org.apache.hadoop.net.Node; -import org.apache.hadoop.test.PathUtils; -import org.junit.After; -import org.junit.Before; import org.junit.Test; public class TestReplicationPolicyWithNodeGroup { @@ -50,10 +48,10 @@ private static final int NUM_OF_DATANODES = 8; private static final int NUM_OF_DATANODES_BOUNDARY = 6; private static final int NUM_OF_DATANODES_MORE_TARGETS = 12; - private final Configuration CONF = new HdfsConfiguration(); - private NetworkTopology cluster; - private NameNode namenode; - private BlockPlacementPolicy replicator; + private static final Configuration CONF = new HdfsConfiguration(); + private static final NetworkTopology cluster; + private static final NameNode namenode; + private static final BlockPlacementPolicy replicator; private static final String filename = "/dummyfile.txt"; private final static DatanodeDescriptor dataNodes[] = new DatanodeDescriptor[] { @@ -96,23 +94,27 @@ private final static DatanodeDescriptor NODE = new DatanodeDescriptor(DFSTestUtil.getDatanodeDescriptor("9.9.9.9", "/d2/r4/n7")); - @Before - public void setUp() throws Exception { - FileSystem.setDefaultUri(CONF, "hdfs://localhost:0"); - CONF.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "0.0.0.0:0"); - // Set properties to make HDFS aware of NodeGroup. - CONF.set("dfs.block.replicator.classname", - "org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyWithNodeGroup"); - CONF.set(CommonConfigurationKeysPublic.NET_TOPOLOGY_IMPL_KEY, - "org.apache.hadoop.net.NetworkTopologyWithNodeGroup"); - - File baseDir = PathUtils.getTestDir(TestReplicationPolicyWithNodeGroup.class); - - CONF.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, - new File(baseDir, "name").getPath()); - - DFSTestUtil.formatNameNode(CONF); - namenode = new NameNode(CONF); + static { + try { + FileSystem.setDefaultUri(CONF, "hdfs://localhost:0"); + CONF.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "0.0.0.0:0"); + // Set properties to make HDFS aware of NodeGroup. + CONF.set("dfs.block.replicator.classname", + "org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyWithNodeGroup"); + CONF.set(CommonConfigurationKeysPublic.NET_TOPOLOGY_IMPL_KEY, + "org.apache.hadoop.net.NetworkTopologyWithNodeGroup"); + + File baseDir = new File(System.getProperty( + "test.build.data", "build/test/data"), "dfs/"); + CONF.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, + new File(baseDir, "name").getPath()); + + DFSTestUtil.formatNameNode(CONF); + namenode = new NameNode(CONF); + } catch (IOException e) { + e.printStackTrace(); + throw (RuntimeException)new RuntimeException().initCause(e); + } final BlockManager bm = namenode.getNamesystem().getBlockManager(); replicator = bm.getBlockPlacementPolicy(); cluster = bm.getDatanodeManager().getNetworkTopology(); @@ -123,11 +125,6 @@ public void setUp() throws Exception { setupDataNodeCapacity(); } - @After - public void tearDown() throws Exception { - namenode.stop(); - } - private static void setupDataNodeCapacity() { for(int i=0; i fsImageDirs = new ArrayList(); ArrayList editsDirs = new ArrayList(); File filePath = - new File(PathUtils.getTestDir(getClass()), "storageDirToCheck"); + new File(System.getProperty("test.build.data","/tmp"), "storageDirToCheck"); assertTrue("Couldn't create directory storageDirToCheck", filePath.exists() || filePath.mkdirs()); fsImageDirs.add(filePath.toURI()); @@ -1912,11 +1911,9 @@ public void testReformatNNBetweenCheckpoints() throws IOException { } // Start a new NN with the same host/port. - cluster = new MiniDFSCluster.Builder(conf) - .numDataNodes(0) - .nameNodePort(origPort) - .nameNodeHttpPort(origHttpPort) - .format(true).build(); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0) + .nameNodePort(origPort).nameNodeHttpPort(origHttpPort).format(true) + .build(); try { secondary.doCheckpoint(); @@ -2138,8 +2135,7 @@ public void testCheckpointTriggerOnTxnCount() throws Exception { conf.setInt(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_CHECK_PERIOD_KEY, 1); try { - cluster = new MiniDFSCluster.Builder(conf) - .numDataNodes(0) + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0) .format(true).build(); FileSystem fs = cluster.getFileSystem(); secondary = startSecondaryNameNode(conf); diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestClusterId.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestClusterId.java index 7c23dd5..4330317 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestClusterId.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestClusterId.java @@ -42,7 +42,6 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption; import org.apache.hadoop.hdfs.server.common.Storage; import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory; -import org.apache.hadoop.test.PathUtils; import org.apache.hadoop.util.ExitUtil; import org.apache.hadoop.util.ExitUtil.ExitException; import org.junit.After; @@ -73,7 +72,7 @@ private String getClusterId(Configuration config) throws IOException { public void setUp() throws IOException { ExitUtil.disableSystemExit(); - String baseDir = PathUtils.getTestDirName(getClass()); + String baseDir = System.getProperty("test.build.data", "build/test/data"); hdfsDir = new File(baseDir, "dfs/name"); if (hdfsDir.exists() && !FileUtil.fullyDelete(hdfsDir)) { diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java index 1be4a22..f83a531 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java @@ -69,7 +69,6 @@ import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.test.GenericTestUtils; -import org.apache.hadoop.test.PathUtils; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.Time; import org.apache.log4j.Level; @@ -97,8 +96,9 @@ static final int NUM_TRANSACTIONS = 100; static final int NUM_THREADS = 100; - static final File TEST_DIR = PathUtils.getTestDir(TestEditLog.class); - + static final File TEST_DIR = new File( + System.getProperty("test.build.data","build/test/data")); + /** An edits log with 3 edits from 0.20 - the result of * a fresh namesystem followed by hadoop fs -touchz /myfile */ static final byte[] HADOOP20_SOME_EDITS = @@ -569,7 +569,6 @@ public void testEditChecksum() throws Exception { fail("should not be able to start"); } catch (IOException e) { // expected - assertNotNull("Cause of exception should be ChecksumException", e.getCause()); assertEquals("Cause of exception should be ChecksumException", ChecksumException.class, e.getCause().getClass()); } diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileOutputStream.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileOutputStream.java index e230d5a..a6e170d 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileOutputStream.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileOutputStream.java @@ -24,7 +24,6 @@ import java.io.File; import java.io.IOException; -import org.apache.hadoop.test.PathUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.util.StringUtils; import org.junit.After; @@ -35,7 +34,8 @@ * Test the EditLogFileOutputStream */ public class TestEditLogFileOutputStream { - private final static File TEST_DIR = PathUtils.getTestDir(TestEditLogFileOutputStream.class); + private final static File TEST_DIR = + new File(System.getProperty("test.build.data", "/tmp")); private static final File TEST_EDITS = new File(TEST_DIR, "testEditLogFileOutput.log"); final static int MIN_PREALLOCATION_LENGTH = diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogJournalFailures.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogJournalFailures.java index 610a4a2..486b17c 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogJournalFailures.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogJournalFailures.java @@ -38,7 +38,6 @@ import org.apache.hadoop.hdfs.server.namenode.JournalSet.JournalAndStream; import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.test.GenericTestUtils; -import org.apache.hadoop.test.PathUtils; import org.apache.hadoop.util.ExitUtil.ExitException; import org.junit.After; import org.junit.Before; @@ -193,7 +192,8 @@ public void testMultipleRedundantFailedEditsDirOnSetReadyToFlush() Configuration conf = new HdfsConfiguration(); String[] nameDirs = new String[4]; for (int i = 0; i < nameDirs.length; i++) { - File nameDir = new File(PathUtils.getTestDir(getClass()), "name-dir" + i); + File nameDir = new File(System.getProperty("test.build.data"), + "name-dir" + i); nameDir.mkdirs(); nameDirs[i] = nameDir.getAbsolutePath(); } diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java index 4db7e6a..dd637a9 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java @@ -47,7 +47,6 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLogLoader.EditLogValidation; import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType; import org.apache.hadoop.io.IOUtils; -import org.apache.hadoop.test.PathUtils; import org.apache.log4j.Level; import org.junit.Test; @@ -61,7 +60,8 @@ ((Log4JLogger)FSEditLogLoader.LOG).getLogger().setLevel(Level.ALL); } - private static final File TEST_DIR = PathUtils.getTestDir(TestFSEditLogLoader.class); + private static final File TEST_DIR = new File( + System.getProperty("test.build.data","build/test/data")); private static final int NUM_DATA_NODES = 0; diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java index 9aaeb74..754e569 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java @@ -91,7 +91,7 @@ */ public class TestFsck { static final String auditLogFile = System.getProperty("test.build.dir", - "build/test") + "/TestFsck-audit.log"; + "build/test") + "/audit.log"; // Pattern for: // allowed=true ugi=name ip=/address cmd=FSCK src=/ dst=null perm=null @@ -159,8 +159,7 @@ public void testFsck() throws Exception { cluster.shutdown(); // restart the cluster; bring up namenode but not the data nodes - cluster = new MiniDFSCluster.Builder(conf) - .numDataNodes(0).format(false).build(); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).format(false).build(); outStr = runFsck(conf, 1, true, "/"); // expect the result is corrupt assertTrue(outStr.contains(NamenodeFsck.CORRUPT_STATUS)); diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java index 5c11c1b..4e4914b 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java @@ -37,7 +37,6 @@ import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType; -import org.apache.hadoop.test.PathUtils; import org.junit.Before; import org.junit.Test; @@ -61,7 +60,7 @@ short replication = 3; private File base_dir = new File( - PathUtils.getTestDir(TestNameEditsConfigs.class), "dfs"); + System.getProperty("test.build.data", "build/test/data"), "dfs/"); @Before public void setUp() throws IOException { @@ -69,7 +68,7 @@ public void setUp() throws IOException { throw new IOException("Cannot remove directory " + base_dir); } } - + void checkImageAndEditsFilesExistence(File dir, boolean shouldHaveImages, boolean shouldHaveEdits) diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRecovery.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRecovery.java index 30d3e71..42f3f57 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRecovery.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRecovery.java @@ -46,7 +46,6 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.OpInstanceCache; import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType; import org.apache.hadoop.io.IOUtils; -import org.apache.hadoop.test.PathUtils; import org.apache.hadoop.util.StringUtils; import org.junit.Test; @@ -58,7 +57,8 @@ public class TestNameNodeRecovery { private static final Log LOG = LogFactory.getLog(TestNameNodeRecovery.class); private static StartupOption recoverStartOpt = StartupOption.RECOVER; - private static final File TEST_DIR = PathUtils.getTestDir(TestNameNodeRecovery.class); + private static final File TEST_DIR = new File( + System.getProperty("test.build.data","build/test/data")); static { recoverStartOpt.setForce(MetaRecoveryContext.FORCE_ALL); diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeResourceChecker.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeResourceChecker.java index 2012b6a..e73d71a 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeResourceChecker.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeResourceChecker.java @@ -33,14 +33,12 @@ import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem.NameNodeResourceMonitor; import org.apache.hadoop.hdfs.server.namenode.NameNodeResourceChecker.CheckedVolume; -import org.apache.hadoop.test.PathUtils; import org.apache.hadoop.util.Time; import org.junit.Before; import org.junit.Test; import org.mockito.Mockito; public class TestNameNodeResourceChecker { - private final static File BASE_DIR = PathUtils.getTestDir(TestNameNodeResourceChecker.class); private Configuration conf; private File baseDir; private File nameDir; @@ -48,7 +46,8 @@ @Before public void setUp () throws IOException { conf = new Configuration(); - nameDir = new File(BASE_DIR, "resource-check-name-dir"); + baseDir = new File(System.getProperty("test.build.data")); + nameDir = new File(baseDir, "resource-check-name-dir"); nameDir.mkdirs(); conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY, nameDir.getAbsolutePath()); } @@ -142,8 +141,8 @@ public void testCheckThatNameNodeResourceMonitorIsRunning() @Test public void testChecking2NameDirsOnOneVolume() throws IOException { Configuration conf = new Configuration(); - File nameDir1 = new File(BASE_DIR, "name-dir1"); - File nameDir2 = new File(BASE_DIR, "name-dir2"); + File nameDir1 = new File(System.getProperty("test.build.data"), "name-dir1"); + File nameDir2 = new File(System.getProperty("test.build.data"), "name-dir2"); nameDir1.mkdirs(); nameDir2.mkdirs(); conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY, @@ -163,7 +162,7 @@ public void testChecking2NameDirsOnOneVolume() throws IOException { @Test public void testCheckingExtraVolumes() throws IOException { Configuration conf = new Configuration(); - File nameDir = new File(BASE_DIR, "name-dir"); + File nameDir = new File(System.getProperty("test.build.data"), "name-dir"); nameDir.mkdirs(); conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY, nameDir.getAbsolutePath()); conf.set(DFSConfigKeys.DFS_NAMENODE_CHECKED_VOLUMES_KEY, nameDir.getAbsolutePath()); @@ -183,8 +182,8 @@ public void testCheckingExtraVolumes() throws IOException { @Test public void testLowResourceVolumePolicy() throws IOException, URISyntaxException { Configuration conf = new Configuration(); - File nameDir1 = new File(BASE_DIR, "name-dir1"); - File nameDir2 = new File(BASE_DIR, "name-dir2"); + File nameDir1 = new File(System.getProperty("test.build.data"), "name-dir1"); + File nameDir2 = new File(System.getProperty("test.build.data"), "name-dir2"); nameDir1.mkdirs(); nameDir2.mkdirs(); diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java index 86323ff..6d6dbdf 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java @@ -57,7 +57,6 @@ import org.apache.hadoop.hdfs.util.MD5FileUtils; import org.apache.hadoop.io.MD5Hash; import org.apache.hadoop.test.GenericTestUtils; -import org.apache.hadoop.test.PathUtils; import org.apache.hadoop.util.StringUtils; import org.apache.log4j.Logger; import org.junit.After; @@ -405,7 +404,8 @@ public void testCompression() throws IOException { Configuration conf = new Configuration(); FileSystem.setDefaultUri(conf, "hdfs://localhost:0"); conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "127.0.0.1:0"); - File base_dir = new File(PathUtils.getTestDir(getClass()), "dfs/"); + File base_dir = new File(System.getProperty( + "test.build.data", "build/test/data"), "dfs/"); conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, new File(base_dir, "name").getPath()); conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, false); diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStorageRestore.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStorageRestore.java index 1a612e8..905e368 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStorageRestore.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStorageRestore.java @@ -391,8 +391,7 @@ public void testStorageRestoreFailure() throws Exception { (new File(path3, "current").getAbsolutePath()) : path3.toString(); try { - cluster = new MiniDFSCluster.Builder(config) - .numDataNodes(0) + cluster = new MiniDFSCluster.Builder(config).numDataNodes(0) .manageNameDfsDirs(false).build(); cluster.waitActive(); diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTransferFsImage.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTransferFsImage.java index 5a178d1..cba634f 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTransferFsImage.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTransferFsImage.java @@ -39,7 +39,6 @@ import org.apache.hadoop.http.HttpServer; import org.apache.hadoop.http.HttpServerFunctionalTest; import org.apache.hadoop.net.NetUtils; -import org.apache.hadoop.test.PathUtils; import org.apache.hadoop.util.StringUtils; import org.junit.Test; import org.mockito.Mockito; @@ -49,7 +48,8 @@ public class TestTransferFsImage { - private static final File TEST_DIR = PathUtils.getTestDir(TestTransferFsImage.class); + private static final File TEST_DIR = new File( + System.getProperty("test.build.data","build/test/data")); /** * Regression test for HDFS-1997. Test that, if an exception diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestBootstrapStandby.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestBootstrapStandby.java index 502c9de..678e038 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestBootstrapStandby.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestBootstrapStandby.java @@ -56,8 +56,8 @@ public void setupCluster() throws IOException { MiniDFSNNTopology topology = new MiniDFSNNTopology() .addNameservice(new MiniDFSNNTopology.NSConf("ns1") - .addNN(new MiniDFSNNTopology.NNConf("nn1").setHttpPort(20001)) - .addNN(new MiniDFSNNTopology.NNConf("nn2").setHttpPort(20002))); + .addNN(new MiniDFSNNTopology.NNConf("nn1").setHttpPort(10001)) + .addNN(new MiniDFSNNTopology.NNConf("nn2").setHttpPort(10002))); cluster = new MiniDFSCluster.Builder(conf) .nnTopology(topology) diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDFSZKFailoverController.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDFSZKFailoverController.java index 1897265..ce005b1 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDFSZKFailoverController.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDFSZKFailoverController.java @@ -70,13 +70,13 @@ public void setup() throws Exception { CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY, 0); - conf.setInt(DFSConfigKeys.DFS_HA_ZKFC_PORT_KEY + ".ns1.nn1", 10023); - conf.setInt(DFSConfigKeys.DFS_HA_ZKFC_PORT_KEY + ".ns1.nn2", 10024); + conf.setInt(DFSConfigKeys.DFS_HA_ZKFC_PORT_KEY + ".ns1.nn1", 10003); + conf.setInt(DFSConfigKeys.DFS_HA_ZKFC_PORT_KEY + ".ns1.nn2", 10004); MiniDFSNNTopology topology = new MiniDFSNNTopology() .addNameservice(new MiniDFSNNTopology.NSConf("ns1") - .addNN(new MiniDFSNNTopology.NNConf("nn1").setIpcPort(10021)) - .addNN(new MiniDFSNNTopology.NNConf("nn2").setIpcPort(10022))); + .addNN(new MiniDFSNNTopology.NNConf("nn1").setIpcPort(10001)) + .addNN(new MiniDFSNNTopology.NNConf("nn2").setIpcPort(10002))); cluster = new MiniDFSCluster.Builder(conf) .nnTopology(topology) .numDataNodes(0) diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java index 8c61c92..8675fa3 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java @@ -124,8 +124,8 @@ private static void testStandbyTriggersLogRolls(int activeIndex) // Have to specify IPC ports so the NNs can talk to each other. MiniDFSNNTopology topology = new MiniDFSNNTopology() .addNameservice(new MiniDFSNNTopology.NSConf("ns1") - .addNN(new MiniDFSNNTopology.NNConf("nn1").setIpcPort(10031)) - .addNN(new MiniDFSNNTopology.NNConf("nn2").setIpcPort(10032))); + .addNN(new MiniDFSNNTopology.NNConf("nn1").setIpcPort(10001)) + .addNN(new MiniDFSNNTopology.NNConf("nn2").setIpcPort(10002))); MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf) .nnTopology(topology) diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureToReadEdits.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureToReadEdits.java index 5ec7f7e..37c0b16 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureToReadEdits.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureToReadEdits.java @@ -76,8 +76,8 @@ public void setUpCluster() throws Exception { MiniDFSNNTopology topology = new MiniDFSNNTopology() .addNameservice(new MiniDFSNNTopology.NSConf("ns1") - .addNN(new MiniDFSNNTopology.NNConf("nn1").setHttpPort(10041)) - .addNN(new MiniDFSNNTopology.NNConf("nn2").setHttpPort(10042))); + .addNN(new MiniDFSNNTopology.NNConf("nn1").setHttpPort(10001)) + .addNN(new MiniDFSNNTopology.NNConf("nn2").setHttpPort(10002))); cluster = new MiniDFSCluster.Builder(conf) .nnTopology(topology) .numDataNodes(0) diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAFsck.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAFsck.java index 4f848dc..49d8959 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAFsck.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAFsck.java @@ -52,8 +52,8 @@ public void testHaFsck() throws Exception { // need some HTTP ports MiniDFSNNTopology topology = new MiniDFSNNTopology() .addNameservice(new MiniDFSNNTopology.NSConf("ha-nn-uri-0") - .addNN(new MiniDFSNNTopology.NNConf("nn1").setHttpPort(10051)) - .addNN(new MiniDFSNNTopology.NNConf("nn2").setHttpPort(10052))); + .addNN(new MiniDFSNNTopology.NNConf("nn1").setHttpPort(10001)) + .addNN(new MiniDFSNNTopology.NNConf("nn2").setHttpPort(10002))); MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf) .nnTopology(topology) diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java index 3ff5d54..dff2874 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java @@ -89,8 +89,8 @@ public void setupCluster() throws Exception { MiniDFSNNTopology topology = new MiniDFSNNTopology() .addNameservice(new MiniDFSNNTopology.NSConf("ns1") - .addNN(new MiniDFSNNTopology.NNConf("nn1").setHttpPort(10061)) - .addNN(new MiniDFSNNTopology.NNConf("nn2").setHttpPort(10062))); + .addNN(new MiniDFSNNTopology.NNConf("nn1").setHttpPort(10001)) + .addNN(new MiniDFSNNTopology.NNConf("nn2").setHttpPort(10002))); cluster = new MiniDFSCluster.Builder(conf) .nnTopology(topology) diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TestOfflineEditsViewer.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TestOfflineEditsViewer.java index e6c9a3f..1d189a1 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TestOfflineEditsViewer.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TestOfflineEditsViewer.java @@ -35,7 +35,6 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes; import org.apache.hadoop.hdfs.server.namenode.OfflineEditsViewerHelper; import org.apache.hadoop.hdfs.tools.offlineEditsViewer.OfflineEditsViewer.Flags; -import org.apache.hadoop.test.PathUtils; import org.junit.Before; import org.junit.Test; @@ -54,7 +53,7 @@ } private static String buildDir = - PathUtils.getTestDirName(TestOfflineEditsViewer.class); + System.getProperty("test.build.data", "build/test/data"); private static String cacheDir = System.getProperty("test.cache.data", "build/test/cache"); diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestDelimitedImageVisitor.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestDelimitedImageVisitor.java index c7d3b31..a5501d9 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestDelimitedImageVisitor.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestDelimitedImageVisitor.java @@ -27,7 +27,6 @@ import java.io.IOException; import org.apache.hadoop.hdfs.tools.offlineImageViewer.ImageVisitor.ImageElement; -import org.apache.hadoop.test.PathUtils; import org.junit.Test; /** @@ -35,7 +34,7 @@ * on predetermined inputs */ public class TestDelimitedImageVisitor { - private static String ROOT = PathUtils.getTestDirName(TestDelimitedImageVisitor.class); + private static String ROOT = System.getProperty("test.build.data","/tmp"); private static final String delim = "--"; // Record an element in the visitor and build the expected line in the output diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java index 11aa3b8..50e8164 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java @@ -53,7 +53,6 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction; import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil; import org.apache.hadoop.security.token.Token; -import org.apache.hadoop.test.PathUtils; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; @@ -89,7 +88,8 @@ final static HashMap writtenFiles = new HashMap(); - private static String ROOT = PathUtils.getTestDirName(TestOfflineImageViewer.class); + private static String ROOT = System.getProperty("test.build.data", + "build/test/data"); // Create a populated namespace for later testing. Save its contents to a // data structure and store its fsimage location. diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestAtomicFileOutputStream.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestAtomicFileOutputStream.java index 2a9465a..ebbb4e2 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestAtomicFileOutputStream.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestAtomicFileOutputStream.java @@ -30,7 +30,6 @@ import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.hdfs.DFSTestUtil; -import org.apache.hadoop.test.PathUtils; import org.junit.Before; import org.junit.Test; @@ -41,7 +40,10 @@ private static final String TEST_STRING = "hello world"; private static final String TEST_STRING_2 = "goodbye world"; - private static File TEST_DIR = PathUtils.getTestDir(TestAtomicFileOutputStream.class); + private static File BASE_DIR = new File( + System.getProperty("test.build.data", "build/test/data")); + private static File TEST_DIR = new File(BASE_DIR, + TestAtomicFileOutputStream.class.getName()); private static File DST_FILE = new File(TEST_DIR, "test.txt"); diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestMD5FileUtils.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestMD5FileUtils.java index 35fa46d..6f5b161 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestMD5FileUtils.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestMD5FileUtils.java @@ -29,12 +29,14 @@ import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.io.MD5Hash; -import org.apache.hadoop.test.PathUtils; import org.junit.Before; import org.junit.Test; public class TestMD5FileUtils { - private static final File TEST_DIR = PathUtils.getTestDir(TestMD5FileUtils.class); + private static final File TEST_DIR_ROOT = new File( + System.getProperty("test.build.data","build/test/data")); + private static final File TEST_DIR = new File(TEST_DIR_ROOT, + "TestMD5FileUtils"); private static final File TEST_FILE = new File(TEST_DIR, "testMd5File.dat"); diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsTimeouts.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsTimeouts.java index 7a007a0..2071f6f 100644 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsTimeouts.java +++ hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsTimeouts.java @@ -58,6 +58,7 @@ private static final int CLIENTS_TO_CONSUME_BACKLOG = 100; private static final int CONNECTION_BACKLOG = 1; + private static final int INITIAL_SOCKET_TIMEOUT = URLUtils.SOCKET_TIMEOUT; private static final int SHORT_SOCKET_TIMEOUT = 5; private static final int TEST_TIMEOUT = 10000; @@ -66,22 +67,20 @@ private InetSocketAddress nnHttpAddress; private ServerSocket serverSocket; private Thread serverThread; - private URLConnectionFactory connectionFactory = new URLConnectionFactory(SHORT_SOCKET_TIMEOUT); @Before public void setUp() throws Exception { + URLUtils.SOCKET_TIMEOUT = SHORT_SOCKET_TIMEOUT; Configuration conf = WebHdfsTestUtil.createConf(); nnHttpAddress = NameNode.getHttpAddress(conf); serverSocket = new ServerSocket(nnHttpAddress.getPort(), CONNECTION_BACKLOG); fs = WebHdfsTestUtil.getWebHdfsFileSystem(conf); - fs.connectionFactory = connectionFactory; clients = new ArrayList(); serverThread = null; } @After public void tearDown() throws Exception { - fs.connectionFactory = URLConnectionFactory.DEFAULT_CONNECTION_FACTORY; IOUtils.cleanup(LOG, clients.toArray(new SocketChannel[clients.size()])); IOUtils.cleanup(LOG, fs); if (serverSocket != null) { @@ -241,7 +240,7 @@ public void testTwoStepWriteReadTimeout() throws Exception { */ private void startSingleTemporaryRedirectResponseThread( final boolean consumeConnectionBacklog) { - fs.connectionFactory = URLConnectionFactory.DEFAULT_CONNECTION_FACTORY; + URLUtils.SOCKET_TIMEOUT = INITIAL_SOCKET_TIMEOUT; serverThread = new Thread() { @Override public void run() { @@ -255,7 +254,7 @@ public void run() { clientSocket = serverSocket.accept(); // Immediately setup conditions for subsequent connections. - fs.connectionFactory = connectionFactory; + URLUtils.SOCKET_TIMEOUT = SHORT_SOCKET_TIMEOUT; if (consumeConnectionBacklog) { consumeConnectionBacklog(); } diff --git hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/test/PathUtils.java hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/test/PathUtils.java deleted file mode 100644 index 2ee4aa1..0000000 --- hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/test/PathUtils.java +++ /dev/null @@ -1,54 +0,0 @@ -/** - * 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.test; - -import java.io.File; - -import org.apache.hadoop.fs.Path; - -public class PathUtils { - - public static Path getTestPath(Class caller) { - return getTestPath(caller, true); - } - - public static Path getTestPath(Class caller, boolean create) { - return new Path(getTestDirName(caller)); - } - - public static File getTestDir(Class caller) { - return getTestDir(caller, true); - } - - public static File getTestDir(Class caller, boolean create) { - File dir = new File(System.getProperty("test.build.data", "/tmp"), caller.getSimpleName()); - if (create) { - dir.mkdirs(); - } - return dir; - } - - public static String getTestDirName(Class caller) { - return getTestDirName(caller, true); - } - - public static String getTestDirName(Class caller, boolean create) { - return getTestDir(caller, create).getAbsolutePath(); - } - -} diff --git hadoop-project/pom.xml hadoop-project/pom.xml index bc7e802..8dee23a 100644 --- hadoop-project/pom.xml +++ hadoop-project/pom.xml @@ -713,7 +713,7 @@ org.apache.maven.plugins maven-surefire-plugin - 2.16 + 2.12.3 org.apache.maven.plugins @@ -827,7 +827,7 @@ org.apache.maven.plugins maven-surefire-plugin - false + always 900 -Xmx1024m -XX:+HeapDumpOnOutOfMemoryError @@ -1002,5 +1002,23 @@ + diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateRequestPBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateRequestPBImpl.java index bff252f..6dd156c 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateRequestPBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateRequestPBImpl.java @@ -88,7 +88,7 @@ public String toString() { return TextFormat.shortDebugString(getProto()); } - private void mergeLocalToBuilder() { + private synchronized void mergeLocalToBuilder() { if (this.ask != null) { addAsksToProto(); } @@ -140,19 +140,19 @@ public void setProgress(float progress) { } @Override - public List getAskList() { + public synchronized List getAskList() { initAsks(); return this.ask; } @Override - public void setAskList(final List resourceRequests) { + public synchronized void setAskList( + final List resourceRequests) { + maybeInitBuilder(); if(resourceRequests == null) { - return; + builder.clearAsk(); } - initAsks(); - this.ask.clear(); - this.ask.addAll(resourceRequests); + this.ask = resourceRequests; } @Override @@ -190,7 +190,7 @@ private void initAsks() { } } - private void addAsksToProto() { + private synchronized void addAsksToProto() { maybeInitBuilder(); builder.clearAsk(); if (ask == null) @@ -224,18 +224,17 @@ public void remove() { builder.addAllAsk(iterable); } @Override - public List getReleaseList() { + public synchronized List getReleaseList() { initReleases(); return this.release; } @Override - public void setReleaseList(List releaseContainers) { + public synchronized void setReleaseList(List releaseContainers) { + maybeInitBuilder(); if(releaseContainers == null) { - return; + builder.clearRelease(); } - initReleases(); - this.release.clear(); - this.release.addAll(releaseContainers); + this.release = releaseContainers; } private void initReleases() { @@ -251,7 +250,7 @@ private void initReleases() { } } - private void addReleasesToProto() { + private synchronized void addReleasesToProto() { maybeInitBuilder(); builder.clearRelease(); if (release == null) diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateResponsePBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateResponsePBImpl.java index 37d5971..627b022 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateResponsePBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateResponsePBImpl.java @@ -212,12 +212,11 @@ public synchronized void setAvailableResources(Resource limit) { @Override public synchronized void setUpdatedNodes( final List updatedNodes) { + maybeInitBuilder(); if (updatedNodes == null) { - this.updatedNodes.clear(); - return; + builder.clearUpdatedNodes(); } - this.updatedNodes = new ArrayList(updatedNodes.size()); - this.updatedNodes.addAll(updatedNodes); + this.updatedNodes = updatedNodes; } @Override @@ -229,11 +228,10 @@ public synchronized void setUpdatedNodes( @Override public synchronized void setAllocatedContainers( final List containers) { + maybeInitBuilder(); if (containers == null) - return; - // this looks like a bug because it results in append and not set - initLocalNewContainerList(); - allocatedContainers.addAll(containers); + builder.clearAllocatedContainers(); + this.allocatedContainers = containers; } //// Finished containers @@ -246,24 +244,19 @@ public synchronized void setAllocatedContainers( @Override public synchronized void setCompletedContainersStatuses( final List containers) { + maybeInitBuilder(); if (containers == null) - return; - initLocalFinishedContainerList(); - completedContainersStatuses.addAll(containers); + builder.clearCompletedContainerStatuses(); + this.completedContainersStatuses = containers; } @Override public synchronized void setNMTokens(List nmTokens) { - if (nmTokens == null || nmTokens.isEmpty()) { - if (this.nmTokens != null) { - this.nmTokens.clear(); - } + maybeInitBuilder(); + if (nmTokens == null) { builder.clearNmTokens(); - return; } - // Implementing it as an append rather than set for consistency - initLocalNewNMTokenList(); - this.nmTokens.addAll(nmTokens); + this.nmTokens = nmTokens; } @Override diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetApplicationsRequestPBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetApplicationsRequestPBImpl.java index 33f74f0..7f11303 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetApplicationsRequestPBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetApplicationsRequestPBImpl.java @@ -54,14 +54,14 @@ public GetApplicationsRequestPBImpl(GetApplicationsRequestProto proto) { viaProto = true; } - public GetApplicationsRequestProto getProto() { + public synchronized GetApplicationsRequestProto getProto() { mergeLocalToProto(); proto = viaProto ? proto : builder.build(); viaProto = true; return proto; } - private void mergeLocalToProto() { + private synchronized void mergeLocalToProto() { if (viaProto) maybeInitBuilder(); mergeLocalToBuilder(); @@ -69,7 +69,7 @@ private void mergeLocalToProto() { viaProto = true; } - private void mergeLocalToBuilder() { + private synchronized void mergeLocalToBuilder() { if (this.applicationTypes != null) { addLocalApplicationTypesToProto(); } @@ -109,7 +109,7 @@ public void remove() { } } - private void addLocalApplicationTypesToProto() { + private synchronized void addLocalApplicationTypesToProto() { maybeInitBuilder(); builder.clearApplicationTypes(); if (this.applicationTypes == null) @@ -117,7 +117,7 @@ private void addLocalApplicationTypesToProto() { builder.addAllApplicationTypes(applicationTypes); } - private void maybeInitBuilder() { + private synchronized void maybeInitBuilder() { if (viaProto || builder == null) { builder = GetApplicationsRequestProto.newBuilder(proto); } @@ -149,13 +149,13 @@ private void initApplicationStates() { } @Override - public Set getApplicationTypes() { + public synchronized Set getApplicationTypes() { initApplicationTypes(); return this.applicationTypes; } @Override - public void setApplicationTypes(Set applicationTypes) { + public synchronized void setApplicationTypes(Set applicationTypes) { maybeInitBuilder(); if (applicationTypes == null) builder.clearApplicationTypes(); @@ -163,13 +163,14 @@ public void setApplicationTypes(Set applicationTypes) { } @Override - public EnumSet getApplicationStates() { + public synchronized EnumSet getApplicationStates() { initApplicationStates(); return this.applicationStates; } @Override - public void setApplicationStates(EnumSet applicationStates) { + public synchronized void setApplicationStates( + EnumSet applicationStates) { maybeInitBuilder(); if (applicationStates == null) { builder.clearApplicationStates(); diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetApplicationsResponsePBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetApplicationsResponsePBImpl.java index 453fc89..f2c9740 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetApplicationsResponsePBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetApplicationsResponsePBImpl.java @@ -55,20 +55,21 @@ public GetApplicationsResponsePBImpl(GetApplicationsResponseProto proto) { } @Override - public List getApplicationList() { + public synchronized List getApplicationList() { initLocalApplicationsList(); return this.applicationList; } @Override - public void setApplicationList(List applications) { + public synchronized void setApplicationList( + List applications) { maybeInitBuilder(); if (applications == null) builder.clearApplications(); this.applicationList = applications; } - public GetApplicationsResponseProto getProto() { + public synchronized GetApplicationsResponseProto getProto() { mergeLocalToProto(); proto = viaProto ? proto : builder.build(); viaProto = true; @@ -95,13 +96,13 @@ public String toString() { return TextFormat.shortDebugString(getProto()); } - private void mergeLocalToBuilder() { + private synchronized void mergeLocalToBuilder() { if (this.applicationList != null) { addLocalApplicationsToProto(); } } - private void mergeLocalToProto() { + private synchronized void mergeLocalToProto() { if (viaProto) maybeInitBuilder(); mergeLocalToBuilder(); @@ -109,7 +110,7 @@ private void mergeLocalToProto() { viaProto = true; } - private void maybeInitBuilder() { + private synchronized void maybeInitBuilder() { if (viaProto || builder == null) { builder = GetApplicationsResponseProto.newBuilder(proto); } @@ -131,7 +132,7 @@ private void initLocalApplicationsList() { } } - private void addLocalApplicationsToProto() { + private synchronized void addLocalApplicationsToProto() { maybeInitBuilder(); builder.clearApplications(); if (applicationList == null) diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetClusterNodesRequestPBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetClusterNodesRequestPBImpl.java index 09c0fc7..dfc6829 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetClusterNodesRequestPBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetClusterNodesRequestPBImpl.java @@ -52,7 +52,7 @@ public GetClusterNodesRequestPBImpl(GetClusterNodesRequestProto proto) { viaProto = true; } - public GetClusterNodesRequestProto getProto() { + public synchronized GetClusterNodesRequestProto getProto() { mergeLocalToProto(); proto = viaProto ? proto : builder.build(); viaProto = true; @@ -60,22 +60,21 @@ public GetClusterNodesRequestProto getProto() { } @Override - public EnumSet getNodeStates() { + public synchronized EnumSet getNodeStates() { initNodeStates(); return this.states; } @Override - public void setNodeStates(final EnumSet states) { - initNodeStates(); - this.states.clear(); + public synchronized void setNodeStates(final EnumSet states) { + maybeInitBuilder(); if (states == null) { - return; + builder.clearNodeStates(); } - this.states.addAll(states); + this.states = states; } - private void mergeLocalToProto() { + private synchronized void mergeLocalToProto() { if (viaProto) { maybeInitBuilder(); } @@ -84,14 +83,14 @@ private void mergeLocalToProto() { viaProto = true; } - private void maybeInitBuilder() { + private synchronized void maybeInitBuilder() { if (viaProto || builder == null) { builder = GetClusterNodesRequestProto.newBuilder(proto); } viaProto = false; } - private void mergeLocalToBuilder() { + private synchronized void mergeLocalToBuilder() { if (this.states != null) { maybeInitBuilder(); builder.clearNodeStates(); diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetClusterNodesResponsePBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetClusterNodesResponsePBImpl.java index 04530e5..c39405b 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetClusterNodesResponsePBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetClusterNodesResponsePBImpl.java @@ -54,20 +54,21 @@ public GetClusterNodesResponsePBImpl(GetClusterNodesResponseProto proto) { } @Override - public List getNodeReports() { + public synchronized List getNodeReports() { initLocalNodeManagerInfosList(); return this.nodeManagerInfoList; } @Override - public void setNodeReports(List nodeManagers) { + public synchronized void setNodeReports(List nodeManagers) { + maybeInitBuilder(); if (nodeManagers == null) { builder.clearNodeReports(); } this.nodeManagerInfoList = nodeManagers; } - public GetClusterNodesResponseProto getProto() { + public synchronized GetClusterNodesResponseProto getProto() { mergeLocalToProto(); proto = viaProto ? proto : builder.build(); viaProto = true; @@ -94,13 +95,13 @@ public String toString() { return TextFormat.shortDebugString(getProto()); } - private void mergeLocalToBuilder() { + private synchronized void mergeLocalToBuilder() { if (this.nodeManagerInfoList != null) { addLocalNodeManagerInfosToProto(); } } - private void mergeLocalToProto() { + private synchronized void mergeLocalToProto() { if (viaProto) maybeInitBuilder(); mergeLocalToBuilder(); @@ -108,7 +109,7 @@ private void mergeLocalToProto() { viaProto = true; } - private void maybeInitBuilder() { + private synchronized void maybeInitBuilder() { if (viaProto || builder == null) { builder = GetClusterNodesResponseProto.newBuilder(proto); } @@ -130,7 +131,7 @@ private void initLocalNodeManagerInfosList() { } } - private void addLocalNodeManagerInfosToProto() { + private synchronized void addLocalNodeManagerInfosToProto() { maybeInitBuilder(); builder.clearNodeReports(); if (nodeManagerInfoList == null) diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetContainerStatusesRequestPBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetContainerStatusesRequestPBImpl.java index bbc1492..4d4bfe8 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetContainerStatusesRequestPBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetContainerStatusesRequestPBImpl.java @@ -53,7 +53,7 @@ public GetContainerStatusesRequestPBImpl( viaProto = true; } - public GetContainerStatusesRequestProto getProto() { + public synchronized GetContainerStatusesRequestProto getProto() { mergeLocalToProto(); proto = viaProto ? proto : builder.build(); viaProto = true; @@ -80,13 +80,13 @@ public String toString() { return TextFormat.shortDebugString(getProto()); } - private void mergeLocalToBuilder() { + private synchronized void mergeLocalToBuilder() { if (this.containerIds != null) { addLocalContainerIdsToProto(); } } - private void mergeLocalToProto() { + private synchronized void mergeLocalToProto() { if (viaProto) maybeInitBuilder(); mergeLocalToBuilder(); @@ -94,14 +94,14 @@ private void mergeLocalToProto() { viaProto = true; } - private void maybeInitBuilder() { + private synchronized void maybeInitBuilder() { if (viaProto || builder == null) { builder = GetContainerStatusesRequestProto.newBuilder(proto); } viaProto = false; } - private void addLocalContainerIdsToProto() { + private synchronized void addLocalContainerIdsToProto() { maybeInitBuilder(); builder.clearContainerId(); if (this.containerIds == null) @@ -126,13 +126,13 @@ private void initLocalContainerIds() { } @Override - public List getContainerIds() { + public synchronized List getContainerIds() { initLocalContainerIds(); return this.containerIds; } @Override - public void setContainerIds(List containerIds) { + public synchronized void setContainerIds(List containerIds) { maybeInitBuilder(); if (containerIds == null) builder.clearContainerId(); diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetContainerStatusesResponsePBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetContainerStatusesResponsePBImpl.java index fb8885b..6d8b8bf 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetContainerStatusesResponsePBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetContainerStatusesResponsePBImpl.java @@ -63,7 +63,7 @@ public GetContainerStatusesResponsePBImpl( viaProto = true; } - public GetContainerStatusesResponseProto getProto() { + public synchronized GetContainerStatusesResponseProto getProto() { mergeLocalToProto(); proto = viaProto ? proto : builder.build(); viaProto = true; @@ -90,7 +90,7 @@ public String toString() { return TextFormat.shortDebugString(getProto()); } - private void mergeLocalToBuilder() { + private synchronized void mergeLocalToBuilder() { if (this.containerStatuses != null) { addLocalContainerStatusesToProto(); } @@ -99,7 +99,7 @@ private void mergeLocalToBuilder() { } } - private void mergeLocalToProto() { + private synchronized void mergeLocalToProto() { if (viaProto) maybeInitBuilder(); mergeLocalToBuilder(); @@ -107,14 +107,14 @@ private void mergeLocalToProto() { viaProto = true; } - private void maybeInitBuilder() { + private synchronized void maybeInitBuilder() { if (viaProto || builder == null) { builder = GetContainerStatusesResponseProto.newBuilder(proto); } viaProto = false; } - private void addLocalContainerStatusesToProto() { + private synchronized void addLocalContainerStatusesToProto() { maybeInitBuilder(); builder.clearStatus(); if (this.containerStatuses == null) @@ -127,7 +127,7 @@ private void addLocalContainerStatusesToProto() { builder.addAllStatus(protoList); } - private void addFailedRequestsToProto() { + private synchronized void addFailedRequestsToProto() { maybeInitBuilder(); builder.clearFailedRequests(); if (this.failedRequests == null) @@ -169,13 +169,13 @@ private void initFailedRequests() { } @Override - public List getContainerStatuses() { + public synchronized List getContainerStatuses() { initLocalContainerStatuses(); return this.containerStatuses; } @Override - public void setContainerStatuses(List statuses) { + public synchronized void setContainerStatuses(List statuses) { maybeInitBuilder(); if (statuses == null) builder.clearStatus(); @@ -183,13 +183,13 @@ public void setContainerStatuses(List statuses) { } @Override - public Map getFailedRequests() { + public synchronized Map getFailedRequests() { initFailedRequests(); return this.failedRequests; } @Override - public void setFailedRequests( + public synchronized void setFailedRequests( Map failedRequests) { maybeInitBuilder(); if (failedRequests == null) diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetQueueUserAclsInfoResponsePBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetQueueUserAclsInfoResponsePBImpl.java index 8d405d9..e8898a5 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetQueueUserAclsInfoResponsePBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetQueueUserAclsInfoResponsePBImpl.java @@ -55,20 +55,22 @@ public GetQueueUserAclsInfoResponsePBImpl( } @Override - public List getUserAclsInfoList() { + public synchronized List getUserAclsInfoList() { initLocalQueueUserAclsList(); return queueUserAclsInfoList; } @Override - public void setUserAclsInfoList(List queueUserAclsList) { + public synchronized void setUserAclsInfoList( + List queueUserAclsList) { + maybeInitBuilder(); if (queueUserAclsList == null) { builder.clearQueueUserAcls(); } this.queueUserAclsInfoList = queueUserAclsList; } - public GetQueueUserAclsInfoResponseProto getProto() { + public synchronized GetQueueUserAclsInfoResponseProto getProto() { mergeLocalToProto(); proto = viaProto ? proto : builder.build(); viaProto = true; @@ -95,13 +97,13 @@ public String toString() { return TextFormat.shortDebugString(getProto()); } - private void mergeLocalToBuilder() { + private synchronized void mergeLocalToBuilder() { if (this.queueUserAclsInfoList != null) { addLocalQueueUserACLInfosToProto(); } } - private void mergeLocalToProto() { + private synchronized void mergeLocalToProto() { if (viaProto) maybeInitBuilder(); mergeLocalToBuilder(); @@ -109,7 +111,7 @@ private void mergeLocalToProto() { viaProto = true; } - private void maybeInitBuilder() { + private synchronized void maybeInitBuilder() { if (viaProto || builder == null) { builder = GetQueueUserAclsInfoResponseProto.newBuilder(proto); } @@ -131,7 +133,7 @@ private void initLocalQueueUserAclsList() { } } - private void addLocalQueueUserACLInfosToProto() { + private synchronized void addLocalQueueUserACLInfosToProto() { maybeInitBuilder(); builder.clearQueueUserAcls(); if (queueUserAclsInfoList == null) diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/RegisterApplicationMasterResponsePBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/RegisterApplicationMasterResponsePBImpl.java index 486304c..7233d71 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/RegisterApplicationMasterResponsePBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/RegisterApplicationMasterResponsePBImpl.java @@ -97,7 +97,7 @@ private void mergeLocalToProto() { viaProto = true; } - private void mergeLocalToBuilder() { + private synchronized void mergeLocalToBuilder() { if (this.maximumResourceCapability != null) { builder.setMaximumCapability( convertToProtoFormat(this.maximumResourceCapability)); @@ -140,7 +140,7 @@ public void setMaximumResourceCapability(Resource capability) { } @Override - public Map getApplicationACLs() { + public synchronized Map getApplicationACLs() { initApplicationACLs(); return this.applicationACLS; } @@ -161,7 +161,7 @@ private void initApplicationACLs() { } } - private void addApplicationACLs() { + private synchronized void addApplicationACLs() { maybeInitBuilder(); builder.clearApplicationACLs(); if (applicationACLS == null) { @@ -200,13 +200,12 @@ public void remove() { } @Override - public void setApplicationACLs( + public synchronized void setApplicationACLs( final Map appACLs) { + maybeInitBuilder(); if (appACLs == null) - return; - initApplicationACLs(); - this.applicationACLS.clear(); - this.applicationACLS.putAll(appACLs); + builder.clearApplicationACLs(); + this.applicationACLS = appACLs; } @Override diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/StartContainersRequestPBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/StartContainersRequestPBImpl.java index 2233705..3e88e59 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/StartContainersRequestPBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/StartContainersRequestPBImpl.java @@ -44,7 +44,7 @@ public StartContainersRequestPBImpl(StartContainersRequestProto proto) { viaProto = true; } - public StartContainersRequestProto getProto() { + public synchronized StartContainersRequestProto getProto() { mergeLocalToProto(); proto = viaProto ? proto : builder.build(); viaProto = true; @@ -66,7 +66,7 @@ public boolean equals(Object other) { return false; } - private void mergeLocalToProto() { + private synchronized void mergeLocalToProto() { if (viaProto) maybeInitBuilder(); mergeLocalToBuilder(); @@ -74,21 +74,21 @@ private void mergeLocalToProto() { viaProto = true; } - private void mergeLocalToBuilder() { + private synchronized void mergeLocalToBuilder() { if (requests != null) { addLocalRequestsToProto(); } } - private void maybeInitBuilder() { + private synchronized void maybeInitBuilder() { if (viaProto || builder == null) { builder = StartContainersRequestProto.newBuilder(proto); } viaProto = false; } - private void addLocalRequestsToProto() { + private synchronized void addLocalRequestsToProto() { maybeInitBuilder(); builder.clearStartContainerRequest(); List protoList = @@ -110,7 +110,8 @@ private void initLocalRequests() { } @Override - public void setStartContainerRequests(List requests) { + public synchronized void setStartContainerRequests( + List requests) { maybeInitBuilder(); if (requests == null) { builder.clearStartContainerRequest(); @@ -119,7 +120,7 @@ public void setStartContainerRequests(List requests) { } @Override - public List getStartContainerRequests() { + public synchronized List getStartContainerRequests() { if (this.requests != null) { return this.requests; } diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/StartContainersResponsePBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/StartContainersResponsePBImpl.java index 8f5c740..8cb4cdd 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/StartContainersResponsePBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/StartContainersResponsePBImpl.java @@ -64,7 +64,7 @@ public StartContainersResponsePBImpl(StartContainersResponseProto proto) { viaProto = true; } - public StartContainersResponseProto getProto() { + public synchronized StartContainersResponseProto getProto() { mergeLocalToProto(); proto = viaProto ? proto : builder.build(); viaProto = true; @@ -91,7 +91,7 @@ public String toString() { return TextFormat.shortDebugString(getProto()); } - private void mergeLocalToBuilder() { + private synchronized void mergeLocalToBuilder() { if (this.servicesMetaData != null) { addServicesMetaDataToProto(); } @@ -128,7 +128,7 @@ private SerializedExceptionProto convertToProtoFormat(SerializedException t) { return ((SerializedExceptionPBImpl) t).getProto(); } - private void mergeLocalToProto() { + private synchronized void mergeLocalToProto() { if (viaProto) { maybeInitBuilder(); } @@ -137,7 +137,7 @@ private void mergeLocalToProto() { viaProto = true; } - private void maybeInitBuilder() { + private synchronized void maybeInitBuilder() { if (viaProto || builder == null) { builder = StartContainersResponseProto.newBuilder(proto); } @@ -145,19 +145,19 @@ private void maybeInitBuilder() { } @Override - public Map getAllServicesMetaData() { + public synchronized Map getAllServicesMetaData() { initServicesMetaData(); return this.servicesMetaData; } @Override - public void setAllServicesMetaData(Map servicesMetaData) { + public synchronized void setAllServicesMetaData( + Map servicesMetaData) { + maybeInitBuilder(); if (servicesMetaData == null) { - return; + builder.clearServicesMetaData(); } - initServicesMetaData(); - this.servicesMetaData.clear(); - this.servicesMetaData.putAll(servicesMetaData); + this.servicesMetaData = servicesMetaData; } private void initServicesMetaData() { @@ -174,7 +174,7 @@ private void initServicesMetaData() { } } - private void addServicesMetaDataToProto() { + private synchronized void addServicesMetaDataToProto() { maybeInitBuilder(); builder.clearServicesMetaData(); if (servicesMetaData == null) @@ -211,7 +211,7 @@ public boolean hasNext() { builder.addAllServicesMetaData(iterable); } - private void addFailedContainersToProto() { + private synchronized void addFailedContainersToProto() { maybeInitBuilder(); builder.clearFailedRequests(); if (this.failedContainers == null) @@ -228,7 +228,7 @@ private void addFailedContainersToProto() { builder.addAllFailedRequests(protoList); } - private void addSucceededContainersToProto() { + private synchronized void addSucceededContainersToProto() { maybeInitBuilder(); builder.clearSucceededRequests(); if (this.succeededContainers == null) { @@ -274,13 +274,13 @@ private void initSucceededContainers() { } @Override - public List getSuccessfullyStartedContainers() { + public synchronized List getSuccessfullyStartedContainers() { initSucceededContainers(); return this.succeededContainers; } @Override - public void setSuccessfullyStartedContainers( + public synchronized void setSuccessfullyStartedContainers( List succeededContainers) { maybeInitBuilder(); if (succeededContainers == null) { @@ -303,13 +303,13 @@ private void initFailedContainers() { } @Override - public Map getFailedRequests() { + public synchronized Map getFailedRequests() { initFailedContainers(); return this.failedContainers; } @Override - public void setFailedRequests( + public synchronized void setFailedRequests( Map failedContainers) { maybeInitBuilder(); if (failedContainers == null) diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/StopContainersRequestPBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/StopContainersRequestPBImpl.java index 27e092b..95bf177 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/StopContainersRequestPBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/StopContainersRequestPBImpl.java @@ -51,7 +51,7 @@ public StopContainersRequestPBImpl(StopContainersRequestProto proto) { viaProto = true; } - public StopContainersRequestProto getProto() { + public synchronized StopContainersRequestProto getProto() { mergeLocalToProto(); proto = viaProto ? proto : builder.build(); viaProto = true; @@ -78,13 +78,13 @@ public String toString() { return TextFormat.shortDebugString(getProto()); } - private void mergeLocalToBuilder() { + private synchronized void mergeLocalToBuilder() { if (this.containerIds != null) { addLocalContainerIdsToProto(); } } - private void mergeLocalToProto() { + private synchronized void mergeLocalToProto() { if (viaProto) maybeInitBuilder(); mergeLocalToBuilder(); @@ -92,14 +92,14 @@ private void mergeLocalToProto() { viaProto = true; } - private void maybeInitBuilder() { + private synchronized void maybeInitBuilder() { if (viaProto || builder == null) { builder = StopContainersRequestProto.newBuilder(proto); } viaProto = false; } - private void addLocalContainerIdsToProto() { + private synchronized void addLocalContainerIdsToProto() { maybeInitBuilder(); builder.clearContainerId(); if (this.containerIds == null) @@ -124,13 +124,13 @@ private void initLocalContainerIds() { } @Override - public List getContainerIds() { + public synchronized List getContainerIds() { initLocalContainerIds(); return this.containerIds; } @Override - public void setContainerIds(List containerIds) { + public synchronized void setContainerIds(List containerIds) { maybeInitBuilder(); if (containerIds == null) builder.clearContainerId(); diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/StopContainersResponsePBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/StopContainersResponsePBImpl.java index dd28b06..c84060b 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/StopContainersResponsePBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/StopContainersResponsePBImpl.java @@ -58,7 +58,7 @@ public StopContainersResponsePBImpl(StopContainersResponseProto proto) { viaProto = true; } - public StopContainersResponseProto getProto() { + public synchronized StopContainersResponseProto getProto() { mergeLocalToProto(); proto = viaProto ? proto : builder.build(); viaProto = true; @@ -85,7 +85,7 @@ public String toString() { return TextFormat.shortDebugString(getProto()); } - private void mergeLocalToProto() { + private synchronized void mergeLocalToProto() { if (viaProto) { maybeInitBuilder(); } @@ -94,14 +94,14 @@ private void mergeLocalToProto() { viaProto = true; } - private void maybeInitBuilder() { + private synchronized void maybeInitBuilder() { if (viaProto || builder == null) { builder = StopContainersResponseProto.newBuilder(proto); } viaProto = false; } - private void mergeLocalToBuilder() { + private synchronized void mergeLocalToBuilder() { if (this.succeededRequests != null) { addSucceededRequestsToProto(); @@ -111,7 +111,7 @@ private void mergeLocalToBuilder() { } } - private void addSucceededRequestsToProto() { + private synchronized void addSucceededRequestsToProto() { maybeInitBuilder(); builder.clearSucceededRequests(); if (this.succeededRequests == null) { @@ -145,7 +145,7 @@ public void remove() { builder.addAllSucceededRequests(iterable); } - private void addFailedRequestsToProto() { + private synchronized void addFailedRequestsToProto() { maybeInitBuilder(); builder.clearFailedRequests(); if (this.failedRequests == null) @@ -187,13 +187,14 @@ private void initFailedRequests() { } @Override - public List getSuccessfullyStoppedContainers() { + public synchronized List getSuccessfullyStoppedContainers() { initSucceededRequests(); return this.succeededRequests; } @Override - public void setSuccessfullyStoppedContainers(List succeededRequests) { + public synchronized void setSuccessfullyStoppedContainers( + List succeededRequests) { maybeInitBuilder(); if (succeededRequests == null) { builder.clearSucceededRequests(); @@ -202,13 +203,13 @@ public void setSuccessfullyStoppedContainers(List succeededRequests } @Override - public Map getFailedRequests() { + public synchronized Map getFailedRequests() { initFailedRequests(); return this.failedRequests; } @Override - public void setFailedRequests( + public synchronized void setFailedRequests( Map failedRequests) { maybeInitBuilder(); if (failedRequests == null) diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerLaunchContextPBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerLaunchContextPBImpl.java index 12dcfcd..755c94a 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerLaunchContextPBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerLaunchContextPBImpl.java @@ -66,7 +66,7 @@ public ContainerLaunchContextPBImpl(ContainerLaunchContextProto proto) { viaProto = true; } - public ContainerLaunchContextProto getProto() { + public synchronized ContainerLaunchContextProto getProto() { mergeLocalToProto(); proto = viaProto ? proto : builder.build(); viaProto = true; @@ -101,7 +101,7 @@ protected final ByteString convertToProtoFormat(ByteBuffer byteBuffer) { return ProtoUtils.convertToProtoFormat(byteBuffer); } - private void mergeLocalToBuilder() { + private synchronized void mergeLocalToBuilder() { if (this.localResources != null) { addLocalResourcesToProto(); } @@ -122,7 +122,7 @@ private void mergeLocalToBuilder() { } } - private void mergeLocalToProto() { + private synchronized void mergeLocalToProto() { if (viaProto) maybeInitBuilder(); mergeLocalToBuilder(); @@ -130,7 +130,7 @@ private void mergeLocalToProto() { viaProto = true; } - private void maybeInitBuilder() { + private synchronized void maybeInitBuilder() { if (viaProto || builder == null) { builder = ContainerLaunchContextProto.newBuilder(proto); } @@ -138,7 +138,7 @@ private void maybeInitBuilder() { } @Override - public List getCommands() { + public synchronized List getCommands() { initCommands(); return this.commands; } @@ -157,15 +157,14 @@ private void initCommands() { } @Override - public void setCommands(final List commands) { + public synchronized void setCommands(final List commands) { + maybeInitBuilder(); if (commands == null) - return; - initCommands(); - this.commands.clear(); - this.commands.addAll(commands); + builder.clearCommand(); + this.commands = commands; } - private void addCommandsToProto() { + private synchronized void addCommandsToProto() { maybeInitBuilder(); builder.clearCommand(); if (this.commands == null) @@ -174,7 +173,7 @@ private void addCommandsToProto() { } @Override - public Map getLocalResources() { + public synchronized Map getLocalResources() { initLocalResources(); return this.localResources; } @@ -193,16 +192,15 @@ private void initLocalResources() { } @Override - public void setLocalResources( + public synchronized void setLocalResources( final Map localResources) { + maybeInitBuilder(); if (localResources == null) - return; - initLocalResources(); - this.localResources.clear(); - this.localResources.putAll(localResources); + builder.clearLocalResources(); + this.localResources = localResources; } - private void addLocalResourcesToProto() { + private synchronized void addLocalResourcesToProto() { maybeInitBuilder(); builder.clearLocalResources(); if (localResources == null) @@ -239,7 +237,7 @@ public boolean hasNext() { } @Override - public ByteBuffer getTokens() { + public synchronized ByteBuffer getTokens() { ContainerLaunchContextProtoOrBuilder p = viaProto ? proto : builder; if (this.tokens != null) { return this.tokens; @@ -252,7 +250,7 @@ public ByteBuffer getTokens() { } @Override - public void setTokens(ByteBuffer tokens) { + public synchronized void setTokens(ByteBuffer tokens) { maybeInitBuilder(); if (tokens == null) { builder.clearTokens(); @@ -261,7 +259,7 @@ public void setTokens(ByteBuffer tokens) { } @Override - public Map getServiceData() { + public synchronized Map getServiceData() { initServiceData(); return this.serviceData; } @@ -280,14 +278,15 @@ private void initServiceData() { } @Override - public void setServiceData(final Map serviceData) { + public synchronized void setServiceData( + final Map serviceData) { + maybeInitBuilder(); if (serviceData == null) - return; - initServiceData(); - this.serviceData.putAll(serviceData); + builder.clearServiceData(); + this.serviceData = serviceData; } - private void addServiceDataToProto() { + private synchronized void addServiceDataToProto() { maybeInitBuilder(); builder.clearServiceData(); if (serviceData == null) @@ -324,7 +323,7 @@ public boolean hasNext() { } @Override - public Map getEnvironment() { + public synchronized Map getEnvironment() { initEnv(); return this.environment; } @@ -343,15 +342,14 @@ private void initEnv() { } @Override - public void setEnvironment(final Map env) { + public synchronized void setEnvironment(final Map env) { + maybeInitBuilder(); if (env == null) - return; - initEnv(); - this.environment.clear(); - this.environment.putAll(env); + builder.clearEnvironment(); + this.environment = env; } - private void addEnvToProto() { + private synchronized void addEnvToProto() { maybeInitBuilder(); builder.clearEnvironment(); if (environment == null) @@ -388,7 +386,7 @@ public boolean hasNext() { } @Override - public Map getApplicationACLs() { + public synchronized Map getApplicationACLs() { initApplicationACLs(); return this.applicationACLS; } @@ -408,7 +406,7 @@ private void initApplicationACLs() { } } - private void addApplicationACLs() { + private synchronized void addApplicationACLs() { maybeInitBuilder(); builder.clearApplicationACLs(); if (applicationACLS == null) { @@ -447,13 +445,12 @@ public void remove() { } @Override - public void setApplicationACLs( + public synchronized void setApplicationACLs( final Map appACLs) { + maybeInitBuilder(); if (appACLs == null) - return; - initApplicationACLs(); - this.applicationACLS.clear(); - this.applicationACLS.putAll(appACLs); + builder.clearApplicationACLs(); + this.applicationACLS = appACLs; } private LocalResourcePBImpl convertFromProtoFormat(LocalResourceProto p) { diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/PreemptionContractPBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/PreemptionContractPBImpl.java index 2d234f2..0f9846b 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/PreemptionContractPBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/PreemptionContractPBImpl.java @@ -115,6 +115,7 @@ private void maybeInitBuilder() { @Override public synchronized void setContainers( final Set containers) { + maybeInitBuilder(); if (null == containers) { builder.clearContainer(); } @@ -130,6 +131,7 @@ public synchronized void setContainers( @Override public synchronized void setResourceRequest( final List req) { + maybeInitBuilder(); if (null == resources) { builder.clearResource(); } diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/QueueInfoPBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/QueueInfoPBImpl.java index 56a5b58..77a1cea 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/QueueInfoPBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/QueueInfoPBImpl.java @@ -55,7 +55,7 @@ public QueueInfoPBImpl(QueueInfoProto proto) { } @Override - public List getApplications() { + public synchronized List getApplications() { initLocalApplicationsList(); return this.applicationsList; } @@ -67,7 +67,7 @@ public float getCapacity() { } @Override - public List getChildQueues() { + public synchronized List getChildQueues() { initLocalChildQueuesList(); return this.childQueuesList; } @@ -100,7 +100,9 @@ public QueueState getQueueState() { } @Override - public void setApplications(List applications) { + public synchronized void + setApplications(List applications) { + maybeInitBuilder(); if (applications == null) { builder.clearApplications(); } @@ -114,7 +116,8 @@ public void setCapacity(float capacity) { } @Override - public void setChildQueues(List childQueues) { + public synchronized void setChildQueues(List childQueues) { + maybeInitBuilder(); if (childQueues == null) { builder.clearChildQueues(); } @@ -193,7 +196,7 @@ private void initLocalApplicationsList() { } } - private void addApplicationsToProto() { + private synchronized void addApplicationsToProto() { maybeInitBuilder(); builder.clearApplications(); if (applicationsList == null) @@ -240,7 +243,7 @@ private void initLocalChildQueuesList() { } } - private void addChildQueuesInfoToProto() { + private synchronized void addChildQueuesInfoToProto() { maybeInitBuilder(); builder.clearChildQueues(); if (childQueuesList == null) @@ -274,7 +277,7 @@ public void remove() { builder.addAllChildQueues(iterable); } - private void mergeLocalToBuilder() { + private synchronized void mergeLocalToBuilder() { if (this.childQueuesList != null) { addChildQueuesInfoToProto(); } diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/QueueUserACLInfoPBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/QueueUserACLInfoPBImpl.java index 4aa9b09..4d92ae4 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/QueueUserACLInfoPBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/QueueUserACLInfoPBImpl.java @@ -58,7 +58,7 @@ public String getQueueName() { } @Override - public List getUserAcls() { + public synchronized List getUserAcls() { initLocalQueueUserAclsList(); return this.userAclsList; } @@ -74,7 +74,8 @@ public void setQueueName(String queueName) { } @Override - public void setUserAcls(List userAclsList) { + public synchronized void setUserAcls(List userAclsList) { + maybeInitBuilder(); if (userAclsList == null) { builder.clearUserAcls(); } @@ -121,7 +122,7 @@ private void initLocalQueueUserAclsList() { } } - private void addQueueACLsToProto() { + private synchronized void addQueueACLsToProto() { maybeInitBuilder(); builder.clearUserAcls(); if (userAclsList == null) @@ -162,7 +163,7 @@ private void maybeInitBuilder() { viaProto = false; } - private void mergeLocalToBuilder() { + private synchronized void mergeLocalToBuilder() { if (this.userAclsList != null) { addQueueACLsToProto(); } diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceBlacklistRequestPBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceBlacklistRequestPBImpl.java index 743e5d1..e129ac5 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceBlacklistRequestPBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceBlacklistRequestPBImpl.java @@ -47,21 +47,21 @@ public ResourceBlacklistRequestPBImpl(ResourceBlacklistRequestProto proto) { viaProto = true; } - public ResourceBlacklistRequestProto getProto() { + public synchronized ResourceBlacklistRequestProto getProto() { mergeLocalToProto(); proto = viaProto ? proto : builder.build(); viaProto = true; return proto; } - private void maybeInitBuilder() { + private synchronized void maybeInitBuilder() { if (viaProto || builder == null) { builder = ResourceBlacklistRequestProto.newBuilder(proto); } viaProto = false; } - private void mergeLocalToProto() { + private synchronized void mergeLocalToProto() { if (viaProto) { maybeInitBuilder(); } @@ -70,7 +70,7 @@ private void mergeLocalToProto() { viaProto = true; } - private void mergeLocalToBuilder() { + private synchronized void mergeLocalToBuilder() { if (this.blacklistAdditions != null) { addBlacklistAdditionsToProto(); } @@ -79,7 +79,7 @@ private void mergeLocalToBuilder() { } } - private void addBlacklistAdditionsToProto() { + private synchronized void addBlacklistAdditionsToProto() { maybeInitBuilder(); builder.clearBlacklistAdditions(); if (this.blacklistAdditions == null) { @@ -88,7 +88,7 @@ private void addBlacklistAdditionsToProto() { builder.addAllBlacklistAdditions(this.blacklistAdditions); } - private void addBlacklistRemovalsToProto() { + private synchronized void addBlacklistRemovalsToProto() { maybeInitBuilder(); builder.clearBlacklistAdditions(); if (this.blacklistRemovals == null) { @@ -118,41 +118,33 @@ private void initBlacklistRemovals() { } @Override - public List getBlacklistAdditions() { + public synchronized List getBlacklistAdditions() { initBlacklistAdditions(); return this.blacklistAdditions; } @Override - public void setBlacklistAdditions(List resourceNames) { - if (resourceNames == null || resourceNames.isEmpty()) { - if (this.blacklistAdditions != null) { - this.blacklistAdditions.clear(); - } - return; + public synchronized void setBlacklistAdditions(List resourceNames) { + maybeInitBuilder(); + if (resourceNames == null) { + builder.clearBlacklistAdditions(); } - initBlacklistAdditions(); - this.blacklistAdditions.clear(); - this.blacklistAdditions.addAll(resourceNames); + this.blacklistAdditions = resourceNames; } @Override - public List getBlacklistRemovals() { + public synchronized List getBlacklistRemovals() { initBlacklistRemovals(); return this.blacklistRemovals; } @Override - public void setBlacklistRemovals(List resourceNames) { - if (resourceNames == null || resourceNames.isEmpty()) { - if (this.blacklistRemovals != null) { - this.blacklistRemovals.clear(); - } - return; + public synchronized void setBlacklistRemovals(List resourceNames) { + maybeInitBuilder(); + if (resourceNames == null) { + builder.clearBlacklistRemovals(); } - initBlacklistRemovals(); - this.blacklistRemovals.clear(); - this.blacklistRemovals.addAll(resourceNames); + this.blacklistRemovals = resourceNames; } @Override diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/StrictPreemptionContractPBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/StrictPreemptionContractPBImpl.java index 28569b4..56e5413 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/StrictPreemptionContractPBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/StrictPreemptionContractPBImpl.java @@ -109,6 +109,7 @@ private void maybeInitBuilder() { @Override public synchronized void setContainers( final Set containers) { + maybeInitBuilder(); if (null == containers) { builder.clearContainer(); }