diff --git a/hadoop-common-project/hadoop-auth-examples/src/main/java/org/apache/hadoop/security/authentication/examples/WhoClient.java b/hadoop-common-project/hadoop-auth-examples/src/main/java/org/apache/hadoop/security/authentication/examples/WhoClient.java index f5cff2b529a5f..72f0f1630949a 100644 --- a/hadoop-common-project/hadoop-auth-examples/src/main/java/org/apache/hadoop/security/authentication/examples/WhoClient.java +++ b/hadoop-common-project/hadoop-auth-examples/src/main/java/org/apache/hadoop/security/authentication/examples/WhoClient.java @@ -19,7 +19,7 @@ import java.io.InputStreamReader; import java.net.HttpURLConnection; import java.net.URL; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; /** * Example that uses AuthenticatedURL. @@ -42,7 +42,7 @@ public static void main(String[] args) { if (conn.getResponseCode() == HttpURLConnection.HTTP_OK) { BufferedReader reader = new BufferedReader( new InputStreamReader( - conn.getInputStream(), Charset.forName("UTF-8"))); + conn.getInputStream(), StandardCharsets.UTF_8)); String line = reader.readLine(); while (line != null) { System.out.println(line); diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/PseudoAuthenticationHandler.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/PseudoAuthenticationHandler.java index 19947aafad232..7bf3398a210ca 100644 --- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/PseudoAuthenticationHandler.java +++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/PseudoAuthenticationHandler.java @@ -23,7 +23,7 @@ import javax.servlet.http.HttpServletResponse; import java.io.IOException; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.util.List; import java.util.Properties; @@ -53,8 +53,6 @@ public class PseudoAuthenticationHandler implements AuthenticationHandler { */ public static final String ANONYMOUS_ALLOWED = TYPE + ".anonymous.allowed"; - private static final Charset UTF8_CHARSET = Charset.forName("UTF-8"); - private static final String PSEUDO_AUTH = "PseudoAuth"; private boolean acceptAnonymous; @@ -146,7 +144,7 @@ private String getUserName(HttpServletRequest request) { if(queryString == null || queryString.length() == 0) { return null; } - List list = URLEncodedUtils.parse(queryString, UTF8_CHARSET); + List list = URLEncodedUtils.parse(queryString, StandardCharsets.UTF_8); if (list != null) { for (NameValuePair nv : list) { if (PseudoAuthenticator.USER_NAME.equals(nv.getName())) { diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/ZKSignerSecretProvider.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/ZKSignerSecretProvider.java index 53b6f4d239f98..b0604c85c39f4 100644 --- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/ZKSignerSecretProvider.java +++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/ZKSignerSecretProvider.java @@ -16,25 +16,13 @@ import org.apache.hadoop.classification.VisibleForTesting; import java.nio.ByteBuffer; import java.security.SecureRandom; -import java.util.Collections; -import java.util.List; import java.util.Properties; import java.util.Random; -import javax.security.auth.login.Configuration; import javax.servlet.ServletContext; -import org.apache.curator.RetryPolicy; import org.apache.curator.framework.CuratorFramework; -import org.apache.curator.framework.CuratorFrameworkFactory; -import org.apache.curator.framework.api.ACLProvider; -import org.apache.curator.framework.imps.DefaultACLProvider; -import org.apache.curator.retry.ExponentialBackoffRetry; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.ZooDefs.Perms; -import org.apache.zookeeper.client.ZKClientConfig; -import org.apache.zookeeper.data.ACL; -import org.apache.zookeeper.data.Id; import org.apache.zookeeper.data.Stat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -92,6 +80,16 @@ public class ZKSignerSecretProvider extends RolloverSignerSecretProvider { public static final String ZOOKEEPER_KERBEROS_PRINCIPAL = CONFIG_PREFIX + "kerberos.principal"; + public static final String ZOOKEEPER_SSL_ENABLED = CONFIG_PREFIX + "ssl.enabled"; + public static final String ZOOKEEPER_SSL_KEYSTORE_LOCATION = + CONFIG_PREFIX + "ssl.keystore.location"; + public static final String ZOOKEEPER_SSL_KEYSTORE_PASSWORD = + CONFIG_PREFIX + "ssl.keystore.password"; + public static final String ZOOKEEPER_SSL_TRUSTSTORE_LOCATION = + CONFIG_PREFIX + "ssl.truststore.location"; + public static final String ZOOKEEPER_SSL_TRUSTSTORE_PASSWORD = + CONFIG_PREFIX + "ssl.truststore.password"; + /** * Constant for the property that specifies whether or not the Curator client * should disconnect from ZooKeeper on shutdown. The default is "true". Only @@ -350,80 +348,33 @@ protected byte[] generateRandomSecret() { * This method creates the Curator client and connects to ZooKeeper. * @param config configuration properties * @return A Curator client - * @throws Exception thrown if an error occurred */ - protected CuratorFramework createCuratorClient(Properties config) - throws Exception { - String connectionString = config.getProperty( - ZOOKEEPER_CONNECTION_STRING, "localhost:2181"); - - RetryPolicy retryPolicy = new ExponentialBackoffRetry(1000, 3); - ACLProvider aclProvider; + protected CuratorFramework createCuratorClient(Properties config) { + String connectionString = config.getProperty(ZOOKEEPER_CONNECTION_STRING, "localhost:2181"); String authType = config.getProperty(ZOOKEEPER_AUTH_TYPE, "none"); - if (authType.equals("sasl")) { - LOG.info("Connecting to ZooKeeper with SASL/Kerberos" - + "and using 'sasl' ACLs"); - String principal = setJaasConfiguration(config); - System.setProperty(ZKClientConfig.LOGIN_CONTEXT_NAME_KEY, - JAAS_LOGIN_ENTRY_NAME); - System.setProperty("zookeeper.authProvider.1", - "org.apache.zookeeper.server.auth.SASLAuthenticationProvider"); - aclProvider = new SASLOwnerACLProvider(principal); - } else { // "none" - LOG.info("Connecting to ZooKeeper without authentication"); - aclProvider = new DefaultACLProvider(); // open to everyone - } - CuratorFramework cf = CuratorFrameworkFactory.builder() - .connectString(connectionString) - .retryPolicy(retryPolicy) - .aclProvider(aclProvider) - .build(); - cf.start(); - return cf; - } - - private String setJaasConfiguration(Properties config) throws Exception { - String keytabFile = config.getProperty(ZOOKEEPER_KERBEROS_KEYTAB).trim(); - if (keytabFile == null || keytabFile.length() == 0) { - throw new IllegalArgumentException(ZOOKEEPER_KERBEROS_KEYTAB - + " must be specified"); - } - String principal = config.getProperty(ZOOKEEPER_KERBEROS_PRINCIPAL) - .trim(); - if (principal == null || principal.length() == 0) { - throw new IllegalArgumentException(ZOOKEEPER_KERBEROS_PRINCIPAL - + " must be specified"); - } + String keytab = config.getProperty(ZOOKEEPER_KERBEROS_KEYTAB, "").trim(); + String principal = config.getProperty(ZOOKEEPER_KERBEROS_PRINCIPAL, "").trim(); - // This is equivalent to writing a jaas.conf file and setting the system - // property, "java.security.auth.login.config", to point to it - JaasConfiguration jConf = - new JaasConfiguration(JAAS_LOGIN_ENTRY_NAME, principal, keytabFile); - Configuration.setConfiguration(jConf); - return principal.split("[/@]")[0]; - } + boolean sslEnabled = Boolean.parseBoolean(config.getProperty(ZOOKEEPER_SSL_ENABLED, "false")); + String keystoreLocation = config.getProperty(ZOOKEEPER_SSL_KEYSTORE_LOCATION, ""); + String keystorePassword = config.getProperty(ZOOKEEPER_SSL_KEYSTORE_PASSWORD, ""); + String truststoreLocation = config.getProperty(ZOOKEEPER_SSL_TRUSTSTORE_LOCATION, ""); + String truststorePassword = config.getProperty(ZOOKEEPER_SSL_TRUSTSTORE_PASSWORD, ""); - /** - * Simple implementation of an {@link ACLProvider} that simply returns an ACL - * that gives all permissions only to a single principal. - */ - private static class SASLOwnerACLProvider implements ACLProvider { - - private final List saslACL; - - private SASLOwnerACLProvider(String principal) { - this.saslACL = Collections.singletonList( - new ACL(Perms.ALL, new Id("sasl", principal))); - } - - @Override - public List getDefaultAcl() { - return saslACL; - } - - @Override - public List getAclForPath(String path) { - return saslACL; - } + CuratorFramework zkClient = + ZookeeperClient.configure() + .withConnectionString(connectionString) + .withAuthType(authType) + .withKeytab(keytab) + .withPrincipal(principal) + .withJaasLoginEntryName(JAAS_LOGIN_ENTRY_NAME) + .enableSSL(sslEnabled) + .withKeystore(keystoreLocation) + .withKeystorePassword(keystorePassword) + .withTruststore(truststoreLocation) + .withTruststorePassword(truststorePassword) + .create(); + zkClient.start(); + return zkClient; } } diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/ZookeeperClient.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/ZookeeperClient.java new file mode 100644 index 0000000000000..38b06a0ac7d9b --- /dev/null +++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/ZookeeperClient.java @@ -0,0 +1,318 @@ +/** + * Licensed 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. See accompanying LICENSE file. + */ + +package org.apache.hadoop.security.authentication.util; + +import org.apache.curator.RetryPolicy; +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.CuratorFrameworkFactory; +import org.apache.curator.framework.api.ACLProvider; +import org.apache.curator.framework.imps.DefaultACLProvider; +import org.apache.curator.retry.ExponentialBackoffRetry; +import org.apache.curator.utils.ConfigurableZookeeperFactory; +import org.apache.curator.utils.ZookeeperFactory; +import org.apache.hadoop.classification.VisibleForTesting; +import org.apache.zookeeper.ZooDefs; +import org.apache.zookeeper.client.ZKClientConfig; +import org.apache.zookeeper.common.ClientX509Util; +import org.apache.zookeeper.data.ACL; +import org.apache.zookeeper.data.Id; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.security.auth.login.Configuration; +import java.util.Collections; +import java.util.List; + +/** + * Utility class to create a CuratorFramework object that can be used to connect to Zookeeper + * based on configuration values that can be supplied from different configuration properties. + * It is used from ZKDelegationTokenSecretManager in hadoop-common, and from + * {@link ZKSignerSecretProvider}. + * + * The class implements a fluid API to set up all the different properties. A very basic setup + * would seem like: + *
+ *   ZookeeperClient.configure()
+ *     .withConnectionString(<connectionString>)
+ *     .create();
+ * 
+ * + * Mandatory parameters to be set: + *
    + *
  • connectionString: A Zookeeper connection string.
  • + *
  • if authentication type is set to 'sasl': + *
      + *
    • keytab: the location of the keytab to be used for Kerberos authentication
    • + *
    • principal: the Kerberos principal to be used from the supplied Kerberos keytab file.
    • + *
    • jaasLoginEntryName: the login entry name in the JAAS configuration that is created for + * the KerberosLoginModule to be used by the Zookeeper client code.
    • + *
    + *
  • + *
  • if SSL is enabled: + *
      + *
    • the location of the Truststore file to be used
    • + *
    • the location of the Keystore file to be used
    • + *
    • if the Truststore is protected by a password, then the password of the Truststore
    • + *
    • if the Keystore is protected by a password, then the password if the Keystore
    • + *
    + *
  • + *
+ * + * When using 'sasl' authentication type, the JAAS configuration to be used by the Zookeeper client + * withing CuratorFramework is set to use the supplied keytab and principal for Kerberos login, + * moreover an ACL provider is set to provide a default ACL that requires SASL auth and the same + * principal to have access to the used paths. + * + * When using SSL/TLS, the Zookeeper client will set to use the secure channel towards Zookeeper, + * with the specified Keystore and Truststore. + * + * Default values: + *
    + *
  • authentication type: 'none'
  • + *
  • sessionTimeout: either the system property curator-default-session-timeout, or 60 + * seconds
  • + *
  • connectionTimeout: either the system property curator-default-connection-timeout, or 15 + * seconds
  • + *
  • retryPolicy: an ExponentialBackoffRetry, with a starting interval of 1 seconds and 3 + * retries
  • + *
  • zkFactory: a ConfigurableZookeeperFactory instance, to allow SSL setup via + * ZKClientConfig
  • + *
+ * + * @see ZKSignerSecretProvider + */ +public class ZookeeperClient { + + private static final Logger LOG = LoggerFactory.getLogger(ZookeeperClient.class); + + private String connectionString; + private String namespace; + + private String authenticationType = "none"; + private String keytab; + private String principal; + private String jaasLoginEntryName; + + private int sessionTimeout = + Integer.getInteger("curator-default-session-timeout", 60 * 1000); + private int connectionTimeout = + Integer.getInteger("curator-default-connection-timeout", 15 * 1000); + + private RetryPolicy retryPolicy = new ExponentialBackoffRetry(1000, 3); + + private ZookeeperFactory zkFactory = new ConfigurableZookeeperFactory(); + + private boolean isSSLEnabled; + private String keystoreLocation; + private String keystorePassword; + private String truststoreLocation; + private String truststorePassword; + + public static ZookeeperClient configure() { + return new ZookeeperClient(); + } + + public ZookeeperClient withConnectionString(String conn) { + connectionString = conn; + return this; + } + + public ZookeeperClient withNamespace(String ns) { + this.namespace = ns; + return this; + } + + public ZookeeperClient withAuthType(String authType) { + this.authenticationType = authType; + return this; + } + + public ZookeeperClient withKeytab(String keytabPath) { + this.keytab = keytabPath; + return this; + } + + public ZookeeperClient withPrincipal(String princ) { + this.principal = princ; + return this; + } + + public ZookeeperClient withJaasLoginEntryName(String entryName) { + this.jaasLoginEntryName = entryName; + return this; + } + + public ZookeeperClient withSessionTimeout(int timeoutMS) { + this.sessionTimeout = timeoutMS; + return this; + } + + public ZookeeperClient withConnectionTimeout(int timeoutMS) { + this.connectionTimeout = timeoutMS; + return this; + } + + public ZookeeperClient withRetryPolicy(RetryPolicy policy) { + this.retryPolicy = policy; + return this; + } + + public ZookeeperClient withZookeeperFactory(ZookeeperFactory factory) { + this.zkFactory = factory; + return this; + } + + public ZookeeperClient enableSSL(boolean enable) { + this.isSSLEnabled = enable; + return this; + } + + public ZookeeperClient withKeystore(String keystorePath) { + this.keystoreLocation = keystorePath; + return this; + } + + public ZookeeperClient withKeystorePassword(String keystorePass) { + this.keystorePassword = keystorePass; + return this; + } + + public ZookeeperClient withTruststore(String truststorePath) { + this.truststoreLocation = truststorePath; + return this; + } + + public ZookeeperClient withTruststorePassword(String truststorePass) { + this.truststorePassword = truststorePass; + return this; + } + + public CuratorFramework create() { + checkNotNull(connectionString, "Zookeeper connection string cannot be null!"); + checkNotNull(retryPolicy, "Zookeeper connection retry policy cannot be null!"); + + return createFrameworkFactoryBuilder() + .connectString(connectionString) + .zookeeperFactory(zkFactory) + .namespace(namespace) + .sessionTimeoutMs(sessionTimeout) + .connectionTimeoutMs(connectionTimeout) + .retryPolicy(retryPolicy) + .aclProvider(aclProvider()) + .zkClientConfig(zkClientConfig()) + .build(); + } + + @VisibleForTesting + CuratorFrameworkFactory.Builder createFrameworkFactoryBuilder() { + return CuratorFrameworkFactory.builder(); + } + + private ACLProvider aclProvider() { + // AuthType has to be explicitly set to 'none' or 'sasl' + checkNotNull(authenticationType, "Zookeeper authType cannot be null!"); + checkArgument(authenticationType.equals("sasl") || authenticationType.equals("none"), + "Zookeeper authType must be one of [none, sasl]!"); + + ACLProvider aclProvider; + if (authenticationType.equals("sasl")) { + LOG.info("Connecting to ZooKeeper with SASL/Kerberos and using 'sasl' ACLs."); + + checkArgument(!isEmpty(keytab), "Zookeeper client's Kerberos Keytab must be specified!"); + checkArgument(!isEmpty(principal), + "Zookeeper client's Kerberos Principal must be specified!"); + checkArgument(!isEmpty(jaasLoginEntryName), "JAAS Login Entry name must be specified!"); + + JaasConfiguration jConf = new JaasConfiguration(jaasLoginEntryName, principal, keytab); + Configuration.setConfiguration(jConf); + System.setProperty(ZKClientConfig.LOGIN_CONTEXT_NAME_KEY, jaasLoginEntryName); + System.setProperty("zookeeper.authProvider.1", + "org.apache.zookeeper.server.auth.SASLAuthenticationProvider"); + aclProvider = new SASLOwnerACLProvider(principal.split("[/@]")[0]); + } else { // "none" + LOG.info("Connecting to ZooKeeper without authentication."); + aclProvider = new DefaultACLProvider(); // open to everyone + } + return aclProvider; + } + + private ZKClientConfig zkClientConfig() { + ZKClientConfig zkClientConfig = new ZKClientConfig(); + if (isSSLEnabled){ + LOG.info("Zookeeper client will use SSL connection. (keystore = {}; truststore = {};)", + keystoreLocation, truststoreLocation); + checkArgument(!isEmpty(keystoreLocation), + "The keystore location parameter is empty for the ZooKeeper client connection."); + checkArgument(!isEmpty(truststoreLocation), + "The truststore location parameter is empty for the ZooKeeper client connection."); + + try (ClientX509Util sslOpts = new ClientX509Util()) { + zkClientConfig.setProperty(ZKClientConfig.SECURE_CLIENT, "true"); + zkClientConfig.setProperty(ZKClientConfig.ZOOKEEPER_CLIENT_CNXN_SOCKET, + "org.apache.zookeeper.ClientCnxnSocketNetty"); + zkClientConfig.setProperty(sslOpts.getSslKeystoreLocationProperty(), keystoreLocation); + zkClientConfig.setProperty(sslOpts.getSslKeystorePasswdProperty(), keystorePassword); + zkClientConfig.setProperty(sslOpts.getSslTruststoreLocationProperty(), truststoreLocation); + zkClientConfig.setProperty(sslOpts.getSslTruststorePasswdProperty(), truststorePassword); + } + } else { + LOG.info("Zookeeper client will use Plain connection."); + } + return zkClientConfig; + } + + /** + * Simple implementation of an {@link ACLProvider} that simply returns an ACL + * that gives all permissions only to a single principal. + */ + @VisibleForTesting + static final class SASLOwnerACLProvider implements ACLProvider { + + private final List saslACL; + + private SASLOwnerACLProvider(String principal) { + this.saslACL = Collections.singletonList( + new ACL(ZooDefs.Perms.ALL, new Id("sasl", principal))); + } + + @Override + public List getDefaultAcl() { + return saslACL; + } + + @Override + public List getAclForPath(String path) { + return saslACL; + } + } + + private boolean isEmpty(String str) { + return str == null || str.length() == 0; + } + + //Preconditions allowed to be imported from hadoop-common, but that results + // in a circular dependency + private void checkNotNull(Object reference, String errorMessage) { + if (reference == null) { + throw new NullPointerException(errorMessage); + } + } + + private void checkArgument(boolean expression, String errorMessage) { + if (!expression) { + throw new IllegalArgumentException(errorMessage); + } + } +} diff --git a/hadoop-common-project/hadoop-auth/src/site/markdown/Configuration.md b/hadoop-common-project/hadoop-auth/src/site/markdown/Configuration.md index 43597b688112d..147ba52055cc4 100644 --- a/hadoop-common-project/hadoop-auth/src/site/markdown/Configuration.md +++ b/hadoop-common-project/hadoop-auth/src/site/markdown/Configuration.md @@ -404,6 +404,21 @@ The following configuration properties are specific to the `zookeeper` implement * `signer.secret.provider.zookeeper.kerberos.principal`: Set this to the Kerberos principal to use. This only required if using Kerberos. +* `signer.secret.provider.zookeeper.ssl.enabled` : Set this to true to enable SSL/TLS + communication between the server and Zookeeper, if the SignerSecretProvider is zookeeper. + +* `signer.secret.provider.zookeeper.ssl.keystore.location` : Specifies the location of the + Zookeeper client's keystore file. + +* `signer.secret.provider.zookeeper.ssl.keystore.password` : Specifies the location of the + Zookeeper client's keystore password. + +* `signer.secret.provider.zookeeper.ssl.truststore.location` : Specifies the location of the + Zookeeper client's truststore file. + +* `signer.secret.provider.zookeeper.ssl.truststore.password` : Specifies the location of the + Zookeeper client's truststore password. + * `signer.secret.provider.zookeeper.disconnect.on.shutdown`: Whether to close the ZooKeeper connection when the provider is shutdown. The default value is `true`. Only set this to `false` if a custom Curator client is being provided and diff --git a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/StringSignerSecretProvider.java b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/StringSignerSecretProvider.java index ed6b1aeccc7c2..9d857640bcf48 100644 --- a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/StringSignerSecretProvider.java +++ b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/StringSignerSecretProvider.java @@ -13,7 +13,7 @@ */ package org.apache.hadoop.security.authentication.util; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.util.Properties; import javax.servlet.ServletContext; @@ -38,7 +38,7 @@ public void init(Properties config, ServletContext servletContext, long tokenValidity) throws Exception { String signatureSecret = config.getProperty( AuthenticationFilter.SIGNATURE_SECRET, null); - secret = signatureSecret.getBytes(Charset.forName("UTF-8")); + secret = signatureSecret.getBytes(StandardCharsets.UTF_8); secrets = new byte[][]{secret}; } diff --git a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/TestZKSignerSecretProvider.java b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/TestZKSignerSecretProvider.java index 628342e40dc4a..4f090c234eece 100644 --- a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/TestZKSignerSecretProvider.java +++ b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/TestZKSignerSecretProvider.java @@ -13,7 +13,7 @@ */ package org.apache.hadoop.security.authentication.util; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.util.Properties; import java.util.Random; import javax.servlet.ServletContext; @@ -140,11 +140,11 @@ public void testUpgradeChangeSecretLength() throws Exception { long seed = System.currentTimeMillis(); Random rand = new Random(seed); byte[] secret2 = Long.toString(rand.nextLong()) - .getBytes(Charset.forName("UTF-8")); + .getBytes(StandardCharsets.UTF_8); byte[] secret1 = Long.toString(rand.nextLong()) - .getBytes(Charset.forName("UTF-8")); + .getBytes(StandardCharsets.UTF_8); byte[] secret3 = Long.toString(rand.nextLong()) - .getBytes(Charset.forName("UTF-8")); + .getBytes(StandardCharsets.UTF_8); rand = new Random(seed); // Secrets 4 and 5 get thrown away by ZK when the new secret provider tries // to init @@ -238,7 +238,7 @@ private class OldMockZKSignerSecretProvider @Override protected byte[] generateRandomSecret() { - return Long.toString(rand.nextLong()).getBytes(Charset.forName("UTF-8")); + return Long.toString(rand.nextLong()).getBytes(StandardCharsets.UTF_8); } } diff --git a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/TestZookeeperClientCreation.java b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/TestZookeeperClientCreation.java new file mode 100644 index 0000000000000..65cf3939ab44b --- /dev/null +++ b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/TestZookeeperClientCreation.java @@ -0,0 +1,498 @@ +/** + * Licensed 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. See accompanying LICENSE file. + */ + +package org.apache.hadoop.security.authentication.util; + +import org.apache.curator.RetryPolicy; +import org.apache.curator.framework.CuratorFrameworkFactory; +import org.apache.curator.framework.imps.DefaultACLProvider; +import org.apache.curator.retry.ExponentialBackoffRetry; +import org.apache.curator.utils.ConfigurableZookeeperFactory; +import org.apache.curator.utils.ZookeeperFactory; +import org.apache.hadoop.security.authentication.util.ZookeeperClient.SASLOwnerACLProvider; +import org.apache.zookeeper.ZooDefs; +import org.apache.zookeeper.client.ZKClientConfig; +import org.apache.zookeeper.common.ClientX509Util; +import org.hamcrest.Matcher; +import org.hamcrest.core.IsNull; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; + +import javax.security.auth.login.AppConfigurationEntry; +import javax.security.auth.login.Configuration; + +import java.util.Arrays; + +import static org.hamcrest.CoreMatchers.anyOf; +import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.ArgumentCaptor.forClass; +import static org.mockito.ArgumentMatchers.isA; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +/** + * Tests for ZookeeperClient class, to check if it creates CuratorFramework by providing expected + * parameter values to the CuratorFrameworkFactory.Builder instance. + */ +public class TestZookeeperClientCreation { + + private ZookeeperClient clientConfigurer; + private CuratorFrameworkFactory.Builder cfBuilder; + + @BeforeEach + public void setup() { + clientConfigurer = spy(ZookeeperClient.configure()); + clientConfigurer.withConnectionString("dummy"); + cfBuilder = spy(CuratorFrameworkFactory.builder()); + + when(clientConfigurer.createFrameworkFactoryBuilder()).thenReturn(cfBuilder); + } + + //Positive tests + @Test + public void testConnectionStringSet() { + clientConfigurer.withConnectionString("conn").create(); + + verify(cfBuilder).connectString("conn"); + + verifyDefaultZKFactory(); + verifyDefaultNamespace(); + verifyDefaultSessionTimeout(); + verifyDefaultConnectionTimeout(); + verifyDefaultRetryPolicy(); + verifyDefaultAclProvider(); + verifyDefaultZKClientConfig(); + } + + @Test + public void testZookeeperFactorySet() { + ZookeeperFactory zkFactory = mock(ZookeeperFactory.class); + clientConfigurer.withZookeeperFactory(zkFactory).create(); + + verify(cfBuilder).zookeeperFactory(zkFactory); + + verifyDummyConnectionString(); + verifyDefaultNamespace(); + verifyDefaultSessionTimeout(); + verifyDefaultConnectionTimeout(); + verifyDefaultRetryPolicy(); + verifyDefaultAclProvider(); + verifyDefaultZKClientConfig(); + } + + @Test + public void testNameSpaceSet() { + clientConfigurer.withNamespace("someNS/someSubSpace").create(); + + verify(cfBuilder).namespace("someNS/someSubSpace"); + + verifyDummyConnectionString(); + verifyDefaultZKFactory(); + verifyDefaultSessionTimeout(); + verifyDefaultConnectionTimeout(); + verifyDefaultRetryPolicy(); + verifyDefaultAclProvider(); + verifyDefaultZKClientConfig(); + } + + @Test + public void testSessionTimeoutSet() { + clientConfigurer.withSessionTimeout(20000).create(); + + verify(cfBuilder).sessionTimeoutMs(20000); + + verifyDummyConnectionString(); + verifyDefaultZKFactory(); + verifyDefaultNamespace(); + verifyDefaultConnectionTimeout(); + verifyDefaultRetryPolicy(); + verifyDefaultAclProvider(); + verifyDefaultZKClientConfig(); + } + + + @Test + public void testDefaultSessionTimeoutIsAffectedBySystemProperty() { + System.setProperty("curator-default-session-timeout", "20000"); + setup(); + clientConfigurer.create(); + + verify(cfBuilder).sessionTimeoutMs(20000); + + verifyDummyConnectionString(); + verifyDefaultZKFactory(); + verifyDefaultNamespace(); + verifyDefaultConnectionTimeout(); + verifyDefaultRetryPolicy(); + verifyDefaultAclProvider(); + verifyDefaultZKClientConfig(); + System.clearProperty("curator-default-session-timeout"); + } + + @Test + public void testConnectionTimeoutSet() { + clientConfigurer.withConnectionTimeout(50).create(); + + verify(cfBuilder).connectionTimeoutMs(50); + + verifyDummyConnectionString(); + verifyDefaultZKFactory(); + verifyDefaultNamespace(); + verifyDefaultSessionTimeout(); + verifyDefaultRetryPolicy(); + verifyDefaultAclProvider(); + verifyDefaultZKClientConfig(); + } + + @Test + public void testDefaultConnectionTimeoutIsAffectedBySystemProperty() { + System.setProperty("curator-default-connection-timeout", "50"); + setup(); + clientConfigurer.create(); + + verify(cfBuilder).connectionTimeoutMs(50); + + verifyDummyConnectionString(); + verifyDefaultZKFactory(); + verifyDefaultNamespace(); + verifyDefaultSessionTimeout(); + verifyDefaultRetryPolicy(); + verifyDefaultAclProvider(); + verifyDefaultZKClientConfig(); + System.clearProperty("curator-default-connection-timeout"); + } + + @Test + public void testRetryPolicySet() { + RetryPolicy policy = mock(RetryPolicy.class); + clientConfigurer.withRetryPolicy(policy).create(); + + verify(cfBuilder).retryPolicy(policy); + + verifyDummyConnectionString(); + verifyDefaultZKFactory(); + verifyDefaultNamespace(); + verifyDefaultSessionTimeout(); + verifyDefaultConnectionTimeout(); + verifyDefaultAclProvider(); + verifyDefaultZKClientConfig(); + } + + @Test + public void testSaslAutTypeWithIBMJava() { + testSaslAuthType("IBMJava"); + } + + @Test + public void testSaslAuthTypeWithNonIBMJava() { + testSaslAuthType("OracleJava"); + } + + @Test + public void testSSLConfiguration() { + clientConfigurer + .enableSSL(true) + .withKeystore("keystoreLoc") + .withKeystorePassword("ksPass") + .withTruststore("truststoreLoc") + .withTruststorePassword("tsPass") + .create(); + + ArgumentCaptor clientConfCaptor = forClass(ZKClientConfig.class); + verify(cfBuilder).zkClientConfig(clientConfCaptor.capture()); + ZKClientConfig conf = clientConfCaptor.getValue(); + + assertThat(conf.getProperty(ZKClientConfig.SECURE_CLIENT), is("true")); + assertThat(conf.getProperty(ZKClientConfig.ZOOKEEPER_CLIENT_CNXN_SOCKET), + is("org.apache.zookeeper.ClientCnxnSocketNetty")); + try (ClientX509Util sslOpts = new ClientX509Util()) { + assertThat(conf.getProperty(sslOpts.getSslKeystoreLocationProperty()), is("keystoreLoc")); + assertThat(conf.getProperty(sslOpts.getSslKeystorePasswdProperty()), is("ksPass")); + assertThat(conf.getProperty(sslOpts.getSslTruststoreLocationProperty()), is("truststoreLoc")); + assertThat(conf.getProperty(sslOpts.getSslTruststorePasswdProperty()), is("tsPass")); + } + + verifyDummyConnectionString(); + verifyDefaultZKFactory(); + verifyDefaultNamespace(); + verifyDefaultSessionTimeout(); + verifyDefaultConnectionTimeout(); + verifyDefaultRetryPolicy(); + verifyDefaultAclProvider(); + } + + //Negative tests + @Test + public void testNoConnectionString(){ + clientConfigurer.withConnectionString(null); + + Throwable t = assertThrows(NullPointerException.class, () -> clientConfigurer.create()); + assertThat(t.getMessage(), containsString("Zookeeper connection string cannot be null!")); + } + + @Test + public void testNoRetryPolicy() { + clientConfigurer.withRetryPolicy(null); + + Throwable t = assertThrows(NullPointerException.class, () -> clientConfigurer.create()); + assertThat(t.getMessage(), containsString("Zookeeper connection retry policy cannot be null!")); + } + + @Test + public void testNoAuthType() { + clientConfigurer.withAuthType(null); + + Throwable t = assertThrows(NullPointerException.class, () -> clientConfigurer.create()); + assertThat(t.getMessage(), containsString("Zookeeper authType cannot be null!")); + } + + @Test + public void testUnrecognizedAuthType() { + clientConfigurer.withAuthType("something"); + + Throwable t = assertThrows(IllegalArgumentException.class, () -> clientConfigurer.create()); + assertThat(t.getMessage(), is("Zookeeper authType must be one of [none, sasl]!")); + } + + @Test + public void testSaslAuthTypeWithoutKeytab() { + clientConfigurer.withAuthType("sasl"); + + Throwable t = assertThrows(IllegalArgumentException.class, () -> clientConfigurer.create()); + assertThat(t.getMessage(), is("Zookeeper client's Kerberos Keytab must be specified!")); + } + + @Test + public void testSaslAuthTypeWithEmptyKeytab() { + clientConfigurer + .withAuthType("sasl") + .withKeytab(""); + + Throwable t = assertThrows(IllegalArgumentException.class, () -> clientConfigurer.create()); + assertThat(t.getMessage(), is("Zookeeper client's Kerberos Keytab must be specified!")); + } + + @Test + public void testSaslAuthTypeWithoutPrincipal() { + clientConfigurer + .withAuthType("sasl") + .withKeytab("keytabLoc"); + + Throwable t = assertThrows(IllegalArgumentException.class, () -> clientConfigurer.create()); + assertThat(t.getMessage(), is("Zookeeper client's Kerberos Principal must be specified!")); + } + + @Test + public void testSaslAuthTypeWithEmptyPrincipal() { + clientConfigurer + .withAuthType("sasl") + .withKeytab("keytabLoc") + .withPrincipal(""); + + Throwable t = assertThrows(IllegalArgumentException.class, () -> clientConfigurer.create()); + assertThat(t.getMessage(), is("Zookeeper client's Kerberos Principal must be specified!")); + } + + @Test + public void testSaslAuthTypeWithoutJaasLoginEntryName() { + clientConfigurer + .withAuthType("sasl") + .withKeytab("keytabLoc") + .withPrincipal("principal") + .withJaasLoginEntryName(null); + + Throwable t = assertThrows(IllegalArgumentException.class, () -> clientConfigurer.create()); + assertThat(t.getMessage(), is("JAAS Login Entry name must be specified!")); + } + + @Test + public void testSaslAuthTypeWithEmptyJaasLoginEntryName() { + clientConfigurer + .withAuthType("sasl") + .withKeytab("keytabLoc") + .withPrincipal("principal") + .withJaasLoginEntryName(""); + + Throwable t = assertThrows(IllegalArgumentException.class, () -> clientConfigurer.create()); + assertThat(t.getMessage(), is("JAAS Login Entry name must be specified!")); + } + + @Test + public void testSSLWithoutKeystore() { + clientConfigurer + .enableSSL(true); + + Throwable t = assertThrows(IllegalArgumentException.class, () -> clientConfigurer.create()); + assertThat(t.getMessage(), + is("The keystore location parameter is empty for the ZooKeeper client connection.")); + } + + @Test + public void testSSLWithEmptyKeystore() { + clientConfigurer + .enableSSL(true) + .withKeystore(""); + + Throwable t = assertThrows(IllegalArgumentException.class, () -> clientConfigurer.create()); + assertThat(t.getMessage(), + is("The keystore location parameter is empty for the ZooKeeper client connection.")); + } + + @Test + public void testSSLWithoutTruststore() { + clientConfigurer + .enableSSL(true) + .withKeystore("keyStoreLoc"); + + Throwable t = assertThrows(IllegalArgumentException.class, () -> clientConfigurer.create()); + assertThat(t.getMessage(), + is("The truststore location parameter is empty for the ZooKeeper client connection.")); + } + + @Test + public void testSSLWithEmptyTruststore() { + clientConfigurer + .enableSSL(true) + .withKeystore("keyStoreLoc") + .withTruststore(""); + + Throwable t = assertThrows(IllegalArgumentException.class, () -> clientConfigurer.create()); + assertThat(t.getMessage(), + is("The truststore location parameter is empty for the ZooKeeper client connection.")); + } + + private void testSaslAuthType(String vendor) { + String origVendor = System.getProperty("java.vendor"); + System.setProperty("java.vendor", vendor); + Configuration origConf = Configuration.getConfiguration(); + + try { + clientConfigurer + .withAuthType("sasl") + .withKeytab("keytabLoc") + .withPrincipal("principal@some.host/SOME.REALM") + .withJaasLoginEntryName("TestEntry") + .create(); + + ArgumentCaptor aclProviderCaptor = forClass(SASLOwnerACLProvider.class); + verify(cfBuilder).aclProvider(aclProviderCaptor.capture()); + SASLOwnerACLProvider aclProvider = aclProviderCaptor.getValue(); + + assertThat(aclProvider.getDefaultAcl().size(), is(1)); + assertThat(aclProvider.getDefaultAcl().get(0).getId().getScheme(), is("sasl")); + assertThat(aclProvider.getDefaultAcl().get(0).getId().getId(), is("principal")); + assertThat(aclProvider.getDefaultAcl().get(0).getPerms(), is(ZooDefs.Perms.ALL)); + + Arrays.stream(new String[] {"/", "/foo", "/foo/bar/baz", "/random/path"}) + .forEach(s -> { + assertThat(aclProvider.getAclForPath(s).size(), is(1)); + assertThat(aclProvider.getAclForPath(s).get(0).getId().getScheme(), is("sasl")); + assertThat(aclProvider.getAclForPath(s).get(0).getId().getId(), is("principal")); + assertThat(aclProvider.getAclForPath(s).get(0).getPerms(), is(ZooDefs.Perms.ALL)); + }); + + assertThat(System.getProperty(ZKClientConfig.LOGIN_CONTEXT_NAME_KEY), is("TestEntry")); + assertThat(System.getProperty("zookeeper.authProvider.1"), + is("org.apache.zookeeper.server.auth.SASLAuthenticationProvider")); + + Configuration config = Configuration.getConfiguration(); + assertThat(config.getAppConfigurationEntry("TestEntry").length, is(1)); + AppConfigurationEntry entry = config.getAppConfigurationEntry("TestEntry")[0]; + assertThat(entry.getOptions().get("keyTab"), is("keytabLoc")); + assertThat(entry.getOptions().get("principal"), is("principal@some.host/SOME.REALM")); + assertThat(entry.getOptions().get("useKeyTab"), is("true")); + assertThat(entry.getOptions().get("storeKey"), is("true")); + assertThat(entry.getOptions().get("useTicketCache"), is("false")); + assertThat(entry.getOptions().get("refreshKrb5Config"), is("true")); + if (System.getProperty("java.vendor").contains("IBM")){ + assertThat(entry.getLoginModuleName(), is("com.ibm.security.auth.module.Krb5LoginModule")); + } else { + assertThat(entry.getLoginModuleName(), is("com.sun.security.auth.module.Krb5LoginModule")); + } + } finally { + Configuration.setConfiguration(origConf); + System.setProperty("java.vendor", origVendor); + } + + verifyDummyConnectionString(); + verifyDefaultZKFactory(); + verifyDefaultNamespace(); + verifyDefaultSessionTimeout(); + verifyDefaultConnectionTimeout(); + verifyDefaultRetryPolicy(); + verifyDefaultZKClientConfig(); + } + + private void verifyDummyConnectionString() { + verify(cfBuilder).connectString("dummy"); + } + + private void verifyDefaultNamespace() { + verify(cfBuilder).namespace(null); + } + + private void verifyDefaultZKFactory() { + verify(cfBuilder).zookeeperFactory(isA(ConfigurableZookeeperFactory.class)); + } + + private void verifyDefaultSessionTimeout() { + verify(cfBuilder).sessionTimeoutMs(60000); + } + + private void verifyDefaultConnectionTimeout() { + verify(cfBuilder).connectionTimeoutMs(15000); + } + + private void verifyDefaultRetryPolicy() { + ArgumentCaptor retry = forClass(ExponentialBackoffRetry.class); + verify(cfBuilder).retryPolicy(retry.capture()); + ExponentialBackoffRetry policy = retry.getValue(); + + assertThat(policy.getBaseSleepTimeMs(), is(1000)); + assertThat(policy.getN(), is(3)); + } + + private void verifyDefaultAclProvider() { + verify(cfBuilder).aclProvider(isA(DefaultACLProvider.class)); + } + + private void verifyDefaultZKClientConfig() { + ArgumentCaptor clientConfCaptor = forClass(ZKClientConfig.class); + verify(cfBuilder).zkClientConfig(clientConfCaptor.capture()); + ZKClientConfig conf = clientConfCaptor.getValue(); + + assertThat(conf.getProperty(ZKClientConfig.SECURE_CLIENT), isEmptyOrFalse()); + try (ClientX509Util sslOpts = new ClientX509Util()) { + assertThat(conf.getProperty(sslOpts.getSslKeystoreLocationProperty()), isEmpty()); + assertThat(conf.getProperty(sslOpts.getSslKeystorePasswdProperty()), isEmpty()); + assertThat(conf.getProperty(sslOpts.getSslTruststoreLocationProperty()), isEmpty()); + assertThat(conf.getProperty(sslOpts.getSslTruststorePasswdProperty()), isEmpty()); + } + } + + private Matcher isEmptyOrFalse() { + return anyOf(isEmpty(), is("false")); + } + + private Matcher isEmpty() { + return anyOf(new IsNull<>(), is("")); + } + +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java index ab7ff0bd40cc2..7c4f617b179e0 100755 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java @@ -43,6 +43,7 @@ import java.net.URISyntaxException; import java.net.URL; import java.net.URLConnection; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.util.ArrayList; import java.util.Arrays; @@ -82,7 +83,6 @@ import javax.xml.transform.dom.DOMSource; import javax.xml.transform.stream.StreamResult; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; import org.apache.commons.collections.map.UnmodifiableMap; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -2903,7 +2903,7 @@ public Reader getConfResourceAsReader(String name) { LOG.info("found resource " + name + " at " + url); } - return new InputStreamReader(url.openStream(), Charsets.UTF_8); + return new InputStreamReader(url.openStream(), StandardCharsets.UTF_8); } catch (Exception e) { return null; } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java index 63f494a20ac37..31b6654afc578 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java @@ -417,6 +417,10 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic { /** How often to retry a ZooKeeper operation in milliseconds. */ public static final String ZK_RETRY_INTERVAL_MS = ZK_PREFIX + "retry-interval-ms"; + + /** SSL enablement for all Hadoop->ZK communication. */ + //Note: except ZKSignerSecretProvider in hadoop-auth to avoid circular dependency. + public static final String ZK_CLIENT_SSL_ENABLED = ZK_PREFIX + "ssl.enabled"; /** Keystore location for ZooKeeper client connection over SSL. */ public static final String ZK_SSL_KEYSTORE_LOCATION = ZK_PREFIX + "ssl.keystore.location"; /** Keystore password for ZooKeeper client connection over SSL. */ @@ -425,6 +429,7 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic { public static final String ZK_SSL_TRUSTSTORE_LOCATION = ZK_PREFIX + "ssl.truststore.location"; /** Truststore password for ZooKeeper client connection over SSL. */ public static final String ZK_SSL_TRUSTSTORE_PASSWORD = ZK_PREFIX + "ssl.truststore.password"; + public static final int ZK_RETRY_INTERVAL_MS_DEFAULT = 1000; /** Default domain name resolver for hadoop to use. */ public static final String HADOOP_DOMAINNAME_RESOLVER_IMPL = diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java index a1a325b925d4f..28799349a8974 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java @@ -899,7 +899,7 @@ private static void runCommandOnStream( try (BufferedReader reader = new BufferedReader( new InputStreamReader(process.getInputStream(), - Charset.forName("UTF-8")))) { + StandardCharsets.UTF_8))) { String line; while((line = reader.readLine()) != null) { LOG.debug(line); @@ -922,7 +922,7 @@ private static void runCommandOnStream( try (BufferedReader reader = new BufferedReader( new InputStreamReader(process.getErrorStream(), - Charset.forName("UTF-8")))) { + StandardCharsets.UTF_8))) { String line; while((line = reader.readLine()) != null) { LOG.debug(line); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FileSystemMultipartUploader.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FileSystemMultipartUploader.java index 481d927672dc3..28a4bce0489cd 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FileSystemMultipartUploader.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FileSystemMultipartUploader.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.io.InputStream; import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Comparator; import java.util.HashSet; @@ -30,7 +31,6 @@ import java.util.concurrent.CompletableFuture; import java.util.stream.Collectors; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; import org.apache.hadoop.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -104,7 +104,7 @@ public CompletableFuture startUpload(Path filePath) fs.mkdirs(collectorPath, FsPermission.getDirDefault()); ByteBuffer byteBuffer = ByteBuffer.wrap( - collectorPath.toString().getBytes(Charsets.UTF_8)); + collectorPath.toString().getBytes(StandardCharsets.UTF_8)); return BBUploadHandle.from(byteBuffer); }); } @@ -130,7 +130,7 @@ private PartHandle innerPutPart(Path filePath, byte[] uploadIdByteArray = uploadId.toByteArray(); checkUploadId(uploadIdByteArray); Path collectorPath = new Path(new String(uploadIdByteArray, 0, - uploadIdByteArray.length, Charsets.UTF_8)); + uploadIdByteArray.length, StandardCharsets.UTF_8)); Path partPath = mergePaths(collectorPath, mergePaths(new Path(Path.SEPARATOR), new Path(partNumber + ".part"))); @@ -149,7 +149,7 @@ private PartHandle innerPutPart(Path filePath, cleanupWithLogger(LOG, inputStream); } return BBPartHandle.from(ByteBuffer.wrap( - partPath.toString().getBytes(Charsets.UTF_8))); + partPath.toString().getBytes(StandardCharsets.UTF_8))); } private Path createCollectorPath(Path filePath) { @@ -210,7 +210,7 @@ private PathHandle innerComplete( .map(pair -> { byte[] byteArray = pair.getValue().toByteArray(); return new Path(new String(byteArray, 0, byteArray.length, - Charsets.UTF_8)); + StandardCharsets.UTF_8)); }) .collect(Collectors.toList()); @@ -223,7 +223,7 @@ private PathHandle innerComplete( "Duplicate PartHandles"); byte[] uploadIdByteArray = multipartUploadId.toByteArray(); Path collectorPath = new Path(new String(uploadIdByteArray, 0, - uploadIdByteArray.length, Charsets.UTF_8)); + uploadIdByteArray.length, StandardCharsets.UTF_8)); boolean emptyFile = totalPartsLen(partHandles) == 0; if (emptyFile) { @@ -250,7 +250,7 @@ public CompletableFuture abort(UploadHandle uploadId, byte[] uploadIdByteArray = uploadId.toByteArray(); checkUploadId(uploadIdByteArray); Path collectorPath = new Path(new String(uploadIdByteArray, 0, - uploadIdByteArray.length, Charsets.UTF_8)); + uploadIdByteArray.length, StandardCharsets.UTF_8)); return FutureIO.eval(() -> { // force a check for a file existing; raises FNFE if not found diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CopyCommands.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CopyCommands.java index 1ac204f5f8a82..70c7fe381244d 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CopyCommands.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CopyCommands.java @@ -22,6 +22,7 @@ import java.io.InputStream; import java.net.URI; import java.net.URISyntaxException; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.util.Iterator; import java.util.LinkedList; @@ -114,7 +115,7 @@ protected void processArguments(LinkedList items) private void writeDelimiter(FSDataOutputStream out) throws IOException { if (delimiter != null) { - out.write(delimiter.getBytes("UTF-8")); + out.write(delimiter.getBytes(StandardCharsets.UTF_8)); } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HtmlQuoting.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HtmlQuoting.java index 5f47ddb339212..ca2687ce5f2d1 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HtmlQuoting.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HtmlQuoting.java @@ -120,7 +120,7 @@ public static String quoteHtmlChars(String item) { ByteArrayOutputStream buffer = new ByteArrayOutputStream(); try { quoteHtmlChars(buffer, bytes, 0, bytes.length); - return buffer.toString("UTF-8"); + return new String(buffer.toByteArray(), StandardCharsets.UTF_8); } catch (IOException ioe) { // Won't happen, since it is a bytearrayoutputstream return null; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/DefaultStringifier.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/DefaultStringifier.java index 7be50b0c539b9..92eff36ced289 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/DefaultStringifier.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/DefaultStringifier.java @@ -20,7 +20,6 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; -import java.nio.charset.UnsupportedCharsetException; import java.util.ArrayList; import org.apache.commons.codec.binary.Base64; @@ -75,14 +74,10 @@ public DefaultStringifier(Configuration conf, Class c) { @Override public T fromString(String str) throws IOException { - try { - byte[] bytes = Base64.decodeBase64(str.getBytes("UTF-8")); - inBuf.reset(bytes, bytes.length); - T restored = deserializer.deserialize(null); - return restored; - } catch (UnsupportedCharsetException ex) { - throw new IOException(ex.toString()); - } + byte[] bytes = Base64.decodeBase64(str.getBytes(StandardCharsets.UTF_8)); + inBuf.reset(bytes, bytes.length); + T restored = deserializer.deserialize(null); + return restored; } @Override diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/WritableUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/WritableUtils.java index 187398de0ec86..b46aed1208d76 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/WritableUtils.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/WritableUtils.java @@ -25,6 +25,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.util.ReflectionUtils; +import java.nio.charset.StandardCharsets; import java.util.zip.GZIPInputStream; import java.util.zip.GZIPOutputStream; @@ -86,12 +87,12 @@ public static int writeCompressedByteArray(DataOutput out, public static String readCompressedString(DataInput in) throws IOException { byte[] bytes = readCompressedByteArray(in); if (bytes == null) return null; - return new String(bytes, "UTF-8"); + return new String(bytes, StandardCharsets.UTF_8); } public static int writeCompressedString(DataOutput out, String s) throws IOException { - return writeCompressedByteArray(out, (s != null) ? s.getBytes("UTF-8") : null); + return writeCompressedByteArray(out, (s != null) ? s.getBytes(StandardCharsets.UTF_8) : null); } /* @@ -103,7 +104,7 @@ public static int writeCompressedString(DataOutput out, String s) throws IOExce */ public static void writeString(DataOutput out, String s) throws IOException { if (s != null) { - byte[] buffer = s.getBytes("UTF-8"); + byte[] buffer = s.getBytes(StandardCharsets.UTF_8); int len = buffer.length; out.writeInt(len); out.write(buffer, 0, len); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/log/LogLevel.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/log/LogLevel.java index 32879597a9c0a..a765ab83ca0e8 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/log/LogLevel.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/log/LogLevel.java @@ -23,6 +23,7 @@ import java.io.PrintWriter; import java.net.URL; import java.net.URLConnection; +import java.nio.charset.StandardCharsets; import java.util.regex.Pattern; import javax.net.ssl.HttpsURLConnection; @@ -33,7 +34,6 @@ import javax.servlet.http.HttpServletResponse; import org.apache.hadoop.classification.VisibleForTesting; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; import org.slf4j.LoggerFactory; import org.apache.hadoop.HadoopIllegalArgumentException; @@ -297,7 +297,7 @@ private void process(String urlString) throws Exception { // read from the servlet BufferedReader in = new BufferedReader( - new InputStreamReader(connection.getInputStream(), Charsets.UTF_8)); + new InputStreamReader(connection.getInputStream(), StandardCharsets.UTF_8)); for (String line;;) { line = in.readLine(); if (line == null) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsConfig.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsConfig.java index 2d22b75841b33..f4848fed519d8 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsConfig.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsConfig.java @@ -23,6 +23,8 @@ import java.net.URL; import java.net.URLClassLoader; import static java.security.AccessController.*; + +import java.nio.charset.StandardCharsets; import java.security.PrivilegedAction; import java.util.Iterator; import java.util.Map; @@ -289,7 +291,7 @@ static String toString(Configuration c) { PropertiesConfiguration tmp = new PropertiesConfiguration(); tmp.copy(c); tmp.write(pw); - return buffer.toString("UTF-8"); + return new String(buffer.toByteArray(), StandardCharsets.UTF_8); } catch (Exception e) { throw new MetricsConfigException(e); } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslPlainServer.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslPlainServer.java index 270b579324c86..8e3de21064e7e 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslPlainServer.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslPlainServer.java @@ -18,6 +18,7 @@ package org.apache.hadoop.security; +import java.nio.charset.StandardCharsets; import java.security.Provider; import java.util.Map; @@ -82,7 +83,7 @@ public byte[] evaluateResponse(byte[] response) throws SaslException { try { String payload; try { - payload = new String(response, "UTF-8"); + payload = new String(response, StandardCharsets.UTF_8); } catch (Exception e) { throw new IllegalArgumentException("Received corrupt response", e); } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/AbstractJavaKeyStoreProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/AbstractJavaKeyStoreProvider.java index 260f1d22496f0..d594d26515bfb 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/AbstractJavaKeyStoreProvider.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/AbstractJavaKeyStoreProvider.java @@ -24,7 +24,6 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.security.ProviderUtils; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -33,6 +32,7 @@ import java.io.InputStream; import java.io.OutputStream; import java.net.URI; +import java.nio.charset.StandardCharsets; import java.security.GeneralSecurityException; import java.security.KeyStore; import java.security.KeyStoreException; @@ -199,7 +199,7 @@ public CredentialEntry getCredentialEntry(String alias) public static char[] bytesToChars(byte[] bytes) throws IOException { String pass; - pass = new String(bytes, Charsets.UTF_8); + pass = new String(bytes, StandardCharsets.UTF_8); return pass.toCharArray(); } @@ -268,7 +268,7 @@ CredentialEntry innerSetCredential(String alias, char[] material) writeLock.lock(); try { keyStore.setKeyEntry(alias, - new SecretKeySpec(new String(material).getBytes("UTF-8"), + new SecretKeySpec(new String(material).getBytes(StandardCharsets.UTF_8), getAlgorithm()), password, null); } catch (KeyStoreException e) { throw new IOException("Can't store credential " + alias + " in " + this, diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/UserProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/UserProvider.java index 0c960d891b53b..2ae98f033735e 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/UserProvider.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/UserProvider.java @@ -70,7 +70,7 @@ public synchronized CredentialEntry createCredentialEntry(String name, char[] cr " already exists in " + this); } credentials.addSecretKey(new Text(name), - new String(credential).getBytes("UTF-8")); + new String(credential).getBytes(StandardCharsets.UTF_8)); return new CredentialEntry(name, credential); } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java index 925bc030c2cd8..0642d3d581066 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java @@ -24,18 +24,12 @@ import java.io.DataOutputStream; import java.io.IOException; import java.io.UncheckedIOException; -import java.util.Collections; -import java.util.List; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.ReentrantLock; import java.util.stream.Stream; -import org.apache.curator.ensemble.fixed.FixedEnsembleProvider; +import org.apache.curator.RetryPolicy; import org.apache.curator.framework.CuratorFramework; -import org.apache.curator.framework.CuratorFrameworkFactory; -import org.apache.curator.framework.CuratorFrameworkFactory.Builder; -import org.apache.curator.framework.api.ACLProvider; -import org.apache.curator.framework.imps.DefaultACLProvider; import org.apache.curator.framework.recipes.cache.ChildData; import org.apache.curator.framework.recipes.cache.CuratorCache; import org.apache.curator.framework.recipes.cache.CuratorCacheBridge; @@ -43,28 +37,28 @@ import org.apache.curator.framework.recipes.shared.SharedCount; import org.apache.curator.framework.recipes.shared.VersionedValue; import org.apache.curator.retry.RetryNTimes; +import org.apache.curator.utils.ZookeeperFactory; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.security.SecurityUtil; -import org.apache.hadoop.security.authentication.util.JaasConfiguration; +import org.apache.hadoop.fs.CommonConfigurationKeys; +import org.apache.hadoop.security.SecurityUtil.TruststoreKeystore; +import org.apache.hadoop.security.authentication.util.ZookeeperClient; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.delegation.web.DelegationTokenManager; + +import static org.apache.hadoop.security.SecurityUtil.getServerPrincipal; import static org.apache.hadoop.util.Time.now; -import org.apache.hadoop.util.curator.ZKCuratorManager; + +import org.apache.hadoop.util.curator.ZKCuratorManager.HadoopZookeeperFactory; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.NoNodeException; -import org.apache.zookeeper.ZooDefs.Perms; -import org.apache.zookeeper.client.ZKClientConfig; -import org.apache.zookeeper.data.ACL; -import org.apache.zookeeper.data.Id; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.classification.VisibleForTesting; -import org.apache.hadoop.util.Preconditions; /** * An implementation of {@link AbstractDelegationTokenSecretManager} that @@ -104,6 +98,16 @@ public abstract class ZKDelegationTokenSecretManager saslACL; - - private SASLOwnerACLProvider(String principal) { - this.saslACL = Collections.singletonList( - new ACL(Perms.ALL, new Id("sasl", principal))); - } - - @Override - public List getDefaultAcl() { - return saslACL; - } - - @Override - public List getAclForPath(String path) { - return saslACL; - } - } - @VisibleForTesting @Private @Unstable diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationFilter.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationFilter.java index 571e54c5f907c..1400e572d94f0 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationFilter.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationFilter.java @@ -51,7 +51,7 @@ import javax.servlet.http.HttpServletResponse; import java.io.IOException; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.security.Principal; import java.util.Enumeration; import java.util.List; @@ -94,8 +94,6 @@ public class DelegationTokenAuthenticationFilter public static final String DELEGATION_TOKEN_SECRET_MANAGER_ATTR = "hadoop.http.delegation-token-secret-manager"; - private static final Charset UTF8_CHARSET = Charset.forName("UTF-8"); - private static final ThreadLocal UGI_TL = new ThreadLocal(); public static final String PROXYUSER_PREFIX = "proxyuser"; @@ -226,7 +224,7 @@ static String getDoAs(HttpServletRequest request) { if (queryString == null) { return null; } - List list = URLEncodedUtils.parse(queryString, UTF8_CHARSET); + List list = URLEncodedUtils.parse(queryString, StandardCharsets.UTF_8); if (list != null) { for (NameValuePair nv : list) { if (DelegationTokenAuthenticatedURL.DO_AS. diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/ServletUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/ServletUtils.java index 078dfa44bddae..e43668e2c60d5 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/ServletUtils.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/ServletUtils.java @@ -23,7 +23,7 @@ import javax.servlet.http.HttpServletRequest; import java.io.IOException; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.util.List; /** @@ -31,7 +31,6 @@ */ @InterfaceAudience.Private class ServletUtils { - private static final Charset UTF8_CHARSET = Charset.forName("UTF-8"); /** * Extract a query string parameter without triggering http parameters @@ -49,7 +48,7 @@ public static String getParameter(HttpServletRequest request, String name) if (queryString == null) { return null; } - List list = URLEncodedUtils.parse(queryString, UTF8_CHARSET); + List list = URLEncodedUtils.parse(queryString, StandardCharsets.UTF_8); if (list != null) { for (NameValuePair nv : list) { if (name.equals(nv.getName())) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfoLinux.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfoLinux.java index 38777d8f66465..574d9062c3851 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfoLinux.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfoLinux.java @@ -22,7 +22,7 @@ import java.io.InputStreamReader; import java.io.IOException; import java.math.BigInteger; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Paths; import java.util.HashMap; @@ -247,7 +247,7 @@ private void readProcMemInfoFile(boolean readAgain) { try { fReader = new InputStreamReader( Files.newInputStream(Paths.get(procfsMemFile)), - Charset.forName("UTF-8")); + StandardCharsets.UTF_8); in = new BufferedReader(fReader); } catch (IOException f) { // shouldn't happen.... @@ -319,7 +319,7 @@ private void readProcCpuInfoFile() { try { fReader = new InputStreamReader(Files.newInputStream(Paths.get(procfsCpuFile)), - Charset.forName("UTF-8")); + StandardCharsets.UTF_8); in = new BufferedReader(fReader); } catch (IOException f) { // shouldn't happen.... @@ -380,7 +380,7 @@ private void readProcStatFile() { try { fReader = new InputStreamReader( Files.newInputStream(Paths.get(procfsStatFile)), - Charset.forName("UTF-8")); + StandardCharsets.UTF_8); in = new BufferedReader(fReader); } catch (IOException f) { // shouldn't happen.... @@ -435,7 +435,7 @@ private void readProcNetInfoFile() { try { fReader = new InputStreamReader( Files.newInputStream(Paths.get(procfsNetFile)), - Charset.forName("UTF-8")); + StandardCharsets.UTF_8); in = new BufferedReader(fReader); } catch (IOException f) { return; @@ -490,7 +490,7 @@ private void readProcDisksInfoFile() { try { in = new BufferedReader(new InputStreamReader( Files.newInputStream(Paths.get(procfsDisksFile)), - Charset.forName("UTF-8"))); + StandardCharsets.UTF_8)); } catch (IOException f) { return; } @@ -558,7 +558,7 @@ int readDiskBlockInformation(String diskName, int defSector) { try { in = new BufferedReader(new InputStreamReader( Files.newInputStream(Paths.get(procfsDiskSectorFile)), - Charset.forName("UTF-8"))); + StandardCharsets.UTF_8)); } catch (IOException f) { return defSector; } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ZKUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ZKUtil.java index 17d8233342743..4c9f418dafa83 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ZKUtil.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ZKUtil.java @@ -19,6 +19,7 @@ import java.io.File; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.List; import org.apache.hadoop.classification.InterfaceAudience; @@ -27,7 +28,6 @@ import org.apache.zookeeper.data.ACL; import org.apache.zookeeper.data.Id; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; import org.apache.hadoop.thirdparty.com.google.common.base.Splitter; import org.apache.hadoop.thirdparty.com.google.common.io.Files; @@ -148,7 +148,7 @@ public static List parseAuth(String authString) throws "Auth '" + comp + "' not of expected form scheme:auth"); } ret.add(new ZKAuthInfo(parts[0], - parts[1].getBytes(Charsets.UTF_8))); + parts[1].getBytes(StandardCharsets.UTF_8))); } return ret; } @@ -172,7 +172,7 @@ public static String resolveConfIndirection(String valInConf) return valInConf; } String path = valInConf.substring(1).trim(); - return Files.asCharSource(new File(path), Charsets.UTF_8).read().trim(); + return Files.asCharSource(new File(path), StandardCharsets.UTF_8).read().trim(); } /** diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/curator/ZKCuratorManager.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/curator/ZKCuratorManager.java index 3055e7bf659a6..a0cbf86fb9f9a 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/curator/ZKCuratorManager.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/curator/ZKCuratorManager.java @@ -18,7 +18,7 @@ package org.apache.hadoop.util.curator; import java.io.IOException; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.LinkedList; import java.util.List; @@ -260,7 +260,7 @@ public byte[] getData(final String path, Stat stat) throws Exception { public String getStringData(final String path) throws Exception { byte[] bytes = getData(path); if (bytes != null) { - return new String(bytes, Charset.forName("UTF-8")); + return new String(bytes, StandardCharsets.UTF_8); } return null; } @@ -275,7 +275,7 @@ public String getStringData(final String path) throws Exception { public String getStringData(final String path, Stat stat) throws Exception { byte[] bytes = getData(path, stat); if (bytes != null) { - return new String(bytes, Charset.forName("UTF-8")); + return new String(bytes, StandardCharsets.UTF_8); } return null; } @@ -299,7 +299,7 @@ public void setData(String path, byte[] data, int version) throws Exception { * @throws Exception If it cannot contact Zookeeper. */ public void setData(String path, String data, int version) throws Exception { - byte[] bytes = data.getBytes(Charset.forName("UTF-8")); + byte[] bytes = data.getBytes(StandardCharsets.UTF_8); setData(path, bytes, version); } diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml index c86fd8b98609b..7333f840f37e7 100644 --- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml +++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml @@ -4255,6 +4255,18 @@ The switch to turn S3A auditing on or off. + + hadoop.zk.ssl.enabled + + Enable SSL/TLS encryption for the ZooKeeper communication. + Note: this setting overrides dfs.ha.zkfc.client.ssl.enabled, + yarn.resourcemanager.zk-client-ssl.enabled and also + hadoop.kms.authentication.zk-dt-secret-manager.ssl.enabled in order to unify the SSL based + Zookeeper access across Hadoop. Leaving this property empty ensures that service specific + enablement can be done separately. + + + hadoop.zk.ssl.keystore.location diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java index b07ba76e8eec0..210b36f2dbd1f 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java @@ -60,7 +60,7 @@ public class TestCommonConfigurationFields extends TestConfigurationFieldsBase { @SuppressWarnings("deprecation") @Override public void initializeMemberVariables() { - xmlFilename = new String("core-default.xml"); + xmlFilename = "core-default.xml"; configurationClasses = new Class[] { CommonConfigurationKeys.class, CommonConfigurationKeysPublic.class, diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileUtil.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileUtil.java index 94d90b2eb978a..177223dc08254 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileUtil.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileUtil.java @@ -698,7 +698,7 @@ public void testUnTar() throws Exception { OutputStream os = new FileOutputStream(simpleTar); try (TarOutputStream tos = new TarOutputStream(os)) { TarEntry te = new TarEntry("/bar/foo"); - byte[] data = "some-content".getBytes("UTF-8"); + byte[] data = "some-content".getBytes(StandardCharsets.UTF_8); te.setSize(data.length); tos.putNextEntry(te); tos.write(data); @@ -782,7 +782,7 @@ public void testUnZip() throws Exception { ZipArchiveList.add(new ZipArchiveEntry("foo_" + i)); ZipArchiveEntry archiveEntry = ZipArchiveList.get(i); archiveEntry.setUnixMode(count += 0100); - byte[] data = "some-content".getBytes("UTF-8"); + byte[] data = "some-content".getBytes(StandardCharsets.UTF_8); archiveEntry.setSize(data.length); tos.putArchiveEntry(archiveEntry); tos.write(data); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystemBasics.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystemBasics.java index 471d2458f4f46..eccf491cca8e3 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystemBasics.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystemBasics.java @@ -30,6 +30,7 @@ import java.io.File; import java.io.IOException; import java.net.URI; +import java.nio.charset.StandardCharsets; import java.util.HashSet; import java.util.Set; @@ -117,7 +118,7 @@ private void writeVersionToMasterIndexImpl(int version, Path masterIndexPath) th final FSDataOutputStream fsdos = localFileSystem.create(masterIndexPath); try { String versionString = version + "\n"; - fsdos.write(versionString.getBytes("UTF-8")); + fsdos.write(versionString.getBytes(StandardCharsets.UTF_8)); fsdos.flush(); } finally { fsdos.close(); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMultipartUploaderTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMultipartUploaderTest.java index c395afdb3779b..7420b47a98495 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMultipartUploaderTest.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMultipartUploaderTest.java @@ -22,13 +22,13 @@ import java.io.IOException; import java.io.InputStream; import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; import java.security.MessageDigest; import java.util.HashMap; import java.util.Map; import java.util.Random; import java.util.concurrent.CompletableFuture; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; import org.assertj.core.api.Assertions; import org.junit.Assume; import org.junit.Test; @@ -596,8 +596,8 @@ public void testMultipartUploadAbort() throws Exception { abortUpload(uploadHandle, file); String contents = "ThisIsPart49\n"; - int len = contents.getBytes(Charsets.UTF_8).length; - InputStream is = IOUtils.toInputStream(contents, "UTF-8"); + int len = contents.getBytes(StandardCharsets.UTF_8).length; + InputStream is = IOUtils.toInputStream(contents, StandardCharsets.UTF_8); intercept(IOException.class, () -> awaitFuture( @@ -624,7 +624,7 @@ public void testMultipartUploadAbort() throws Exception { public void testAbortUnknownUpload() throws Exception { Path file = methodPath(); ByteBuffer byteBuffer = ByteBuffer.wrap( - "invalid-handle".getBytes(Charsets.UTF_8)); + "invalid-handle".getBytes(StandardCharsets.UTF_8)); intercept(FileNotFoundException.class, () -> abortUpload(BBUploadHandle.from(byteBuffer), file)); } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java index b61abddd43426..bbccbfbc16eef 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java @@ -45,6 +45,7 @@ import java.io.InputStream; import java.io.OutputStream; import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -806,7 +807,7 @@ public static String readUTF8(FileSystem fs, try (FSDataInputStream in = fs.open(path)) { byte[] buf = new byte[length]; in.readFully(0, buf); - return new String(buf, "UTF-8"); + return new String(buf, StandardCharsets.UTF_8); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestHAAdmin.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestHAAdmin.java index 13f7eccd55aea..a027b4d682b9f 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestHAAdmin.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestHAAdmin.java @@ -23,6 +23,7 @@ import java.io.IOException; import java.io.PrintStream; import java.net.InetSocketAddress; +import java.nio.charset.StandardCharsets; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState; @@ -30,7 +31,6 @@ import org.junit.Before; import org.junit.Test; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; import org.apache.hadoop.thirdparty.com.google.common.base.Joiner; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -97,8 +97,8 @@ private Object runTool(String ... args) throws Exception { outBytes.reset(); LOG.info("Running: HAAdmin " + Joiner.on(" ").join(args)); int ret = tool.run(args); - errOutput = new String(errOutBytes.toByteArray(), Charsets.UTF_8); - output = new String(outBytes.toByteArray(), Charsets.UTF_8); + errOutput = new String(errOutBytes.toByteArray(), StandardCharsets.UTF_8); + output = new String(outBytes.toByteArray(), StandardCharsets.UTF_8); LOG.info("Err_output:\n" + errOutput + "\nOutput:\n" + output); return ret; } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestIsActiveServlet.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestIsActiveServlet.java index cfc7e359e94d4..22bea17a7c063 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestIsActiveServlet.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestIsActiveServlet.java @@ -27,6 +27,7 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.PrintWriter; +import java.nio.charset.StandardCharsets; import static org.junit.Assert.assertEquals; import static org.mockito.ArgumentMatchers.anyInt; @@ -90,6 +91,6 @@ protected boolean isActive() { private String doGet() throws IOException { servlet.doGet(req, resp); - return new String(respOut.toByteArray(), "UTF-8"); + return new String(respOut.toByteArray(), StandardCharsets.UTF_8); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestSecureIOUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestSecureIOUtils.java index 91c0f1b4429d5..f9a5a30966419 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestSecureIOUtils.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestSecureIOUtils.java @@ -23,6 +23,7 @@ import java.io.File; import java.io.FileOutputStream; import java.io.IOException; +import java.nio.charset.StandardCharsets; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; @@ -57,7 +58,7 @@ public static void makeTestFile() throws Exception { for (File f : new File[] { testFilePathIs, testFilePathRaf, testFilePathFadis }) { FileOutputStream fos = new FileOutputStream(f); - fos.write("hello".getBytes("UTF-8")); + fos.write("hello".getBytes(StandardCharsets.UTF_8)); fos.close(); } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestText.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestText.java index 7ae5d7d7ca051..4eb260c033596 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestText.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestText.java @@ -22,8 +22,8 @@ import java.nio.BufferUnderflowException; import java.nio.ByteBuffer; import java.nio.charset.CharacterCodingException; +import java.nio.charset.StandardCharsets; import java.util.Random; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; import org.apache.hadoop.thirdparty.com.google.common.primitives.Bytes; import org.junit.Test; @@ -105,7 +105,7 @@ public void testCoding() throws Exception { ByteBuffer bb = Text.encode(before); byte[] utf8Text = bb.array(); - byte[] utf8Java = before.getBytes("UTF-8"); + byte[] utf8Java = before.getBytes(StandardCharsets.UTF_8); assertEquals(0, WritableComparator.compareBytes( utf8Text, 0, bb.limit(), utf8Java, 0, utf8Java.length)); @@ -392,7 +392,7 @@ public void testReadWriteOperations() { @Test public void testReadWithKnownLength() throws IOException { String line = "hello world"; - byte[] inputBytes = line.getBytes(Charsets.UTF_8); + byte[] inputBytes = line.getBytes(StandardCharsets.UTF_8); DataInputBuffer in = new DataInputBuffer(); Text text = new Text(); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestUTF8.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestUTF8.java index 2d60b5ecca184..6899d1cdcabf7 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestUTF8.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestUTF8.java @@ -23,6 +23,7 @@ import java.io.IOException; import java.io.UTFDataFormatException; import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; import java.util.Random; import org.apache.hadoop.test.GenericTestUtils; @@ -110,7 +111,7 @@ public void testNullEncoding() throws Exception { DataOutputBuffer dob = new DataOutputBuffer(); new UTF8(s).write(dob); - assertEquals(s, new String(dob.getData(), 2, dob.getLength()-2, "UTF-8")); + assertEquals(s, new String(dob.getData(), 2, dob.getLength()-2, StandardCharsets.UTF_8)); } /** @@ -125,7 +126,7 @@ public void testNonBasicMultilingualPlane() throws Exception { String catFace = "\uD83D\uDC31"; // This encodes to 4 bytes in UTF-8: - byte[] encoded = catFace.getBytes("UTF-8"); + byte[] encoded = catFace.getBytes(StandardCharsets.UTF_8); assertEquals(4, encoded.length); assertEquals("f09f90b1", StringUtils.byteToHexString(encoded)); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/sink/TestFileSink.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/sink/TestFileSink.java index 67889405c1068..420c16bef577e 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/sink/TestFileSink.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/sink/TestFileSink.java @@ -23,6 +23,7 @@ import java.io.FileInputStream; import java.io.IOException; import java.io.InputStream; +import java.nio.charset.StandardCharsets; import java.util.regex.Pattern; import org.apache.hadoop.io.IOUtils; @@ -113,7 +114,7 @@ public void testFileSink() throws IOException { is = new FileInputStream(outFile); baos = new ByteArrayOutputStream((int)outFile.length()); IOUtils.copyBytes(is, baos, 1024, true); - outFileContent = new String(baos.toByteArray(), "UTF-8"); + outFileContent = new String(baos.toByteArray(), StandardCharsets.UTF_8); } finally { IOUtils.cleanupWithLogger(null, baos, is); } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/sink/TestStatsDMetrics.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/sink/TestStatsDMetrics.java index 99a75787ad841..2ce02f74f196d 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/sink/TestStatsDMetrics.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/sink/TestStatsDMetrics.java @@ -25,7 +25,7 @@ import java.io.IOException; import java.net.DatagramPacket; import java.net.DatagramSocket; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.HashSet; import java.util.List; @@ -75,7 +75,7 @@ public void testPutMetrics() throws IOException, IllegalAccessException { sock.receive(p); String result =new String(p.getData(), 0, p.getLength(), - Charset.forName("UTF-8")); + StandardCharsets.UTF_8); assertTrue( "Received data did not match data sent", result.equals("host.process.jvm.Context.foo1:1.25|c") || @@ -109,7 +109,7 @@ public void testPutMetrics2() throws IOException, IllegalAccessException { sink.putMetrics(record); sock.receive(p); String result = - new String(p.getData(), 0, p.getLength(), Charset.forName("UTF-8")); + new String(p.getData(), 0, p.getLength(), StandardCharsets.UTF_8); assertTrue("Received data did not match data sent", result.equals("process.jvm.Context.foo1:1|c") || diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestTableMapping.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestTableMapping.java index 697b0bad43757..50fe0c098f478 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestTableMapping.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestTableMapping.java @@ -21,11 +21,11 @@ import static org.junit.Assert.assertEquals; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; import org.apache.hadoop.thirdparty.com.google.common.io.Files; import java.io.File; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.List; @@ -41,7 +41,7 @@ public class TestTableMapping { public void testResolve() throws IOException { File mapFile = File.createTempFile(getClass().getSimpleName() + ".testResolve", ".txt"); - Files.asCharSink(mapFile, Charsets.UTF_8).write( + Files.asCharSink(mapFile, StandardCharsets.UTF_8).write( hostName1 + " /rack1\n" + hostName2 + "\t/rack2\n"); mapFile.deleteOnExit(); TableMapping mapping = new TableMapping(); @@ -64,7 +64,7 @@ public void testResolve() throws IOException { public void testTableCaching() throws IOException { File mapFile = File.createTempFile(getClass().getSimpleName() + ".testTableCaching", ".txt"); - Files.asCharSink(mapFile, Charsets.UTF_8).write( + Files.asCharSink(mapFile, StandardCharsets.UTF_8).write( hostName1 + " /rack1\n" + hostName2 + "\t/rack2\n"); mapFile.deleteOnExit(); TableMapping mapping = new TableMapping(); @@ -128,7 +128,7 @@ public void testFileDoesNotExist() { public void testClearingCachedMappings() throws IOException { File mapFile = File.createTempFile(getClass().getSimpleName() + ".testClearingCachedMappings", ".txt"); - Files.asCharSink(mapFile, Charsets.UTF_8).write( + Files.asCharSink(mapFile, StandardCharsets.UTF_8).write( hostName1 + " /rack1\n" + hostName2 + "\t/rack2\n"); mapFile.deleteOnExit(); @@ -147,7 +147,7 @@ public void testClearingCachedMappings() throws IOException { assertEquals("/rack1", result.get(0)); assertEquals("/rack2", result.get(1)); - Files.asCharSink(mapFile, Charsets.UTF_8).write(""); + Files.asCharSink(mapFile, StandardCharsets.UTF_8).write(""); mapping.reloadCachedMappings(); @@ -166,7 +166,7 @@ public void testClearingCachedMappings() throws IOException { public void testBadFile() throws IOException { File mapFile = File.createTempFile(getClass().getSimpleName() + ".testBadFile", ".txt"); - Files.asCharSink(mapFile, Charsets.UTF_8).write("bad contents"); + Files.asCharSink(mapFile, StandardCharsets.UTF_8).write("bad contents"); mapFile.deleteOnExit(); TableMapping mapping = new TableMapping(); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/TestZKDelegationTokenSecretManager.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/TestZKDelegationTokenSecretManager.java index 2312af3c79dfa..25dae7e4fd5cc 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/TestZKDelegationTokenSecretManager.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/TestZKDelegationTokenSecretManager.java @@ -19,6 +19,7 @@ package org.apache.hadoop.security.token.delegation; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -400,7 +401,7 @@ public List getDefaultAcl() { .connectString(connectString) .retryPolicy(retryPolicy) .aclProvider(digestAclProvider) - .authorization("digest", userPass.getBytes("UTF-8")) + .authorization("digest", userPass.getBytes(StandardCharsets.UTF_8)) .build(); curatorFramework.start(); ZKDelegationTokenSecretManager.setCurator(curatorFramework); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestClasspath.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestClasspath.java index 529887b297553..716dfe0c36d56 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestClasspath.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestClasspath.java @@ -24,6 +24,7 @@ import java.io.IOException; import java.io.PrintStream; import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.util.jar.Attributes; import java.util.jar.JarFile; import java.util.jar.Manifest; @@ -46,7 +47,7 @@ public class TestClasspath { .class); private static final File TEST_DIR = GenericTestUtils.getTestDir( "TestClasspath"); - private static final Charset UTF8 = Charset.forName("UTF-8"); + private static final Charset UTF8 = StandardCharsets.UTF_8; static { ExitUtil.disableSystemExit(); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestPureJavaCrc32.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestPureJavaCrc32.java index 7fb90a15e3ad5..bf3e58793bb24 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestPureJavaCrc32.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestPureJavaCrc32.java @@ -21,6 +21,7 @@ import java.io.FileOutputStream; import java.io.PrintStream; import java.lang.reflect.Constructor; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.List; import java.util.Properties; @@ -49,7 +50,7 @@ public void testCorrectness() throws Exception { checkOnBytes(new byte[] {40, 60, 97, -70}, false); - checkOnBytes("hello world!".getBytes("UTF-8"), false); + checkOnBytes("hello world!".getBytes(StandardCharsets.UTF_8), false); for (int i = 0; i < 10000; i++) { byte randomBytes[] = new byte[new Random().nextInt(2048)]; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestZKUtil.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestZKUtil.java index 93790eb1350a7..d12fff2732cd6 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestZKUtil.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestZKUtil.java @@ -22,6 +22,7 @@ import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.List; import org.apache.hadoop.test.GenericTestUtils; @@ -31,7 +32,6 @@ import org.apache.zookeeper.data.ACL; import org.junit.Test; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; import org.apache.hadoop.thirdparty.com.google.common.io.Files; public class TestZKUtil { @@ -131,7 +131,7 @@ public void testConfIndirection() throws IOException { assertEquals("x", ZKUtil.resolveConfIndirection("x")); TEST_FILE.getParentFile().mkdirs(); - Files.asCharSink(TEST_FILE, Charsets.UTF_8).write("hello world"); + Files.asCharSink(TEST_FILE, StandardCharsets.UTF_8).write("hello world"); assertEquals("hello world", ZKUtil.resolveConfIndirection( "@" + TEST_FILE.getAbsolutePath())); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/curator/TestZKCuratorManager.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/curator/TestZKCuratorManager.java index aced6e8d28b27..69cba8f4ad01d 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/curator/TestZKCuratorManager.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/curator/TestZKCuratorManager.java @@ -22,6 +22,7 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -117,7 +118,7 @@ public void testGetStringData() throws Exception { curator.create(node1); assertNull(curator.getStringData(node1)); - byte[] setData = "setData".getBytes("UTF-8"); + byte[] setData = "setData".getBytes(StandardCharsets.UTF_8); curator.setData(node1, setData, -1); assertEquals("setData", curator.getStringData(node1)); @@ -136,7 +137,7 @@ public void testTransaction() throws Exception { String fencingNodePath = "/fencing"; String node1 = "/node1"; String node2 = "/node2"; - byte[] testData = "testData".getBytes("UTF-8"); + byte[] testData = "testData".getBytes(StandardCharsets.UTF_8); assertFalse(curator.exists(fencingNodePath)); assertFalse(curator.exists(node1)); assertFalse(curator.exists(node2)); @@ -154,7 +155,7 @@ public void testTransaction() throws Exception { assertTrue(Arrays.equals(testData, curator.getData(node1))); assertTrue(Arrays.equals(testData, curator.getData(node2))); - byte[] setData = "setData".getBytes("UTF-8"); + byte[] setData = "setData".getBytes(StandardCharsets.UTF_8); txn = curator.createTransaction(zkAcl, fencingNodePath); txn.setData(node1, setData, -1); txn.delete(node2); diff --git a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSJSONWriter.java b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSJSONWriter.java index b9b8d9cee6673..e29ce4ba5f0f5 100644 --- a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSJSONWriter.java +++ b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSJSONWriter.java @@ -33,7 +33,7 @@ import java.io.Writer; import java.lang.annotation.Annotation; import java.lang.reflect.Type; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.util.List; import java.util.Map; @@ -64,8 +64,7 @@ public void writeTo(Object obj, Class aClass, Type type, Annotation[] annotations, MediaType mediaType, MultivaluedMap stringObjectMultivaluedMap, OutputStream outputStream) throws IOException, WebApplicationException { - Writer writer = new OutputStreamWriter(outputStream, Charset - .forName("UTF-8")); + Writer writer = new OutputStreamWriter(outputStream, StandardCharsets.UTF_8); JsonSerialization.writer().writeValue(writer, obj); } diff --git a/hadoop-common-project/hadoop-registry/src/main/java/org/apache/hadoop/registry/client/binding/JsonSerDeser.java b/hadoop-common-project/hadoop-registry/src/main/java/org/apache/hadoop/registry/client/binding/JsonSerDeser.java index 04aabfc635bed..dafd3e2d00b4a 100644 --- a/hadoop-common-project/hadoop-registry/src/main/java/org/apache/hadoop/registry/client/binding/JsonSerDeser.java +++ b/hadoop-common-project/hadoop-registry/src/main/java/org/apache/hadoop/registry/client/binding/JsonSerDeser.java @@ -28,6 +28,7 @@ import java.io.EOFException; import java.io.IOException; +import java.nio.charset.StandardCharsets; /** * Support for marshalling objects to and from JSON. @@ -47,7 +48,6 @@ @InterfaceStability.Evolving public class JsonSerDeser extends JsonSerialization { - private static final String UTF_8 = "UTF-8"; public static final String E_NO_DATA = "No data at path"; public static final String E_DATA_TOO_SHORT = "Data at path too short"; public static final String E_MISSING_MARKER_STRING = @@ -102,7 +102,7 @@ public T fromBytes(String path, byte[] bytes, String marker) if (StringUtils.isNotEmpty(marker) && len < marker.length()) { throw new NoRecordException(path, E_DATA_TOO_SHORT); } - String json = new String(bytes, 0, len, UTF_8); + String json = new String(bytes, 0, len, StandardCharsets.UTF_8); if (StringUtils.isNotEmpty(marker) && !json.contains(marker)) { throw new NoRecordException(path, E_MISSING_MARKER_STRING + marker); diff --git a/hadoop-common-project/hadoop-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistrySecurity.java b/hadoop-common-project/hadoop-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistrySecurity.java index d48920a222333..0dc587f64e613 100644 --- a/hadoop-common-project/hadoop-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistrySecurity.java +++ b/hadoop-common-project/hadoop-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistrySecurity.java @@ -42,6 +42,7 @@ import javax.security.auth.login.AppConfigurationEntry; import java.io.File; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.security.NoSuchAlgorithmException; import java.util.ArrayList; import java.util.Collections; @@ -295,7 +296,7 @@ private void initSecurity() throws IOException { digestAuthUser = id; digestAuthPassword = pass; String authPair = id + ":" + pass; - digestAuthData = authPair.getBytes("UTF-8"); + digestAuthData = authPair.getBytes(StandardCharsets.UTF_8); if (LOG.isDebugEnabled()) { LOG.debug("Auth is Digest ACL: {}", aclToString(acl)); } diff --git a/hadoop-common-project/hadoop-registry/src/main/java/org/apache/hadoop/registry/server/dns/RegistryDNS.java b/hadoop-common-project/hadoop-registry/src/main/java/org/apache/hadoop/registry/server/dns/RegistryDNS.java index 02d866f4d3bc1..b6de757fc3c17 100644 --- a/hadoop-common-project/hadoop-registry/src/main/java/org/apache/hadoop/registry/server/dns/RegistryDNS.java +++ b/hadoop-common-project/hadoop-registry/src/main/java/org/apache/hadoop/registry/server/dns/RegistryDNS.java @@ -80,6 +80,7 @@ import java.nio.channels.DatagramChannel; import java.nio.channels.ServerSocketChannel; import java.nio.channels.SocketChannel; +import java.nio.charset.StandardCharsets; import java.security.KeyFactory; import java.security.NoSuchAlgorithmException; import java.security.PrivateKey; @@ -628,7 +629,7 @@ private void enableDNSSECIfNecessary(Zone zone, Configuration conf, Name zoneName = zone.getOrigin(); DNSKEYRecord dnskeyRecord = dnsKeyRecs.get(zoneName); if (dnskeyRecord == null) { - byte[] key = Base64.decodeBase64(publicKey.getBytes("UTF-8")); + byte[] key = Base64.decodeBase64(publicKey.getBytes(StandardCharsets.UTF_8)); dnskeyRecord = new DNSKEYRecord(zoneName, DClass.IN, ttl, DNSKEYRecord.Flags.ZONE_KEY, diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java index 71cff2e3915b0..b2fc472aad835 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java @@ -661,9 +661,12 @@ public static boolean isValidName(String src) { String[] components = StringUtils.split(src, '/'); for (int i = 0; i < components.length; i++) { String element = components[i]; + // For Windows, we must allow the : in the drive letter. + if (Shell.WINDOWS && i == 1 && element.endsWith(":")) { + continue; + } if (element.equals(".") || - // For Windows, we must allow the : in the drive letter. - (!Shell.WINDOWS && i == 1 && element.contains(":")) || + (element.contains(":")) || (element.contains("/"))) { return false; } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataTransferSaslUtil.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataTransferSaslUtil.java index ab5cd0608d9d4..4749890ce139a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataTransferSaslUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataTransferSaslUtil.java @@ -30,6 +30,7 @@ import java.io.InputStream; import java.io.OutputStream; import java.net.InetAddress; +import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.List; import java.util.Map; @@ -59,7 +60,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet; import org.apache.hadoop.thirdparty.com.google.common.collect.Maps; import org.apache.hadoop.thirdparty.com.google.common.net.InetAddresses; @@ -147,7 +147,7 @@ public static Map createSaslPropertiesForEncryption( * @return key encoded as SASL password */ public static char[] encryptionKeyToPassword(byte[] encryptionKey) { - return new String(Base64.encodeBase64(encryptionKey, false), Charsets.UTF_8) + return new String(Base64.encodeBase64(encryptionKey, false), StandardCharsets.UTF_8) .toCharArray(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferClient.java index 641c7a0ff4790..043439130d5dc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferClient.java @@ -30,6 +30,7 @@ import java.io.OutputStream; import java.net.InetAddress; import java.net.Socket; +import java.nio.charset.StandardCharsets; import java.util.List; import java.util.Map; import java.util.concurrent.atomic.AtomicBoolean; @@ -65,7 +66,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; /** * Negotiates SASL for DataTransferProtocol on behalf of a client. There are @@ -347,7 +347,7 @@ private static String getUserNameFromEncryptionKey( return encryptionKey.keyId + NAME_DELIMITER + encryptionKey.blockPoolId + NAME_DELIMITER + new String(Base64.encodeBase64(encryptionKey.nonce, false), - Charsets.UTF_8); + StandardCharsets.UTF_8); } /** @@ -450,7 +450,7 @@ private IOStreamPair getSaslStreams(InetAddress addr, private void updateToken(Token accessToken, SecretKey secretKey, Map saslProps) throws IOException { - byte[] newSecret = saslProps.get(Sasl.QOP).getBytes(Charsets.UTF_8); + byte[] newSecret = saslProps.get(Sasl.QOP).getBytes(StandardCharsets.UTF_8); BlockTokenIdentifier bkid = accessToken.decodeIdentifier(); bkid.setHandshakeMsg(newSecret); byte[] bkidBytes = bkid.getBytes(); @@ -471,7 +471,7 @@ private void updateToken(Token accessToken, */ private static String buildUserName(Token blockToken) { return new String(Base64.encodeBase64(blockToken.getIdentifier(), false), - Charsets.UTF_8); + StandardCharsets.UTF_8); } /** @@ -483,7 +483,7 @@ private static String buildUserName(Token blockToken) { */ private char[] buildClientPassword(Token blockToken) { return new String(Base64.encodeBase64(blockToken.getPassword(), false), - Charsets.UTF_8).toCharArray(); + StandardCharsets.UTF_8).toCharArray(); } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/CombinedHostsFileReader.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/CombinedHostsFileReader.java index 91ab48fa9d471..c7724ce6db486 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/CombinedHostsFileReader.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/CombinedHostsFileReader.java @@ -27,6 +27,7 @@ import java.io.InputStreamReader; import java.io.IOException; import java.io.Reader; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Paths; import java.util.ArrayList; @@ -84,7 +85,7 @@ private CombinedHostsFileReader() { if (hostFile.length() > 0) { try (Reader input = new InputStreamReader( - Files.newInputStream(hostFile.toPath()), "UTF-8")) { + Files.newInputStream(hostFile.toPath()), StandardCharsets.UTF_8)) { allDNs = objectMapper.readValue(input, DatanodeAdminProperties[].class); } catch (JsonMappingException jme) { // The old format doesn't have json top-level token to enclose @@ -103,7 +104,7 @@ private CombinedHostsFileReader() { List all = new ArrayList<>(); try (Reader input = new InputStreamReader(Files.newInputStream(Paths.get(hostsFilePath)), - "UTF-8")) { + StandardCharsets.UTF_8)) { Iterator iterator = objectReader.readValues(jsonFactory.createParser(input)); while (iterator.hasNext()) { diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/CombinedHostsFileWriter.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/CombinedHostsFileWriter.java index 7897dc1ebf6e2..de4c12d556cc7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/CombinedHostsFileWriter.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/CombinedHostsFileWriter.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.io.OutputStreamWriter; import java.io.Writer; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Paths; import java.util.Set; @@ -62,7 +63,7 @@ public static void writeFile(final String hostsFile, try (Writer output = new OutputStreamWriter(Files.newOutputStream(Paths.get(hostsFile)), - "UTF-8")) { + StandardCharsets.UTF_8)) { objectMapper.writeValue(output, allDNs); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java index 689a7c78d4e84..a8bd95b32a252 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java @@ -137,7 +137,6 @@ import org.slf4j.LoggerFactory; import org.apache.hadoop.classification.VisibleForTesting; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; import org.apache.hadoop.util.Preconditions; /** A FileSystem for HDFS over the web. */ @@ -1792,7 +1791,7 @@ public DirectoryEntries listStatusBatch(Path f, byte[] token) throws } DirectoryListing listing = new FsPathResponseRunner( GetOpParam.Op.LISTSTATUS_BATCH, - f, new StartAfterParam(new String(prevKey, Charsets.UTF_8))) { + f, new StartAfterParam(new String(prevKey, StandardCharsets.UTF_8))) { @Override DirectoryListing decodeResponse(Map json) throws IOException { return JsonUtilClient.toDirectoryListing(json); diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsContentLength.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsContentLength.java index 6ee8858df991e..5577bb6266486 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsContentLength.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsContentLength.java @@ -22,6 +22,7 @@ import java.net.InetSocketAddress; import java.net.ServerSocket; import java.net.Socket; +import java.nio.charset.StandardCharsets; import java.util.concurrent.Callable; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -204,7 +205,7 @@ public String call() throws Exception { if (n <= 0) { break; } - sb.append(new String(buf, 0, n, "UTF-8")); + sb.append(new String(buf, 0, n, StandardCharsets.UTF_8)); } return sb.toString(); } finally { diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java index 65b49cc9cf1ac..dab4776575bff 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.fs.http.client; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -24,7 +25,6 @@ import java.util.List; import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyInfo; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.type.MapType; import org.apache.hadoop.classification.InterfaceAudience; @@ -796,7 +796,7 @@ public DirectoryEntries listStatusBatch(Path f, byte[] token) throws Map params = new HashMap(); params.put(OP_PARAM, Operation.LISTSTATUS_BATCH.toString()); if (token != null) { - params.put(START_AFTER_PARAM, new String(token, Charsets.UTF_8)); + params.put(START_AFTER_PARAM, new String(token, StandardCharsets.UTF_8)); } HttpURLConnection conn = getConnection( Operation.LISTSTATUS_BATCH.getMethod(), @@ -811,7 +811,7 @@ public DirectoryEntries listStatusBatch(Path f, byte[] token) throws byte[] newToken = null; if (statuses.length > 0) { newToken = statuses[statuses.length - 1].getPath().getName().toString() - .getBytes(Charsets.UTF_8); + .getBytes(StandardCharsets.UTF_8); } // Parse the remainingEntries boolean into hasMore final long remainingEntries = (Long) listing.get(REMAINING_ENTRIES_JSON); diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java index 57a79a184741c..de4cf83248145 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java @@ -18,7 +18,6 @@ package org.apache.hadoop.fs.http.server; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -91,6 +90,7 @@ import java.io.IOException; import java.io.InputStream; import java.net.URI; +import java.nio.charset.StandardCharsets; import java.security.AccessControlException; import java.security.PrivilegedExceptionAction; import java.text.MessageFormat; @@ -422,7 +422,7 @@ public InputStream run() throws Exception { HttpFSParametersProvider.StartAfterParam.class); byte[] token = HttpFSUtils.EMPTY_BYTES; if (startAfter != null) { - token = startAfter.getBytes(Charsets.UTF_8); + token = startAfter.getBytes(StandardCharsets.UTF_8); } FSOperations.FSListStatusBatch command = new FSOperations .FSListStatusBatch(path, token); diff --git a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java index 30a6a844e28a9..9b96a92cb4bf0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java +++ b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java @@ -25,7 +25,7 @@ import java.net.InetSocketAddress; import java.net.SocketAddress; import java.nio.ByteBuffer; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.util.EnumSet; import io.netty.buffer.ByteBuf; @@ -681,15 +681,15 @@ READLINK3Response readlink(XDR xdr, SecurityHandler securityHandler, } int rtmax = config.getInt(NfsConfigKeys.DFS_NFS_MAX_READ_TRANSFER_SIZE_KEY, NfsConfigKeys.DFS_NFS_MAX_READ_TRANSFER_SIZE_DEFAULT); - if (rtmax < target.getBytes(Charset.forName("UTF-8")).length) { + if (rtmax < target.getBytes(StandardCharsets.UTF_8).length) { LOG.error("Link size: {} is larger than max transfer size: {}", - target.getBytes(Charset.forName("UTF-8")).length, rtmax); + target.getBytes(StandardCharsets.UTF_8).length, rtmax); return new READLINK3Response(Nfs3Status.NFS3ERR_IO, postOpAttr, new byte[0]); } return new READLINK3Response(Nfs3Status.NFS3_OK, postOpAttr, - target.getBytes(Charset.forName("UTF-8"))); + target.getBytes(StandardCharsets.UTF_8)); } catch (IOException e) { LOG.warn("Readlink error", e); @@ -1515,7 +1515,7 @@ private DirectoryListing listPaths(DFSClient dfsClient, String dirFileIdPath, } // This happens when startAfter was just deleted LOG.info("Cookie couldn't be found: {}, do listing from beginning", - new String(startAfter, Charset.forName("UTF-8"))); + new String(startAfter, StandardCharsets.UTF_8)); dlisting = dfsClient .listPaths(dirFileIdPath, HdfsFileStatus.EMPTY_NAME); } @@ -1628,7 +1628,7 @@ public READDIR3Response readdir(XDR xdr, SecurityHandler securityHandler, startAfter = HdfsFileStatus.EMPTY_NAME; } else { String inodeIdPath = Nfs3Utils.getFileIdPath(cookie); - startAfter = inodeIdPath.getBytes(Charset.forName("UTF-8")); + startAfter = inodeIdPath.getBytes(StandardCharsets.UTF_8); } dlisting = listPaths(dfsClient, dirFileIdPath, startAfter); @@ -1800,7 +1800,7 @@ READDIRPLUS3Response readdirplus(XDR xdr, SecurityHandler securityHandler, startAfter = HdfsFileStatus.EMPTY_NAME; } else { String inodeIdPath = Nfs3Utils.getFileIdPath(cookie); - startAfter = inodeIdPath.getBytes(Charset.forName("UTF-8")); + startAfter = inodeIdPath.getBytes(StandardCharsets.UTF_8); } dlisting = listPaths(dfsClient, dirFileIdPath, startAfter); diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MembershipNamenodeResolver.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MembershipNamenodeResolver.java index c0e800e0430d4..933ecf070091a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MembershipNamenodeResolver.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MembershipNamenodeResolver.java @@ -79,7 +79,7 @@ public class MembershipNamenodeResolver * name and a boolean indicating if observer namenodes should be listed first. * If true, observer namenodes are listed first. If false, active namenodes are listed first. * Invalidated on cache refresh. */ - private Map, List> cacheNS; + private Map, List> cacheNS; /** Cached lookup of NN for block pool. Invalidated on cache refresh. */ private Map> cacheBP; @@ -483,9 +483,9 @@ public void setRouterId(String router) { * Rotate cache, make the current namenode have the lowest priority, * to ensure that the current namenode will not be accessed first next time. * - * @param nsId name service id - * @param namenode namenode contexts - * @param listObserversFirst Observer read case, observer NN will be ranked first + * @param nsId name service id. + * @param namenode namenode contexts. + * @param listObserversFirst Observer read case, observer NN will be ranked first. */ @Override public void rotateCache( @@ -494,29 +494,32 @@ public void rotateCache( if (namenodeContexts == null || namenodeContexts.size() <= 1) { return namenodeContexts; } - FederationNamenodeContext firstNamenodeContext = namenodeContexts.get(0); - /* - * If the first nn in the cache is active, the active nn priority cannot be lowered. - * This happens when other threads have already updated the cache. - */ - if (firstNamenodeContext.getState().equals(ACTIVE)) { - return namenodeContexts; + + // If there is active nn, rotateCache is not needed + // because the router has already loaded the cache. + for (FederationNamenodeContext namenodeContext : namenodeContexts) { + if (namenodeContext.getState() == ACTIVE) { + return namenodeContexts; + } } - /* - * If the first nn in the cache at this time is not the nn - * that needs to be lowered in priority, there is no need to rotate. - * This happens when other threads have already rotated the cache. - */ - if (firstNamenodeContext.getRpcAddress().equals(namenode.getRpcAddress())) { - List rotatedNnContexts = new ArrayList<>(namenodeContexts); - Collections.rotate(rotatedNnContexts, -1); - String firstNamenodeId = namenodeContexts.get(0).getNamenodeId(); - LOG.info("Rotate cache of pair , put namenode: {} in the " + - "first position of the cache and namenode: {} in the last position of the cache", - nsId, listObserversFirst, firstNamenodeId, namenode.getNamenodeId()); - return rotatedNnContexts; + + // If the last namenode in the cache at this time + // is the namenode whose priority needs to be lowered. + // No need to rotate cache, because other threads have already rotated the cache. + FederationNamenodeContext lastNamenode = namenodeContexts.get(namenodeContexts.size()-1); + if (lastNamenode.getRpcAddress().equals(namenode.getRpcAddress())) { + return namenodeContexts; } - return namenodeContexts; + + // Move the inaccessible namenode to the end of the cache, + // to ensure that the namenode will not be accessed first next time. + List rotateNamenodeContexts = + (List) namenodeContexts; + rotateNamenodeContexts.remove(namenode); + rotateNamenodeContexts.add(namenode); + LOG.info("Rotate cache of pair<{}, {}> -> {}", + nsId, listObserversFirst, rotateNamenodeContexts); + return rotateNamenodeContexts; }); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java index b38900c3bc264..ad95e9532a3e1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java @@ -457,14 +457,17 @@ private static IOException toIOException(Exception e) { * @param ioe IOException reported. * @param retryCount Number of retries. * @param nsId Nameservice ID. + * @param namenode namenode context. + * @param listObserverFirst Observer read case, observer NN will be ranked first. * @return Retry decision. - * @throws NoNamenodesAvailableException Exception that the retry policy - * generates for no available namenodes. + * @throws IOException An IO Error occurred. */ - private RetryDecision shouldRetry(final IOException ioe, final int retryCount, - final String nsId) throws IOException { + private RetryDecision shouldRetry( + final IOException ioe, final int retryCount, final String nsId, + final FederationNamenodeContext namenode, + final boolean listObserverFirst) throws IOException { // check for the case of cluster unavailable state - if (isClusterUnAvailable(nsId)) { + if (isClusterUnAvailable(nsId, namenode, listObserverFirst)) { // we allow to retry once if cluster is unavailable if (retryCount == 0) { return RetryDecision.RETRY; @@ -538,7 +541,7 @@ public Object invokeMethod( ProxyAndInfo client = connection.getClient(); final Object proxy = client.getProxy(); - ret = invoke(nsId, 0, method, proxy, params); + ret = invoke(nsId, namenode, useObserver, 0, method, proxy, params); if (failover && FederationNamenodeServiceState.OBSERVER != namenode.getState()) { // Success on alternate server, update @@ -594,13 +597,16 @@ public Object invokeMethod( se.initCause(ioe); throw se; } else if (ioe instanceof NoNamenodesAvailableException) { + IOException cause = (IOException) ioe.getCause(); if (this.rpcMonitor != null) { this.rpcMonitor.proxyOpNoNamenodes(nsId); } LOG.error("Cannot get available namenode for {} {} error: {}", nsId, rpcAddress, ioe.getMessage()); // Rotate cache so that client can retry the next namenode in the cache - this.namenodeResolver.rotateCache(nsId, namenode, shouldUseObserver); + if (shouldRotateCache(cause)) { + this.namenodeResolver.rotateCache(nsId, namenode, useObserver); + } // Throw RetriableException so that client can retry throw new RetriableException(ioe); } else { @@ -708,7 +714,9 @@ private void addClientInfoToCallerContext(UserGroupInformation ugi) { * @return Response from the remote server * @throws IOException If error occurs. */ - private Object invoke(String nsId, int retryCount, final Method method, + private Object invoke( + String nsId, FederationNamenodeContext namenode, Boolean listObserverFirst, + int retryCount, final Method method, final Object obj, final Object... params) throws IOException { try { return method.invoke(obj, params); @@ -721,14 +729,14 @@ private Object invoke(String nsId, int retryCount, final Method method, IOException ioe = (IOException) cause; // Check if we should retry. - RetryDecision decision = shouldRetry(ioe, retryCount, nsId); + RetryDecision decision = shouldRetry(ioe, retryCount, nsId, namenode, listObserverFirst); if (decision == RetryDecision.RETRY) { if (this.rpcMonitor != null) { this.rpcMonitor.proxyOpRetries(); } // retry - return invoke(nsId, ++retryCount, method, obj, params); + return invoke(nsId, namenode, listObserverFirst, ++retryCount, method, obj, params); } else if (decision == RetryDecision.FAILOVER_AND_RETRY) { // failover, invoker looks for standby exceptions for failover. if (ioe instanceof StandbyException) { @@ -772,13 +780,23 @@ public static boolean isUnavailableException(IOException ioe) { * Check if the cluster of given nameservice id is available. * * @param nsId nameservice ID. + * @param namenode namenode context. + * @param listObserverFirst Observer read case, observer NN will be ranked first. * @return true if the cluster with given nameservice id is available. * @throws IOException if error occurs. */ - private boolean isClusterUnAvailable(String nsId) throws IOException { + private boolean isClusterUnAvailable( + String nsId, FederationNamenodeContext namenode, + boolean listObserverFirst) throws IOException { + // If the operation is an observer read + // and the namenode that caused the exception is an observer, + // false is returned so that the observer can be marked as unavailable,so other observers + // or active namenode which is standby in the cache of the router can be retried. + if (listObserverFirst && namenode.getState() == FederationNamenodeServiceState.OBSERVER) { + return false; + } List nnState = this.namenodeResolver - .getNamenodesForNameserviceId(nsId, false); - + .getNamenodesForNameserviceId(nsId, listObserverFirst); if (nnState != null) { for (FederationNamenodeContext nnContext : nnState) { // Once we find one NN is in active state, we assume this @@ -1830,4 +1848,24 @@ private LongAccumulator getTimeOfLastCallToActive(String namespaceId) { return lastActiveNNRefreshTimes .computeIfAbsent(namespaceId, key -> new LongAccumulator(Math::max, 0)); } + + /** + * Determine whether router rotated cache is required when NoNamenodesAvailableException occurs. + * + * @param ioe cause of the NoNamenodesAvailableException. + * @return true if NoNamenodesAvailableException occurs due to + * {@link RouterRpcClient#isUnavailableException(IOException) unavailable exception}, + * otherwise false. + */ + private boolean shouldRotateCache(IOException ioe) { + if (isUnavailableException(ioe)) { + return true; + } + if (ioe instanceof RemoteException) { + RemoteException re = (RemoteException) ioe; + ioe = re.unwrapRemoteException(); + ioe = getCleanException(ioe); + } + return isUnavailableException(ioe); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/MiniRouterDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/MiniRouterDFSCluster.java index bf22cf01148a3..ea19dd1e1ba33 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/MiniRouterDFSCluster.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/MiniRouterDFSCluster.java @@ -132,9 +132,9 @@ public class MiniRouterDFSCluster { /** Mini cluster. */ private MiniDFSCluster cluster; - protected static final long DEFAULT_HEARTBEAT_INTERVAL_MS = + public static final long DEFAULT_HEARTBEAT_INTERVAL_MS = TimeUnit.SECONDS.toMillis(5); - protected static final long DEFAULT_CACHE_INTERVAL_MS = + public static final long DEFAULT_CACHE_INTERVAL_MS = TimeUnit.SECONDS.toMillis(5); /** Heartbeat interval in milliseconds. */ private long heartbeatInterval; @@ -240,17 +240,26 @@ public FileSystem getFileSystem(Configuration configuration) throws IOException } public FileSystem getFileSystemWithObserverReadProxyProvider() throws IOException { - Configuration observerReadConf = new Configuration(conf); - observerReadConf.set(DFS_NAMESERVICES, - observerReadConf.get(DFS_NAMESERVICES)+ ",router-service"); - observerReadConf.set(DFS_HA_NAMENODES_KEY_PREFIX + ".router-service", "router1"); - observerReadConf.set(DFS_NAMENODE_RPC_ADDRESS_KEY+ ".router-service.router1", + return getFileSystemWithProxyProvider(ObserverReadProxyProvider.class.getName()); + } + + public FileSystem getFileSystemWithConfiguredFailoverProxyProvider() throws IOException { + return getFileSystemWithProxyProvider(ConfiguredFailoverProxyProvider.class.getName()); + } + + private FileSystem getFileSystemWithProxyProvider( + String proxyProviderClassName) throws IOException { + conf.set(DFS_NAMESERVICES, + conf.get(DFS_NAMESERVICES)+ ",router-service"); + conf.set(DFS_HA_NAMENODES_KEY_PREFIX + ".router-service", "router1"); + conf.set(DFS_NAMENODE_RPC_ADDRESS_KEY+ ".router-service.router1", getFileSystemURI().toString()); - observerReadConf.set(HdfsClientConfigKeys.Failover.PROXY_PROVIDER_KEY_PREFIX - + "." + "router-service", ObserverReadProxyProvider.class.getName()); - DistributedFileSystem.setDefaultUri(observerReadConf, "hdfs://router-service"); - return DistributedFileSystem.get(observerReadConf); + conf.set(HdfsClientConfigKeys.Failover.PROXY_PROVIDER_KEY_PREFIX + + "." + "router-service", proxyProviderClassName); + DistributedFileSystem.setDefaultUri(conf, "hdfs://router-service"); + + return DistributedFileSystem.get(conf); } public DFSClient getClient(UserGroupInformation user) diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestNoNamenodesAvailableLongTime.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestNoNamenodesAvailableLongTime.java new file mode 100644 index 0000000000000..ad5eb2c8d25fa --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestNoNamenodesAvailableLongTime.java @@ -0,0 +1,430 @@ +/** + * 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.server.federation.router; + +import static org.apache.hadoop.fs.permission.AclEntryType.USER; +import static org.apache.hadoop.fs.permission.FsAction.ALL; +import static org.apache.hadoop.fs.permission.AclEntryScope.DEFAULT; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.AclEntry; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hadoop.hdfs.server.federation.MiniRouterDFSCluster; +import org.apache.hadoop.hdfs.server.federation.MiniRouterDFSCluster.RouterContext; +import org.apache.hadoop.hdfs.server.federation.RouterConfigBuilder; +import org.apache.hadoop.hdfs.server.federation.StateStoreDFSCluster; +import org.apache.hadoop.hdfs.server.federation.metrics.FederationRPCMetrics; +import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeContext; +import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeServiceState; +import org.apache.hadoop.hdfs.server.namenode.NameNode; +import org.apache.hadoop.ipc.RemoteException; +import org.apache.hadoop.util.Lists; +import org.junit.After; +import org.junit.Test; + + +import java.io.IOException; +import java.util.Collection; +import java.util.List; + +import static org.apache.hadoop.ha.HAServiceProtocol.HAServiceState.ACTIVE; +import static org.apache.hadoop.hdfs.server.federation.MiniRouterDFSCluster.DEFAULT_HEARTBEAT_INTERVAL_MS; +import static org.apache.hadoop.hdfs.server.namenode.AclTestHelpers.aclEntry; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertTrue; + +/** + * When failover occurs, the router may record that the ns has no active namenode + * even if there is actually an active namenode. + * Only when the router updates the cache next time can the memory status be updated, + * causing the router to report NoNamenodesAvailableException for a long time, + * + * @see org.apache.hadoop.hdfs.server.federation.router.NoNamenodesAvailableException + */ +public class TestNoNamenodesAvailableLongTime { + + // router load cache interval 10s + private static final long CACHE_FLUSH_INTERVAL_MS = 10000; + private StateStoreDFSCluster cluster; + private FileSystem fileSystem; + private RouterContext routerContext; + private FederationRPCMetrics rpcMetrics; + + @After + public void cleanup() throws IOException { + rpcMetrics = null; + routerContext = null; + if (fileSystem != null) { + fileSystem.close(); + fileSystem = null; + } + if (cluster != null) { + cluster.shutdown(); + cluster = null; + } + } + + /** + * Set up state store cluster. + * + * @param numNameservices number of name services + * @param numberOfObserver number of observer + * @param useObserver whether to use observer + */ + private void setupCluster(int numNameservices, int numberOfObserver, boolean useObserver) + throws Exception { + if (!useObserver) { + numberOfObserver = 0; + } + int numberOfNamenode = 2 + numberOfObserver; + cluster = new StateStoreDFSCluster(true, numNameservices, numberOfNamenode, + DEFAULT_HEARTBEAT_INTERVAL_MS, CACHE_FLUSH_INTERVAL_MS); + Configuration routerConf = new RouterConfigBuilder() + .stateStore() + .metrics() + .admin() + .rpc() + .heartbeat() + .build(); + + // Set router observer related configs + if (useObserver) { + routerConf.setBoolean(RBFConfigKeys.DFS_ROUTER_OBSERVER_READ_DEFAULT_KEY, true); + routerConf.setBoolean(DFSConfigKeys.DFS_HA_TAILEDITS_INPROGRESS_KEY, true); + routerConf.set(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, "0ms"); + } + + // Reduce the number of RPC clients threads to overload the Router easy + routerConf.setInt(RBFConfigKeys.DFS_ROUTER_CLIENT_THREADS_SIZE, 4); + + // No need for datanodes + cluster.setNumDatanodesPerNameservice(0); + cluster.addRouterOverrides(routerConf); + + cluster.startCluster(); + + // Making one Namenode active per nameservice + if (cluster.isHighAvailability()) { + for (String ns : cluster.getNameservices()) { + List nnList = cluster.getNamenodes(ns); + cluster.switchToActive(ns, nnList.get(0).getNamenodeId()); + cluster.switchToStandby(ns, nnList.get(1).getNamenodeId()); + for (int i = 2; i < numberOfNamenode; i++) { + cluster.switchToObserver(ns, nnList.get(i).getNamenodeId()); + } + } + } + + cluster.startRouters(); + cluster.waitClusterUp(); + } + + /** + * Initialize the test environment and start the cluster so that + * there is no active namenode record in the router cache, + * but the second non-observer namenode in the router cache is actually active. + */ + private void initEnv(int numberOfObserver, boolean useObserver) throws Exception { + setupCluster(1, numberOfObserver, useObserver); + // Transition all namenodes in the cluster are standby. + transitionActiveToStandby(); + // + allRoutersHeartbeat(); + allRoutersLoadCache(); + + List namenodes = cluster.getNamenodes(); + + // Make sure all namenodes are in standby state + for (MiniRouterDFSCluster.NamenodeContext namenodeContext : namenodes) { + assertNotEquals(ACTIVE.ordinal(), namenodeContext.getNamenode().getNameNodeState()); + } + + routerContext = cluster.getRandomRouter(); + + // Get the second namenode in the router cache and make it active + setSecondNonObserverNamenodeInTheRouterCacheActive(numberOfObserver, false); + allRoutersHeartbeat(); + + // Get router metrics + rpcMetrics = routerContext.getRouter().getRpcServer().getRPCMetrics(); + + assertTrue(routerCacheNoActiveNamenode(routerContext, "ns0", useObserver)); + + // Retries is 2 (see FailoverOnNetworkExceptionRetry#shouldRetry, will fail + // when reties > max.attempts), so total access is 3. + routerContext.getConf().setInt("dfs.client.retry.max.attempts", 1); + + if (useObserver) { + fileSystem = routerContext.getFileSystemWithObserverReadProxyProvider(); + } else { + fileSystem = routerContext.getFileSystemWithConfiguredFailoverProxyProvider(); + } + } + + /** + * If NoNamenodesAvailableException occurs due to + * {@link RouterRpcClient#isUnavailableException(IOException) unavailable exception}, + * should rotated Cache. + */ + @Test + public void testShouldRotatedCache() throws Exception { + // 2 namenodes: 1 active, 1 standby. + // But there is no active namenode in router cache. + initEnv(0, false); + // At this time, the router has recorded 2 standby namenodes in memory. + assertTrue(routerCacheNoActiveNamenode(routerContext, "ns0", false)); + + Path path = new Path("/test.file"); + // The first create operation will cause NoNamenodesAvailableException and RotatedCache. + // After retrying, create and complete operation will be executed successfully. + fileSystem.create(path); + assertEquals(1, rpcMetrics.getProxyOpNoNamenodes()); + + // At this time, the router has recorded 2 standby namenodes in memory, + // the operation can be successful without waiting for the router load cache. + assertTrue(routerCacheNoActiveNamenode(routerContext, "ns0", false)); + } + + /** + * If a request still fails even if it is sent to active, + * then the change operation itself is illegal, + * the cache should not be rotated due to illegal operations. + */ + @Test + public void testShouldNotBeRotatedCache() throws Exception { + testShouldRotatedCache(); + long proxyOpNoNamenodes = rpcMetrics.getProxyOpNoNamenodes(); + Path path = new Path("/test.file"); + /* + * we have put the actually active namenode at the front of the cache by rotating the cache. + * Therefore, the setPermission operation does not cause NoNamenodesAvailableException. + */ + fileSystem.setPermission(path, FsPermission.createImmutable((short)0640)); + assertEquals(proxyOpNoNamenodes, rpcMetrics.getProxyOpNoNamenodes()); + + // At this time, the router has recorded 2 standby namenodes in memory + assertTrue(routerCacheNoActiveNamenode(routerContext, "ns0", false)); + + /* + * Even if the router transfers the illegal request to active, + * NoNamenodesAvailableException will still be generated. + * Therefore, rotated cache is not needed. + */ + List aclSpec = Lists.newArrayList(aclEntry(DEFAULT, USER, "foo", ALL)); + try { + fileSystem.setAcl(path, aclSpec); + }catch (RemoteException e) { + assertTrue(e.getMessage().contains( + "org.apache.hadoop.hdfs.server.federation.router.NoNamenodesAvailableException: " + + "No namenodes available under nameservice ns0")); + assertTrue(e.getMessage().contains( + "org.apache.hadoop.hdfs.protocol.AclException: Invalid ACL: " + + "only directories may have a default ACL. Path: /test.file")); + } + // Retries is 2 (see FailoverOnNetworkExceptionRetry#shouldRetry, will fail + // when reties > max.attempts), so total access is 3. + assertEquals(proxyOpNoNamenodes + 3, rpcMetrics.getProxyOpNoNamenodes()); + proxyOpNoNamenodes = rpcMetrics.getProxyOpNoNamenodes(); + + // So legal operations can be accessed normally without reporting NoNamenodesAvailableException. + assertTrue(routerCacheNoActiveNamenode(routerContext, "ns0", false)); + fileSystem.getFileStatus(path); + assertEquals(proxyOpNoNamenodes, rpcMetrics.getProxyOpNoNamenodes()); + + // At this time, the router has recorded 2 standby namenodes in memory, + // the operation can be successful without waiting for the router load cache. + assertTrue(routerCacheNoActiveNamenode(routerContext, "ns0", false)); + } + + /** + * In the observer scenario, NoNamenodesAvailableException occurs, + * the operation can be successful without waiting for the router load cache. + */ + @Test + public void testUseObserver() throws Exception { + // 4 namenodes: 2 observers, 1 active, 1 standby. + // But there is no active namenode in router cache. + initEnv(2, true); + + Path path = new Path("/"); + // At this time, the router has recorded 2 standby namenodes in memory. + assertTrue(routerCacheNoActiveNamenode(routerContext, "ns0", true)); + + // The first msync operation will cause NoNamenodesAvailableException and RotatedCache. + // After retrying, msync and getFileInfo operation will be executed successfully. + fileSystem.getFileStatus(path); + assertEquals(1, rpcMetrics.getObserverProxyOps()); + assertEquals(1, rpcMetrics.getProxyOpNoNamenodes()); + + // At this time, the router has recorded 2 standby namenodes in memory, + // the operation can be successful without waiting for the router load cache. + assertTrue(routerCacheNoActiveNamenode(routerContext, "ns0", true)); + } + + /** + * In a multi-observer environment, if at least one observer is normal, + * read requests can still succeed even if NoNamenodesAvailableException occurs. + */ + @Test + public void testAtLeastOneObserverNormal() throws Exception { + // 4 namenodes: 2 observers, 1 active, 1 standby. + // But there is no active namenode in router cache. + initEnv(2, true); + // Shutdown one observer. + stopObserver(1); + + /* + * The first msync operation will cause NoNamenodesAvailableException and RotatedCache. + * After retrying, msync operation will be executed successfully. + * Each read request will shuffle the observer, + * if the getFileInfo operation is sent to the downed observer, + * it will cause NoNamenodesAvailableException, + * at this time, the request can be retried to the normal observer, + * no NoNamenodesAvailableException will be generated and the operation will be successful. + */ + fileSystem.getFileStatus(new Path("/")); + assertEquals(1, rpcMetrics.getProxyOpNoNamenodes()); + assertEquals(1, rpcMetrics.getObserverProxyOps()); + + // At this time, the router has recorded 2 standby namenodes in memory, + // the operation can be successful without waiting for the router load cache. + assertTrue(routerCacheNoActiveNamenode(routerContext, "ns0", true)); + } + + /** + * If all obervers are down, read requests can succeed, + * even if a NoNamenodesAvailableException occurs. + */ + @Test + public void testAllObserverAbnormality() throws Exception { + // 4 namenodes: 2 observers, 1 active, 1 standby. + // But there is no active namenode in router cache. + initEnv(2, true); + // Shutdown all observers. + stopObserver(2); + + /* + * The first msync operation will cause NoNamenodesAvailableException and RotatedCache. + * After retrying, msync operation will be executed successfully. + * The getFileInfo operation retried 2 namenodes, both causing UnavailableException, + * and continued to retry to the standby namenode, + * causing NoNamenodesAvailableException and RotatedCache, + * and the execution was successful after retrying. + */ + fileSystem.getFileStatus(new Path("/")); + assertEquals(2, rpcMetrics.getProxyOpFailureCommunicate()); + assertEquals(2, rpcMetrics.getProxyOpNoNamenodes()); + + // At this time, the router has recorded 2 standby namenodes in memory, + // the operation can be successful without waiting for the router load cache. + assertTrue(routerCacheNoActiveNamenode(routerContext, "ns0", true)); + } + + /** + * Determine whether cache of the router has an active namenode. + * + * @return true if no active namenode, otherwise false. + */ + private boolean routerCacheNoActiveNamenode( + RouterContext context, String nsId, boolean useObserver) throws IOException { + List namenodes + = context.getRouter().getNamenodeResolver().getNamenodesForNameserviceId(nsId, useObserver); + for (FederationNamenodeContext namenode : namenodes) { + if (namenode.getState() == FederationNamenodeServiceState.ACTIVE){ + return false; + } + } + return true; + } + + /** + * All routers in the cluster force loadcache. + */ + private void allRoutersLoadCache() { + for (MiniRouterDFSCluster.RouterContext context : cluster.getRouters()) { + // Update service cache + context.getRouter().getStateStore().refreshCaches(true); + } + } + + /** + * Set the second non-observer state namenode in the router cache to active. + */ + private void setSecondNonObserverNamenodeInTheRouterCacheActive( + int numberOfObserver, boolean useObserver) throws IOException { + List ns0 = routerContext.getRouter() + .getNamenodeResolver() + .getNamenodesForNameserviceId("ns0", useObserver); + + String nsId = ns0.get(numberOfObserver+1).getNamenodeId(); + cluster.switchToActive("ns0", nsId); + assertEquals(ACTIVE.ordinal(), + cluster.getNamenode("ns0", nsId).getNamenode().getNameNodeState()); + + } + + /** + * All routers in the cluster force heartbeat. + */ + private void allRoutersHeartbeat() throws IOException { + for (RouterContext context : cluster.getRouters()) { + // Manually trigger the heartbeat, but the router does not manually load the cache + Collection heartbeatServices = context + .getRouter().getNamenodeHeartbeatServices(); + for (NamenodeHeartbeatService service : heartbeatServices) { + service.periodicInvoke(); + } + } + } + + /** + * Transition the active namenode in the cluster to standby. + */ + private void transitionActiveToStandby() { + if (cluster.isHighAvailability()) { + for (String ns : cluster.getNameservices()) { + List nnList = cluster.getNamenodes(ns); + for (MiniRouterDFSCluster.NamenodeContext namenodeContext : nnList) { + if (namenodeContext.getNamenode().isActiveState()) { + cluster.switchToStandby(ns, namenodeContext.getNamenodeId()); + } + } + } + } + } + + /** + * Shutdown observer namenode in the cluster. + * + * @param num The number of shutdown observer. + */ + private void stopObserver(int num) { + int nnIndex; + int numNns = cluster.getNamenodes().size(); + for (nnIndex = 0; nnIndex < numNns && num > 0; nnIndex++) { + NameNode nameNode = cluster.getCluster().getNameNode(nnIndex); + if (nameNode != null && nameNode.isObserverState()) { + cluster.getCluster().shutdownNameNode(nnIndex); + num--; + } + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferServer.java index fcb6b7d7bc575..1d8928f75c705 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferServer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferServer.java @@ -28,6 +28,7 @@ import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import java.nio.charset.StandardCharsets; import java.util.List; import java.util.Map; @@ -62,7 +63,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; /** * Negotiates SASL for DataTransferProtocol on behalf of a server. There are @@ -326,7 +326,7 @@ private char[] buildServerPassword(String userName) throws IOException { byte[] tokenPassword = blockPoolTokenSecretManager.retrievePassword( identifier); return (new String(Base64.encodeBase64(tokenPassword, false), - Charsets.UTF_8)).toCharArray(); + StandardCharsets.UTF_8)).toCharArray(); } /** @@ -381,7 +381,7 @@ private IOStreamPair doSaslHandshake(Peer peer, OutputStream underlyingOut, if (secret != null || bpid != null) { // sanity check, if one is null, the other must also not be null assert(secret != null && bpid != null); - String qop = new String(secret, Charsets.UTF_8); + String qop = new String(secret, StandardCharsets.UTF_8); saslProps.put(Sasl.QOP, qop); } SaslParticipant sasl = SaslParticipant.createServerSaslParticipant( diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java index ffa613018c6a6..e0ac692e60afd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java @@ -25,6 +25,7 @@ import java.io.OutputStreamWriter; import java.net.URL; import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.StandardCopyOption; import java.security.PrivilegedExceptionAction; @@ -72,7 +73,6 @@ import org.apache.hadoop.util.Time; import org.apache.hadoop.classification.VisibleForTesting; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; import org.apache.hadoop.util.Preconditions; import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableList; import org.apache.hadoop.thirdparty.protobuf.TextFormat; @@ -1105,7 +1105,7 @@ private void persistPaxosData(long segmentTxId, // Write human-readable data after the protobuf. This is only // to assist in debugging -- it's not parsed at all. try(OutputStreamWriter writer = - new OutputStreamWriter(fos, Charsets.UTF_8)) { + new OutputStreamWriter(fos, StandardCharsets.UTF_8)) { writer.write(String.valueOf(newData)); writer.write('\n'); writer.flush(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenSecretManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenSecretManager.java index 18cdeaebfda82..b9f817db519f4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenSecretManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenSecretManager.java @@ -18,10 +18,10 @@ package org.apache.hadoop.hdfs.security.token.block; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; import java.io.ByteArrayInputStream; import java.io.DataInputStream; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.security.MessageDigest; import java.security.SecureRandom; import java.util.Arrays; @@ -293,7 +293,7 @@ public Token generateToken(String userId, if (shouldWrapQOP) { String qop = Server.getAuxiliaryPortEstablishedQOP(); if (qop != null) { - id.setHandshakeMsg(qop.getBytes(Charsets.UTF_8)); + id.setHandshakeMsg(qop.getBytes(StandardCharsets.UTF_8)); } } return new Token(id, this); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java index 16afb8496f2a6..93303bcf807de 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java @@ -25,6 +25,7 @@ import java.lang.management.ManagementFactory; import java.nio.channels.FileLock; import java.nio.channels.OverlappingFileLockException; +import java.nio.charset.StandardCharsets; import java.nio.file.DirectoryStream; import java.nio.file.Files; import java.nio.file.attribute.PosixFilePermission; @@ -53,7 +54,6 @@ import org.apache.hadoop.util.ToolRunner; import org.apache.hadoop.util.VersionInfo; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; import org.apache.hadoop.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -944,7 +944,7 @@ FileLock tryLock() throws IOException { LOG.error("Unable to acquire file lock on path {}", lockF); throw new OverlappingFileLockException(); } - file.write(jvmName.getBytes(Charsets.UTF_8)); + file.write(jvmName.getBytes(StandardCharsets.UTF_8)); LOG.info("Lock on {} acquired by nodename {}", lockF, jvmName); } catch(OverlappingFileLockException oe) { // Cannot read from the locked file on Windows. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java index e2f9877483156..2a1e43105600b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java @@ -42,7 +42,7 @@ import org.slf4j.LoggerFactory; import java.io.IOException; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.HashMap; import java.util.List; @@ -450,7 +450,7 @@ private NodePlan verifyPlanHash(String planID, String plan) if ((planID == null) || (planID.length() != sha1Length) || - !DigestUtils.sha1Hex(plan.getBytes(Charset.forName("UTF-8"))) + !DigestUtils.sha1Hex(plan.getBytes(StandardCharsets.UTF_8)) .equalsIgnoreCase(planID)) { LOG.error("Disk Balancer - Invalid plan hash."); throw new DiskBalancerException("Invalid or mis-matched hash.", diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java index bf68f96a6dab0..2935e6ae3221a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java @@ -171,7 +171,8 @@ public class FsVolumeImpl implements FsVolumeSpi { this.usage = usage; if (this.usage != null) { reserved = new ReservedSpaceCalculator.Builder(conf) - .setUsage(this.usage).setStorageType(storageType).build(); + .setUsage(this.usage).setStorageType(storageType) + .setDir(currentDir != null ? currentDir.getParent() : "NULL").build(); boolean fixedSizeVolume = conf.getBoolean( DFSConfigKeys.DFS_DATANODE_FIXED_VOLUME_SIZE_KEY, DFSConfigKeys.DFS_DATANODE_FIXED_VOLUME_SIZE_DEFAULT); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/PmemVolumeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/PmemVolumeManager.java index 601eb19fe2168..22172f4a01915 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/PmemVolumeManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/PmemVolumeManager.java @@ -35,6 +35,7 @@ import java.io.RandomAccessFile; import java.nio.MappedByteBuffer; import java.nio.channels.FileChannel; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Collection; import java.util.List; @@ -333,7 +334,7 @@ static File verifyIfValidPmemVolume(File pmemDir) String uuidStr = UUID.randomUUID().toString(); String testFilePath = realPmemDir.getPath() + "/.verify.pmem." + uuidStr; - byte[] contents = uuidStr.getBytes("UTF-8"); + byte[] contents = uuidStr.getBytes(StandardCharsets.UTF_8); RandomAccessFile testFile = null; MappedByteBuffer out = null; try { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ReservedSpaceCalculator.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ReservedSpaceCalculator.java index 749e16e659c44..0ab4032104ac9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ReservedSpaceCalculator.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ReservedSpaceCalculator.java @@ -46,6 +46,8 @@ public static class Builder { private DF usage; private StorageType storageType; + private String dir; + public Builder(Configuration conf) { this.conf = conf; } @@ -61,6 +63,11 @@ public Builder setStorageType( return this; } + public Builder setDir(String newDir) { + this.dir = newDir; + return this; + } + ReservedSpaceCalculator build() { try { Class clazz = conf.getClass( @@ -69,10 +76,10 @@ ReservedSpaceCalculator build() { ReservedSpaceCalculator.class); Constructor constructor = clazz.getConstructor( - Configuration.class, DF.class, StorageType.class); + Configuration.class, DF.class, StorageType.class, String.class); return (ReservedSpaceCalculator) constructor.newInstance( - conf, usage, storageType); + conf, usage, storageType, dir); } catch (Exception e) { throw new IllegalStateException( "Error instantiating ReservedSpaceCalculator", e); @@ -84,20 +91,30 @@ ReservedSpaceCalculator build() { private final Configuration conf; private final StorageType storageType; + private final String dir; + ReservedSpaceCalculator(Configuration conf, DF usage, - StorageType storageType) { + StorageType storageType, String dir) { this.usage = usage; this.conf = conf; this.storageType = storageType; + this.dir = dir; } DF getUsage() { return usage; } + String getDir() { + return dir; + } + long getReservedFromConf(String key, long defaultValue) { - return conf.getLong(key + "." + StringUtils.toLowerCase( - storageType.toString()), conf.getLongBytes(key, defaultValue)); + return conf.getLong( + key + "." + getDir() + "." + StringUtils.toLowerCase(storageType.toString()), + conf.getLong(key + "." + getDir(), + conf.getLong(key + "." + StringUtils.toLowerCase(storageType.toString()), + conf.getLongBytes(key, defaultValue)))); } /** @@ -117,8 +134,8 @@ public static class ReservedSpaceCalculatorAbsolute extends private final long reservedBytes; public ReservedSpaceCalculatorAbsolute(Configuration conf, DF usage, - StorageType storageType) { - super(conf, usage, storageType); + StorageType storageType, String dir) { + super(conf, usage, storageType, dir); this.reservedBytes = getReservedFromConf(DFS_DATANODE_DU_RESERVED_KEY, DFS_DATANODE_DU_RESERVED_DEFAULT); } @@ -138,8 +155,8 @@ public static class ReservedSpaceCalculatorPercentage extends private final long reservedPct; public ReservedSpaceCalculatorPercentage(Configuration conf, DF usage, - StorageType storageType) { - super(conf, usage, storageType); + StorageType storageType, String dir) { + super(conf, usage, storageType, dir); this.reservedPct = getReservedFromConf( DFS_DATANODE_DU_RESERVED_PERCENTAGE_KEY, DFS_DATANODE_DU_RESERVED_PERCENTAGE_DEFAULT); @@ -162,8 +179,8 @@ public static class ReservedSpaceCalculatorConservative extends private final long reservedPct; public ReservedSpaceCalculatorConservative(Configuration conf, DF usage, - StorageType storageType) { - super(conf, usage, storageType); + StorageType storageType, String dir) { + super(conf, usage, storageType, dir); this.reservedBytes = getReservedFromConf(DFS_DATANODE_DU_RESERVED_KEY, DFS_DATANODE_DU_RESERVED_DEFAULT); this.reservedPct = getReservedFromConf( @@ -197,8 +214,8 @@ public static class ReservedSpaceCalculatorAggressive extends private final long reservedPct; public ReservedSpaceCalculatorAggressive(Configuration conf, DF usage, - StorageType storageType) { - super(conf, usage, storageType); + StorageType storageType, String dir) { + super(conf, usage, storageType, dir); this.reservedBytes = getReservedFromConf(DFS_DATANODE_DU_RESERVED_KEY, DFS_DATANODE_DU_RESERVED_DEFAULT); this.reservedPct = getReservedFromConf( diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/webhdfs/ExceptionHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/webhdfs/ExceptionHandler.java index 6fe0851bb3f70..a222af7c1cdf2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/webhdfs/ExceptionHandler.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/webhdfs/ExceptionHandler.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.hdfs.server.datanode.web.webhdfs; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; import com.sun.jersey.api.ParamException; import com.sun.jersey.api.container.ContainerException; import io.netty.buffer.Unpooled; @@ -32,6 +31,7 @@ import java.io.FileNotFoundException; import java.io.IOException; +import java.nio.charset.StandardCharsets; import static io.netty.handler.codec.http.HttpHeaderNames.CONTENT_LENGTH; import static io.netty.handler.codec.http.HttpHeaderNames.CONTENT_TYPE; @@ -83,7 +83,7 @@ static DefaultFullHttpResponse exceptionCaught(Throwable cause) { s = INTERNAL_SERVER_ERROR; } - final byte[] js = JsonUtil.toJsonString(e).getBytes(Charsets.UTF_8); + final byte[] js = JsonUtil.toJsonString(e).getBytes(StandardCharsets.UTF_8); DefaultFullHttpResponse resp = new DefaultFullHttpResponse(HTTP_1_1, s, Unpooled.wrappedBuffer(js)); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java index 3c6efb78dd1fd..c14526b6a11b3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java @@ -96,6 +96,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_DIFF_LISTING_LIMIT_DEFAULT; import static org.apache.hadoop.hdfs.DFSUtil.isParentEntry; +import java.nio.charset.StandardCharsets; import java.util.concurrent.atomic.AtomicLong; import org.apache.commons.text.CaseUtils; @@ -343,7 +344,6 @@ import org.slf4j.LoggerFactory; import org.apache.hadoop.classification.VisibleForTesting; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; import org.apache.hadoop.util.Preconditions; import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableMap; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; @@ -1982,7 +1982,7 @@ void metaSave(String filename) throws IOException { File file = new File(System.getProperty("hadoop.log.dir"), filename); PrintWriter out = new PrintWriter(new BufferedWriter( new OutputStreamWriter(Files.newOutputStream(file.toPath()), - Charsets.UTF_8))); + StandardCharsets.UTF_8))); metaSave(out); out.flush(); out.close(); @@ -4217,7 +4217,7 @@ DirectoryListing getListing(String src, byte[] startAfter, public byte[] getSrcPathsHash(String[] srcs) { synchronized (digest) { for (String src : srcs) { - digest.update(src.getBytes(Charsets.UTF_8)); + digest.update(src.getBytes(StandardCharsets.UTF_8)); } byte[] result = digest.digest(); digest.reset(); @@ -9124,4 +9124,31 @@ Path getEnclosingRoot(final String srcArg) throws IOException { return new Path("/"); } } + + public void setMetricsEnabled(boolean metricsEnabled) { + this.fsLock.setMetricsEnabled(metricsEnabled); + } + + @VisibleForTesting + public boolean isMetricsEnabled() { + return this.fsLock.isMetricsEnabled(); + } + + public void setReadLockReportingThresholdMs(long readLockReportingThresholdMs) { + this.fsLock.setReadLockReportingThresholdMs(readLockReportingThresholdMs); + } + + @VisibleForTesting + public long getReadLockReportingThresholdMs() { + return this.fsLock.getReadLockReportingThresholdMs(); + } + + public void setWriteLockReportingThresholdMs(long writeLockReportingThresholdMs) { + this.fsLock.setWriteLockReportingThresholdMs(writeLockReportingThresholdMs); + } + + @VisibleForTesting + public long getWriteLockReportingThresholdMs() { + return this.fsLock.getWriteLockReportingThresholdMs(); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystemLock.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystemLock.java index e510bd625e738..af641e964f46c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystemLock.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystemLock.java @@ -68,7 +68,7 @@ class FSNamesystemLock { @VisibleForTesting protected ReentrantReadWriteLock coarseLock; - private final boolean metricsEnabled; + private volatile boolean metricsEnabled; private final MutableRatesWithAggregation detailedHoldTimeMetrics; private final Timer timer; @@ -79,14 +79,14 @@ class FSNamesystemLock { private final long lockSuppressWarningIntervalMs; /** Threshold (ms) for long holding write lock report. */ - private final long writeLockReportingThresholdMs; + private volatile long writeLockReportingThresholdMs; /** Last time stamp for write lock. Keep the longest one for multi-entrance.*/ private long writeLockHeldTimeStampNanos; /** Frequency limiter used for reporting long write lock hold times. */ private final LogThrottlingHelper writeLockReportLogger; /** Threshold (ms) for long holding read lock report. */ - private final long readLockReportingThresholdMs; + private volatile long readLockReportingThresholdMs; /** * Last time stamp for read lock. Keep the longest one for * multi-entrance. This is ThreadLocal since there could be @@ -462,6 +462,33 @@ private static String getMetricName(String operationName, boolean isWrite) { LOCK_METRIC_SUFFIX; } + @VisibleForTesting + public void setMetricsEnabled(boolean metricsEnabled) { + this.metricsEnabled = metricsEnabled; + } + + public boolean isMetricsEnabled() { + return metricsEnabled; + } + + public void setReadLockReportingThresholdMs(long readLockReportingThresholdMs) { + this.readLockReportingThresholdMs = readLockReportingThresholdMs; + } + + @VisibleForTesting + public long getReadLockReportingThresholdMs() { + return readLockReportingThresholdMs; + } + + public void setWriteLockReportingThresholdMs(long writeLockReportingThresholdMs) { + this.writeLockReportingThresholdMs = writeLockReportingThresholdMs; + } + + @VisibleForTesting + public long getWriteLockReportingThresholdMs() { + return writeLockReportingThresholdMs; + } + /** * Read lock Held Info. */ diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java index df490ea0d9fe0..cd74511e6bbca 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java @@ -139,6 +139,12 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_NN_NOT_BECOME_ACTIVE_IN_SAFEMODE_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_IMAGE_PARALLEL_LOAD_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_IMAGE_PARALLEL_LOAD_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LOCK_DETAILED_METRICS_DEFAULT; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LOCK_DETAILED_METRICS_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_READ_LOCK_REPORTING_THRESHOLD_MS_DEFAULT; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_READ_LOCK_REPORTING_THRESHOLD_MS_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_DEFAULT; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_BLOCKPLACEMENTPOLICY_MIN_BLOCKS_FOR_WRITE_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_BLOCKPLACEMENTPOLICY_MIN_BLOCKS_FOR_WRITE_KEY; import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_NAMENODE_RPC_PORT_DEFAULT; @@ -371,7 +377,10 @@ public enum OperationCategory { DFS_NAMENODE_DECOMMISSION_BACKOFF_MONITOR_PENDING_BLOCKS_PER_LOCK, DFS_NAMENODE_BLOCKPLACEMENTPOLICY_MIN_BLOCKS_FOR_WRITE_KEY, IPC_SERVER_LOG_SLOW_RPC, - IPC_SERVER_LOG_SLOW_RPC_THRESHOLD_MS_KEY)); + IPC_SERVER_LOG_SLOW_RPC_THRESHOLD_MS_KEY, + DFS_NAMENODE_LOCK_DETAILED_METRICS_KEY, + DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_KEY, + DFS_NAMENODE_READ_LOCK_REPORTING_THRESHOLD_MS_KEY)); private static final String USAGE = "Usage: hdfs namenode [" + StartupOption.BACKUP.getName() + "] | \n\t[" @@ -2378,6 +2387,10 @@ protected String reconfigurePropertyImpl(String property, String newVal) } else if (property.equals(IPC_SERVER_LOG_SLOW_RPC) || (property.equals(IPC_SERVER_LOG_SLOW_RPC_THRESHOLD_MS_KEY))) { return reconfigureLogSlowRPC(property, newVal); + } else if (property.equals(DFS_NAMENODE_LOCK_DETAILED_METRICS_KEY) + || property.equals(DFS_NAMENODE_READ_LOCK_REPORTING_THRESHOLD_MS_KEY) + || property.equals(DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_KEY)) { + return reconfigureFSNamesystemLockMetricsParameters(property, newVal); } else { throw new ReconfigurationException(property, newVal, getConf().get( property)); @@ -2736,6 +2749,48 @@ private String reconfigureMinBlocksForWrite(String property, String newValue) } } + private String reconfigureFSNamesystemLockMetricsParameters(final String property, + final String newVal) throws ReconfigurationException { + String result; + try { + switch (property) { + case DFS_NAMENODE_LOCK_DETAILED_METRICS_KEY: { + if (newVal != null && !newVal.equalsIgnoreCase("true") && + !newVal.equalsIgnoreCase("false")) { + throw new IllegalArgumentException(newVal + " is not boolean value"); + } + boolean enable = (newVal == null ? + DFS_NAMENODE_LOCK_DETAILED_METRICS_DEFAULT : Boolean.parseBoolean(newVal)); + result = Boolean.toString(enable); + namesystem.setMetricsEnabled(enable); + break; + } + case DFS_NAMENODE_READ_LOCK_REPORTING_THRESHOLD_MS_KEY: { + long readLockReportingThresholdMs = (newVal == null ? + DFS_NAMENODE_READ_LOCK_REPORTING_THRESHOLD_MS_DEFAULT : Long.parseLong(newVal)); + result = Long.toString(readLockReportingThresholdMs); + namesystem.setReadLockReportingThresholdMs(readLockReportingThresholdMs); + break; + } + case DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_KEY: { + long writeLockReportingThresholdMs = (newVal == null ? + DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_DEFAULT : Long.parseLong(newVal)); + result = Long.toString(writeLockReportingThresholdMs); + namesystem.setWriteLockReportingThresholdMs(writeLockReportingThresholdMs); + break; + } + default: { + throw new IllegalArgumentException("Unexpected property " + property + " in " + + "reconfigureFSNamesystemLockMetricsParameters"); + } + } + LOG.info("RECONFIGURE* changed FSNamesystemLockMetricsParameters {} to {}", property, result); + return result; + } catch (IllegalArgumentException e){ + throw new ReconfigurationException(property, newVal, getConf().get(property), e); + } + } + @Override // ReconfigurableBase protected Configuration getNewConf() { return new HdfsConfiguration(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java index affab74e4f1db..40cdbaa637bd6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java @@ -26,6 +26,7 @@ import java.net.URI; import java.net.URISyntaxException; import java.net.UnknownHostException; +import java.nio.charset.StandardCharsets; import java.security.Principal; import java.security.PrivilegedExceptionAction; import java.util.Base64; @@ -124,7 +125,6 @@ import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.classification.VisibleForTesting; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; import com.sun.jersey.spi.container.ResourceFilters; /** Web-hdfs NameNode implementation. */ @@ -1339,7 +1339,7 @@ protected Response get( { byte[] start = HdfsFileStatus.EMPTY_NAME; if (startAfter != null && startAfter.getValue() != null) { - start = startAfter.getValue().getBytes(Charsets.UTF_8); + start = startAfter.getValue().getBytes(StandardCharsets.UTF_8); } final DirectoryListing listing = getDirectoryListing(cp, fullpath, start); final String js = JsonUtil.toJsonString(listing); @@ -1532,7 +1532,7 @@ private static StreamingOutput getListingStream(final ClientProtocol cp, @Override public void write(final OutputStream outstream) throws IOException { final PrintWriter out = new PrintWriter(new OutputStreamWriter( - outstream, Charsets.UTF_8)); + outstream, StandardCharsets.UTF_8)); out.println("{\"" + FileStatus.class.getSimpleName() + "es\":{\"" + FileStatus.class.getSimpleName() + "\":["); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java index 2b09f81301dde..7860129196656 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java @@ -30,6 +30,7 @@ import java.util.List; import org.apache.hadoop.classification.VisibleForTesting; +import org.apache.hadoop.fs.CommonConfigurationKeys; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.HadoopIllegalArgumentException; @@ -297,8 +298,8 @@ public List getAllOtherNodes() { @Override protected boolean isSSLEnabled() { - return conf.getBoolean( - DFSConfigKeys.ZK_CLIENT_SSL_ENABLED, - DFSConfigKeys.DEFAULT_ZK_CLIENT_SSL_ENABLED); + return conf.getBoolean(CommonConfigurationKeys.ZK_CLIENT_SSL_ENABLED, + conf.getBoolean(DFSConfigKeys.ZK_CLIENT_SSL_ENABLED, + DFSConfigKeys.DEFAULT_ZK_CLIENT_SSL_ENABLED)); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsXmlLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsXmlLoader.java index fc5f30e883001..ac43b21d840ab 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsXmlLoader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsXmlLoader.java @@ -22,6 +22,7 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStreamReader; +import java.nio.charset.StandardCharsets; import java.util.Stack; import org.apache.hadoop.classification.InterfaceAudience; @@ -41,7 +42,6 @@ import org.xml.sax.helpers.DefaultHandler; import org.xml.sax.helpers.XMLReaderFactory; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; /** * OfflineEditsXmlLoader walks an EditsVisitor over an OEV XML file @@ -75,7 +75,7 @@ public OfflineEditsXmlLoader(OfflineEditsVisitor visitor, File inputFile, OfflineEditsViewer.Flags flags) throws FileNotFoundException { this.visitor = visitor; this.fileReader = - new InputStreamReader(new FileInputStream(inputFile), Charsets.UTF_8); + new InputStreamReader(new FileInputStream(inputFile), StandardCharsets.UTF_8); this.fixTxIds = flags.getFixTxIds(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/StatisticsEditsVisitor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/StatisticsEditsVisitor.java index 411df1062771b..fe477058a05a3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/StatisticsEditsVisitor.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/StatisticsEditsVisitor.java @@ -21,6 +21,7 @@ import java.io.OutputStream; import java.io.OutputStreamWriter; import java.io.PrintWriter; +import java.nio.charset.StandardCharsets; import java.util.Map; import java.util.HashMap; @@ -30,7 +31,6 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp; import org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; /** * StatisticsEditsVisitor implements text version of EditsVisitor @@ -53,7 +53,7 @@ public class StatisticsEditsVisitor implements OfflineEditsVisitor { * @param out Name of file to write output to */ public StatisticsEditsVisitor(OutputStream out) throws IOException { - this.out = new PrintWriter(new OutputStreamWriter(out, Charsets.UTF_8)); + this.out = new PrintWriter(new OutputStreamWriter(out, StandardCharsets.UTF_8)); } /** Start the visitor */ diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FSImageHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FSImageHandler.java index 963406771b18a..e32cadcee039f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FSImageHandler.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FSImageHandler.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.hdfs.tools.offlineImageViewer; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; import io.netty.channel.ChannelFutureListener; @@ -37,6 +36,7 @@ import java.io.FileNotFoundException; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.List; import java.util.Map; @@ -124,7 +124,7 @@ public void channelRead0(ChannelHandlerContext ctx, HttpRequest request) DefaultFullHttpResponse resp = new DefaultFullHttpResponse(HTTP_1_1, HttpResponseStatus.OK, Unpooled.wrappedBuffer(content - .getBytes(Charsets.UTF_8))); + .getBytes(StandardCharsets.UTF_8))); resp.headers().set(CONTENT_TYPE, APPLICATION_JSON_UTF8); resp.headers().set(CONTENT_LENGTH, resp.content().readableBytes()); resp.headers().set(CONNECTION, CLOSE); @@ -142,7 +142,7 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) Exception e = cause instanceof Exception ? (Exception) cause : new Exception(cause); final String output = JsonUtil.toJsonString(e); - ByteBuf content = Unpooled.wrappedBuffer(output.getBytes(Charsets.UTF_8)); + ByteBuf content = Unpooled.wrappedBuffer(output.getBytes(StandardCharsets.UTF_8)); final DefaultFullHttpResponse resp = new DefaultFullHttpResponse( HTTP_1_1, INTERNAL_SERVER_ERROR, content); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java index 2e2eaf4e4d46e..6b55c7f205b9d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java @@ -19,6 +19,7 @@ import java.io.DataInputStream; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.text.DateFormat; import java.text.SimpleDateFormat; import java.util.Date; @@ -320,7 +321,7 @@ private void processINodesUC(DataInputStream in, ImageVisitor v, for(int i = 0; i < numINUC; i++) { v.visitEnclosingElement(ImageElement.INODE_UNDER_CONSTRUCTION); byte [] name = FSImageSerialization.readBytes(in); - String n = new String(name, "UTF8"); + String n = new String(name, StandardCharsets.UTF_8); v.visit(ImageElement.INODE_PATH, n); if (NameNodeLayoutVersion.supports(Feature.ADD_INODE_ID, imageVersion)) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageReconstructor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageReconstructor.java index 6a2049acb4b54..0c075ff6dac7d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageReconstructor.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageReconstructor.java @@ -36,7 +36,7 @@ import java.io.InputStreamReader; import java.io.OutputStream; import java.nio.ByteBuffer; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Paths; import java.security.DigestOutputStream; @@ -1840,7 +1840,7 @@ public static void run(String inputPath, String outputPath) Files.deleteIfExists(Paths.get(outputPath)); fout = Files.newOutputStream(Paths.get(outputPath)); fis = Files.newInputStream(Paths.get(inputPath)); - reader = new InputStreamReader(fis, Charset.forName("UTF-8")); + reader = new InputStreamReader(fis, StandardCharsets.UTF_8); out = new CountingOutputStream( new DigestOutputStream( new BufferedOutputStream(fout), digester)); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageTextWriter.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageTextWriter.java index f2b329fa2ffd8..20affa93c33e1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageTextWriter.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageTextWriter.java @@ -26,11 +26,11 @@ import java.io.InputStream; import java.io.PrintStream; import java.io.RandomAccessFile; -import java.io.UnsupportedEncodingException; import java.io.ByteArrayInputStream; import java.io.DataInputStream; import java.nio.ByteBuffer; import java.nio.channels.FileChannel; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; @@ -419,9 +419,8 @@ private static byte[] toBytes(long value) { return ByteBuffer.allocate(8).putLong(value).array(); } - private static byte[] toBytes(String value) - throws UnsupportedEncodingException { - return value.getBytes("UTF-8"); + private static byte[] toBytes(String value) { + return value.getBytes(StandardCharsets.UTF_8); } private static long toLong(byte[] bytes) { @@ -430,11 +429,7 @@ private static long toLong(byte[] bytes) { } private static String toString(byte[] bytes) throws IOException { - try { - return new String(bytes, "UTF-8"); - } catch (UnsupportedEncodingException e) { - throw new IOException(e); - } + return new String(bytes, StandardCharsets.UTF_8); } @Override diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TextWriterImageVisitor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TextWriterImageVisitor.java index f732bd6a7f542..1d7c8a4a139ca 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TextWriterImageVisitor.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TextWriterImageVisitor.java @@ -19,10 +19,10 @@ import java.io.IOException; import java.io.OutputStreamWriter; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Paths; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; /** * TextWriterImageProcessor mixes in the ability for ImageVisitor @@ -61,7 +61,7 @@ public TextWriterImageVisitor(String filename, boolean printToScreen) super(); this.printToScreen = printToScreen; fw = new OutputStreamWriter(Files.newOutputStream(Paths.get(filename)), - Charsets.UTF_8); + StandardCharsets.UTF_8); okToWrite = true; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/MD5FileUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/MD5FileUtils.java index 77ec789058831..02ef9ba88c31d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/MD5FileUtils.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/MD5FileUtils.java @@ -23,6 +23,7 @@ import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.security.DigestInputStream; import java.security.MessageDigest; @@ -35,7 +36,6 @@ import org.apache.hadoop.io.MD5Hash; import org.apache.hadoop.util.StringUtils; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; /** * Static functions for dealing with files of the same format @@ -75,7 +75,7 @@ public static void verifySavedMD5(File dataFile, MD5Hash expectedMD5) private static Matcher readStoredMd5(File md5File) throws IOException { BufferedReader reader = new BufferedReader(new InputStreamReader( - Files.newInputStream(md5File.toPath()), Charsets.UTF_8)); + Files.newInputStream(md5File.toPath()), StandardCharsets.UTF_8)); String md5Line; try { md5Line = reader.readLine(); @@ -155,7 +155,7 @@ private static void saveMD5File(File dataFile, String digestString) String md5Line = digestString + " *" + dataFile.getName() + "\n"; AtomicFileOutputStream afos = new AtomicFileOutputStream(md5File); - afos.write(md5Line.getBytes(Charsets.UTF_8)); + afos.write(md5Line.getBytes(StandardCharsets.UTF_8)); afos.close(); if (LOG.isDebugEnabled()) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/PersistentLongFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/PersistentLongFile.java index 78834e0161a4f..309e27d82c00f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/PersistentLongFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/PersistentLongFile.java @@ -22,14 +22,13 @@ import java.io.FileInputStream; import java.io.IOException; import java.io.InputStreamReader; +import java.nio.charset.StandardCharsets; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.io.IOUtils; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; - /** * Class that represents a file on disk which persistently stores * a single long value. The file is updated atomically @@ -77,7 +76,7 @@ public void set(long newVal) throws IOException { public static void writeFile(File file, long val) throws IOException { AtomicFileOutputStream fos = new AtomicFileOutputStream(file); try { - fos.write(String.valueOf(val).getBytes(Charsets.UTF_8)); + fos.write(String.valueOf(val).getBytes(StandardCharsets.UTF_8)); fos.write('\n'); fos.close(); fos = null; @@ -93,7 +92,7 @@ public static long readFile(File file, long defaultVal) throws IOException { if (file.exists()) { BufferedReader br = new BufferedReader(new InputStreamReader(new FileInputStream( - file), Charsets.UTF_8)); + file), StandardCharsets.UTF_8)); try { val = Long.parseLong(br.readLine()); br.close(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml index 52075a24f1e32..b64a7a664b918 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml @@ -397,12 +397,19 @@ dfs.datanode.du.reserved 0 Reserved space in bytes per volume. Always leave this much space free for non dfs use. + Specific directory based reservation is supported. The property can be followed with directory + name which is set at 'dfs.datanode.data.dir'. For example, reserved space for /data/hdfs1/data + can be configured using property 'dfs.datanode.du.reserved./data/hdfs1/data'. If specific directory + reservation is not configured then dfs.datanode.du.reserved will be used. Specific storage type based reservation is also supported. The property can be followed with corresponding storage types ([ssd]/[disk]/[archive]/[ram_disk]/[nvdimm]) for cluster with heterogeneous storage. For example, reserved space for RAM_DISK storage can be configured using property 'dfs.datanode.du.reserved.ram_disk'. If specific storage type reservation is not configured then dfs.datanode.du.reserved will be used. Support multiple size unit suffix(case insensitive), - as described in dfs.blocksize. + as described in dfs.blocksize. Use directory name and storage type based reservation at the + same time is also allowed if both are configured. + Property priority example: dfs.datanode.du.reserved./data/hdfs1/data.ram_disk > + dfs.datanode.du.reserved./data/hdfs1/data > dfs.datanode.du.reserved.ram_disk > dfs.datanode.du.reserved Note: In case of using tune2fs to set reserved-blocks-percentage, or other filesystem tools, then you can possibly run into out of disk errors because hadoop will not check those external tool configurations. @@ -414,12 +421,19 @@ 0 Reserved space in percentage. Read dfs.datanode.du.reserved.calculator to see when this takes effect. The actual number of bytes reserved will be calculated by using the - total capacity of the data directory in question. Specific storage type based reservation + total capacity of the data directory in question. Specific directory based reservation is + supported. The property can be followed with directory name which is set at 'dfs.datanode.data.dir'. + For example, reserved percentage space for /data/hdfs1/data can be configured using property + 'dfs.datanode.du.reserved.pct./data/hdfs1/data'. If specific directory reservation is not + configured then dfs.datanode.du.reserved.pct will be used. Specific storage type based reservation is also supported. The property can be followed with corresponding storage types ([ssd]/[disk]/[archive]/[ram_disk]/[nvdimm]) for cluster with heterogeneous storage. For example, reserved percentage space for RAM_DISK storage can be configured using property 'dfs.datanode.du.reserved.pct.ram_disk'. If specific storage type reservation is not configured - then dfs.datanode.du.reserved.pct will be used. + then dfs.datanode.du.reserved.pct will be used. Use directory and storage type based reservation + is also allowed if both are configured. + Priority example: dfs.datanode.du.reserved.pct./data/hdfs1/data.ram_disk > dfs.datanode.du.reserved.pct./data/hdfs1/data + > dfs.datanode.du.reserved.pct.ram_disk > dfs.datanode.du.reserved.pct @@ -3788,6 +3802,8 @@ false Enable SSL/TLS encryption for the ZooKeeper communication from ZKFC. + Note: if hadoop.zk.ssl.enabled is set to a value, then that central setting has precedence, + and this value will be overridden by the value of hadoop.zk.ssl.enabled. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java index e816edd3110de..4ff57dd7e16b2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java @@ -53,6 +53,7 @@ import java.net.URL; import java.net.URLConnection; import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; import java.security.NoSuchAlgorithmException; import java.security.PrivilegedExceptionAction; import java.util.ArrayList; @@ -70,7 +71,6 @@ import java.util.UUID; import java.util.concurrent.TimeoutException; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; import org.apache.hadoop.thirdparty.com.google.common.base.Joiner; import org.apache.hadoop.util.Preconditions; import org.apache.hadoop.thirdparty.com.google.common.base.Strings; @@ -985,7 +985,7 @@ public static void appendFileNewBlock(DistributedFileSystem fs, * @return url content as string (UTF-8 encoding assumed) */ public static String urlGet(URL url) throws IOException { - return new String(urlGetBytes(url), Charsets.UTF_8); + return new String(urlGetBytes(url), StandardCharsets.UTF_8); } /** @@ -1438,7 +1438,7 @@ public static void runOperations(MiniDFSCluster cluster, Short permission = 0777; filesystem.setPermission(pathFileCreate, new FsPermission(permission)); // OP_SET_OWNER 8 - filesystem.setOwner(pathFileCreate, new String("newOwner"), null); + filesystem.setOwner(pathFileCreate, "newOwner", null); // OP_CLOSE 9 see above // OP_SET_GENSTAMP 10 see above // OP_SET_NS_QUOTA 11 obsolete diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBalancerBandwidth.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBalancerBandwidth.java index 3191fbdf8fe1f..bc83a704720f9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBalancerBandwidth.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBalancerBandwidth.java @@ -23,6 +23,7 @@ import java.io.ByteArrayOutputStream; import java.io.PrintStream; import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.concurrent.TimeoutException; @@ -46,7 +47,7 @@ public class TestBalancerBandwidth { final static private int DEFAULT_BANDWIDTH = 1024*1024; public static final Logger LOG = LoggerFactory.getLogger(TestBalancerBandwidth.class); - private static final Charset UTF8 = Charset.forName("UTF-8"); + private static final Charset UTF8 = StandardCharsets.UTF_8; private final ByteArrayOutputStream outContent = new ByteArrayOutputStream(); private final PrintStream outStream = new PrintStream(outContent); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java index 1092f74a3bd51..7fc6cd9cfafda 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java @@ -23,6 +23,7 @@ import java.io.File; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.Collections; import java.util.List; @@ -41,7 +42,6 @@ import org.junit.After; import org.junit.Test; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; /** * This test ensures the appropriate response (successful or failure) from @@ -312,8 +312,8 @@ public void testRollback() throws Exception { for (File f : baseDirs) { UpgradeUtilities.corruptFile( new File(f,"VERSION"), - "layoutVersion".getBytes(Charsets.UTF_8), - "xxxxxxxxxxxxx".getBytes(Charsets.UTF_8)); + "layoutVersion".getBytes(StandardCharsets.UTF_8), + "xxxxxxxxxxxxx".getBytes(StandardCharsets.UTF_8)); } startNameNodeShouldFail("file VERSION has layoutVersion missing"); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgrade.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgrade.java index ac58f320b20b9..ebc0379bc60be 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgrade.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgrade.java @@ -29,6 +29,7 @@ import java.io.File; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.regex.Pattern; import org.slf4j.Logger; @@ -49,7 +50,6 @@ import org.junit.Ignore; import org.junit.Test; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; import org.apache.hadoop.thirdparty.com.google.common.base.Joiner; /** @@ -335,8 +335,8 @@ public void testUpgrade() throws Exception { for (File f : baseDirs) { UpgradeUtilities.corruptFile( new File(f,"VERSION"), - "layoutVersion".getBytes(Charsets.UTF_8), - "xxxxxxxxxxxxx".getBytes(Charsets.UTF_8)); + "layoutVersion".getBytes(StandardCharsets.UTF_8), + "xxxxxxxxxxxxx".getBytes(StandardCharsets.UTF_8)); } startNameNodeShouldFail(StartupOption.UPGRADE); UpgradeUtilities.createEmptyDirs(nameNodeDirs); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java index f8e8e4120c43f..5d7110d3d9a8b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java @@ -83,6 +83,7 @@ import org.apache.hadoop.security.alias.JavaKeyStoreProvider; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.LambdaTestUtils; +import org.apache.hadoop.util.Shell; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -865,13 +866,25 @@ public void testLocalhostReverseLookup() { @Test (timeout=15000) public void testIsValidName() { - assertFalse(DFSUtil.isValidName("/foo/../bar")); - assertFalse(DFSUtil.isValidName("/foo/./bar")); - assertFalse(DFSUtil.isValidName("/foo//bar")); - assertTrue(DFSUtil.isValidName("/")); - assertTrue(DFSUtil.isValidName("/bar/")); - assertFalse(DFSUtil.isValidName("/foo/:/bar")); - assertFalse(DFSUtil.isValidName("/foo:bar")); + String validPaths[] = new String[]{"/", "/bar/"}; + for (String path : validPaths) { + assertTrue("Should have been accepted '" + path + "'", DFSUtil.isValidName(path)); + } + + String invalidPaths[] = + new String[]{"/foo/../bar", "/foo/./bar", "/foo//bar", "/foo/:/bar", "/foo:bar"}; + for (String path : invalidPaths) { + assertFalse("Should have been rejected '" + path + "'", DFSUtil.isValidName(path)); + } + + String windowsPath = "/C:/foo/bar"; + if (Shell.WINDOWS) { + assertTrue("Should have been accepted '" + windowsPath + "' in windows os.", + DFSUtil.isValidName(windowsPath)); + } else { + assertFalse("Should have been rejected '" + windowsPath + "' in unix os.", + DFSUtil.isValidName(windowsPath)); + } } @Test(timeout=5000) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeReport.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeReport.java index 9638f71ef8286..239555a8b0065 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeReport.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeReport.java @@ -160,7 +160,7 @@ public void testDatanodeReportMissingBlock() throws Exception { cluster.waitActive(); DistributedFileSystem fs = cluster.getFileSystem(); Path p = new Path("/testDatanodeReportMissingBlock"); - DFSTestUtil.writeFile(fs, p, new String("testdata")); + DFSTestUtil.writeFile(fs, p, "testdata"); LocatedBlock lb = fs.getClient().getLocatedBlocks(p.toString(), 0).get(0); assertEquals(3, lb.getLocations().length); ExtendedBlock b = lb.getBlock(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMultipleNNPortQOP.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMultipleNNPortQOP.java index d536c5e8a969a..8121d8454a183 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMultipleNNPortQOP.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMultipleNNPortQOP.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hdfs; import java.net.URI; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import org.apache.hadoop.fs.BlockLocation; import org.apache.hadoop.fs.CommonConfigurationKeys; @@ -287,7 +288,7 @@ public void testMultipleNNPortOverwriteDownStream() throws Exception { private void doTest(FileSystem fs, Path path) throws Exception { FileSystemTestHelper.createFile(fs, path, NUM_BLOCKS, BLOCK_SIZE); assertArrayEquals(FileSystemTestHelper.getFileData(NUM_BLOCKS, BLOCK_SIZE), - DFSTestUtil.readFile(fs, path).getBytes("UTF-8")); + DFSTestUtil.readFile(fs, path).getBytes(StandardCharsets.UTF_8)); BlockLocation[] blockLocations = fs.getFileBlockLocations(path, 0, Long.MAX_VALUE); assertNotNull(blockLocations); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestQuota.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestQuota.java index a11af7220481a..2e6feaf5445fb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestQuota.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestQuota.java @@ -32,6 +32,7 @@ import java.io.IOException; import java.io.OutputStream; import java.io.PrintStream; +import java.nio.charset.StandardCharsets; import java.security.PrivilegedExceptionAction; import java.util.List; import java.util.Scanner; @@ -66,7 +67,6 @@ import org.junit.Rule; import org.junit.Test; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; import org.junit.rules.Timeout; import org.slf4j.Logger; import org.slf4j.event.Level; @@ -1216,7 +1216,7 @@ public void testSetSpaceQuotaWhenStorageTypeIsWrong() throws Exception { String[] args = { "-setSpaceQuota", "100", "-storageType", "COLD", "/testDir" }; admin.run(args); - String errOutput = new String(err.toByteArray(), Charsets.UTF_8); + String errOutput = new String(err.toByteArray(), StandardCharsets.UTF_8); assertTrue( errOutput.contains(StorageType.getTypesSupportingQuota().toString())); } finally { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/TestSaslDataTransfer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/TestSaslDataTransfer.java index 3dd0b7eb99ea1..85e43f65c37b2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/TestSaslDataTransfer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/TestSaslDataTransfer.java @@ -32,6 +32,7 @@ import java.net.ServerSocket; import java.net.Socket; import java.net.SocketTimeoutException; +import java.nio.charset.StandardCharsets; import java.util.concurrent.atomic.AtomicBoolean; import org.slf4j.LoggerFactory; @@ -200,7 +201,7 @@ private void doTest(HdfsConfiguration conf) throws IOException { fs = FileSystem.get(cluster.getURI(), conf); FileSystemTestHelper.createFile(fs, PATH, NUM_BLOCKS, BLOCK_SIZE); assertArrayEquals(FileSystemTestHelper.getFileData(NUM_BLOCKS, BLOCK_SIZE), - DFSTestUtil.readFile(fs, PATH).getBytes("UTF-8")); + DFSTestUtil.readFile(fs, PATH).getBytes(StandardCharsets.UTF_8)); BlockLocation[] blockLocations = fs.getFileBlockLocations(PATH, 0, Long.MAX_VALUE); assertNotNull(blockLocations); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java index 064dd9e5dd86e..f959b142a3116 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.hdfs.qjournal.server; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; import org.apache.hadoop.thirdparty.com.google.common.primitives.Bytes; import org.apache.hadoop.thirdparty.com.google.common.primitives.Ints; import org.apache.hadoop.conf.Configuration; @@ -54,6 +53,7 @@ import java.io.IOException; import java.net.HttpURLConnection; import java.net.URL; +import java.nio.charset.StandardCharsets; import java.util.Collection; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; @@ -278,7 +278,7 @@ public void testJournal() throws Exception { ch.newEpoch(1).get(); ch.setEpoch(1); ch.startLogSegment(1, NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION).get(); - ch.sendEdits(1L, 1, 1, "hello".getBytes(Charsets.UTF_8)).get(); + ch.sendEdits(1L, 1, 1, "hello".getBytes(StandardCharsets.UTF_8)).get(); metrics = MetricsAsserts.getMetrics( journal.getMetrics().getName()); @@ -291,7 +291,7 @@ public void testJournal() throws Exception { beginTimestamp = lastJournalTimestamp; ch.setCommittedTxId(100L); - ch.sendEdits(1L, 2, 1, "goodbye".getBytes(Charsets.UTF_8)).get(); + ch.sendEdits(1L, 2, 1, "goodbye".getBytes(StandardCharsets.UTF_8)).get(); metrics = MetricsAsserts.getMetrics( journal.getMetrics().getName()); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMetrics.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMetrics.java index 35f7924be110e..89a9fdd1a8733 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMetrics.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMetrics.java @@ -553,7 +553,7 @@ public void testDNShouldNotDeleteBlockONTooManyOpenFiles() cluster.waitActive(); DistributedFileSystem fs = cluster.getFileSystem(); Path p = new Path("/testShouldThrowTMP"); - DFSTestUtil.writeFile(fs, p, new String("testdata")); + DFSTestUtil.writeFile(fs, p, "testdata"); //Before DN throws too many open files verifyBlockLocations(fs, p, 1); Mockito.doThrow(new FileNotFoundException("Too many open files")). diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestReservedSpaceCalculator.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestReservedSpaceCalculator.java index fa666f2a691a7..10136863964b9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestReservedSpaceCalculator.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestReservedSpaceCalculator.java @@ -168,6 +168,55 @@ public void testReservedSpaceAggresivePerStorageType() { checkReserved(StorageType.ARCHIVE, 100000, 5000); } + @Test + public void testReservedSpaceAbsolutePerDir() { + conf.setClass(DFS_DATANODE_DU_RESERVED_CALCULATOR_KEY, ReservedSpaceCalculatorAbsolute.class, + ReservedSpaceCalculator.class); + + String dir1 = "/data/hdfs1/data"; + String dir2 = "/data/hdfs2/data"; + String dir3 = "/data/hdfs3/data"; + + conf.setLong(DFS_DATANODE_DU_RESERVED_KEY + "." + dir1 + ".ssd", 900); + conf.setLong(DFS_DATANODE_DU_RESERVED_KEY + "." + dir1, 1800); + conf.setLong(DFS_DATANODE_DU_RESERVED_KEY + "." + dir2, 2700); + conf.setLong(DFS_DATANODE_DU_RESERVED_KEY + ".ssd", 3600); + conf.setLong(DFS_DATANODE_DU_RESERVED_KEY, 4500); + + checkReserved(StorageType.SSD, 10000, 900, dir1); + checkReserved(StorageType.DISK, 10000, 1800, dir1); + checkReserved(StorageType.SSD, 10000, 2700, dir2); + checkReserved(StorageType.DISK, 10000, 2700, dir2); + checkReserved(StorageType.SSD, 10000, 3600, dir3); + checkReserved(StorageType.DISK, 10000, 4500, dir3); + } + + @Test + public void testReservedSpacePercentagePerDir() { + conf.setClass(DFS_DATANODE_DU_RESERVED_CALCULATOR_KEY, + ReservedSpaceCalculatorPercentage.class, + ReservedSpaceCalculator.class); + + String dir1 = "/data/hdfs1/data"; + String dir2 = "/data/hdfs2/data"; + String dir3 = "/data/hdfs3/data"; + + // Set percentage reserved values for different directories + conf.setLong(DFS_DATANODE_DU_RESERVED_PERCENTAGE_KEY + "." + dir1 + ".ssd", 20); + conf.setLong(DFS_DATANODE_DU_RESERVED_PERCENTAGE_KEY + "." + dir1, 10); + conf.setLong(DFS_DATANODE_DU_RESERVED_PERCENTAGE_KEY + "." + dir2, 25); + conf.setLong(DFS_DATANODE_DU_RESERVED_PERCENTAGE_KEY + ".ssd", 30); + conf.setLong(DFS_DATANODE_DU_RESERVED_PERCENTAGE_KEY, 40); + + // Verify reserved space calculations for different directories and storage types + checkReserved(StorageType.SSD, 10000, 2000, dir1); + checkReserved(StorageType.DISK, 10000, 1000, dir1); + checkReserved(StorageType.SSD, 10000, 2500, dir2); + checkReserved(StorageType.DISK, 10000, 2500, dir2); + checkReserved(StorageType.SSD, 10000, 3000, dir3); + checkReserved(StorageType.DISK, 10000, 4000, dir3); + } + @Test(expected = IllegalStateException.class) public void testInvalidCalculator() { conf.set(DFS_DATANODE_DU_RESERVED_CALCULATOR_KEY, "INVALIDTYPE"); @@ -179,10 +228,15 @@ public void testInvalidCalculator() { private void checkReserved(StorageType storageType, long totalCapacity, long reservedExpected) { + checkReserved(storageType, totalCapacity, reservedExpected, "NULL"); + } + + private void checkReserved(StorageType storageType, + long totalCapacity, long reservedExpected, String dir) { when(usage.getCapacity()).thenReturn(totalCapacity); reserved = new ReservedSpaceCalculator.Builder(conf).setUsage(usage) - .setStorageType(storageType).build(); + .setStorageType(storageType).setDir(dir).build(); assertEquals(reservedExpected, reserved.getReserved()); } } \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFavoredNodesEndToEnd.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFavoredNodesEndToEnd.java index b8cc32e43e2d8..14fd75c72c39a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFavoredNodesEndToEnd.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFavoredNodesEndToEnd.java @@ -60,7 +60,7 @@ public class TestFavoredNodesEndToEnd { private static Configuration conf; private final static int NUM_DATA_NODES = 10; private final static int NUM_FILES = 10; - private final static byte[] SOME_BYTES = new String("foo").getBytes(); + private final static byte[] SOME_BYTES = "foo".getBytes(); private static DistributedFileSystem dfs; private static ArrayList datanodes; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java index 1782a4644bc91..892c5ce020a4d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java @@ -919,7 +919,7 @@ public void testFsckReplicaDetails() throws Exception { dfs = cluster.getFileSystem(); // create files - final String testFile = new String("/testfile"); + final String testFile = "/testfile"; final Path path = new Path(testFile); DFSTestUtil.createFile(dfs, path, fileSize, replFactor, 1000L); DFSTestUtil.waitReplication(dfs, path, replFactor); @@ -1202,7 +1202,7 @@ public void testFsckMissingReplicas() throws IOException { assertNotNull("Failed to get FileSystem", dfs); // Create a file that will be intentionally under-replicated - final String pathString = new String("/testfile"); + final String pathString = "/testfile"; final Path path = new Path(pathString); long fileLen = blockSize * numBlocks; DFSTestUtil.createFile(dfs, path, fileLen, replFactor, 1); @@ -1263,7 +1263,7 @@ public void testFsckMisPlacedReplicas() throws IOException { assertNotNull("Failed to get FileSystem", dfs); // Create a file that will be intentionally under-replicated - final String pathString = new String("/testfile"); + final String pathString = "/testfile"; final Path path = new Path(pathString); long fileLen = blockSize * numBlocks; DFSTestUtil.createFile(dfs, path, fileLen, replFactor, 1); @@ -1436,7 +1436,7 @@ public void testBlockIdCK() throws Exception { DFSTestUtil util = new DFSTestUtil.Builder(). setName(getClass().getSimpleName()).setNumFiles(1).build(); //create files - final String pathString = new String("/testfile"); + final String pathString = "/testfile"; final Path path = new Path(pathString); util.createFile(dfs, path, 1024, replFactor, 1000L); util.waitReplication(dfs, path, replFactor); @@ -1490,7 +1490,7 @@ public void testBlockIdCKDecommission() throws Exception { DFSTestUtil util = new DFSTestUtil.Builder(). setName(getClass().getSimpleName()).setNumFiles(1).build(); //create files - final String pathString = new String("/testfile"); + final String pathString = "/testfile"; final Path path = new Path(pathString); util.createFile(dfs, path, 1024, replFactor, 1000L); util.waitReplication(dfs, path, replFactor); @@ -1577,7 +1577,7 @@ public void testBlockIdCKMaintenance() throws Exception { DFSTestUtil util = new DFSTestUtil.Builder(). setName(getClass().getSimpleName()).setNumFiles(1).build(); //create files - final String pathString = new String("/testfile"); + final String pathString = "/testfile"; final Path path = new Path(pathString); util.createFile(dfs, path, 1024, replFactor, 1000L); util.waitReplication(dfs, path, replFactor); @@ -1694,7 +1694,7 @@ public void testBlockIdCKStaleness() throws Exception { setName(getClass().getSimpleName()).setNumFiles(1).build(); // Create one file. - final String pathString = new String("/testfile"); + final String pathString = "/testfile"; final Path path = new Path(pathString); util.createFile(fs, path, 1024L, replFactor, 1024L); util.waitReplication(fs, path, replFactor); @@ -1780,7 +1780,7 @@ public void testBlockIdCKCorruption() throws Exception { DFSTestUtil util = new DFSTestUtil.Builder(). setName(getClass().getSimpleName()).setNumFiles(1).build(); //create files - final String pathString = new String("/testfile"); + final String pathString = "/testfile"; final Path path = new Path(pathString); util.createFile(dfs, path, 1024, repFactor, 1000L); util.waitReplication(dfs, path, repFactor); @@ -1937,7 +1937,7 @@ public void testFsckWithDecommissionedReplicas() throws Exception { setName(getClass().getSimpleName()).setNumFiles(1).build(); //create files - final String testFile = new String("/testfile"); + final String testFile = "/testfile"; final Path path = new Path(testFile); util.createFile(dfs, path, fileSize, replFactor, 1000L); util.waitReplication(dfs, path, replFactor); @@ -2020,7 +2020,7 @@ public void testFsckWithMaintenanceReplicas() throws Exception { DFSTestUtil util = new DFSTestUtil.Builder(). setName(getClass().getSimpleName()).setNumFiles(1).build(); //create files - final String testFile = new String("/testfile"); + final String testFile = "/testfile"; final Path path = new Path(testFile); util.createFile(dfs, path, 1024, replFactor, 1000L); util.waitReplication(dfs, path, replFactor); @@ -2394,7 +2394,7 @@ private void testUpgradeDomain(boolean defineUpgradeDomain, } // create files - final String testFile = new String("/testfile"); + final String testFile = "/testfile"; final Path path = new Path(testFile); DFSTestUtil.createFile(dfs, path, fileSize, replFactor, 1000L); DFSTestUtil.waitReplication(dfs, path, replFactor); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java index b32f8fe759d1e..4f18baf1aeaa9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java @@ -1163,7 +1163,7 @@ public void testFilesInGetListingOps() throws Exception { HdfsFileStatus.EMPTY_NAME, false); assertTrue(dl.getPartialListing().length == 3); - String f2 = new String("f2"); + String f2 = "f2"; dl = cluster.getNameNodeRpc().getListing("/tmp", f2.getBytes(), false); assertTrue(dl.getPartialListing().length == 1); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeReconfigure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeReconfigure.java index 5a0f62a8117e0..d116da5a8167b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeReconfigure.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeReconfigure.java @@ -34,6 +34,9 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IPC_SERVER_LOG_SLOW_RPC_THRESHOLD_MS_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MAX_NODES_TO_REPORT_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_IMAGE_PARALLEL_LOAD_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LOCK_DETAILED_METRICS_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_READ_LOCK_REPORTING_THRESHOLD_MS_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_KEY; import static org.junit.Assert.*; import org.slf4j.Logger; @@ -747,6 +750,78 @@ public void testReconfigureLogSlowRPC() throws ReconfigurationException { assertEquals(nnrs.getClientRpcServer().getLogSlowRPCThresholdTime(), 20000); } + @Test + public void testReconfigureFSNamesystemLockMetricsParameters() + throws ReconfigurationException, IOException { + Configuration conf = new HdfsConfiguration(); + conf.setBoolean(DFS_NAMENODE_LOCK_DETAILED_METRICS_KEY, false); + long defaultReadLockMS = 1000L; + conf.setLong(DFS_NAMENODE_READ_LOCK_REPORTING_THRESHOLD_MS_KEY, defaultReadLockMS); + long defaultWriteLockMS = 1000L; + conf.setLong(DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_KEY, defaultWriteLockMS); + + try (MiniDFSCluster newCluster = new MiniDFSCluster.Builder(conf).build()) { + newCluster.waitActive(); + final NameNode nameNode = newCluster.getNameNode(); + final FSNamesystem fsNamesystem = nameNode.getNamesystem(); + // verify default value. + assertFalse(fsNamesystem.isMetricsEnabled()); + assertEquals(defaultReadLockMS, fsNamesystem.getReadLockReportingThresholdMs()); + assertEquals(defaultWriteLockMS, fsNamesystem.getWriteLockReportingThresholdMs()); + + // try invalid metricsEnabled. + try { + nameNode.reconfigurePropertyImpl(DFS_NAMENODE_LOCK_DETAILED_METRICS_KEY, + "non-boolean"); + fail("should not reach here"); + } catch (ReconfigurationException e) { + assertEquals( + "Could not change property dfs.namenode.lock.detailed-metrics.enabled from " + + "'false' to 'non-boolean'", e.getMessage()); + } + + // try correct metricsEnabled. + nameNode.reconfigurePropertyImpl(DFS_NAMENODE_LOCK_DETAILED_METRICS_KEY, "true"); + assertTrue(fsNamesystem.isMetricsEnabled()); + + nameNode.reconfigurePropertyImpl(DFS_NAMENODE_LOCK_DETAILED_METRICS_KEY, null); + assertFalse(fsNamesystem.isMetricsEnabled()); + + // try invalid readLockMS. + try { + nameNode.reconfigureProperty(DFS_NAMENODE_READ_LOCK_REPORTING_THRESHOLD_MS_KEY, + "non-numeric"); + fail("Should not reach here"); + } catch (ReconfigurationException e) { + assertEquals("Could not change property " + + "dfs.namenode.read-lock-reporting-threshold-ms from '" + + defaultReadLockMS + "' to 'non-numeric'", e.getMessage()); + } + + // try correct readLockMS. + nameNode.reconfigureProperty(DFS_NAMENODE_READ_LOCK_REPORTING_THRESHOLD_MS_KEY, + "20000"); + assertEquals(fsNamesystem.getReadLockReportingThresholdMs(), 20000); + + + // try invalid writeLockMS. + try { + nameNode.reconfigureProperty( + DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_KEY, "non-numeric"); + fail("Should not reach here"); + } catch (ReconfigurationException e) { + assertEquals("Could not change property " + + "dfs.namenode.write-lock-reporting-threshold-ms from '" + + defaultWriteLockMS + "' to 'non-numeric'", e.getMessage()); + } + + // try correct writeLockMS. + nameNode.reconfigureProperty( + DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_KEY, "100000"); + assertEquals(fsNamesystem.getWriteLockReportingThresholdMs(), 100000); + } + } + @After public void shutDown() throws IOException { if (cluster != null) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartupProgressServlet.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartupProgressServlet.java index 245602ee9bfa7..d28f0a4c2920e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartupProgressServlet.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartupProgressServlet.java @@ -24,6 +24,7 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.PrintWriter; +import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.Collections; import java.util.Map; @@ -245,7 +246,7 @@ public void testFinalState() throws Exception { */ private String doGetAndReturnResponseBody() throws IOException { servlet.doGet(req, resp); - return new String(respOut.toByteArray(), "UTF-8"); + return new String(respOut.toByteArray(), StandardCharsets.UTF_8); } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java index 1712c620d2c82..2a3fba83ffcb2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java @@ -99,7 +99,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_MAX_SLOWPEER_COLLECT_NODES_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LOCK_DETAILED_METRICS_KEY; import static org.apache.hadoop.hdfs.client.HdfsAdmin.TRASH_PERMISSION; import static org.hamcrest.CoreMatchers.allOf; import static org.hamcrest.CoreMatchers.anyOf; @@ -442,7 +442,7 @@ public void testNameNodeGetReconfigurableProperties() throws IOException, Interr final List outs = Lists.newArrayList(); final List errs = Lists.newArrayList(); getReconfigurableProperties("namenode", address, outs, errs); - assertEquals(25, outs.size()); + assertEquals(28, outs.size()); assertTrue(outs.get(0).contains("Reconfigurable properties:")); assertEquals(DFS_BLOCK_INVALIDATE_LIMIT_KEY, outs.get(1)); assertEquals(DFS_BLOCK_PLACEMENT_EC_CLASSNAME_KEY, outs.get(2)); @@ -457,7 +457,7 @@ public void testNameNodeGetReconfigurableProperties() throws IOException, Interr assertEquals(DFS_NAMENODE_DECOMMISSION_BACKOFF_MONITOR_PENDING_BLOCKS_PER_LOCK, outs.get(11)); assertEquals(DFS_NAMENODE_DECOMMISSION_BACKOFF_MONITOR_PENDING_LIMIT, outs.get(12)); assertEquals(DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, outs.get(13)); - assertEquals(DFS_NAMENODE_MAX_SLOWPEER_COLLECT_NODES_KEY, outs.get(14)); + assertEquals(DFS_NAMENODE_LOCK_DETAILED_METRICS_KEY, outs.get(14)); assertEquals(errs.size(), 0); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdminWithHA.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdminWithHA.java index 5681f3bdb25eb..3d4ddfe88eb14 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdminWithHA.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdminWithHA.java @@ -19,8 +19,8 @@ import java.io.ByteArrayOutputStream; import java.io.PrintStream; +import java.nio.charset.StandardCharsets; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeys; @@ -56,8 +56,8 @@ public class TestDFSAdminWithHA { private static String newLine = System.getProperty("line.separator"); private void assertOutputMatches(String string) { - String errOutput = new String(err.toByteArray(), Charsets.UTF_8); - String output = new String(out.toByteArray(), Charsets.UTF_8); + String errOutput = new String(err.toByteArray(), StandardCharsets.UTF_8); + String output = new String(out.toByteArray(), StandardCharsets.UTF_8); if (!errOutput.matches(string) && !output.matches(string)) { fail("Expected output to match '" + string + @@ -70,8 +70,8 @@ private void assertOutputMatches(String string) { } private void assertOutputMatches(String outMessage, String errMessage) { - String errOutput = new String(err.toByteArray(), Charsets.UTF_8); - String output = new String(out.toByteArray(), Charsets.UTF_8); + String errOutput = new String(err.toByteArray(), StandardCharsets.UTF_8); + String output = new String(out.toByteArray(), StandardCharsets.UTF_8); if (!errOutput.matches(errMessage) || !output.matches(outMessage)) { fail("Expected output to match '" + outMessage + " and " + errMessage + diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdmin.java index 0086134d78817..51d5b90e38d85 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdmin.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdmin.java @@ -26,6 +26,7 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.PrintStream; +import java.nio.charset.StandardCharsets; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -48,7 +49,6 @@ import org.mockito.ArgumentCaptor; import org.mockito.Mockito; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; import org.apache.hadoop.thirdparty.com.google.common.base.Joiner; public class TestDFSHAAdmin { @@ -435,8 +435,8 @@ private Object runTool(String ... args) throws Exception { outBytes.reset(); LOG.info("Running: DFSHAAdmin " + Joiner.on(" ").join(args)); int ret = tool.run(args); - errOutput = new String(errOutBytes.toByteArray(), Charsets.UTF_8); - output = new String(outBytes.toByteArray(), Charsets.UTF_8); + errOutput = new String(errOutBytes.toByteArray(), StandardCharsets.UTF_8); + output = new String(outBytes.toByteArray(), StandardCharsets.UTF_8); LOG.info("Err_output:\n" + errOutput + "\nOutput:\n" + output); return ret; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdminMiniCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdminMiniCluster.java index d0edd175ec1ec..bf557deaea6ee 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdminMiniCluster.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdminMiniCluster.java @@ -27,6 +27,7 @@ import java.io.File; import java.io.IOException; import java.io.PrintStream; +import java.nio.charset.StandardCharsets; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -45,7 +46,6 @@ import org.junit.Before; import org.junit.Test; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; import org.apache.hadoop.thirdparty.com.google.common.base.Joiner; import org.apache.hadoop.thirdparty.com.google.common.io.Files; @@ -232,7 +232,7 @@ public void testFencer() throws Exception { assertEquals(0, runTool("-ns", "minidfs-ns", "-failover", "nn2", "nn1")); // Fencer has not run yet, since none of the above required fencing - assertEquals("", Files.asCharSource(tmpFile, Charsets.UTF_8).read()); + assertEquals("", Files.asCharSource(tmpFile, StandardCharsets.UTF_8).read()); // Test failover with fencer and forcefence option assertEquals(0, runTool("-failover", "nn1", "nn2", "--forcefence")); @@ -240,7 +240,7 @@ public void testFencer() throws Exception { // The fence script should run with the configuration from the target // node, rather than the configuration from the fencing node. Strip // out any trailing spaces and CR/LFs which may be present on Windows. - String fenceCommandOutput = Files.asCharSource(tmpFile, Charsets.UTF_8) + String fenceCommandOutput = Files.asCharSource(tmpFile, StandardCharsets.UTF_8) .read().replaceAll(" *[\r\n]+", ""); assertEquals("minidfs-ns.nn1 " + nn1Port + " nn1", fenceCommandOutput); tmpFile.delete(); @@ -325,7 +325,7 @@ private int runTool(String ... args) throws Exception { errOutBytes.reset(); LOG.info("Running: DFSHAAdmin " + Joiner.on(" ").join(args)); int ret = tool.run(args); - errOutput = new String(errOutBytes.toByteArray(), Charsets.UTF_8); + errOutput = new String(errOutBytes.toByteArray(), StandardCharsets.UTF_8); LOG.info("Output:\n" + errOutput); return ret; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsFileSystemContract.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsFileSystemContract.java index c7df1f7c1d2d1..2a9eda3ca5e7a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsFileSystemContract.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsFileSystemContract.java @@ -27,6 +27,7 @@ import java.io.InputStreamReader; import java.net.HttpURLConnection; import java.net.URL; +import java.nio.charset.StandardCharsets; import java.text.MessageFormat; import java.util.Arrays; import java.util.Map; @@ -341,7 +342,7 @@ public void testLengthParamLongerThanFile() throws IOException { byte[] respBody = new byte[content.length()]; is = conn.getInputStream(); IOUtils.readFully(is, respBody, 0, content.length()); - assertEquals(content, new String(respBody, "US-ASCII")); + assertEquals(content, new String(respBody, StandardCharsets.US_ASCII)); } finally { IOUtils.closeStream(is); if (conn != null) { @@ -392,7 +393,7 @@ public void testOffsetPlusLengthParamsLongerThanFile() throws IOException { byte[] respBody = new byte[content.length() - 1]; is = conn.getInputStream(); IOUtils.readFully(is, respBody, 0, content.length() - 1); - assertEquals(content.substring(1), new String(respBody, "US-ASCII")); + assertEquals(content.substring(1), new String(respBody, StandardCharsets.US_ASCII)); } finally { IOUtils.closeStream(is); if (conn != null) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsTimeouts.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsTimeouts.java index a693ac3d5e981..1f5c89a03efaa 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsTimeouts.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsTimeouts.java @@ -31,6 +31,7 @@ import java.net.Socket; import java.net.SocketTimeoutException; import java.nio.channels.SocketChannel; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -332,7 +333,7 @@ public void run() { // Write response. out = clientSocket.getOutputStream(); - out.write(temporaryRedirect().getBytes("UTF-8")); + out.write(temporaryRedirect().getBytes(StandardCharsets.UTF_8)); } catch (IOException e) { // Fail the test on any I/O error in the server thread. LOG.error("unexpected IOException in server thread", e); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestHdfsConfigFields.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestHdfsConfigFields.java index fab4fcd0409b2..da57cab60a340 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestHdfsConfigFields.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestHdfsConfigFields.java @@ -39,7 +39,7 @@ public class TestHdfsConfigFields extends TestConfigurationFieldsBase { @Override public void initializeMemberVariables() { - xmlFilename = new String("hdfs-default.xml"); + xmlFilename = "hdfs-default.xml"; configurationClasses = new Class[] { HdfsClientConfigKeys.class, HdfsClientConfigKeys.Failover.class, HdfsClientConfigKeys.StripedRead.class, DFSConfigKeys.class, diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRecovery.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRecovery.java index 5a23b58875a0b..d0b9acee8ec80 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRecovery.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRecovery.java @@ -28,6 +28,7 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import java.nio.charset.StandardCharsets; import java.util.function.Supplier; import java.io.File; import java.io.FileInputStream; @@ -2097,7 +2098,7 @@ public static String slurp(File f) throws IOException { String contents = null; try { in.read(buf, 0, len); - contents = new String(buf, "UTF-8"); + contents = new String(buf, StandardCharsets.UTF_8); } finally { in.close(); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobQueueClient.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobQueueClient.java index 9158ec3b33720..3b41f418640c0 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobQueueClient.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobQueueClient.java @@ -21,6 +21,7 @@ import java.io.OutputStreamWriter; import java.io.PrintWriter; import java.io.Writer; +import java.nio.charset.StandardCharsets; import java.util.List; import java.util.ArrayList; import java.util.Arrays; @@ -31,7 +32,6 @@ import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; /** * JobQueueClient is interface provided to the user in order to get @@ -148,7 +148,7 @@ private void displayQueueList() throws IOException { JobQueueInfo[] rootQueues = jc.getRootQueues(); for (JobQueueInfo queue : rootQueues) { printJobQueueInfo(queue, new PrintWriter(new OutputStreamWriter( - System.out, Charsets.UTF_8))); + System.out, StandardCharsets.UTF_8))); } } @@ -187,7 +187,7 @@ private void displayQueueInfo(String queue, boolean showJobs) return; } printJobQueueInfo(jobQueueInfo, new PrintWriter(new OutputStreamWriter( - System.out, Charsets.UTF_8))); + System.out, StandardCharsets.UTF_8))); if (showJobs && (jobQueueInfo.getChildren() == null || jobQueueInfo.getChildren().size() == 0)) { JobStatus[] jobs = jobQueueInfo.getJobStatuses(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskLog.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskLog.java index a0223dedd64ce..36405328f5fae 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskLog.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskLog.java @@ -27,6 +27,7 @@ import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Enumeration; import java.util.List; @@ -56,7 +57,6 @@ import org.apache.log4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; /** * A simple logger to handle the task-specific user logs. @@ -114,7 +114,7 @@ private static LogFileDetail getLogFileDetail(TaskAttemptID taskid, File indexFile = getIndexFile(taskid, isCleanup); BufferedReader fis = new BufferedReader(new InputStreamReader( SecureIOUtils.openForRead(indexFile, obtainLogDirOwner(taskid), null), - Charsets.UTF_8)); + StandardCharsets.UTF_8)); //the format of the index file is //LOG_DIR: //stdout: diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TextInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TextInputFormat.java index 45b4fd6f717ba..62f4031566f7f 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TextInputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TextInputFormat.java @@ -19,6 +19,7 @@ package org.apache.hadoop.mapred; import java.io.*; +import java.nio.charset.StandardCharsets; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -27,7 +28,6 @@ import org.apache.hadoop.io.Text; import org.apache.hadoop.io.compress.*; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; /** * An {@link InputFormat} for plain text files. Files are broken into lines. @@ -62,7 +62,7 @@ public RecordReader getRecordReader( String delimiter = job.get("textinputformat.record.delimiter"); byte[] recordDelimiterBytes = null; if (null != delimiter) { - recordDelimiterBytes = delimiter.getBytes(Charsets.UTF_8); + recordDelimiterBytes = delimiter.getBytes(StandardCharsets.UTF_8); } return new LineRecordReader(job, (FileSplit) genericSplit, recordDelimiterBytes); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmitter.java index 232b8bf13bc97..0f7f9dfc14766 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmitter.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmitter.java @@ -22,6 +22,7 @@ import java.net.InetAddress; import java.net.URI; import java.net.URISyntaxException; +import java.nio.charset.StandardCharsets; import java.security.NoSuchAlgorithmException; import java.util.ArrayList; import java.util.Arrays; @@ -63,7 +64,6 @@ import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.yarn.api.records.ReservationId; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; @InterfaceAudience.Private @InterfaceStability.Unstable @@ -409,7 +409,7 @@ private void readTokensFromFiles(Configuration conf, Credentials credentials) for(Map.Entry ent: nm.entrySet()) { credentials.addSecretKey(new Text(ent.getKey()), ent.getValue() - .getBytes(Charsets.UTF_8)); + .getBytes(StandardCharsets.UTF_8)); } } catch (JsonMappingException | JsonParseException e) { LOG.warn("couldn't parse Token Cache JSON file with user secret keys"); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JSONHistoryViewerPrinter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JSONHistoryViewerPrinter.java index 5f8e9ad4b6858..88e8d07a293d5 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JSONHistoryViewerPrinter.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JSONHistoryViewerPrinter.java @@ -33,6 +33,7 @@ import java.io.OutputStreamWriter; import java.io.PrintStream; import java.io.Writer; +import java.nio.charset.StandardCharsets; import java.util.Iterator; import java.util.Map; @@ -72,7 +73,7 @@ public void print(PrintStream ps) throws IOException { printTaskSummary(); printTasks(); - writer = new OutputStreamWriter(ps, "UTF-8"); + writer = new OutputStreamWriter(ps, StandardCharsets.UTF_8); json.write(writer); writer.flush(); } catch (JSONException je) { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/TextInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/TextInputFormat.java index d15ee7ca4c3fb..77bea97e149f9 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/TextInputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/TextInputFormat.java @@ -32,7 +32,8 @@ import org.apache.hadoop.mapreduce.RecordReader; import org.apache.hadoop.mapreduce.TaskAttemptContext; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; +import java.nio.charset.StandardCharsets; + /** An {@link InputFormat} for plain text files. Files are broken into lines. * Either linefeed or carriage-return are used to signal end of line. Keys are @@ -49,7 +50,7 @@ public class TextInputFormat extends FileInputFormat { "textinputformat.record.delimiter"); byte[] recordDelimiterBytes = null; if (null != delimiter) - recordDelimiterBytes = delimiter.getBytes(Charsets.UTF_8); + recordDelimiterBytes = delimiter.getBytes(StandardCharsets.UTF_8); return new LineRecordReader(recordDelimiterBytes); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/partition/KeyFieldBasedPartitioner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/partition/KeyFieldBasedPartitioner.java index 69377e73e6d46..f7b68f75e8fb1 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/partition/KeyFieldBasedPartitioner.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/partition/KeyFieldBasedPartitioner.java @@ -18,7 +18,7 @@ package org.apache.hadoop.mapreduce.lib.partition; -import java.io.UnsupportedEncodingException; +import java.nio.charset.StandardCharsets; import java.util.List; import org.apache.hadoop.classification.InterfaceAudience; @@ -90,12 +90,7 @@ public int getPartition(K2 key, V2 value, int numReduceTasks) { return getPartition(key.toString().hashCode(), numReduceTasks); } - try { - keyBytes = key.toString().getBytes("UTF-8"); - } catch (UnsupportedEncodingException e) { - throw new RuntimeException("The current system does not " + - "support UTF-8 encoding!", e); - } + keyBytes = key.toString().getBytes(StandardCharsets.UTF_8); // return 0 if the key is empty if (keyBytes.length == 0) { return 0; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/partition/KeyFieldHelper.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/partition/KeyFieldHelper.java index 21ca3fae33d82..03ef14c9ba698 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/partition/KeyFieldHelper.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/partition/KeyFieldHelper.java @@ -18,7 +18,7 @@ package org.apache.hadoop.mapreduce.lib.partition; -import java.io.UnsupportedEncodingException; +import java.nio.charset.StandardCharsets; import java.util.List; import java.util.ArrayList; import java.util.StringTokenizer; @@ -61,13 +61,8 @@ public String toString() { private boolean keySpecSeen = false; public void setKeyFieldSeparator(String keyFieldSeparator) { - try { - this.keyFieldSeparator = - keyFieldSeparator.getBytes("UTF-8"); - } catch (UnsupportedEncodingException e) { - throw new RuntimeException("The current system does not " + - "support UTF-8 encoding!", e); - } + this.keyFieldSeparator = + keyFieldSeparator.getBytes(StandardCharsets.UTF_8); } /** Required for backcompatibility with num.key.fields.for.partition in diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/security/SecureShuffleUtils.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/security/SecureShuffleUtils.java index cdd656ca48907..47c8d12cb4b2f 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/security/SecureShuffleUtils.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/security/SecureShuffleUtils.java @@ -23,6 +23,7 @@ import java.io.PrintStream; import java.io.UnsupportedEncodingException; import java.net.URL; +import java.nio.charset.StandardCharsets; import javax.crypto.SecretKey; import javax.servlet.http.HttpServletRequest; @@ -34,7 +35,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; /** * @@ -56,7 +56,7 @@ public class SecureShuffleUtils { */ public static String generateHash(byte[] msg, SecretKey key) { return new String(Base64.encodeBase64(generateByteHash(msg, key)), - Charsets.UTF_8); + StandardCharsets.UTF_8); } /** @@ -70,7 +70,6 @@ private static byte[] generateByteHash(byte[] msg, SecretKey key) { /** * verify that hash equals to HMacHash(msg) - * @param newHash * @return true if is the same */ private static boolean verifyHash(byte[] hash, byte[] msg, SecretKey key) { @@ -87,7 +86,7 @@ private static boolean verifyHash(byte[] hash, byte[] msg, SecretKey key) { */ public static String hashFromString(String enc_str, SecretKey key) throws IOException { - return generateHash(enc_str.getBytes(Charsets.UTF_8), key); + return generateHash(enc_str.getBytes(StandardCharsets.UTF_8), key); } /** @@ -98,9 +97,9 @@ public static String hashFromString(String enc_str, SecretKey key) */ public static void verifyReply(String base64Hash, String msg, SecretKey key) throws IOException { - byte[] hash = Base64.decodeBase64(base64Hash.getBytes(Charsets.UTF_8)); + byte[] hash = Base64.decodeBase64(base64Hash.getBytes(StandardCharsets.UTF_8)); - boolean res = verifyHash(hash, msg.getBytes(Charsets.UTF_8), key); + boolean res = verifyHash(hash, msg.getBytes(StandardCharsets.UTF_8), key); if(res != true) { throw new IOException("Verification of the hashReply failed"); @@ -148,7 +147,7 @@ public static String toHex(byte[] ba) { for (byte b : ba) { ps.printf("%x", b); } - strHex = baos.toString("UTF-8"); + strHex = new String(baos.toByteArray(), StandardCharsets.UTF_8); } catch (UnsupportedEncodingException e) { } return strHex; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/split/JobSplit.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/split/JobSplit.java index 7d08fb3acd791..d725196b9b884 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/split/JobSplit.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/split/JobSplit.java @@ -20,7 +20,7 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; -import java.io.UnsupportedEncodingException; +import java.nio.charset.StandardCharsets; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Writable; @@ -47,15 +47,8 @@ @InterfaceStability.Unstable public class JobSplit { static final int META_SPLIT_VERSION = 1; - static final byte[] META_SPLIT_FILE_HEADER; - static { - try { - META_SPLIT_FILE_HEADER = "META-SPL".getBytes("UTF-8"); - } catch (UnsupportedEncodingException u) { - throw new RuntimeException(u); - } - } - public static final TaskSplitMetaInfo EMPTY_TASK_SPLIT = + static final byte[] META_SPLIT_FILE_HEADER = "META-SPL".getBytes(StandardCharsets.UTF_8); + public static final TaskSplitMetaInfo EMPTY_TASK_SPLIT = new TaskSplitMetaInfo(); /** diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/split/JobSplitWriter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/split/JobSplitWriter.java index 76234bd17fe9e..a32e254af4a01 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/split/JobSplitWriter.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/split/JobSplitWriter.java @@ -19,7 +19,7 @@ package org.apache.hadoop.mapreduce.split; import java.io.IOException; -import java.io.UnsupportedEncodingException; +import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.List; @@ -54,16 +54,8 @@ public class JobSplitWriter { private static final Logger LOG = LoggerFactory.getLogger(JobSplitWriter.class); private static final int splitVersion = JobSplit.META_SPLIT_VERSION; - private static final byte[] SPLIT_FILE_HEADER; + private static final byte[] SPLIT_FILE_HEADER = "SPL".getBytes(StandardCharsets.UTF_8); - static { - try { - SPLIT_FILE_HEADER = "SPL".getBytes("UTF-8"); - } catch (UnsupportedEncodingException u) { - throw new RuntimeException(u); - } - } - @SuppressWarnings("unchecked") public static void createSplitFiles(Path jobSubmitDir, Configuration conf, FileSystem fs, List splits) diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/tools/CLI.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/tools/CLI.java index dcd4edd6ff1ca..0f65a29b13489 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/tools/CLI.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/tools/CLI.java @@ -24,6 +24,7 @@ import java.io.OutputStreamWriter; import java.io.PrintStream; import java.io.PrintWriter; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.List; import java.util.Set; @@ -64,7 +65,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; /** * Interprets the map reduce cli options @@ -767,7 +767,7 @@ protected void displayTasks(Job job, String type, String state) public void displayJobList(JobStatus[] jobs) throws IOException, InterruptedException { displayJobList(jobs, new PrintWriter(new OutputStreamWriter(System.out, - Charsets.UTF_8))); + StandardCharsets.UTF_8))); } @Private diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestFileOutputCommitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestFileOutputCommitter.java index bb5c30e9511fa..b646b04b74034 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestFileOutputCommitter.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestFileOutputCommitter.java @@ -22,6 +22,7 @@ import java.io.FileInputStream; import java.io.IOException; import java.net.URI; +import java.nio.charset.StandardCharsets; import org.junit.Test; import static org.assertj.core.api.Assertions.assertThat; @@ -571,7 +572,7 @@ public static String slurp(File f) throws IOException { String contents = null; try { in.read(buf, 0, len); - contents = new String(buf, "UTF-8"); + contents = new String(buf, StandardCharsets.UTF_8); } finally { in.close(); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestLineRecordReader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestLineRecordReader.java index f4f2d18c3823b..1b3ebdfa31da0 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestLineRecordReader.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestLineRecordReader.java @@ -238,7 +238,7 @@ public String[] readRecordsDirectly(URL testFileUrl, boolean bzip) } fis.close(); assertTrue("Test file data too big for buffer", count < data.length); - return new String(data, 0, count, "UTF-8").split("\n"); + return new String(data, 0, count, StandardCharsets.UTF_8).split("\n"); } public void checkRecordSpanningMultipleSplits(String testFile, diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestLineRecordReader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestLineRecordReader.java index f6fbbd58dabbf..b2b2dd62586d2 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestLineRecordReader.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestLineRecordReader.java @@ -183,7 +183,7 @@ public String[] readRecordsDirectly(URL testFileUrl, boolean bzip) } fis.close(); assertTrue("Test file data too big for buffer", count < data.length); - return new String(data, 0, count, "UTF-8").split("\n"); + return new String(data, 0, count, StandardCharsets.UTF_8).split("\n"); } public void checkRecordSpanningMultipleSplits(String testFile, diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestFileOutputCommitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestFileOutputCommitter.java index 9c58c0d773f99..2aa7b34a007c8 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestFileOutputCommitter.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestFileOutputCommitter.java @@ -23,6 +23,7 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.net.URI; +import java.nio.charset.StandardCharsets; import java.util.concurrent.Callable; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; @@ -843,7 +844,7 @@ public static String slurp(File f) throws IOException { String contents = null; try { in.read(buf, 0, len); - contents = new String(buf, "UTF-8"); + contents = new String(buf, StandardCharsets.UTF_8); } finally { in.close(); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestConcatenatedCompressedInput.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestConcatenatedCompressedInput.java index af6b9529e02a8..ae68d74d8d4d3 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestConcatenatedCompressedInput.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestConcatenatedCompressedInput.java @@ -38,6 +38,7 @@ import java.io.FileInputStream; import java.io.IOException; import java.io.OutputStream; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.List; import java.util.zip.Inflater; @@ -295,7 +296,7 @@ public void testPrototypeInflaterGzip() throws IOException { try { int numBytesUncompressed = inflater.inflate(uncompressedBuf); String outString = - new String(uncompressedBuf, 0, numBytesUncompressed, "UTF-8"); + new String(uncompressedBuf, 0, numBytesUncompressed, StandardCharsets.UTF_8); System.out.println("uncompressed data of first gzip member = [" + outString + "]"); } catch (java.util.zip.DataFormatException ex) { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFixedLengthInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFixedLengthInputFormat.java index 51347296a861c..5fec24a1b1317 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFixedLengthInputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFixedLengthInputFormat.java @@ -97,7 +97,7 @@ public void testFormatCompressedIn() throws IOException { @Test (timeout=5000) public void testNoRecordLength() throws IOException { localFs.delete(workDir, true); - Path file = new Path(workDir, new String("testFormat.txt")); + Path file = new Path(workDir, "testFormat.txt"); createFile(file, null, 10, 10); // Set the fixed length record length config property JobConf job = new JobConf(defaultConf); @@ -124,7 +124,7 @@ public void testNoRecordLength() throws IOException { @Test (timeout=5000) public void testZeroRecordLength() throws IOException { localFs.delete(workDir, true); - Path file = new Path(workDir, new String("testFormat.txt")); + Path file = new Path(workDir, "testFormat.txt"); createFile(file, null, 10, 10); // Set the fixed length record length config property JobConf job = new JobConf(defaultConf); @@ -152,7 +152,7 @@ public void testZeroRecordLength() throws IOException { @Test (timeout=5000) public void testNegativeRecordLength() throws IOException { localFs.delete(workDir, true); - Path file = new Path(workDir, new String("testFormat.txt")); + Path file = new Path(workDir, "testFormat.txt"); createFile(file, null, 10, 10); // Set the fixed length record length config property JobConf job = new JobConf(defaultConf); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/UtilsForTests.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/UtilsForTests.java index 4a7c3283d48a3..fd73410918d85 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/UtilsForTests.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/UtilsForTests.java @@ -24,6 +24,7 @@ import java.io.FileOutputStream; import java.io.IOException; import java.io.InputStream; +import java.nio.charset.StandardCharsets; import java.text.DecimalFormat; import java.util.ArrayList; import java.util.Arrays; @@ -180,7 +181,7 @@ public static String slurp(File f) throws IOException { String contents = null; try { in.read(buf, 0, len); - contents = new String(buf, "UTF-8"); + contents = new String(buf, StandardCharsets.UTF_8); } finally { in.close(); } @@ -194,7 +195,7 @@ public static String slurpHadoop(Path p, FileSystem fs) throws IOException { String contents = null; try { in.read(buf, 0, len); - contents = new String(buf, "UTF-8"); + contents = new String(buf, StandardCharsets.UTF_8); } finally { in.close(); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/JobControlTestUtils.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/JobControlTestUtils.java index d160de5db61a6..c971ccc6c02be 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/JobControlTestUtils.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/JobControlTestUtils.java @@ -19,6 +19,7 @@ package org.apache.hadoop.mapred.jobcontrol; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.text.NumberFormat; import java.util.Iterator; import java.util.List; @@ -100,7 +101,7 @@ static void generateData(FileSystem fs, Path dirPath) throws IOException { FSDataOutputStream out = fs.create(new Path(dirPath, "data.txt")); for (int i = 0; i < 10000; i++) { String line = generateRandomLine(); - out.write(line.getBytes("UTF-8")); + out.write(line.getBytes(StandardCharsets.UTF_8)); } out.close(); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/MapReduceTestUtil.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/MapReduceTestUtil.java index 2f30bb5ec0c50..4141d26933f82 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/MapReduceTestUtil.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/MapReduceTestUtil.java @@ -25,6 +25,7 @@ import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; +import java.nio.charset.StandardCharsets; import java.text.NumberFormat; import java.util.ArrayList; import java.util.Iterator; @@ -116,7 +117,7 @@ public static void generateData(FileSystem fs, Path dirPath) FSDataOutputStream out = fs.create(new Path(dirPath, "data.txt")); for (int i = 0; i < 10000; i++) { String line = generateRandomLine(); - out.write(line.getBytes("UTF-8")); + out.write(line.getBytes(StandardCharsets.UTF_8)); } out.close(); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMRJobClient.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMRJobClient.java index 31b90aa0e506c..17cd5bfaace33 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMRJobClient.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMRJobClient.java @@ -47,6 +47,7 @@ import java.io.PipedInputStream; import java.io.PipedOutputStream; import java.io.PrintStream; +import java.nio.charset.StandardCharsets; import java.util.Arrays; import static org.junit.Assert.assertEquals; @@ -193,7 +194,7 @@ private void testfailTask(Configuration conf) throws Exception { assertEquals("Exit code", -1, exitCode); runTool(conf, jc, new String[] { "-fail-task", taid.toString() }, out); - String answer = new String(out.toByteArray(), "UTF-8"); + String answer = new String(out.toByteArray(), StandardCharsets.UTF_8); assertTrue(answer.contains("Killed task " + taid + " by failing it")); } @@ -211,7 +212,7 @@ private void testKillTask(Configuration conf) throws Exception { assertEquals("Exit code", -1, exitCode); runTool(conf, jc, new String[] { "-kill-task", taid.toString() }, out); - String answer = new String(out.toByteArray(), "UTF-8"); + String answer = new String(out.toByteArray(), StandardCharsets.UTF_8); assertTrue(answer.contains("Killed task " + taid)); } @@ -231,7 +232,7 @@ private void testKillJob(Configuration conf) throws Exception { exitCode = runTool(conf, jc, new String[] { "-kill", jobId }, out); assertEquals("Exit code", 0, exitCode); - String answer = new String(out.toByteArray(), "UTF-8"); + String answer = new String(out.toByteArray(), StandardCharsets.UTF_8); assertTrue(answer.contains("Killed job " + jobId)); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestFixedLengthInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestFixedLengthInputFormat.java index 684d3e13d1f6a..be9e6deff3fc8 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestFixedLengthInputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestFixedLengthInputFormat.java @@ -102,7 +102,7 @@ public void testFormatCompressedIn() throws Exception { @Test (timeout=5000) public void testNoRecordLength() throws Exception { localFs.delete(workDir, true); - Path file = new Path(workDir, new String("testFormat.txt")); + Path file = new Path(workDir, "testFormat.txt"); createFile(file, null, 10, 10); // Create the job and do not set fixed record length Job job = Job.getInstance(defaultConf); @@ -136,7 +136,7 @@ public void testNoRecordLength() throws Exception { @Test (timeout=5000) public void testZeroRecordLength() throws Exception { localFs.delete(workDir, true); - Path file = new Path(workDir, new String("testFormat.txt")); + Path file = new Path(workDir, "testFormat.txt"); createFile(file, null, 10, 10); Job job = Job.getInstance(defaultConf); // Set the fixed length record length config property @@ -172,7 +172,7 @@ public void testZeroRecordLength() throws Exception { @Test (timeout=5000) public void testNegativeRecordLength() throws Exception { localFs.delete(workDir, true); - Path file = new Path(workDir, new String("testFormat.txt")); + Path file = new Path(workDir, "testFormat.txt"); createFile(file, null, 10, 10); // Set the fixed length record length config property Job job = Job.getInstance(defaultConf); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/NativeMapOutputCollectorDelegator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/NativeMapOutputCollectorDelegator.java index 2c8d13fff1922..12d449ebbacf2 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/NativeMapOutputCollectorDelegator.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/NativeMapOutputCollectorDelegator.java @@ -18,8 +18,8 @@ package org.apache.hadoop.mapred.nativetask; import java.io.IOException; +import java.nio.charset.StandardCharsets; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.io.RawComparator; @@ -131,7 +131,7 @@ public void init(Context context) throws IOException, ClassNotFoundException { if (ret) { if (job.getBoolean(MRJobConfig.MAP_OUTPUT_COMPRESS, false)) { String codec = job.get(MRJobConfig.MAP_OUTPUT_COMPRESS_CODEC); - if (!NativeRuntime.supportsCompressionCodec(codec.getBytes(Charsets.UTF_8))) { + if (!NativeRuntime.supportsCompressionCodec(codec.getBytes(StandardCharsets.UTF_8))) { String message = "Native output collector doesn't support compression codec " + codec; LOG.error(message); throw new InvalidJobConfException(message); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/NativeRuntime.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/NativeRuntime.java index 311ee223b9c76..df1c7ade18cda 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/NativeRuntime.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/NativeRuntime.java @@ -19,8 +19,8 @@ package org.apache.hadoop.mapred.nativetask; import java.io.IOException; +import java.nio.charset.StandardCharsets; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.DataInputBuffer; @@ -83,7 +83,7 @@ public static void configure(Configuration jobConf) { */ public synchronized static long createNativeObject(String clazz) { assertNativeLibraryLoaded(); - final long ret = JNICreateNativeObject(clazz.getBytes(Charsets.UTF_8)); + final long ret = JNICreateNativeObject(clazz.getBytes(StandardCharsets.UTF_8)); if (ret == 0) { LOG.warn("Can't create NativeObject for class " + clazz + ", probably not exist."); } @@ -95,8 +95,8 @@ public synchronized static long createNativeObject(String clazz) { */ public synchronized static long registerLibrary(String libraryName, String clazz) { assertNativeLibraryLoaded(); - final long ret = JNIRegisterModule(libraryName.getBytes(Charsets.UTF_8), - clazz.getBytes(Charsets.UTF_8)); + final long ret = JNIRegisterModule(libraryName.getBytes(StandardCharsets.UTF_8), + clazz.getBytes(StandardCharsets.UTF_8)); if (ret != 0) { LOG.warn("Can't create NativeObject for class " + clazz + ", probably not exist."); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/util/ConfigUtil.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/util/ConfigUtil.java index 5f7be806ed954..57476836cc793 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/util/ConfigUtil.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/util/ConfigUtil.java @@ -17,11 +17,11 @@ */ package org.apache.hadoop.mapred.nativetask.util; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.List; import java.util.Map; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.classification.InterfaceAudience; @@ -31,8 +31,8 @@ public abstract class ConfigUtil { public static byte[][] toBytes(Configuration conf) { List nativeConfigs = new ArrayList(); for (Map.Entry e : conf) { - nativeConfigs.add(e.getKey().getBytes(Charsets.UTF_8)); - nativeConfigs.add(e.getValue().getBytes(Charsets.UTF_8)); + nativeConfigs.add(e.getKey().getBytes(StandardCharsets.UTF_8)); + nativeConfigs.add(e.getValue().getBytes(StandardCharsets.UTF_8)); } return nativeConfigs.toArray(new byte[nativeConfigs.size()][]); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/util/ReadWriteBuffer.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/util/ReadWriteBuffer.java index af2c496eb6c64..c0161439980e5 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/util/ReadWriteBuffer.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/util/ReadWriteBuffer.java @@ -17,9 +17,10 @@ */ package org.apache.hadoop.mapred.nativetask.util; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; import org.apache.hadoop.classification.InterfaceAudience; +import java.nio.charset.StandardCharsets; + @InterfaceAudience.Private public class ReadWriteBuffer { private byte[] _buff; @@ -135,13 +136,13 @@ public byte[] readBytes() { } public void writeString(String str) { - final byte[] bytes = str.getBytes(Charsets.UTF_8); + final byte[] bytes = str.getBytes(StandardCharsets.UTF_8); writeBytes(bytes, 0, bytes.length); } public String readString() { final byte[] bytes = readBytes(); - return new String(bytes, Charsets.UTF_8); + return new String(bytes, StandardCharsets.UTF_8); } private void checkWriteSpaceAndResizeIfNecessary(int toBeWritten) { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/buffer/TestByteBufferReadWrite.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/buffer/TestByteBufferReadWrite.java index 98d0697100928..8dfa5322e84e6 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/buffer/TestByteBufferReadWrite.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/buffer/TestByteBufferReadWrite.java @@ -19,7 +19,7 @@ import java.io.ByteArrayInputStream; import java.io.IOException; -import java.io.UnsupportedEncodingException; +import java.nio.charset.StandardCharsets; import org.apache.hadoop.mapred.nativetask.NativeDataTarget; @@ -143,8 +143,8 @@ public void testFlush() throws IOException { Mockito.verify(target).finishSendData(); } - private static String toString(byte[] str) throws UnsupportedEncodingException { - return new String(str, 0, str.length, "UTF-8"); + private static String toString(byte[] str) { + return new String(str, 0, str.length, StandardCharsets.UTF_8); } private static class MockDataTarget implements NativeDataTarget { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleChannelHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleChannelHandler.java index 49c0bb288b534..d0d0e74c9970b 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleChannelHandler.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleChannelHandler.java @@ -45,6 +45,7 @@ import java.io.RandomAccessFile; import java.net.URL; import java.nio.channels.ClosedChannelException; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -60,7 +61,6 @@ import org.apache.hadoop.io.SecureIOUtils; import org.apache.hadoop.mapreduce.security.SecureShuffleUtils; import org.apache.hadoop.mapreduce.task.reduce.ShuffleHeader; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; import org.eclipse.jetty.http.HttpHeader; import static io.netty.buffer.Unpooled.wrappedBuffer; @@ -469,7 +469,7 @@ protected void verifyRequest(String appid, ChannelHandlerContext ctx, // verify - throws exception SecureShuffleUtils.verifyReply(urlHashStr, encryptedURL, tokenSecret); // verification passed - encode the reply - String reply = SecureShuffleUtils.generateHash(urlHashStr.getBytes(Charsets.UTF_8), + String reply = SecureShuffleUtils.generateHash(urlHashStr.getBytes(StandardCharsets.UTF_8), tokenSecret); response.headers().set( SecureShuffleUtils.HTTP_HEADER_REPLY_URL_HASH, reply); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleChannelHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleChannelHandler.java index 66fa3de94f89f..f052e4bc70c34 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleChannelHandler.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleChannelHandler.java @@ -79,7 +79,6 @@ import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; import org.apache.hadoop.security.token.SecretManager; import org.apache.hadoop.security.token.Token; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; import org.eclipse.jetty.http.HttpHeader; import org.junit.Test; import org.slf4j.LoggerFactory; @@ -336,7 +335,7 @@ public DefaultHttpResponse getExpectedHttpResponse( SecretKey tokenSecret = ctx.secretManager.retrieveTokenSecret(TEST_JOB_ID); headers.set(SecureShuffleUtils.HTTP_HEADER_REPLY_URL_HASH, SecureShuffleUtils.generateHash( - request.headers().get(HTTP_HEADER_URL_HASH).getBytes(Charsets.UTF_8), + request.headers().get(HTTP_HEADER_URL_HASH).getBytes(StandardCharsets.UTF_8), tokenSecret)); } catch (SecretManager.InvalidToken e) { fail("Could not generate reply hash"); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/BaileyBorweinPlouffe.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/BaileyBorweinPlouffe.java index f0f9a34f55444..1cc099e14baee 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/BaileyBorweinPlouffe.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/BaileyBorweinPlouffe.java @@ -25,6 +25,7 @@ import java.io.OutputStreamWriter; import java.io.PrintStream; import java.io.PrintWriter; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Iterator; import java.util.List; @@ -53,7 +54,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; /** * A map/reduce program that uses Bailey-Borwein-Plouffe to compute exact @@ -158,7 +158,7 @@ protected void cleanup(Context context final OutputStream outputstream = fs.create(outfile); try { final PrintWriter out = new PrintWriter( - new OutputStreamWriter(outputstream, Charsets.UTF_8), true); + new OutputStreamWriter(outputstream, StandardCharsets.UTF_8), true); // write hex text print(out, hex.iterator(), "Pi = 0x3.", "%02X", 5, 5); out.println("Total number of hexadecimal digits is " diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/WordMean.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/WordMean.java index 26a3009918674..58518de084d35 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/WordMean.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/WordMean.java @@ -21,6 +21,7 @@ import java.io.BufferedReader; import java.io.IOException; import java.io.InputStreamReader; +import java.nio.charset.StandardCharsets; import java.util.StringTokenizer; import org.apache.hadoop.conf.Configuration; @@ -37,8 +38,6 @@ import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; - public class WordMean extends Configured implements Tool { private double mean = 0; @@ -96,7 +95,7 @@ public static class WordMeanReducer extends public void reduce(Text key, Iterable values, Context context) throws IOException, InterruptedException { - int theSum = 0; + long theSum = 0; for (LongWritable val : values) { theSum += val.get(); } @@ -127,7 +126,7 @@ private double readAndCalcMean(Path path, Configuration conf) // average = total sum / number of elements; try { - br = new BufferedReader(new InputStreamReader(fs.open(file), Charsets.UTF_8)); + br = new BufferedReader(new InputStreamReader(fs.open(file), StandardCharsets.UTF_8)); long count = 0; long length = 0; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/WordMedian.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/WordMedian.java index 9acf62bd17e24..c209da4ee0e5b 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/WordMedian.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/WordMedian.java @@ -21,6 +21,7 @@ import java.io.BufferedReader; import java.io.IOException; import java.io.InputStreamReader; +import java.nio.charset.StandardCharsets; import java.util.StringTokenizer; import org.apache.hadoop.conf.Configuration; @@ -39,7 +40,6 @@ import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; public class WordMedian extends Configured implements Tool { @@ -130,7 +130,7 @@ private double readAndFindMedian(String path, int medianIndex1, BufferedReader br = null; try { - br = new BufferedReader(new InputStreamReader(fs.open(file), Charsets.UTF_8)); + br = new BufferedReader(new InputStreamReader(fs.open(file), StandardCharsets.UTF_8)); int num = 0; String line; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/WordStandardDeviation.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/WordStandardDeviation.java index 2a7733b875c09..57c35eb6e0766 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/WordStandardDeviation.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/WordStandardDeviation.java @@ -21,6 +21,7 @@ import java.io.BufferedReader; import java.io.IOException; import java.io.InputStreamReader; +import java.nio.charset.StandardCharsets; import java.util.StringTokenizer; import org.apache.hadoop.conf.Configuration; @@ -37,7 +38,6 @@ import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; public class WordStandardDeviation extends Configured implements Tool { @@ -137,7 +137,7 @@ private double readAndCalcStdDev(Path path, Configuration conf) double stddev = 0; BufferedReader br = null; try { - br = new BufferedReader(new InputStreamReader(fs.open(file), Charsets.UTF_8)); + br = new BufferedReader(new InputStreamReader(fs.open(file), StandardCharsets.UTF_8)); long count = 0; long length = 0; long square = 0; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/dancing/DistributedPentomino.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/dancing/DistributedPentomino.java index e2d034193beb9..56b580ecc63be 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/dancing/DistributedPentomino.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/dancing/DistributedPentomino.java @@ -19,6 +19,7 @@ package org.apache.hadoop.examples.dancing; import java.io.*; +import java.nio.charset.StandardCharsets; import java.util.List; import java.util.StringTokenizer; @@ -33,7 +34,6 @@ import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; import org.apache.hadoop.util.*; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; /** * Launch a distributed pentomino solver. @@ -141,7 +141,7 @@ private static long createInputDirectory(FileSystem fs, Path input = new Path(dir, "part1"); PrintWriter file = new PrintWriter(new OutputStreamWriter(new BufferedOutputStream - (fs.create(input), 64*1024), Charsets.UTF_8)); + (fs.create(input), 64*1024), StandardCharsets.UTF_8)); for(int[] prefix: splits) { for(int i=0; i < prefix.length; ++i) { if (i != 0) { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/dancing/Sudoku.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/dancing/Sudoku.java index aa2df72af2805..402ff028dfb42 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/dancing/Sudoku.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/dancing/Sudoku.java @@ -19,9 +19,9 @@ package org.apache.hadoop.examples.dancing; import java.io.*; +import java.nio.charset.StandardCharsets; import java.util.*; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; /** * This class uses the dancing links algorithm from Knuth to solve sudoku @@ -136,7 +136,7 @@ public void solution(List> names) { */ public Sudoku(InputStream stream) throws IOException { BufferedReader file = new BufferedReader( - new InputStreamReader(stream, Charsets.UTF_8)); + new InputStreamReader(stream, StandardCharsets.UTF_8)); String line = file.readLine(); List result = new ArrayList(); while (line != null) { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/pi/Parser.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/pi/Parser.java index 16273fd0baaa4..bffaf8fd1574d 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/pi/Parser.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/pi/Parser.java @@ -25,6 +25,7 @@ import java.io.InputStreamReader; import java.io.OutputStreamWriter; import java.io.PrintWriter; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -34,7 +35,6 @@ import org.apache.hadoop.examples.pi.math.Bellard; import org.apache.hadoop.examples.pi.math.Bellard.Parameter; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; /** A class for parsing outputs */ public final class Parser { @@ -80,7 +80,7 @@ private void parse(File f, Map> sums) throws IOExcep m.put(p, new ArrayList()); final BufferedReader in = new BufferedReader( - new InputStreamReader(new FileInputStream(f), Charsets.UTF_8)); + new InputStreamReader(new FileInputStream(f), StandardCharsets.UTF_8)); try { for(String line; (line = in.readLine()) != null; ) try { @@ -137,7 +137,7 @@ Map> parse(String inputpath, String outputdir final PrintWriter out = new PrintWriter( new OutputStreamWriter(new FileOutputStream( - new File(outputdir, p + ".txt")), Charsets.UTF_8), true); + new File(outputdir, p + ".txt")), StandardCharsets.UTF_8), true); try { for(int i = 0; i < results.size(); i++) out.println(DistSum.taskResult2string(p + "." + i, results.get(i))); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/pi/Util.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/pi/Util.java index ddbbf6334379d..4e8461525ae3f 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/pi/Util.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/pi/Util.java @@ -25,6 +25,7 @@ import java.io.OutputStreamWriter; import java.io.PrintStream; import java.io.PrintWriter; +import java.nio.charset.StandardCharsets; import java.text.SimpleDateFormat; import java.util.ArrayList; import java.util.Arrays; @@ -46,7 +47,6 @@ import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.util.ToolRunner; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; import org.apache.hadoop.util.concurrent.HadoopExecutors; /** Utility methods */ @@ -216,7 +216,8 @@ public static PrintWriter createWriter(File dir, String prefix) throws IOExcepti final File f = new File(dir, prefix + dateFormat.format(new Date(System.currentTimeMillis())) + ".txt"); if (!f.exists()) - return new PrintWriter(new OutputStreamWriter(new FileOutputStream(f), Charsets.UTF_8)); + return new PrintWriter(new OutputStreamWriter( + new FileOutputStream(f), StandardCharsets.UTF_8)); try {Thread.sleep(10);} catch (InterruptedException e) {} } @@ -291,7 +292,7 @@ static List readJobOutputs(FileSystem fs, Path outdir) throws IOExce for(FileStatus status : fs.listStatus(outdir)) { if (status.getPath().getName().startsWith("part-")) { final BufferedReader in = new BufferedReader( - new InputStreamReader(fs.open(status.getPath()), Charsets.UTF_8)); + new InputStreamReader(fs.open(status.getPath()), StandardCharsets.UTF_8)); try { for(String line; (line = in.readLine()) != null; ) results.add(TaskResult.valueOf(line)); @@ -310,13 +311,14 @@ static List readJobOutputs(FileSystem fs, Path outdir) throws IOExce static void writeResults(String name, List results, FileSystem fs, String dir) throws IOException { final Path outfile = new Path(dir, name + ".txt"); Util.out.println(name + "> writing results to " + outfile); - final PrintWriter out = new PrintWriter(new OutputStreamWriter(fs.create(outfile), Charsets.UTF_8), true); + final PrintWriter printWriter = new PrintWriter(new OutputStreamWriter( + fs.create(outfile), StandardCharsets.UTF_8), true); try { for(TaskResult r : results) - out.println(r); + printWriter.println(r); } finally { - out.close(); + printWriter.close(); } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraScheduler.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraScheduler.java index 7998d4a8f6133..6df1f1e497783 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraScheduler.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraScheduler.java @@ -19,6 +19,7 @@ package org.apache.hadoop.examples.terasort; import java.io.*; +import java.nio.charset.StandardCharsets; import java.util.*; import org.apache.hadoop.conf.Configuration; @@ -28,7 +29,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; class TeraScheduler { private static final Logger LOG = @@ -75,7 +75,7 @@ public String toString() { List readFile(String filename) throws IOException { List result = new ArrayList(10000); try (BufferedReader in = new BufferedReader( - new InputStreamReader(new FileInputStream(filename), Charsets.UTF_8))) { + new InputStreamReader(new FileInputStream(filename), StandardCharsets.UTF_8))) { String line = in.readLine(); while (line != null) { result.add(line); diff --git a/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/util/Exec.java b/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/util/Exec.java index 528163103da44..86e8d9c2a0ed9 100644 --- a/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/util/Exec.java +++ b/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/util/Exec.java @@ -19,7 +19,7 @@ import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; -import java.io.UnsupportedEncodingException; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -108,11 +108,7 @@ public static class OutputBufferThread extends Thread { public OutputBufferThread(InputStream is) { this.setDaemon(true); output = new ArrayList(); - try { - reader = new BufferedReader(new InputStreamReader(is, "UTF-8")); - } catch (UnsupportedEncodingException e) { - throw new RuntimeException("Unsupported encoding " + e.toString()); - } + reader = new BufferedReader(new InputStreamReader(is, StandardCharsets.UTF_8)); } @Override diff --git a/hadoop-tools/hadoop-archives/src/main/java/org/apache/hadoop/tools/HadoopArchives.java b/hadoop-tools/hadoop-archives/src/main/java/org/apache/hadoop/tools/HadoopArchives.java index 6d082380ffa91..c72a926b1317b 100644 --- a/hadoop-tools/hadoop-archives/src/main/java/org/apache/hadoop/tools/HadoopArchives.java +++ b/hadoop-tools/hadoop-archives/src/main/java/org/apache/hadoop/tools/HadoopArchives.java @@ -23,6 +23,7 @@ import java.io.IOException; import java.io.UnsupportedEncodingException; import java.net.URLEncoder; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.HashSet; import java.util.Iterator; @@ -74,8 +75,6 @@ import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; - /** * a archive creation utility. * This class provides methods that can be used @@ -754,7 +753,7 @@ public void configure(JobConf conf) { indexStream = fs.create(index); outStream = fs.create(masterIndex); String version = VERSION + " \n"; - outStream.write(version.getBytes(Charsets.UTF_8)); + outStream.write(version.getBytes(StandardCharsets.UTF_8)); } catch(IOException e) { throw new RuntimeException(e); @@ -773,7 +772,7 @@ public void reduce(IntWritable key, Iterator values, while(values.hasNext()) { Text value = values.next(); String towrite = value.toString() + "\n"; - indexStream.write(towrite.getBytes(Charsets.UTF_8)); + indexStream.write(towrite.getBytes(StandardCharsets.UTF_8)); written++; if (written > numIndexes -1) { // every 1000 indexes we report status @@ -782,7 +781,7 @@ public void reduce(IntWritable key, Iterator values, endIndex = keyVal; String masterWrite = startIndex + " " + endIndex + " " + startPos + " " + indexStream.getPos() + " \n" ; - outStream.write(masterWrite.getBytes(Charsets.UTF_8)); + outStream.write(masterWrite.getBytes(StandardCharsets.UTF_8)); startPos = indexStream.getPos(); startIndex = endIndex; written = 0; @@ -795,7 +794,7 @@ public void close() throws IOException { if (written > 0) { String masterWrite = startIndex + " " + keyVal + " " + startPos + " " + indexStream.getPos() + " \n"; - outStream.write(masterWrite.getBytes(Charsets.UTF_8)); + outStream.write(masterWrite.getBytes(StandardCharsets.UTF_8)); } // close the streams outStream.close(); diff --git a/hadoop-tools/hadoop-archives/src/test/java/org/apache/hadoop/tools/TestHadoopArchives.java b/hadoop-tools/hadoop-archives/src/test/java/org/apache/hadoop/tools/TestHadoopArchives.java index b1755affa8833..3267a683c275c 100644 --- a/hadoop-tools/hadoop-archives/src/test/java/org/apache/hadoop/tools/TestHadoopArchives.java +++ b/hadoop-tools/hadoop-archives/src/test/java/org/apache/hadoop/tools/TestHadoopArchives.java @@ -23,6 +23,7 @@ import java.io.IOException; import java.io.PrintStream; import java.net.URI; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -81,7 +82,7 @@ public class TestHadoopArchives { private static String createFile(Path root, FileSystem fs, String... dirsAndFile ) throws IOException { String fileBaseName = dirsAndFile[dirsAndFile.length - 1]; - return createFile(root, fs, fileBaseName.getBytes("UTF-8"), dirsAndFile); + return createFile(root, fs, fileBaseName.getBytes(StandardCharsets.UTF_8), dirsAndFile); } private static String createFile(Path root, FileSystem fs, byte[] fileContent, String... dirsAndFile @@ -395,7 +396,7 @@ public void testReadFileContent() throws Exception { } else if ("zero-length".equals(baseName)) { assertEquals(0, actualContentSimple.length); } else { - String actual = new String(actualContentSimple, "UTF-8"); + String actual = new String(actualContentSimple, StandardCharsets.UTF_8); assertEquals(baseName, actual); } readFileCount++; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/AbstractDelegationTokenBinding.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/AbstractDelegationTokenBinding.java index 6af413e44d6c1..f33944070d94d 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/AbstractDelegationTokenBinding.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/AbstractDelegationTokenBinding.java @@ -20,7 +20,7 @@ import java.io.IOException; import java.net.URI; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.util.Optional; import org.slf4j.Logger; @@ -304,7 +304,7 @@ public String getUserAgentField() { * @return a password. */ protected static byte[] getSecretManagerPasssword() { - return "non-password".getBytes(Charset.forName("UTF-8")); + return "non-password".getBytes(StandardCharsets.UTF_8); } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploader.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploader.java index b7eae8ead7096..58e38c2873bd0 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploader.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploader.java @@ -25,6 +25,7 @@ import java.io.IOException; import java.io.InputStream; import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Comparator; import java.util.HashSet; @@ -40,8 +41,6 @@ import software.amazon.awssdk.services.s3.model.UploadPartRequest; import software.amazon.awssdk.services.s3.model.UploadPartResponse; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; - import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -132,7 +131,7 @@ public CompletableFuture startUpload( PutObjectOptions.keepingDirs()); statistics.uploadStarted(); return BBUploadHandle.from(ByteBuffer.wrap( - uploadId.getBytes(Charsets.UTF_8))); + uploadId.getBytes(StandardCharsets.UTF_8))); })); } @@ -151,7 +150,7 @@ public CompletableFuture putPart( checkUploadId(uploadIdBytes); String key = context.pathToKey(dest); String uploadIdString = new String(uploadIdBytes, 0, uploadIdBytes.length, - Charsets.UTF_8); + StandardCharsets.UTF_8); return context.submit(new CompletableFuture<>(), () -> { UploadPartRequest request = writeOperations.newUploadPartRequestBuilder(key, @@ -189,7 +188,7 @@ public CompletableFuture complete( String key = context.pathToKey(dest); String uploadIdStr = new String(uploadIdBytes, 0, uploadIdBytes.length, - Charsets.UTF_8); + StandardCharsets.UTF_8); ArrayList eTags = new ArrayList<>(); eTags.ensureCapacity(handles.size()); long totalLength = 0; @@ -221,7 +220,7 @@ public CompletableFuture complete( finalLen ); - byte[] eTag = result.eTag().getBytes(Charsets.UTF_8); + byte[] eTag = result.eTag().getBytes(StandardCharsets.UTF_8); statistics.uploadCompleted(); return (PathHandle) () -> ByteBuffer.wrap(eTag); })); @@ -237,7 +236,7 @@ public CompletableFuture abort( final byte[] uploadIdBytes = uploadId.toByteArray(); checkUploadId(uploadIdBytes); String uploadIdString = new String(uploadIdBytes, 0, uploadIdBytes.length, - Charsets.UTF_8); + StandardCharsets.UTF_8); return context.submit(new CompletableFuture<>(), () -> { writeOperations.abortMultipartCommit( diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java index eaa84c6086bd9..10bb4a8032fd8 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java @@ -52,7 +52,6 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.service.Service; import org.apache.hadoop.service.ServiceOperations; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListeningExecutorService; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.MoreExecutors; import org.apache.hadoop.util.BlockingThreadPoolExecutorService; @@ -75,6 +74,7 @@ import java.io.InputStream; import java.net.URI; import java.net.URISyntaxException; +import java.nio.charset.StandardCharsets; import java.text.DateFormat; import java.text.SimpleDateFormat; import java.util.ArrayList; @@ -912,7 +912,7 @@ private static CompletableFuture put(FileSystem fs, return submit(EXECUTOR, () -> { try (DurationInfo ignore = new DurationInfo(LOG, false, "Creating %s", path)) { - createFile(fs, path, true, text.getBytes(Charsets.UTF_8)); + createFile(fs, path, true, text.getBytes(StandardCharsets.UTF_8)); return path; } }); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ILoadTestSessionCredentials.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ILoadTestSessionCredentials.java index c3030aa227d2c..3b21a08e30a0a 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ILoadTestSessionCredentials.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ILoadTestSessionCredentials.java @@ -21,6 +21,7 @@ import java.io.File; import java.io.FileWriter; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.List; import java.util.concurrent.CompletionService; @@ -134,7 +135,7 @@ protected String getFilePrefix() { @Test public void testCreate10Tokens() throws Throwable { File file = fetchTokens(10); - String csv = FileUtils.readFileToString(file, "UTF-8"); + String csv = FileUtils.readFileToString(file, StandardCharsets.UTF_8); LOG.info("CSV data\n{}", csv); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperations.java index 64def00fd2871..7499e10da04a0 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperations.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperations.java @@ -21,6 +21,7 @@ import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.List; import java.util.UUID; @@ -495,7 +496,7 @@ public void testUploadEmptyFile() throws Throwable { public void testUploadSmallFile() throws Throwable { File tempFile = File.createTempFile("commit", ".txt"); String text = "hello, world"; - FileUtils.write(tempFile, text, "UTF-8"); + FileUtils.write(tempFile, text, StandardCharsets.UTF_8); CommitOperations actions = newCommitOperations(); Path dest = methodSubPath("testUploadSmallFile"); S3AFileSystem fs = getFileSystem(); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java index e37717bfa1e34..0ecbe4d5b8ded 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java @@ -107,7 +107,8 @@ public Configuration createConfiguration() { removeBaseAndBucketOverrides(bucketName, conf, DIRECTORY_MARKER_POLICY, - AUTHORITATIVE_PATH); + AUTHORITATIVE_PATH, + FS_S3A_CREATE_PERFORMANCE); // directory marker options conf.set(DIRECTORY_MARKER_POLICY, keepMarkers @@ -235,6 +236,21 @@ protected Path file(Path path, final boolean overwrite) return path; } + /** + * Create a file with a specific body, returning its path. + * @param path path to file. + * @param overwrite overwrite flag + * @param body body of file + * @return path of new file + */ + protected Path file(Path path, final boolean overwrite, byte[] body) + throws IOException { + ContractTestUtils.createFile(getFileSystem(), path, overwrite, body); + return path; + } + + + /** * Touch a file, overwriting. * @param path path diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java index 3a503ddfa2b20..45fbf791908a3 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java @@ -26,7 +26,7 @@ import java.io.OutputStream; import java.net.URI; import java.net.URISyntaxException; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.text.SimpleDateFormat; import java.util.ArrayList; import java.util.Date; @@ -178,7 +178,7 @@ public FolderRenamePending(Path redoFile, NativeAzureFileSystem fs) "Error reading pending rename file contents -- " + "maximum file size exceeded"); } - String contents = new String(bytes, 0, l, Charset.forName("UTF-8")); + String contents = new String(bytes, 0, l, StandardCharsets.UTF_8); // parse the JSON JsonNode json = null; @@ -301,7 +301,7 @@ public void writeFile(NativeAzureFileSystem fs) throws IOException { // Write file. try { output = fs.createInternal(path, FsPermission.getFileDefault(), false, null); - output.write(contents.getBytes(Charset.forName("UTF-8"))); + output.write(contents.getBytes(StandardCharsets.UTF_8)); } catch (IOException e) { throw new IOException("Unable to write RenamePending file for folder rename from " + srcKey + " to " + dstKey, e); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/AzureADAuthenticator.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/AzureADAuthenticator.java index dd4ec7c2009c2..1a1a27c53b641 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/AzureADAuthenticator.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/AzureADAuthenticator.java @@ -374,7 +374,7 @@ private static AzureADToken getTokenSingleCall(String authEndpoint, conn.getRequestProperties()); if (httpMethod.equals("POST")) { conn.setDoOutput(true); - conn.getOutputStream().write(payload.getBytes("UTF-8")); + conn.getOutputStream().write(payload.getBytes(StandardCharsets.UTF_8)); } int httpResponseCode = conn.getResponseCode(); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index 45cb538d0b007..9c1f590da9c5a 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -260,26 +260,25 @@ AbfsUriQueryBuilder createDefaultUriQueryBuilder() { return abfsUriQueryBuilder; } - public AbfsRestOperation createFilesystem(TracingContext tracingContext) throws AzureBlobFileSystemException { + public AbfsRestOperation createFilesystem(TracingContext tracingContext) + throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); final AbfsUriQueryBuilder abfsUriQueryBuilder = new AbfsUriQueryBuilder(); abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM); final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( - AbfsRestOperationType.CreateFileSystem, - this, - HTTP_METHOD_PUT, - url, - requestHeaders); + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.CreateFileSystem, + HTTP_METHOD_PUT, url, requestHeaders); op.execute(tracingContext); return op; } - public AbfsRestOperation setFilesystemProperties(final String properties, TracingContext tracingContext) throws AzureBlobFileSystemException { + public AbfsRestOperation setFilesystemProperties(final String properties, + TracingContext tracingContext) throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); - // JDK7 does not support PATCH, so to workaround the issue we will use + // JDK7 does not support PATCH, so to work around the issue we will use // PUT and specify the real method in the X-Http-Method-Override header. requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, HTTP_METHOD_PATCH)); @@ -291,9 +290,8 @@ public AbfsRestOperation setFilesystemProperties(final String properties, Tracin abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM); final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( + final AbfsRestOperation op = getAbfsRestOperation( AbfsRestOperationType.SetFileSystemProperties, - this, HTTP_METHOD_PUT, url, requestHeaders); @@ -316,9 +314,8 @@ public AbfsRestOperation listPath(final String relativePath, final boolean recur appendSASTokenToQuery(relativePath, SASTokenProvider.LIST_OPERATION, abfsUriQueryBuilder); final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( + final AbfsRestOperation op = getAbfsRestOperation( AbfsRestOperationType.ListPaths, - this, HTTP_METHOD_GET, url, requestHeaders); @@ -333,9 +330,8 @@ public AbfsRestOperation getFilesystemProperties(TracingContext tracingContext) abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM); final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( + final AbfsRestOperation op = getAbfsRestOperation( AbfsRestOperationType.GetFileSystemProperties, - this, HTTP_METHOD_HEAD, url, requestHeaders); @@ -350,9 +346,8 @@ public AbfsRestOperation deleteFilesystem(TracingContext tracingContext) throws abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM); final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( + final AbfsRestOperation op = getAbfsRestOperation( AbfsRestOperationType.DeleteFileSystem, - this, HTTP_METHOD_DELETE, url, requestHeaders); @@ -396,9 +391,8 @@ public AbfsRestOperation createPath(final String path, final boolean isFile, fin appendSASTokenToQuery(path, operation, abfsUriQueryBuilder); final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( + final AbfsRestOperation op = getAbfsRestOperation( AbfsRestOperationType.CreatePath, - this, HTTP_METHOD_PUT, url, requestHeaders); @@ -431,9 +425,8 @@ public AbfsRestOperation acquireLease(final String path, int duration, TracingCo final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( + final AbfsRestOperation op = getAbfsRestOperation( AbfsRestOperationType.LeasePath, - this, HTTP_METHOD_POST, url, requestHeaders); @@ -451,9 +444,8 @@ public AbfsRestOperation renewLease(final String path, final String leaseId, final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( + final AbfsRestOperation op = getAbfsRestOperation( AbfsRestOperationType.LeasePath, - this, HTTP_METHOD_POST, url, requestHeaders); @@ -471,9 +463,8 @@ public AbfsRestOperation releaseLease(final String path, final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( + final AbfsRestOperation op = getAbfsRestOperation( AbfsRestOperationType.LeasePath, - this, HTTP_METHOD_POST, url, requestHeaders); @@ -491,9 +482,8 @@ public AbfsRestOperation breakLease(final String path, final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( + final AbfsRestOperation op = getAbfsRestOperation( AbfsRestOperationType.LeasePath, - this, HTTP_METHOD_POST, url, requestHeaders); @@ -646,9 +636,8 @@ private boolean checkIsDir(AbfsHttpOperation result) { @VisibleForTesting AbfsRestOperation createRenameRestOperation(URL url, List requestHeaders) { - AbfsRestOperation op = new AbfsRestOperation( + AbfsRestOperation op = getAbfsRestOperation( AbfsRestOperationType.RenamePath, - this, HTTP_METHOD_PUT, url, requestHeaders); @@ -766,7 +755,8 @@ public AbfsRestOperation append(final String path, final byte[] buffer, abfsUriQueryBuilder, cachedSasToken); final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = getAbfsRestOperationForAppend(AbfsRestOperationType.Append, + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.Append, HTTP_METHOD_PUT, url, requestHeaders, @@ -801,7 +791,7 @@ public AbfsRestOperation append(final String path, final byte[] buffer, if (reqParams.isAppendBlob() && appendSuccessCheckOp(op, path, (reqParams.getPosition() + reqParams.getLength()), tracingContext)) { - final AbfsRestOperation successOp = getAbfsRestOperationForAppend( + final AbfsRestOperation successOp = getAbfsRestOperation( AbfsRestOperationType.Append, HTTP_METHOD_PUT, url, @@ -819,38 +809,6 @@ && appendSuccessCheckOp(op, path, return op; } - /** - * Returns the rest operation for append. - * @param operationType The AbfsRestOperationType. - * @param httpMethod specifies the httpMethod. - * @param url specifies the url. - * @param requestHeaders This includes the list of request headers. - * @param buffer The buffer to write into. - * @param bufferOffset The buffer offset. - * @param bufferLength The buffer Length. - * @param sasTokenForReuse The sasToken. - * @return AbfsRestOperation op. - */ - @VisibleForTesting - AbfsRestOperation getAbfsRestOperationForAppend(final AbfsRestOperationType operationType, - final String httpMethod, - final URL url, - final List requestHeaders, - final byte[] buffer, - final int bufferOffset, - final int bufferLength, - final String sasTokenForReuse) { - return new AbfsRestOperation( - operationType, - this, - httpMethod, - url, - requestHeaders, - buffer, - bufferOffset, - bufferLength, sasTokenForReuse); - } - /** * Returns true if the status code lies in the range of user error. * @param responseStatusCode http response status code. @@ -907,9 +865,8 @@ public AbfsRestOperation flush(final String path, final long position, abfsUriQueryBuilder, cachedSasToken); final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( + final AbfsRestOperation op = getAbfsRestOperation( AbfsRestOperationType.Flush, - this, HTTP_METHOD_PUT, url, requestHeaders, sasTokenForReuse); @@ -934,9 +891,8 @@ public AbfsRestOperation setPathProperties(final String path, final String prope appendSASTokenToQuery(path, SASTokenProvider.SET_PROPERTIES_OPERATION, abfsUriQueryBuilder); final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( + final AbfsRestOperation op = getAbfsRestOperation( AbfsRestOperationType.SetPathProperties, - this, HTTP_METHOD_PUT, url, requestHeaders); @@ -963,9 +919,8 @@ public AbfsRestOperation getPathStatus(final String path, final boolean includeP appendSASTokenToQuery(path, operation, abfsUriQueryBuilder); final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( + final AbfsRestOperation op = getAbfsRestOperation( AbfsRestOperationType.GetPathStatus, - this, HTTP_METHOD_HEAD, url, requestHeaders); @@ -988,9 +943,8 @@ public AbfsRestOperation read(final String path, final long position, final byte abfsUriQueryBuilder, cachedSasToken); final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( + final AbfsRestOperation op = getAbfsRestOperation( AbfsRestOperationType.ReadFile, - this, HTTP_METHOD_GET, url, requestHeaders, @@ -1063,9 +1017,8 @@ public AbfsRestOperation deleteIdempotencyCheckOp(final AbfsRestOperation op) { && DEFAULT_DELETE_CONSIDERED_IDEMPOTENT) { // Server has returned HTTP 404, which means path no longer // exists. Assuming delete result to be idempotent, return success. - final AbfsRestOperation successOp = new AbfsRestOperation( + final AbfsRestOperation successOp = getAbfsRestOperation( AbfsRestOperationType.DeletePath, - this, HTTP_METHOD_DELETE, op.getUrl(), op.getRequestHeaders()); @@ -1098,9 +1051,8 @@ public AbfsRestOperation setOwner(final String path, final String owner, final S appendSASTokenToQuery(path, SASTokenProvider.SET_OWNER_OPERATION, abfsUriQueryBuilder); final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( + final AbfsRestOperation op = getAbfsRestOperation( AbfsRestOperationType.SetOwner, - this, AbfsHttpConstants.HTTP_METHOD_PUT, url, requestHeaders); @@ -1124,9 +1076,8 @@ public AbfsRestOperation setPermission(final String path, final String permissio appendSASTokenToQuery(path, SASTokenProvider.SET_PERMISSION_OPERATION, abfsUriQueryBuilder); final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( + final AbfsRestOperation op = getAbfsRestOperation( AbfsRestOperationType.SetPermissions, - this, AbfsHttpConstants.HTTP_METHOD_PUT, url, requestHeaders); @@ -1159,9 +1110,8 @@ public AbfsRestOperation setAcl(final String path, final String aclSpecString, f appendSASTokenToQuery(path, SASTokenProvider.SET_ACL_OPERATION, abfsUriQueryBuilder); final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( + final AbfsRestOperation op = getAbfsRestOperation( AbfsRestOperationType.SetAcl, - this, AbfsHttpConstants.HTTP_METHOD_PUT, url, requestHeaders); @@ -1184,9 +1134,8 @@ public AbfsRestOperation getAclStatus(final String path, final boolean useUPN, appendSASTokenToQuery(path, SASTokenProvider.GET_ACL_OPERATION, abfsUriQueryBuilder); final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( + final AbfsRestOperation op = getAbfsRestOperation( AbfsRestOperationType.GetAcl, - this, AbfsHttpConstants.HTTP_METHOD_HEAD, url, requestHeaders); @@ -1211,9 +1160,11 @@ public AbfsRestOperation checkAccess(String path, String rwx, TracingContext tra abfsUriQueryBuilder.addQuery(QUERY_FS_ACTION, rwx); appendSASTokenToQuery(path, SASTokenProvider.CHECK_ACCESS_OPERATION, abfsUriQueryBuilder); URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - AbfsRestOperation op = new AbfsRestOperation( - AbfsRestOperationType.CheckAccess, this, - AbfsHttpConstants.HTTP_METHOD_HEAD, url, createDefaultHeaders()); + AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.CheckAccess, + AbfsHttpConstants.HTTP_METHOD_HEAD, + url, + createDefaultHeaders()); op.execute(tracingContext); return op; } @@ -1238,7 +1189,7 @@ public static String getDirectoryQueryParameter(final String path) { } /** - * If configured for SAS AuthType, appends SAS token to queryBuilder + * If configured for SAS AuthType, appends SAS token to queryBuilder. * @param path * @param operation * @param queryBuilder @@ -1250,7 +1201,7 @@ private String appendSASTokenToQuery(String path, String operation, AbfsUriQuery } /** - * If configured for SAS AuthType, appends SAS token to queryBuilder + * If configured for SAS AuthType, appends SAS token to queryBuilder. * @param path * @param operation * @param queryBuilder @@ -1459,4 +1410,82 @@ public void addCallback(ListenableFuture future, FutureCallback callba protected AccessTokenProvider getTokenProvider() { return tokenProvider; } + + /** + * Creates an AbfsRestOperation with additional parameters for buffer and SAS token. + * + * @param operationType The type of the operation. + * @param httpMethod The HTTP method of the operation. + * @param url The URL associated with the operation. + * @param requestHeaders The list of HTTP headers for the request. + * @param buffer The byte buffer containing data for the operation. + * @param bufferOffset The offset within the buffer where the data starts. + * @param bufferLength The length of the data within the buffer. + * @param sasTokenForReuse The SAS token for reusing authentication. + * @return An AbfsRestOperation instance. + */ + AbfsRestOperation getAbfsRestOperation(final AbfsRestOperationType operationType, + final String httpMethod, + final URL url, + final List requestHeaders, + final byte[] buffer, + final int bufferOffset, + final int bufferLength, + final String sasTokenForReuse) { + return new AbfsRestOperation( + operationType, + this, + httpMethod, + url, + requestHeaders, + buffer, + bufferOffset, + bufferLength, + sasTokenForReuse); + } + + /** + * Creates an AbfsRestOperation with basic parameters and no buffer or SAS token. + * + * @param operationType The type of the operation. + * @param httpMethod The HTTP method of the operation. + * @param url The URL associated with the operation. + * @param requestHeaders The list of HTTP headers for the request. + * @return An AbfsRestOperation instance. + */ + AbfsRestOperation getAbfsRestOperation(final AbfsRestOperationType operationType, + final String httpMethod, + final URL url, + final List requestHeaders) { + return new AbfsRestOperation( + operationType, + this, + httpMethod, + url, + requestHeaders + ); + } + + /** + * Creates an AbfsRestOperation with parameters including request headers and SAS token. + * + * @param operationType The type of the operation. + * @param httpMethod The HTTP method of the operation. + * @param url The URL associated with the operation. + * @param requestHeaders The list of HTTP headers for the request. + * @param sasTokenForReuse The SAS token for reusing authentication. + * @return An AbfsRestOperation instance. + */ + AbfsRestOperation getAbfsRestOperation(final AbfsRestOperationType operationType, + final String httpMethod, + final URL url, + final List requestHeaders, + final String sasTokenForReuse) { + return new AbfsRestOperation( + operationType, + this, + httpMethod, + url, + requestHeaders, sasTokenForReuse); + } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java index 6402be72ddc37..f40cd2cea81ed 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java @@ -82,6 +82,11 @@ public class AbfsRestOperation { */ private String failureReason; + /** + * This variable stores the tracing context used for last Rest Operation. + */ + private TracingContext lastUsedTracingContext; + /** * Checks if there is non-null HTTP response. * @return true if there is a non-null HTTP response from the ABFS call. @@ -197,10 +202,13 @@ String getSasToken() { public void execute(TracingContext tracingContext) throws AzureBlobFileSystemException { + // Since this might be a sub-sequential or parallel rest operation + // triggered by a single file system call, using a new tracing context. + lastUsedTracingContext = createNewTracingContext(tracingContext); try { IOStatisticsBinding.trackDurationOfInvocation(abfsCounters, AbfsStatistic.getStatNameFromHttpCall(method), - () -> completeExecute(tracingContext)); + () -> completeExecute(lastUsedTracingContext)); } catch (AzureBlobFileSystemException aze) { throw aze; } catch (IOException e) { @@ -214,7 +222,7 @@ public void execute(TracingContext tracingContext) * HTTP operations. * @param tracingContext TracingContext instance to track correlation IDs */ - private void completeExecute(TracingContext tracingContext) + void completeExecute(TracingContext tracingContext) throws AzureBlobFileSystemException { // see if we have latency reports from the previous requests String latencyHeader = getClientLatency(); @@ -409,4 +417,25 @@ private void incrementCounter(AbfsStatistic statistic, long value) { abfsCounters.incrementCounter(statistic, value); } } + + /** + * Creates a new Tracing context before entering the retry loop of a rest operation. + * This will ensure all rest operations have unique + * tracing context that will be used for all the retries. + * @param tracingContext original tracingContext. + * @return tracingContext new tracingContext object created from original one. + */ + @VisibleForTesting + public TracingContext createNewTracingContext(final TracingContext tracingContext) { + return new TracingContext(tracingContext); + } + + /** + * Returns the tracing contest used for last rest operation made. + * @return tracingContext lasUserTracingContext. + */ + @VisibleForTesting + public final TracingContext getLastTracingContext() { + return lastUsedTracingContext; + } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestBlobOperationDescriptor.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestBlobOperationDescriptor.java index aca5f810b4b53..598469488a661 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestBlobOperationDescriptor.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestBlobOperationDescriptor.java @@ -32,6 +32,7 @@ import org.junit.Test; import java.net.HttpURLConnection; +import java.nio.charset.StandardCharsets; /** * Tests for BlobOperationDescriptor. @@ -71,7 +72,7 @@ public void testAppendBlockOperations() throws Exception { assertEquals(0, lastContentLengthReceived); String message = "this is a test"; - output.write(message.getBytes("UTF-8")); + output.write(message.getBytes(StandardCharsets.UTF_8)); output.flush(); assertEquals(BlobOperationDescriptor.OperationType.AppendBlock, lastOperationTypeSent); @@ -107,7 +108,7 @@ public void testPutBlockOperations() throws Exception { assertEquals(0, lastContentLengthReceived); String message = "this is a test"; - output.write(message.getBytes("UTF-8")); + output.write(message.getBytes(StandardCharsets.UTF_8)); output.flush(); assertEquals(BlobOperationDescriptor.OperationType.PutBlock, lastOperationTypeSent); @@ -186,7 +187,7 @@ public void testGetBlobOperations() throws Exception { assertNull(lastOperationTypeReceived); assertEquals(0, lastContentLengthReceived); - output.write(message.getBytes("UTF-8")); + output.write(message.getBytes(StandardCharsets.UTF_8)); output.flush(); assertEquals(BlobOperationDescriptor.OperationType.PutBlock, lastOperationTypeSent); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java index 1f0ff667522da..57f5702f74fab 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java @@ -29,10 +29,14 @@ import org.assertj.core.api.Assertions; import org.junit.Assume; import org.junit.Test; +import org.mockito.Mockito; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; import org.apache.hadoop.fs.azurebfs.services.AbfsClient; +import org.apache.hadoop.fs.azurebfs.services.AbfsClientTestUtil; import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation; import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; import org.apache.hadoop.fs.azurebfs.services.ITestAbfsClient; @@ -61,7 +65,6 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathDoesNotExist; import static org.apache.hadoop.test.LambdaTestUtils.intercept; - /** * Test delete operation. */ @@ -257,14 +260,15 @@ public void testDeleteIdempotencyTriggerHttp404() throws Exception { // Case 2: Mimic retried case // Idempotency check on Delete always returns success - AbfsRestOperation idempotencyRetOp = ITestAbfsClient.getRestOp( + AbfsRestOperation idempotencyRetOp = Mockito.spy(ITestAbfsClient.getRestOp( DeletePath, mockClient, HTTP_METHOD_DELETE, ITestAbfsClient.getTestUrl(mockClient, "/NonExistingPath"), - ITestAbfsClient.getTestRequestHeaders(mockClient)); + ITestAbfsClient.getTestRequestHeaders(mockClient))); idempotencyRetOp.hardSetResult(HTTP_OK); doReturn(idempotencyRetOp).when(mockClient).deleteIdempotencyCheckOp(any()); TracingContext tracingContext = getTestTracingContext(fs, false); + doReturn(tracingContext).when(idempotencyRetOp).createNewTracingContext(any()); when(mockClient.deletePath("/NonExistingPath", false, null, tracingContext)) .thenCallRealMethod(); @@ -283,4 +287,25 @@ public void testDeleteIdempotencyTriggerHttp404() throws Exception { mockStore.delete(new Path("/NonExistingPath"), false, getTestTracingContext(fs, false)); } + @Test + public void deleteBlobDirParallelThreadToDeleteOnDifferentTracingContext() + throws Exception { + Configuration configuration = getRawConfiguration(); + AzureBlobFileSystem fs = Mockito.spy( + (AzureBlobFileSystem) FileSystem.newInstance(configuration)); + AzureBlobFileSystemStore spiedStore = Mockito.spy(fs.getAbfsStore()); + AbfsClient spiedClient = Mockito.spy(fs.getAbfsClient()); + + Mockito.doReturn(spiedStore).when(fs).getAbfsStore(); + spiedStore.setClient(spiedClient); + + fs.mkdirs(new Path("/testDir")); + fs.create(new Path("/testDir/file1")); + fs.create(new Path("/testDir/file2")); + + AbfsClientTestUtil.hookOnRestOpsForTracingContextSingularity(spiedClient); + + fs.delete(new Path("/testDir"), true); + fs.close(); + } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java index 8d1330b5ea7dd..e7f57b8af54d0 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java @@ -20,6 +20,7 @@ import java.io.FileNotFoundException; import java.io.IOException; +import java.net.SocketTimeoutException; import java.util.ArrayList; import java.util.List; import java.util.concurrent.Callable; @@ -28,6 +29,8 @@ import java.util.concurrent.Future; import org.junit.Test; +import org.mockito.Mockito; +import org.mockito.stubbing.Stubber; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -36,16 +39,30 @@ import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; +import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; +import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultEntrySchema; +import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultSchema; +import org.apache.hadoop.fs.azurebfs.services.AbfsClient; +import org.apache.hadoop.fs.azurebfs.services.AbfsClientTestUtil; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; +import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderFormat; import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.apache.hadoop.fs.contract.ContractTestUtils; +import static java.net.HttpURLConnection.HTTP_OK; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_LIST_MAX_RESULTS; +import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.CONNECTION_TIMEOUT_JDK_MESSAGE; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertMkdirs; import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathExists; import static org.apache.hadoop.fs.contract.ContractTestUtils.rename; import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.nullable; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.when; /** * Test listStatus operation. @@ -53,6 +70,7 @@ public class ITestAzureBlobFileSystemListStatus extends AbstractAbfsIntegrationTest { private static final int TEST_FILES_NUMBER = 6000; + private static final String TEST_CONTINUATION_TOKEN = "continuation"; public ITestAzureBlobFileSystemListStatus() throws Exception { super(); @@ -62,34 +80,105 @@ public ITestAzureBlobFileSystemListStatus() throws Exception { public void testListPath() throws Exception { Configuration config = new Configuration(this.getRawConfiguration()); config.set(AZURE_LIST_MAX_RESULTS, "5000"); - final AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem - .newInstance(getFileSystem().getUri(), config); - final List> tasks = new ArrayList<>(); - - ExecutorService es = Executors.newFixedThreadPool(10); - for (int i = 0; i < TEST_FILES_NUMBER; i++) { - final Path fileName = new Path("/test" + i); - Callable callable = new Callable() { - @Override - public Void call() throws Exception { - touch(fileName); - return null; - } - }; - - tasks.add(es.submit(callable)); - } - - for (Future task : tasks) { - task.get(); + try (final AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem + .newInstance(getFileSystem().getUri(), config)) { + final List> tasks = new ArrayList<>(); + + ExecutorService es = Executors.newFixedThreadPool(10); + for (int i = 0; i < TEST_FILES_NUMBER; i++) { + final Path fileName = new Path("/test" + i); + Callable callable = new Callable() { + @Override + public Void call() throws Exception { + touch(fileName); + return null; + } + }; + + tasks.add(es.submit(callable)); + } + + for (Future task : tasks) { + task.get(); + } + + es.shutdownNow(); + fs.registerListener( + new TracingHeaderValidator(getConfiguration().getClientCorrelationId(), + fs.getFileSystemId(), FSOperationType.LISTSTATUS, true, 0)); + FileStatus[] files = fs.listStatus(new Path("/")); + assertEquals(TEST_FILES_NUMBER, files.length /* user directory */); } + } - es.shutdownNow(); - fs.registerListener( - new TracingHeaderValidator(getConfiguration().getClientCorrelationId(), - fs.getFileSystemId(), FSOperationType.LISTSTATUS, true, 0)); - FileStatus[] files = fs.listStatus(new Path("/")); - assertEquals(TEST_FILES_NUMBER, files.length /* user directory */); + /** + * Test to verify that each paginated call to ListBlobs uses a new tracing context. + * @throws Exception + */ + @Test + public void testListPathTracingContext() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + final AzureBlobFileSystem spiedFs = Mockito.spy(fs); + final AzureBlobFileSystemStore spiedStore = Mockito.spy(fs.getAbfsStore()); + final AbfsClient spiedClient = Mockito.spy(fs.getAbfsClient()); + final TracingContext spiedTracingContext = Mockito.spy( + new TracingContext( + fs.getClientCorrelationId(), fs.getFileSystemId(), + FSOperationType.LISTSTATUS, true, TracingHeaderFormat.ALL_ID_FORMAT, null)); + + Mockito.doReturn(spiedStore).when(spiedFs).getAbfsStore(); + spiedStore.setClient(spiedClient); + spiedFs.setWorkingDirectory(new Path("/")); + + AbfsClientTestUtil.setMockAbfsRestOperationForListPathOperation(spiedClient, + (httpOperation) -> { + + ListResultEntrySchema entry = new ListResultEntrySchema() + .withName("a") + .withIsDirectory(true); + List paths = new ArrayList<>(); + paths.add(entry); + paths.clear(); + entry = new ListResultEntrySchema() + .withName("abc.txt") + .withIsDirectory(false); + paths.add(entry); + ListResultSchema schema1 = new ListResultSchema().withPaths(paths); + ListResultSchema schema2 = new ListResultSchema().withPaths(paths); + + when(httpOperation.getListResultSchema()).thenReturn(schema1) + .thenReturn(schema2); + when(httpOperation.getResponseHeader( + HttpHeaderConfigurations.X_MS_CONTINUATION)) + .thenReturn(TEST_CONTINUATION_TOKEN) + .thenReturn(EMPTY_STRING); + + Stubber stubber = Mockito.doThrow( + new SocketTimeoutException(CONNECTION_TIMEOUT_JDK_MESSAGE)); + stubber.doNothing().when(httpOperation).processResponse( + nullable(byte[].class), nullable(int.class), nullable(int.class)); + + when(httpOperation.getStatusCode()).thenReturn(-1).thenReturn(HTTP_OK); + return httpOperation; + }); + + List fileStatuses = new ArrayList<>(); + spiedStore.listStatus(new Path("/"), "", fileStatuses, true, null, spiedTracingContext); + + // Assert that there were 2 paginated ListPath calls were made 1 and 2. + // 1. Without continuation token + Mockito.verify(spiedClient, times(1)).listPath( + "/", false, + spiedFs.getAbfsStore().getAbfsConfiguration().getListMaxResults(), + null, spiedTracingContext); + // 2. With continuation token + Mockito.verify(spiedClient, times(1)).listPath( + "/", false, + spiedFs.getAbfsStore().getAbfsConfiguration().getListMaxResults(), + TEST_CONTINUATION_TOKEN, spiedTracingContext); + + // Assert that none of the API calls used the same tracing header. + Mockito.verify(spiedTracingContext, times(0)).constructHeader(any(), any()); } /** diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsConfigurationFieldsValidation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsConfigurationFieldsValidation.java index fe25477beb61e..f041f4bccdc8c 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsConfigurationFieldsValidation.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsConfigurationFieldsValidation.java @@ -20,8 +20,8 @@ import java.io.IOException; import java.lang.reflect.Field; +import java.nio.charset.StandardCharsets; -import org.apache.commons.codec.Charsets; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys; import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys; @@ -99,8 +99,8 @@ public class TestAbfsConfigurationFieldsValidation { public TestAbfsConfigurationFieldsValidation() throws Exception { super(); this.accountName = "testaccount1.blob.core.windows.net"; - this.encodedString = Base64.encode("base64Value".getBytes(Charsets.UTF_8)); - this.encodedAccountKey = Base64.encode("someAccountKey".getBytes(Charsets.UTF_8)); + this.encodedString = Base64.encode("base64Value".getBytes(StandardCharsets.UTF_8)); + this.encodedAccountKey = Base64.encode("someAccountKey".getBytes(StandardCharsets.UTF_8)); Configuration configuration = new Configuration(); configuration.addResource(TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME); configuration.set(INT_KEY, "1234565"); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java index 23e65ed2dd246..2da530364c13b 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java @@ -83,44 +83,46 @@ public void checkCorrelationConfigValidation(String clientCorrelationId, boolean includeInHeader) throws Exception { Configuration conf = getRawConfiguration(); conf.set(FS_AZURE_CLIENT_CORRELATIONID, clientCorrelationId); - AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(conf); - - String correlationID = fs.getClientCorrelationId(); - if (includeInHeader) { - Assertions.assertThat(correlationID) - .describedAs("Correlation ID should match config when valid") - .isEqualTo(clientCorrelationId); - } else { - Assertions.assertThat(correlationID) - .describedAs("Invalid ID should be replaced with empty string") - .isEqualTo(EMPTY_STRING); + try (AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(conf)) { + + String correlationID = fs.getClientCorrelationId(); + if (includeInHeader) { + Assertions.assertThat(correlationID) + .describedAs("Correlation ID should match config when valid") + .isEqualTo(clientCorrelationId); + } else { + Assertions.assertThat(correlationID) + .describedAs("Invalid ID should be replaced with empty string") + .isEqualTo(EMPTY_STRING); + } + TracingContext tracingContext = new TracingContext(clientCorrelationId, + fs.getFileSystemId(), FSOperationType.TEST_OP, + TracingHeaderFormat.ALL_ID_FORMAT, null); + boolean isNamespaceEnabled = fs.getIsNamespaceEnabled(tracingContext); + String path = getRelativePath(new Path("/testDir")); + String permission = isNamespaceEnabled + ? getOctalNotation(FsPermission.getDirDefault()) + : null; + String umask = isNamespaceEnabled + ? getOctalNotation(FsPermission.getUMask(fs.getConf())) + : null; + + //request should not fail for invalid clientCorrelationID + AbfsRestOperation op = fs.getAbfsClient() + .createPath(path, false, true, permission, umask, false, null, + tracingContext); + + int statusCode = op.getResult().getStatusCode(); + Assertions.assertThat(statusCode).describedAs("Request should not fail") + .isEqualTo(HTTP_CREATED); + + String requestHeader = op.getResult().getClientRequestId().replace("[", "") + .replace("]", ""); + Assertions.assertThat(requestHeader) + .describedAs("Client Request Header should match TracingContext") + .isEqualTo(op.getLastTracingContext().getHeader()); + } - TracingContext tracingContext = new TracingContext(clientCorrelationId, - fs.getFileSystemId(), FSOperationType.TEST_OP, - TracingHeaderFormat.ALL_ID_FORMAT, null); - boolean isNamespaceEnabled = fs.getIsNamespaceEnabled(tracingContext); - String path = getRelativePath(new Path("/testDir")); - String permission = isNamespaceEnabled - ? getOctalNotation(FsPermission.getDirDefault()) - : null; - String umask = isNamespaceEnabled - ? getOctalNotation(FsPermission.getUMask(fs.getConf())) - : null; - - //request should not fail for invalid clientCorrelationID - AbfsRestOperation op = fs.getAbfsClient() - .createPath(path, false, true, permission, umask, false, null, - tracingContext); - - int statusCode = op.getResult().getStatusCode(); - Assertions.assertThat(statusCode).describedAs("Request should not fail") - .isEqualTo(HTTP_CREATED); - - String requestHeader = op.getResult().getClientRequestId().replace("[", "") - .replace("]", ""); - Assertions.assertThat(requestHeader) - .describedAs("Client Request Header should match TracingContext") - .isEqualTo(tracingContext.getHeader()); } @Ignore diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/ClassicDelegationTokenManager.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/ClassicDelegationTokenManager.java index 1f0cbc0a1672b..5f131db3d9895 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/ClassicDelegationTokenManager.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/ClassicDelegationTokenManager.java @@ -20,7 +20,7 @@ import java.io.IOException; import java.net.URI; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import org.apache.hadoop.util.Preconditions; import org.slf4j.Logger; @@ -249,8 +249,8 @@ public static Configuration useClassicDTManager(Configuration conf) { * highlighting security risks of shared mutable byte arrays. * @return a password. */ - private static byte[] getSecretManagerPasssword() { - return "non-password".getBytes(Charset.forName("UTF-8")); + private static byte[] getSecretManagerPassword() { + return "non-password".getBytes(StandardCharsets.UTF_8); } /** @@ -265,13 +265,13 @@ public TokenSecretManager() { @Override protected byte[] createPassword(StubAbfsTokenIdentifier identifier) { - return getSecretManagerPasssword(); + return getSecretManagerPassword(); } @Override public byte[] retrievePassword(StubAbfsTokenIdentifier identifier) throws InvalidToken { - return getSecretManagerPasssword(); + return getSecretManagerPassword(); } @Override diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientTestUtil.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientTestUtil.java new file mode 100644 index 0000000000000..875682fe20320 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientTestUtil.java @@ -0,0 +1,162 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.azurebfs.services; + +import java.io.IOException; +import java.net.HttpURLConnection; +import java.net.URL; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.Set; +import java.util.concurrent.locks.ReentrantLock; + +import org.assertj.core.api.Assertions; +import org.mockito.Mockito; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; +import org.apache.hadoop.util.functional.FunctionRaisingIOE; + +import static java.net.HttpURLConnection.HTTP_OK; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_GET; +import static org.apache.hadoop.fs.azurebfs.services.AuthType.OAuth; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.ArgumentMatchers.nullable; + +/** + * Utility class to help defining mock behavior on AbfsClient and AbfsRestOperation + * objects which are protected inside services package. + */ +public final class AbfsClientTestUtil { + + private AbfsClientTestUtil() { + + } + + public static void setMockAbfsRestOperationForListPathOperation( + final AbfsClient spiedClient, + FunctionRaisingIOE functionRaisingIOE) + throws Exception { + ExponentialRetryPolicy retryPolicy = Mockito.mock(ExponentialRetryPolicy.class); + AbfsHttpOperation httpOperation = Mockito.mock(AbfsHttpOperation.class); + AbfsRestOperation abfsRestOperation = Mockito.spy(new AbfsRestOperation( + AbfsRestOperationType.ListPaths, + spiedClient, + HTTP_METHOD_GET, + null, + new ArrayList<>() + )); + + Mockito.doReturn(abfsRestOperation).when(spiedClient).getAbfsRestOperation( + eq(AbfsRestOperationType.ListPaths), any(), any(), any()); + + addGeneralMockBehaviourToAbfsClient(spiedClient, retryPolicy); + addGeneralMockBehaviourToRestOpAndHttpOp(abfsRestOperation, httpOperation); + + functionRaisingIOE.apply(httpOperation); + } + + /** + * Adding general mock behaviour to AbfsRestOperation and AbfsHttpOperation + * to avoid any NPE occurring. These will avoid any network call made and + * will return the relevant exception or return value directly. + * @param abfsRestOperation to be mocked + * @param httpOperation to be mocked + * @throws IOException + */ + public static void addGeneralMockBehaviourToRestOpAndHttpOp(final AbfsRestOperation abfsRestOperation, + final AbfsHttpOperation httpOperation) throws IOException { + HttpURLConnection httpURLConnection = Mockito.mock(HttpURLConnection.class); + Mockito.doNothing().when(httpURLConnection) + .setRequestProperty(nullable(String.class), nullable(String.class)); + Mockito.doReturn(httpURLConnection).when(httpOperation).getConnection(); + Mockito.doReturn("").when(abfsRestOperation).getClientLatency(); + Mockito.doReturn(httpOperation).when(abfsRestOperation).createHttpOperation(); + } + + /** + * Adding general mock behaviour to AbfsClient to avoid any NPE occurring. + * These will avoid any network call made and will return the relevant exception or return value directly. + * @param abfsClient to be mocked + * @param retryPolicy to be mocked + * @throws IOException + */ + public static void addGeneralMockBehaviourToAbfsClient(final AbfsClient abfsClient, + final ExponentialRetryPolicy retryPolicy) throws IOException { + Mockito.doReturn(OAuth).when(abfsClient).getAuthType(); + Mockito.doReturn("").when(abfsClient).getAccessToken(); + AbfsThrottlingIntercept intercept = Mockito.mock( + AbfsThrottlingIntercept.class); + Mockito.doReturn(intercept).when(abfsClient).getIntercept(); + Mockito.doNothing() + .when(intercept) + .sendingRequest(any(), nullable(AbfsCounters.class)); + Mockito.doNothing().when(intercept).updateMetrics(any(), any()); + + Mockito.doReturn(retryPolicy).when(abfsClient).getRetryPolicy(); + Mockito.doReturn(true) + .when(retryPolicy) + .shouldRetry(nullable(Integer.class), nullable(Integer.class)); + Mockito.doReturn(false).when(retryPolicy).shouldRetry(0, HTTP_OK); + Mockito.doReturn(false).when(retryPolicy).shouldRetry(1, HTTP_OK); + Mockito.doReturn(false).when(retryPolicy).shouldRetry(2, HTTP_OK); + } + + public static void hookOnRestOpsForTracingContextSingularity(AbfsClient client) { + Set tracingContextSet = new HashSet<>(); + ReentrantLock lock = new ReentrantLock(); + Answer answer = new Answer() { + @Override + public Object answer(final InvocationOnMock invocationOnMock) + throws Throwable { + AbfsRestOperation op = Mockito.spy((AbfsRestOperation) invocationOnMock.callRealMethod()); + Mockito.doAnswer(completeExecuteInvocation -> { + lock.lock(); + try { + TracingContext context = completeExecuteInvocation.getArgument(0); + Assertions.assertThat(tracingContextSet).doesNotContain(context); + tracingContextSet.add(context); + } finally { + lock.unlock(); + } + return completeExecuteInvocation.callRealMethod(); + }).when(op).completeExecute(Mockito.any(TracingContext.class)); + return op; + } + }; + + Mockito.doAnswer(answer) + .when(client) + .getAbfsRestOperation(Mockito.any(AbfsRestOperationType.class), + Mockito.anyString(), Mockito.any(URL.class), Mockito.anyList(), + Mockito.nullable(byte[].class), Mockito.anyInt(), Mockito.anyInt(), + Mockito.nullable(String.class)); + Mockito.doAnswer(answer) + .when(client) + .getAbfsRestOperation(Mockito.any(AbfsRestOperationType.class), + Mockito.anyString(), Mockito.any(URL.class), Mockito.anyList()); + Mockito.doAnswer(answer) + .when(client) + .getAbfsRestOperation(Mockito.any(AbfsRestOperationType.class), + Mockito.anyString(), Mockito.any(URL.class), Mockito.anyList(), + Mockito.nullable(String.class)); + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java index 18d1e3917f24e..6707c593f5a76 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java @@ -58,6 +58,7 @@ import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT_NAME; import static org.apache.hadoop.test.LambdaTestUtils.intercept; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -592,7 +593,7 @@ public void testExpectHundredContinue() throws Exception { // Mock the restOperation for the client. Mockito.doReturn(op) .when(testClient) - .getAbfsRestOperationForAppend(Mockito.any(), + .getAbfsRestOperation(eq(AbfsRestOperationType.Append), Mockito.any(), Mockito.any(), Mockito.any(), Mockito.any(), Mockito.nullable(int.class), Mockito.nullable(int.class), Mockito.any()); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsRestOperation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsRestOperation.java index 6ffe2e2773bbf..6574a808f92bd 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsRestOperation.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsRestOperation.java @@ -290,6 +290,7 @@ public void testExpectHundredContinue() throws Exception { TracingContext tracingContext = Mockito.spy(new TracingContext("abcd", "abcde", FSOperationType.APPEND, TracingHeaderFormat.ALL_ID_FORMAT, null)); + Mockito.doReturn(tracingContext).when(op).createNewTracingContext(Mockito.any()); switch (errorType) { case WRITE: diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperationMockFailures.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperationMockFailures.java index bfa524a25e600..b302a1fa939e7 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperationMockFailures.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperationMockFailures.java @@ -18,9 +18,7 @@ package org.apache.hadoop.fs.azurebfs.services; -import java.io.IOException; import java.io.InterruptedIOException; -import java.net.HttpURLConnection; import java.net.SocketException; import java.net.SocketTimeoutException; import java.net.UnknownHostException; @@ -39,7 +37,8 @@ import static java.net.HttpURLConnection.HTTP_UNAVAILABLE; import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.EGRESS_OVER_ACCOUNT_LIMIT; import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.INGRESS_OVER_ACCOUNT_LIMIT; -import static org.apache.hadoop.fs.azurebfs.services.AuthType.OAuth; +import static org.apache.hadoop.fs.azurebfs.services.AbfsClientTestUtil.addGeneralMockBehaviourToAbfsClient; +import static org.apache.hadoop.fs.azurebfs.services.AbfsClientTestUtil.addGeneralMockBehaviourToRestOpAndHttpOp; import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.CONNECTION_RESET_ABBREVIATION; import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.CONNECTION_RESET_MESSAGE; import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.CONNECTION_TIMEOUT_ABBREVIATION; @@ -166,7 +165,7 @@ private void testClientRequestIdForStatusRetry(int status, AbfsClient abfsClient = Mockito.mock(AbfsClient.class); ExponentialRetryPolicy retryPolicy = Mockito.mock( ExponentialRetryPolicy.class); - addMockBehaviourToAbfsClient(abfsClient, retryPolicy); + addGeneralMockBehaviourToAbfsClient(abfsClient, retryPolicy); AbfsRestOperation abfsRestOperation = Mockito.spy(new AbfsRestOperation( @@ -178,7 +177,7 @@ private void testClientRequestIdForStatusRetry(int status, )); AbfsHttpOperation httpOperation = Mockito.mock(AbfsHttpOperation.class); - addMockBehaviourToRestOpAndHttpOp(abfsRestOperation, httpOperation); + addGeneralMockBehaviourToRestOpAndHttpOp(abfsRestOperation, httpOperation); Mockito.doNothing() .doNothing() @@ -202,6 +201,8 @@ private void testClientRequestIdForStatusRetry(int status, TracingContext tracingContext = Mockito.mock(TracingContext.class); Mockito.doNothing().when(tracingContext).setRetryCount(nullable(int.class)); + Mockito.doReturn(tracingContext) + .when(abfsRestOperation).createNewTracingContext(any()); int[] count = new int[1]; count[0] = 0; @@ -225,7 +226,7 @@ private void testClientRequestIdForTimeoutRetry(Exception[] exceptions, AbfsClient abfsClient = Mockito.mock(AbfsClient.class); ExponentialRetryPolicy retryPolicy = Mockito.mock( ExponentialRetryPolicy.class); - addMockBehaviourToAbfsClient(abfsClient, retryPolicy); + addGeneralMockBehaviourToAbfsClient(abfsClient, retryPolicy); AbfsRestOperation abfsRestOperation = Mockito.spy(new AbfsRestOperation( @@ -237,7 +238,7 @@ private void testClientRequestIdForTimeoutRetry(Exception[] exceptions, )); AbfsHttpOperation httpOperation = Mockito.mock(AbfsHttpOperation.class); - addMockBehaviourToRestOpAndHttpOp(abfsRestOperation, httpOperation); + addGeneralMockBehaviourToRestOpAndHttpOp(abfsRestOperation, httpOperation); Stubber stubber = Mockito.doThrow(exceptions[0]); for (int iteration = 1; iteration < len; iteration++) { @@ -253,6 +254,7 @@ private void testClientRequestIdForTimeoutRetry(Exception[] exceptions, TracingContext tracingContext = Mockito.mock(TracingContext.class); Mockito.doNothing().when(tracingContext).setRetryCount(nullable(int.class)); + Mockito.doReturn(tracingContext).when(abfsRestOperation).createNewTracingContext(any()); int[] count = new int[1]; count[0] = 0; @@ -268,35 +270,4 @@ private void testClientRequestIdForTimeoutRetry(Exception[] exceptions, abfsRestOperation.execute(tracingContext); Assertions.assertThat(count[0]).isEqualTo(len + 1); } - - private void addMockBehaviourToRestOpAndHttpOp(final AbfsRestOperation abfsRestOperation, - final AbfsHttpOperation httpOperation) throws IOException { - HttpURLConnection httpURLConnection = Mockito.mock(HttpURLConnection.class); - Mockito.doNothing() - .when(httpURLConnection) - .setRequestProperty(nullable(String.class), nullable(String.class)); - Mockito.doReturn(httpURLConnection).when(httpOperation).getConnection(); - Mockito.doReturn("").when(abfsRestOperation).getClientLatency(); - Mockito.doReturn(httpOperation).when(abfsRestOperation).createHttpOperation(); - } - - private void addMockBehaviourToAbfsClient(final AbfsClient abfsClient, - final ExponentialRetryPolicy retryPolicy) throws IOException { - Mockito.doReturn(OAuth).when(abfsClient).getAuthType(); - Mockito.doReturn("").when(abfsClient).getAccessToken(); - AbfsThrottlingIntercept intercept = Mockito.mock( - AbfsThrottlingIntercept.class); - Mockito.doReturn(intercept).when(abfsClient).getIntercept(); - Mockito.doNothing() - .when(intercept) - .sendingRequest(any(), nullable(AbfsCounters.class)); - Mockito.doNothing().when(intercept).updateMetrics(any(), any()); - - Mockito.doReturn(retryPolicy).when(abfsClient).getRetryPolicy(); - Mockito.doReturn(true) - .when(retryPolicy) - .shouldRetry(nullable(Integer.class), nullable(Integer.class)); - Mockito.doReturn(false).when(retryPolicy).shouldRetry(1, HTTP_OK); - Mockito.doReturn(false).when(retryPolicy).shouldRetry(2, HTTP_OK); - } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestShellDecryptionKeyProvider.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestShellDecryptionKeyProvider.java index 1f027411720e9..f039b60156508 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestShellDecryptionKeyProvider.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestShellDecryptionKeyProvider.java @@ -19,7 +19,7 @@ package org.apache.hadoop.fs.azurebfs.services; import java.io.File; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import org.junit.Assert; import org.junit.Test; @@ -75,7 +75,7 @@ public void testValidScript() throws Exception { // expected result (so that we validate both script input and output) File scriptFile = new File(TEST_ROOT_DIR, "testScript.cmd"); FileUtils.writeStringToFile(scriptFile, "@echo %1 " + expectedResult, - Charset.forName("UTF-8")); + StandardCharsets.UTF_8); ShellDecryptionKeyProvider provider = new ShellDecryptionKeyProvider(); Configuration conf = new Configuration(); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestTextFileBasedIdentityHandler.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestTextFileBasedIdentityHandler.java index b0a72b2131cdd..1e578670cb33f 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestTextFileBasedIdentityHandler.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestTextFileBasedIdentityHandler.java @@ -20,7 +20,7 @@ import java.io.File; import java.io.IOException; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.nio.file.NoSuchFileException; import org.junit.Assert; @@ -71,22 +71,22 @@ public static void init() throws IOException { groupMappingFile = tempDir.newFile("group-mapping.conf"); //Stage data for user mapping - FileUtils.writeStringToFile(userMappingFile, testUserDataLine1, Charset.forName("UTF-8"), true); - FileUtils.writeStringToFile(userMappingFile, testUserDataLine2, Charset.forName("UTF-8"), true); - FileUtils.writeStringToFile(userMappingFile, testUserDataLine3, Charset.forName("UTF-8"), true); - FileUtils.writeStringToFile(userMappingFile, testUserDataLine4, Charset.forName("UTF-8"), true); - FileUtils.writeStringToFile(userMappingFile, testUserDataLine5, Charset.forName("UTF-8"), true); - FileUtils.writeStringToFile(userMappingFile, testUserDataLine6, Charset.forName("UTF-8"), true); - FileUtils.writeStringToFile(userMappingFile, testUserDataLine7, Charset.forName("UTF-8"), true); - FileUtils.writeStringToFile(userMappingFile, NEW_LINE, Charset.forName("UTF-8"), true); + FileUtils.writeStringToFile(userMappingFile, testUserDataLine1, StandardCharsets.UTF_8, true); + FileUtils.writeStringToFile(userMappingFile, testUserDataLine2, StandardCharsets.UTF_8, true); + FileUtils.writeStringToFile(userMappingFile, testUserDataLine3, StandardCharsets.UTF_8, true); + FileUtils.writeStringToFile(userMappingFile, testUserDataLine4, StandardCharsets.UTF_8, true); + FileUtils.writeStringToFile(userMappingFile, testUserDataLine5, StandardCharsets.UTF_8, true); + FileUtils.writeStringToFile(userMappingFile, testUserDataLine6, StandardCharsets.UTF_8, true); + FileUtils.writeStringToFile(userMappingFile, testUserDataLine7, StandardCharsets.UTF_8, true); + FileUtils.writeStringToFile(userMappingFile, NEW_LINE, StandardCharsets.UTF_8, true); //Stage data for group mapping - FileUtils.writeStringToFile(groupMappingFile, testGroupDataLine1, Charset.forName("UTF-8"), true); - FileUtils.writeStringToFile(groupMappingFile, testGroupDataLine2, Charset.forName("UTF-8"), true); - FileUtils.writeStringToFile(groupMappingFile, testGroupDataLine3, Charset.forName("UTF-8"), true); - FileUtils.writeStringToFile(groupMappingFile, testGroupDataLine4, Charset.forName("UTF-8"), true); - FileUtils.writeStringToFile(groupMappingFile, testGroupDataLine5, Charset.forName("UTF-8"), true); - FileUtils.writeStringToFile(groupMappingFile, NEW_LINE, Charset.forName("UTF-8"), true); + FileUtils.writeStringToFile(groupMappingFile, testGroupDataLine1, StandardCharsets.UTF_8, true); + FileUtils.writeStringToFile(groupMappingFile, testGroupDataLine2, StandardCharsets.UTF_8, true); + FileUtils.writeStringToFile(groupMappingFile, testGroupDataLine3, StandardCharsets.UTF_8, true); + FileUtils.writeStringToFile(groupMappingFile, testGroupDataLine4, StandardCharsets.UTF_8, true); + FileUtils.writeStringToFile(groupMappingFile, testGroupDataLine5, StandardCharsets.UTF_8, true); + FileUtils.writeStringToFile(groupMappingFile, NEW_LINE, StandardCharsets.UTF_8, true); } private void assertUserLookup(TextFileBasedIdentityHandler handler, String userInTest, String expectedUser) diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/FileBasedCopyListing.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/FileBasedCopyListing.java index c356edd4251c0..d1dd7617e8020 100644 --- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/FileBasedCopyListing.java +++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/FileBasedCopyListing.java @@ -27,7 +27,7 @@ import java.io.BufferedReader; import java.io.IOException; import java.io.InputStreamReader; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.List; @@ -76,7 +76,7 @@ private List fetchFileList(Path sourceListing) throws IOException { BufferedReader input = null; try { input = new BufferedReader(new InputStreamReader(fs.open(sourceListing), - Charset.forName("UTF-8"))); + StandardCharsets.UTF_8)); String line = input.readLine(); while (line != null) { result.add(new Path(line)); diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/RegexCopyFilter.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/RegexCopyFilter.java index f5f23eea05ea7..01787860fc15b 100644 --- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/RegexCopyFilter.java +++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/RegexCopyFilter.java @@ -29,7 +29,7 @@ import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.util.ArrayList; import java.util.List; @@ -66,7 +66,7 @@ public void initialize() { try { InputStream is = Files.newInputStream(filtersFile.toPath()); reader = new BufferedReader(new InputStreamReader(is, - Charset.forName("UTF-8"))); + StandardCharsets.UTF_8)); String line; while ((line = reader.readLine()) != null) { Pattern pattern = Pattern.compile(line); diff --git a/hadoop-tools/hadoop-extras/src/main/java/org/apache/hadoop/tools/DistTool.java b/hadoop-tools/hadoop-extras/src/main/java/org/apache/hadoop/tools/DistTool.java index fb56b90186c6b..7e5b715479019 100644 --- a/hadoop-tools/hadoop-extras/src/main/java/org/apache/hadoop/tools/DistTool.java +++ b/hadoop-tools/hadoop-extras/src/main/java/org/apache/hadoop/tools/DistTool.java @@ -22,7 +22,7 @@ import java.io.DataOutput; import java.io.IOException; import java.io.InputStreamReader; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.List; import java.util.Random; @@ -96,7 +96,7 @@ protected static List readFile(Configuration conf, Path inputfile List result = new ArrayList(); FileSystem fs = inputfile.getFileSystem(conf); try (BufferedReader input = new BufferedReader(new InputStreamReader(fs.open(inputfile), - Charset.forName("UTF-8")))) { + StandardCharsets.UTF_8))) { for(String line; (line = input.readLine()) != null;) { result.add(line); } diff --git a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageWriter.java b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageWriter.java index 9c8dc1f2304c1..2900139093294 100644 --- a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageWriter.java +++ b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageWriter.java @@ -26,6 +26,7 @@ import java.io.OutputStream; import java.io.IOException; import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; import java.security.DigestOutputStream; import java.security.MessageDigest; import java.util.Arrays; @@ -35,7 +36,6 @@ import java.util.Map.Entry; import java.util.concurrent.atomic.AtomicLong; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; import org.apache.hadoop.thirdparty.protobuf.CodedOutputStream; import org.apache.hadoop.classification.InterfaceAudience; @@ -325,7 +325,7 @@ void writeMD5(String imagename) throws IOException { Path chk = new Path(outdir, imagename + ".md5"); try (OutputStream out = outfs.create(chk)) { String md5Line = digestString + " *" + imagename + "\n"; - out.write(md5Line.getBytes(Charsets.UTF_8)); + out.write(md5Line.getBytes(StandardCharsets.UTF_8)); } } diff --git a/hadoop-tools/hadoop-gridmix/src/main/java/org/apache/hadoop/mapred/gridmix/CompressionEmulationUtil.java b/hadoop-tools/hadoop-gridmix/src/main/java/org/apache/hadoop/mapred/gridmix/CompressionEmulationUtil.java index 71db9bfb25c3a..99c621a3e92eb 100644 --- a/hadoop-tools/hadoop-gridmix/src/main/java/org/apache/hadoop/mapred/gridmix/CompressionEmulationUtil.java +++ b/hadoop-tools/hadoop-gridmix/src/main/java/org/apache/hadoop/mapred/gridmix/CompressionEmulationUtil.java @@ -22,6 +22,7 @@ import java.io.InputStream; import java.io.OutputStream; import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.util.HashMap; import java.util.Map; @@ -99,7 +100,7 @@ class CompressionEmulationUtil { private static final CompressionRatioLookupTable COMPRESSION_LOOKUP_TABLE = new CompressionRatioLookupTable(); - private static final Charset charsetUTF8 = Charset.forName("UTF-8"); + private static final Charset charsetUTF8 = StandardCharsets.UTF_8; /** * This is a {@link Mapper} implementation for generating random text data. diff --git a/hadoop-tools/hadoop-gridmix/src/main/java/org/apache/hadoop/mapred/gridmix/DistributedCacheEmulator.java b/hadoop-tools/hadoop-gridmix/src/main/java/org/apache/hadoop/mapred/gridmix/DistributedCacheEmulator.java index 56f67e5a73cb0..a6f986ce2602c 100644 --- a/hadoop-tools/hadoop-gridmix/src/main/java/org/apache/hadoop/mapred/gridmix/DistributedCacheEmulator.java +++ b/hadoop-tools/hadoop-gridmix/src/main/java/org/apache/hadoop/mapred/gridmix/DistributedCacheEmulator.java @@ -42,6 +42,7 @@ import java.net.URI; import java.net.URISyntaxException; import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; @@ -113,7 +114,7 @@ class DistributedCacheEmulator { Configuration conf; // gridmix configuration - private static final Charset charsetUTF8 = Charset.forName("UTF-8"); + private static final Charset charsetUTF8 = StandardCharsets.UTF_8; // Pseudo local file system where local FS based distributed cache files are // created by gridmix. diff --git a/hadoop-tools/hadoop-gridmix/src/main/java/org/apache/hadoop/mapred/gridmix/GenerateDistCacheData.java b/hadoop-tools/hadoop-gridmix/src/main/java/org/apache/hadoop/mapred/gridmix/GenerateDistCacheData.java index 4a75cdedf7388..aa191629cf109 100644 --- a/hadoop-tools/hadoop-gridmix/src/main/java/org/apache/hadoop/mapred/gridmix/GenerateDistCacheData.java +++ b/hadoop-tools/hadoop-gridmix/src/main/java/org/apache/hadoop/mapred/gridmix/GenerateDistCacheData.java @@ -19,6 +19,7 @@ import java.io.IOException; import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.security.PrivilegedExceptionAction; import java.util.ArrayList; import java.util.List; @@ -96,7 +97,7 @@ class GenerateDistCacheData extends GridmixJob { */ static final short GRIDMIX_DISTCACHE_FILE_PERM = 0644; - private static final Charset charsetUTF8 = Charset.forName("UTF-8"); + private static final Charset charsetUTF8 = StandardCharsets.UTF_8; public GenerateDistCacheData(Configuration conf) throws IOException { super(conf, 0L, JOB_NAME); diff --git a/hadoop-tools/hadoop-gridmix/src/main/java/org/apache/hadoop/mapred/gridmix/GridmixRecord.java b/hadoop-tools/hadoop-gridmix/src/main/java/org/apache/hadoop/mapred/gridmix/GridmixRecord.java index 481799f7b5166..afb95cab87c24 100644 --- a/hadoop-tools/hadoop-gridmix/src/main/java/org/apache/hadoop/mapred/gridmix/GridmixRecord.java +++ b/hadoop-tools/hadoop-gridmix/src/main/java/org/apache/hadoop/mapred/gridmix/GridmixRecord.java @@ -21,6 +21,7 @@ import java.io.DataOutput; import java.io.EOFException; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.Arrays; import org.apache.hadoop.io.DataInputBuffer; @@ -111,7 +112,7 @@ private void writeRandomText(DataOutput out, final int size) //TODO Should we use long for size. What if the data is more than 4G? String randomWord = rtg.getRandomWord(); - byte[] bytes = randomWord.getBytes("UTF-8"); + byte[] bytes = randomWord.getBytes(StandardCharsets.UTF_8); long randomWordSize = bytes.length; while (i >= randomWordSize) { out.write(bytes); @@ -119,7 +120,7 @@ private void writeRandomText(DataOutput out, final int size) // get the next random word randomWord = rtg.getRandomWord(); - bytes = randomWord.getBytes("UTF-8"); + bytes = randomWord.getBytes(StandardCharsets.UTF_8); // determine the random word size randomWordSize = bytes.length; } diff --git a/hadoop-tools/hadoop-kafka/src/main/java/org/apache/hadoop/metrics2/sink/KafkaSink.java b/hadoop-tools/hadoop-kafka/src/main/java/org/apache/hadoop/metrics2/sink/KafkaSink.java index 0856d0f4e0eeb..9cb6b93c4e0d2 100644 --- a/hadoop-tools/hadoop-kafka/src/main/java/org/apache/hadoop/metrics2/sink/KafkaSink.java +++ b/hadoop-tools/hadoop-kafka/src/main/java/org/apache/hadoop/metrics2/sink/KafkaSink.java @@ -37,7 +37,7 @@ import java.io.Closeable; import java.io.IOException; import java.net.InetAddress; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.time.Instant; import java.time.LocalDateTime; import java.time.ZoneId; @@ -156,7 +156,7 @@ public void putMetrics(MetricsRecord record) { // Create the record to be sent from the json. ProducerRecord data = new ProducerRecord( - topic, jsonLines.toString().getBytes(Charset.forName("UTF-8"))); + topic, jsonLines.toString().getBytes(StandardCharsets.UTF_8)); // Send the data to the Kafka broker. Here is an example of this data: // {"hostname": "...", "timestamp": 1436913651516, diff --git a/hadoop-tools/hadoop-kafka/src/test/java/org/apache/hadoop/metrics2/impl/TestKafkaMetrics.java b/hadoop-tools/hadoop-kafka/src/test/java/org/apache/hadoop/metrics2/impl/TestKafkaMetrics.java index 03c479fba59c5..665dbddd9a5db 100644 --- a/hadoop-tools/hadoop-kafka/src/test/java/org/apache/hadoop/metrics2/impl/TestKafkaMetrics.java +++ b/hadoop-tools/hadoop-kafka/src/test/java/org/apache/hadoop/metrics2/impl/TestKafkaMetrics.java @@ -159,7 +159,7 @@ StringBuilder recordToJson(MetricsRecord record) { String date = dateFormat.format(currDate); SimpleDateFormat timeFormat = new SimpleDateFormat("HH:mm:ss"); String time = timeFormat.format(currDate); - String hostname = new String("null"); + String hostname = "null"; try { hostname = InetAddress.getLocalHost().getHostName(); } catch (Exception e) { diff --git a/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/solver/impl/TestLpSolver.java b/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/solver/impl/TestLpSolver.java index d32f7c3592b26..1bce63466a5d6 100644 --- a/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/solver/impl/TestLpSolver.java +++ b/hadoop-tools/hadoop-resourceestimator/src/test/java/org/apache/hadoop/resourceestimator/solver/impl/TestLpSolver.java @@ -41,7 +41,7 @@ import java.io.InputStream; import java.io.InputStreamReader; import java.io.Reader; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.text.ParseException; import java.util.List; import java.util.Map; @@ -84,7 +84,7 @@ private void parseLog(final String inputLog) RLESparseResourceAllocation result = solver.solve(jobHistory); String file = "src/test/resources/lp/answer.txt"; Reader fileReader = new InputStreamReader(new FileInputStream(file), - Charset.forName("UTF-8")); + StandardCharsets.UTF_8); BufferedReader bufferedReader = new BufferedReader(fileReader); String line = bufferedReader.readLine(); Configuration config = new Configuration(); diff --git a/hadoop-tools/hadoop-rumen/src/main/java/org/apache/hadoop/tools/rumen/RandomSeedGenerator.java b/hadoop-tools/hadoop-rumen/src/main/java/org/apache/hadoop/tools/rumen/RandomSeedGenerator.java index ecd5f0bbfc1fd..817c5c8b2f704 100644 --- a/hadoop-tools/hadoop-rumen/src/main/java/org/apache/hadoop/tools/rumen/RandomSeedGenerator.java +++ b/hadoop-tools/hadoop-rumen/src/main/java/org/apache/hadoop/tools/rumen/RandomSeedGenerator.java @@ -17,7 +17,7 @@ */ package org.apache.hadoop.tools.rumen; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.security.MessageDigest; import java.security.NoSuchAlgorithmException; @@ -43,8 +43,7 @@ */ public class RandomSeedGenerator { private static Logger LOG = LoggerFactory.getLogger(RandomSeedGenerator.class); - private static final Charset UTF_8 = Charset.forName("UTF-8"); - + /** MD5 algorithm instance, one for each thread. */ private static final ThreadLocal md5Holder = new ThreadLocal() { @@ -74,7 +73,7 @@ public static long getSeed(String streamId, long masterSeed) { // We could have fed the bytes of masterSeed one by one to md5.update() // instead String str = streamId + '/' + masterSeed; - byte[] digest = md5.digest(str.getBytes(UTF_8)); + byte[] digest = md5.digest(str.getBytes(StandardCharsets.UTF_8)); // Create a long from the first 8 bytes of the digest // This is fine as MD5 has the avalanche property. // Paranoids could have XOR folded the other 8 bytes in too. diff --git a/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/record/Record.java b/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/record/Record.java index f0ec99ad8143c..84df8b8187e50 100644 --- a/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/record/Record.java +++ b/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/record/Record.java @@ -22,6 +22,7 @@ import java.io.DataOutput; import java.io.ByteArrayOutputStream; import java.io.IOException; +import java.nio.charset.StandardCharsets; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -94,8 +95,8 @@ public String toString() { ByteArrayOutputStream s = new ByteArrayOutputStream(); CsvRecordOutput a = new CsvRecordOutput(s); this.serialize(a); - return new String(s.toByteArray(), "UTF-8"); - } catch (Throwable ex) { + return new String(s.toByteArray(), StandardCharsets.UTF_8); + } catch (Exception ex) { throw new RuntimeException(ex); } } diff --git a/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/Environment.java b/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/Environment.java index bc92b7149ae20..7871a4c969e7d 100644 --- a/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/Environment.java +++ b/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/Environment.java @@ -20,7 +20,7 @@ import java.io.*; import java.net.InetAddress; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.util.*; import org.apache.hadoop.classification.InterfaceAudience; @@ -65,7 +65,7 @@ public Environment() throws IOException { Process pid = Runtime.getRuntime().exec(command); BufferedReader in = new BufferedReader( - new InputStreamReader(pid.getInputStream(), Charset.forName("UTF-8"))); + new InputStreamReader(pid.getInputStream(), StandardCharsets.UTF_8)); try { while (true) { String line = in.readLine(); diff --git a/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/PipeMapper.java b/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/PipeMapper.java index 9bab1013f2cc7..438a00057ec14 100644 --- a/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/PipeMapper.java +++ b/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/PipeMapper.java @@ -20,6 +20,7 @@ import java.io.*; import java.net.URLDecoder; +import java.nio.charset.StandardCharsets; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.Mapper; @@ -75,13 +76,11 @@ public void configure(JobConf job) { inputFormatClassName.equals(TextInputFormat.class.getCanonicalName())); } - try { - mapOutputFieldSeparator = job.get("stream.map.output.field.separator", "\t").getBytes("UTF-8"); - mapInputFieldSeparator = job.get("stream.map.input.field.separator", "\t").getBytes("UTF-8"); - numOfMapOutputKeyFields = job.getInt("stream.num.map.output.key.fields", 1); - } catch (UnsupportedEncodingException e) { - throw new RuntimeException("The current system does not support UTF-8 encoding!", e); - } + mapOutputFieldSeparator = job.get("stream.map.output.field.separator", "\t") + .getBytes(StandardCharsets.UTF_8); + mapInputFieldSeparator = job.get("stream.map.input.field.separator", "\t") + .getBytes(StandardCharsets.UTF_8); + numOfMapOutputKeyFields = job.getInt("stream.num.map.output.key.fields", 1); } // Do NOT declare default constructor diff --git a/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/PipeReducer.java b/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/PipeReducer.java index ffa7b014131da..1f5a247bb2aa1 100644 --- a/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/PipeReducer.java +++ b/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/PipeReducer.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.io.UnsupportedEncodingException; +import java.nio.charset.StandardCharsets; import java.util.Iterator; import java.net.URLDecoder; @@ -71,13 +72,11 @@ public void configure(JobConf job) { SkipBadRecords.setAutoIncrReducerProcCount(job, false); skipping = job.getBoolean(MRJobConfig.SKIP_RECORDS, false); - try { - reduceOutFieldSeparator = job_.get("stream.reduce.output.field.separator", "\t").getBytes("UTF-8"); - reduceInputFieldSeparator = job_.get("stream.reduce.input.field.separator", "\t").getBytes("UTF-8"); - this.numOfReduceOutputKeyFields = job_.getInt("stream.num.reduce.output.key.fields", 1); - } catch (UnsupportedEncodingException e) { - throw new RuntimeException("The current system does not support UTF-8 encoding!", e); - } + reduceOutFieldSeparator = job_.get("stream.reduce.output.field.separator", "\t") + .getBytes(StandardCharsets.UTF_8); + reduceInputFieldSeparator = job_.get("stream.reduce.input.field.separator", "\t") + .getBytes(StandardCharsets.UTF_8); + this.numOfReduceOutputKeyFields = job_.getInt("stream.num.reduce.output.key.fields", 1); } public void reduce(Object key, Iterator values, OutputCollector output, diff --git a/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/StreamBaseRecordReader.java b/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/StreamBaseRecordReader.java index 063ea51dac693..c757cf6d464f4 100644 --- a/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/StreamBaseRecordReader.java +++ b/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/StreamBaseRecordReader.java @@ -19,11 +19,9 @@ package org.apache.hadoop.streaming; import java.io.*; +import java.nio.charset.StandardCharsets; import org.apache.hadoop.io.Text; -import org.apache.hadoop.io.Writable; -import org.apache.hadoop.io.WritableComparable; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.mapred.Reporter; @@ -103,7 +101,8 @@ public Text createValue() { void numRecStats(byte[] record, int start, int len) throws IOException { numRec_++; if (numRec_ == nextStatusRec_) { - String recordStr = new String(record, start, Math.min(len, statusMaxRecordChars_), "UTF-8"); + String recordStr = new String(record, start, + Math.min(len, statusMaxRecordChars_), StandardCharsets.UTF_8); nextStatusRec_ += 100;//*= 10; String status = getStatus(recordStr); LOG.info(status); diff --git a/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/StreamUtil.java b/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/StreamUtil.java index 8dd987e870cda..a6983e1c6c306 100644 --- a/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/StreamUtil.java +++ b/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/StreamUtil.java @@ -23,6 +23,7 @@ import java.io.IOException; import java.net.InetAddress; import java.net.URL; +import java.nio.charset.StandardCharsets; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; @@ -146,7 +147,7 @@ static String slurp(File f) throws IOException { String contents = null; try { in.read(buf, 0, len); - contents = new String(buf, "UTF-8"); + contents = new String(buf, StandardCharsets.UTF_8); } finally { in.close(); } @@ -160,7 +161,7 @@ static String slurpHadoop(Path p, FileSystem fs) throws IOException { String contents = null; try { in.readFully(in.getPos(), buf); - contents = new String(buf, "UTF-8"); + contents = new String(buf, StandardCharsets.UTF_8); } finally { in.close(); } diff --git a/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/StreamXmlRecordReader.java b/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/StreamXmlRecordReader.java index 7438cb8191a4b..974cdc7c8d001 100644 --- a/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/StreamXmlRecordReader.java +++ b/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/StreamXmlRecordReader.java @@ -19,6 +19,7 @@ package org.apache.hadoop.streaming; import java.io.*; +import java.nio.charset.StandardCharsets; import java.util.regex.*; import org.apache.hadoop.io.DataOutputBuffer; @@ -132,7 +133,7 @@ private boolean slowReadUntilMatch(Pattern markPattern, boolean includePat, read = bin_.read(buf); if (read == -1) return false; - String sbuf = new String(buf, 0, read, "UTF-8"); + String sbuf = new String(buf, 0, read, StandardCharsets.UTF_8); Matcher match = markPattern.matcher(sbuf); firstMatchStart_ = NA; @@ -235,7 +236,7 @@ void addGroup(StringBuffer pat, String escapedGroup) { } boolean fastReadUntilMatch(String textPat, boolean includePat, DataOutputBuffer outBufOrNull) throws IOException { - byte[] cpat = textPat.getBytes("UTF-8"); + byte[] cpat = textPat.getBytes(StandardCharsets.UTF_8); int m = 0; boolean match = false; int msup = cpat.length; diff --git a/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/io/KeyOnlyTextOutputReader.java b/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/io/KeyOnlyTextOutputReader.java index 32bba397cea6b..1c17659b778af 100644 --- a/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/io/KeyOnlyTextOutputReader.java +++ b/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/io/KeyOnlyTextOutputReader.java @@ -21,7 +21,7 @@ import java.io.DataInput; import java.io.IOException; import java.io.InputStream; -import java.io.UnsupportedEncodingException; +import java.nio.charset.StandardCharsets; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.NullWritable; @@ -77,11 +77,7 @@ public NullWritable getCurrentValue() throws IOException { @Override public String getLastOutput() { if (bytes != null) { - try { - return new String(bytes, "UTF-8"); - } catch (UnsupportedEncodingException e) { - return ""; - } + return new String(bytes, StandardCharsets.UTF_8); } else { return null; } diff --git a/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/io/TextInputWriter.java b/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/io/TextInputWriter.java index 6f0fd8bfa5318..31513da71d9c9 100644 --- a/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/io/TextInputWriter.java +++ b/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/io/TextInputWriter.java @@ -20,6 +20,7 @@ import java.io.DataOutput; import java.io.IOException; +import java.nio.charset.StandardCharsets; import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.Text; @@ -66,7 +67,7 @@ protected void writeUTF8(Object object) throws IOException { valSize = val.getLength(); } else { String sval = object.toString(); - bval = sval.getBytes("UTF-8"); + bval = sval.getBytes(StandardCharsets.UTF_8); valSize = bval.length; } clientOut.write(bval, 0, valSize); diff --git a/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/io/TextOutputReader.java b/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/io/TextOutputReader.java index 06c05bc9ef795..11c84a471f7d7 100644 --- a/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/io/TextOutputReader.java +++ b/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/io/TextOutputReader.java @@ -21,8 +21,8 @@ import java.io.DataInput; import java.io.IOException; import java.io.InputStream; -import java.io.UnsupportedEncodingException; import java.nio.charset.CharacterCodingException; +import java.nio.charset.StandardCharsets; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.Text; @@ -84,11 +84,7 @@ public Text getCurrentValue() throws IOException { @Override public String getLastOutput() { if (bytes != null) { - try { - return new String(bytes, "UTF-8"); - } catch (UnsupportedEncodingException e) { - return ""; - } + return new String(bytes, StandardCharsets.UTF_8); } else { return null; } diff --git a/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/mapreduce/StreamBaseRecordReader.java b/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/mapreduce/StreamBaseRecordReader.java index 43c1b1bec0a71..e3c14743cb32b 100644 --- a/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/mapreduce/StreamBaseRecordReader.java +++ b/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/mapreduce/StreamBaseRecordReader.java @@ -19,6 +19,7 @@ package org.apache.hadoop.streaming.mapreduce; import java.io.IOException; +import java.nio.charset.StandardCharsets; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -107,7 +108,7 @@ void numRecStats(byte[] record, int start, int len) throws IOException { numRec_++; if (numRec_ == nextStatusRec_) { String recordStr = new String(record, start, Math.min(len, - statusMaxRecordChars_), "UTF-8"); + statusMaxRecordChars_), StandardCharsets.UTF_8); nextStatusRec_ += 100;// *= 10; String status = getStatus(recordStr); LOG.info(status); diff --git a/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/mapreduce/StreamXmlRecordReader.java b/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/mapreduce/StreamXmlRecordReader.java index c7ee847763f75..aa8a4d8832c39 100644 --- a/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/mapreduce/StreamXmlRecordReader.java +++ b/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/mapreduce/StreamXmlRecordReader.java @@ -20,6 +20,7 @@ import java.io.BufferedInputStream; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -139,7 +140,7 @@ private boolean slowReadUntilMatch(Pattern markPattern, boolean includePat, if (read == -1) return false; - String sbuf = new String(buf, 0, read, "UTF-8"); + String sbuf = new String(buf, 0, read, StandardCharsets.UTF_8); Matcher match = markPattern.matcher(sbuf); firstMatchStart_ = NA; @@ -246,7 +247,7 @@ void addGroup(StringBuffer pat, String escapedGroup) { boolean fastReadUntilMatch(String textPat, boolean includePat, DataOutputBuffer outBufOrNull) throws IOException { - byte[] cpat = textPat.getBytes("UTF-8"); + byte[] cpat = textPat.getBytes(StandardCharsets.UTF_8); int m = 0; boolean match = false; int msup = cpat.length; diff --git a/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/RawBytesMapApp.java b/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/RawBytesMapApp.java index 813c08c6111d3..e9c2740ee8877 100644 --- a/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/RawBytesMapApp.java +++ b/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/RawBytesMapApp.java @@ -22,6 +22,7 @@ import java.io.DataOutputStream; import java.io.IOException; import java.io.InputStreamReader; +import java.nio.charset.StandardCharsets; import org.apache.hadoop.io.IntWritable; @@ -52,7 +53,7 @@ public static void main(String[] args) throws IOException { } private void writeString(String str) throws IOException { - byte[] bytes = str.getBytes("UTF-8"); + byte[] bytes = str.getBytes(StandardCharsets.UTF_8); dos.writeInt(bytes.length); dos.write(bytes); } diff --git a/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/RawBytesReduceApp.java b/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/RawBytesReduceApp.java index 741e3d3a007e7..4a21f11f58ca0 100644 --- a/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/RawBytesReduceApp.java +++ b/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/RawBytesReduceApp.java @@ -21,6 +21,7 @@ import java.io.DataInputStream; import java.io.EOFException; import java.io.IOException; +import java.nio.charset.StandardCharsets; import org.apache.hadoop.io.IntWritable; @@ -62,7 +63,7 @@ private String readString() throws IOException { } byte[] bytes = new byte[length]; dis.readFully(bytes); - return new String(bytes, "UTF-8"); + return new String(bytes, StandardCharsets.UTF_8); } private int readInt() throws IOException { diff --git a/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestFileArgs.java b/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestFileArgs.java index fccd8d51e4b92..901abba885b5f 100644 --- a/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestFileArgs.java +++ b/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestFileArgs.java @@ -21,6 +21,7 @@ import java.io.DataOutputStream; import java.io.File; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.Map; import org.apache.hadoop.hdfs.MiniDFSCluster; @@ -70,7 +71,7 @@ public void setUp() throws IOException { // Set up side file FileSystem localFs = FileSystem.getLocal(conf); DataOutputStream dos = localFs.create(new Path("target/sidefile")); - dos.write("hello world\n".getBytes("UTF-8")); + dos.write("hello world\n".getBytes(StandardCharsets.UTF_8)); dos.close(); // Since ls doesn't read stdin, we don't want to write anything diff --git a/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestGzipInput.java b/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestGzipInput.java index a9fc5fd5a457b..dc12e4eff97a2 100644 --- a/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestGzipInput.java +++ b/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestGzipInput.java @@ -21,6 +21,7 @@ import java.io.File; import java.io.FileOutputStream; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.zip.GZIPOutputStream; /** @@ -37,7 +38,7 @@ protected void createInput() throws IOException { GZIPOutputStream out = new GZIPOutputStream( new FileOutputStream(INPUT_FILE.getAbsoluteFile())); - out.write(input.getBytes("UTF-8")); + out.write(input.getBytes(StandardCharsets.UTF_8)); out.close(); } } diff --git a/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestMultipleArchiveFiles.java b/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestMultipleArchiveFiles.java index 752268de3dc7d..041d527ab173a 100644 --- a/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestMultipleArchiveFiles.java +++ b/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestMultipleArchiveFiles.java @@ -21,6 +21,7 @@ import java.io.File; import java.io.IOException; import java.io.DataOutputStream; +import java.nio.charset.StandardCharsets; import java.util.Map; import java.util.zip.ZipEntry; import java.util.zip.ZipOutputStream; @@ -86,14 +87,14 @@ protected void createInput() throws IOException DataOutputStream dos = fileSys.create(new Path(INPUT_FILE)); String inputFileString = "symlink1" + File.separator + "cacheArchive1\nsymlink2" + File.separator + "cacheArchive2"; - dos.write(inputFileString.getBytes("UTF-8")); + dos.write(inputFileString.getBytes(StandardCharsets.UTF_8)); dos.close(); DataOutputStream out = fileSys.create(new Path(CACHE_ARCHIVE_1.toString())); ZipOutputStream zos = new ZipOutputStream(out); ZipEntry ze = new ZipEntry(CACHE_FILE_1.toString()); zos.putNextEntry(ze); - zos.write(input.getBytes("UTF-8")); + zos.write(input.getBytes(StandardCharsets.UTF_8)); zos.closeEntry(); zos.close(); @@ -101,7 +102,7 @@ protected void createInput() throws IOException zos = new ZipOutputStream(out); ze = new ZipEntry(CACHE_FILE_2.toString()); zos.putNextEntry(ze); - zos.write(input.getBytes("UTF-8")); + zos.write(input.getBytes(StandardCharsets.UTF_8)); zos.closeEntry(); zos.close(); } diff --git a/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestRawBytesStreaming.java b/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestRawBytesStreaming.java index 7621fd1fe8a9c..09adb3d5fd44c 100644 --- a/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestRawBytesStreaming.java +++ b/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestRawBytesStreaming.java @@ -22,6 +22,7 @@ import java.io.File; import java.io.FileOutputStream; import java.io.IOException; +import java.nio.charset.StandardCharsets; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileUtil; @@ -46,7 +47,7 @@ public TestRawBytesStreaming() throws IOException { protected void createInput() throws IOException { DataOutputStream out = new DataOutputStream(new FileOutputStream(INPUT_FILE.getAbsoluteFile())); - out.write(input.getBytes("UTF-8")); + out.write(input.getBytes(StandardCharsets.UTF_8)); out.close(); } diff --git a/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestStreamAggregate.java b/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestStreamAggregate.java index b27a8c65ae866..b303c8c977249 100644 --- a/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestStreamAggregate.java +++ b/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestStreamAggregate.java @@ -21,6 +21,7 @@ import org.junit.Test; import static org.junit.Assert.*; import java.io.*; +import java.nio.charset.StandardCharsets; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.mapreduce.MRJobConfig; @@ -53,7 +54,7 @@ protected void createInput() throws IOException { DataOutputStream out = new DataOutputStream( new FileOutputStream(INPUT_FILE.getAbsoluteFile())); - out.write(input.getBytes("UTF-8")); + out.write(input.getBytes(StandardCharsets.UTF_8)); out.close(); } diff --git a/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestUnconsumedInput.java b/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestUnconsumedInput.java index e1f6da52768d5..b2bc84b4f90a9 100644 --- a/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestUnconsumedInput.java +++ b/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestUnconsumedInput.java @@ -24,14 +24,11 @@ import java.io.File; import java.io.FileOutputStream; import java.io.IOException; +import java.nio.charset.StandardCharsets; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileUtil; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hdfs.HdfsConfiguration; -import org.apache.hadoop.hdfs.MiniDFSCluster; import org.junit.Test; public class TestUnconsumedInput { @@ -54,12 +51,12 @@ public TestUnconsumedInput() throws IOException protected void createInput() throws IOException { - DataOutputStream out = new DataOutputStream( - new FileOutputStream(INPUT_FILE.getAbsoluteFile())); + try (DataOutputStream out = new DataOutputStream( + new FileOutputStream(INPUT_FILE.getAbsoluteFile()))) { for (int i=0; i<10000; ++i) { - out.write(input.getBytes("UTF-8")); + out.write(input.getBytes(StandardCharsets.UTF_8)); } - out.close(); + } } protected String[] genArgs() { diff --git a/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/mapreduce/TestStreamXmlRecordReader.java b/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/mapreduce/TestStreamXmlRecordReader.java index f2d9495efa892..5bf2fe52d447c 100644 --- a/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/mapreduce/TestStreamXmlRecordReader.java +++ b/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/mapreduce/TestStreamXmlRecordReader.java @@ -24,6 +24,7 @@ import java.io.File; import java.io.FileOutputStream; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.HashSet; import java.util.Set; @@ -88,7 +89,7 @@ private String slurpHadoop(Path p, FileSystem fs) throws IOException { String contents = null; try { in.readFully(in.getPos(), buf); - contents = new String(buf, "UTF-8"); + contents = new String(buf, StandardCharsets.UTF_8); } finally { in.close(); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ResourceManagerAdministrationProtocol.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ResourceManagerAdministrationProtocol.java index 916dcd50aa0d9..0b4f1ad6429fa 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ResourceManagerAdministrationProtocol.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ResourceManagerAdministrationProtocol.java @@ -64,6 +64,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.BatchSaveFederationQueuePoliciesResponse; import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesRequest; import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesResponse; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationRequest; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationResponse; @Private public interface ResourceManagerAdministrationProtocol extends GetUserMappingsProtocol { @@ -218,4 +220,17 @@ BatchSaveFederationQueuePoliciesResponse batchSaveFederationQueuePolicies( @Idempotent QueryFederationQueuePoliciesResponse listFederationQueuePolicies( QueryFederationQueuePoliciesRequest request) throws YarnException, IOException; + + /** + * In YARN-Federation mode, this method provides a way to delete federation application. + * + * @param request DeleteFederationApplicationRequest Request. + * @return Response from deleteFederationApplication. + * @throws YarnException exceptions from yarn servers. + * @throws IOException if an IO error occurred. + */ + @Private + @Idempotent + DeleteFederationApplicationResponse deleteFederationApplication( + DeleteFederationApplicationRequest request) throws YarnException, IOException; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/DeleteFederationApplicationRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/DeleteFederationApplicationRequest.java new file mode 100644 index 0000000000000..a035aa8275c12 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/DeleteFederationApplicationRequest.java @@ -0,0 +1,51 @@ +/** + * 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.yarn.server.api.protocolrecords; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.util.Records; + +/** + * This class is used for cleaning up an application that exists in the FederationStateStore. + * This is a user-specified operation; + * we typically use this command to clean up an expired application. + * However, it can also be used to clean up non-expired application, although it is not recommended. + */ +@Private +@Unstable +public abstract class DeleteFederationApplicationRequest { + + @Private + @Unstable + public static DeleteFederationApplicationRequest newInstance(String application) { + DeleteFederationApplicationRequest request = + Records.newRecord(DeleteFederationApplicationRequest.class); + request.setApplication(application); + return request; + } + + @Public + @Unstable + public abstract String getApplication(); + + @Public + @Unstable + public abstract void setApplication(String application); +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/DeleteFederationApplicationResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/DeleteFederationApplicationResponse.java new file mode 100644 index 0000000000000..d75515ff633ec --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/DeleteFederationApplicationResponse.java @@ -0,0 +1,47 @@ +/** + * 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.yarn.server.api.protocolrecords; + +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.util.Records; + +@Private +@Unstable +public abstract class DeleteFederationApplicationResponse { + + public static DeleteFederationApplicationResponse newInstance() { + return Records.newRecord(DeleteFederationApplicationResponse.class); + } + + public static DeleteFederationApplicationResponse newInstance(String msg) { + DeleteFederationApplicationResponse response = + Records.newRecord(DeleteFederationApplicationResponse.class); + response.setMessage(msg); + return response; + } + + @Public + @Unstable + public abstract String getMessage(); + + @Public + @Unstable + public abstract void setMessage(String msg); +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/resourcemanager_administration_protocol.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/resourcemanager_administration_protocol.proto index fcae14128d80f..f0f3fa563f59d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/resourcemanager_administration_protocol.proto +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/resourcemanager_administration_protocol.proto @@ -51,4 +51,5 @@ service ResourceManagerAdministrationProtocolService { rpc saveFederationQueuePolicy(SaveFederationQueuePolicyRequestProto) returns (SaveFederationQueuePolicyResponseProto); rpc batchSaveFederationQueuePolicies(BatchSaveFederationQueuePoliciesRequestProto) returns (BatchSaveFederationQueuePoliciesResponseProto); rpc listFederationQueuePolicies(QueryFederationQueuePoliciesRequestProto) returns (QueryFederationQueuePoliciesResponseProto); + rpc deleteFederationApplication(DeleteFederationApplicationRequestProto) returns (DeleteFederationApplicationResponseProto); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/yarn_server_resourcemanager_service_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/yarn_server_resourcemanager_service_protos.proto index a2945f1eb1ee1..ba18e514a7dfa 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/yarn_server_resourcemanager_service_protos.proto +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/yarn_server_resourcemanager_service_protos.proto @@ -203,6 +203,14 @@ message QueryFederationQueuePoliciesResponseProto { repeated FederationQueueWeightProto federationQueueWeights = 5; } +message DeleteFederationApplicationRequestProto { + optional string application = 1; +} + +message DeleteFederationApplicationResponseProto { + optional string message = 1; +} + ////////////////////////////////////////////////////////////////// ///////////// RM Failover related records //////////////////////// ////////////////////////////////////////////////////////////////// diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java index f7747c6216a86..e4b48c45f33ec 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java @@ -36,7 +36,7 @@ public class TestYarnConfigurationFields extends TestConfigurationFieldsBase { @SuppressWarnings({"deprecation", "methodlength"}) @Override public void initializeMemberVariables() { - xmlFilename = new String("yarn-default.xml"); + xmlFilename = "yarn-default.xml"; configurationClasses = new Class[] { YarnConfiguration.class }; // Allocate for usage diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-unmanaged-am-launcher/src/main/java/org/apache/hadoop/yarn/applications/unmanagedamlauncher/UnmanagedAMLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-unmanaged-am-launcher/src/main/java/org/apache/hadoop/yarn/applications/unmanagedamlauncher/UnmanagedAMLauncher.java index f3bb5fde3a4d3..c6fe29f65fbcc 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-unmanaged-am-launcher/src/main/java/org/apache/hadoop/yarn/applications/unmanagedamlauncher/UnmanagedAMLauncher.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-unmanaged-am-launcher/src/main/java/org/apache/hadoop/yarn/applications/unmanagedamlauncher/UnmanagedAMLauncher.java @@ -25,7 +25,7 @@ import java.io.IOException; import java.io.InputStreamReader; import java.net.InetAddress; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.EnumSet; import java.util.Map; @@ -235,10 +235,10 @@ public void launchAM(ApplicationAttemptId attemptId) final BufferedReader errReader = new BufferedReader(new InputStreamReader( - amProc.getErrorStream(), Charset.forName("UTF-8"))); + amProc.getErrorStream(), StandardCharsets.UTF_8)); final BufferedReader inReader = new BufferedReader(new InputStreamReader( - amProc.getInputStream(), Charset.forName("UTF-8"))); + amProc.getInputStream(), StandardCharsets.UTF_8)); // read error and input streams as this would free up the buffers // free the error stream buffer diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/PublishedConfigurationOutputter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/PublishedConfigurationOutputter.java index 447a3e967abe0..3b39c10bf8931 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/PublishedConfigurationOutputter.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/PublishedConfigurationOutputter.java @@ -18,7 +18,6 @@ package org.apache.hadoop.yarn.service.utils; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; import org.apache.hadoop.util.Preconditions; import org.apache.commons.io.FileUtils; import org.apache.commons.io.IOUtils; @@ -72,7 +71,7 @@ public void save(File dest) throws IOException { * @throws IOException */ public void save(OutputStream out) throws IOException { - IOUtils.write(asString(), out, Charsets.UTF_8); + IOUtils.write(asString(), out, StandardCharsets.UTF_8); } /** * Convert to a string diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java index 00cf2d4f76a8b..cd0d2aa979155 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java @@ -22,7 +22,7 @@ import java.io.IOException; import java.io.OutputStreamWriter; import java.io.PrintWriter; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.text.DecimalFormat; import java.util.*; @@ -357,7 +357,7 @@ private int printApplicationAttemptReport(String applicationAttemptId) // Use PrintWriter.println, which uses correct platform line ending. ByteArrayOutputStream baos = new ByteArrayOutputStream(); PrintWriter appAttemptReportStr = new PrintWriter( - new OutputStreamWriter(baos, Charset.forName("UTF-8"))); + new OutputStreamWriter(baos, StandardCharsets.UTF_8)); if (appAttemptReport != null) { appAttemptReportStr.println("Application Attempt Report : "); appAttemptReportStr.print("\tApplicationAttempt-Id : "); @@ -381,11 +381,11 @@ private int printApplicationAttemptReport(String applicationAttemptId) appAttemptReportStr.print("Application Attempt with id '" + applicationAttemptId + "' doesn't exist in Timeline Server."); appAttemptReportStr.close(); - sysout.println(baos.toString("UTF-8")); + sysout.println(new String(baos.toByteArray(), StandardCharsets.UTF_8)); return -1; } appAttemptReportStr.close(); - sysout.println(baos.toString("UTF-8")); + sysout.println(new String(baos.toByteArray(), StandardCharsets.UTF_8)); return 0; } @@ -417,7 +417,7 @@ private int printContainerReport(String containerId) throws YarnException, // Use PrintWriter.println, which uses correct platform line ending. ByteArrayOutputStream baos = new ByteArrayOutputStream(); PrintWriter containerReportStr = new PrintWriter( - new OutputStreamWriter(baos, Charset.forName("UTF-8"))); + new OutputStreamWriter(baos, StandardCharsets.UTF_8)); if (containerReport != null) { containerReportStr.println("Container Report : "); containerReportStr.print("\tContainer-Id : "); @@ -446,11 +446,11 @@ private int printContainerReport(String containerId) throws YarnException, containerReportStr.print("Container with id '" + containerId + "' doesn't exist in Timeline Server."); containerReportStr.close(); - sysout.println(baos.toString("UTF-8")); + sysout.println(new String(baos.toByteArray(), StandardCharsets.UTF_8)); return -1; } containerReportStr.close(); - sysout.println(baos.toString("UTF-8")); + sysout.println(new String(baos.toByteArray(), StandardCharsets.UTF_8)); return 0; } @@ -468,7 +468,7 @@ private void listApplications(Set appTypes, EnumSet appStates, Set appTags) throws YarnException, IOException { PrintWriter writer = new PrintWriter( - new OutputStreamWriter(sysout, Charset.forName("UTF-8"))); + new OutputStreamWriter(sysout, StandardCharsets.UTF_8)); if (allAppStates) { for (YarnApplicationState appState : YarnApplicationState.values()) { appStates.add(appState); @@ -610,7 +610,7 @@ private int printApplicationReport(String applicationId) // Use PrintWriter.println, which uses correct platform line ending. ByteArrayOutputStream baos = new ByteArrayOutputStream(); PrintWriter appReportStr = new PrintWriter( - new OutputStreamWriter(baos, Charset.forName("UTF-8"))); + new OutputStreamWriter(baos, StandardCharsets.UTF_8)); if (appReport != null) { appReportStr.println("Application Report : "); appReportStr.print("\tApplication-Id : "); @@ -673,11 +673,11 @@ private int printApplicationReport(String applicationId) appReportStr.print("Application with id '" + applicationId + "' doesn't exist in RM."); appReportStr.close(); - sysout.println(baos.toString("UTF-8")); + sysout.println(new String(baos.toByteArray(), StandardCharsets.UTF_8)); return -1; } appReportStr.close(); - sysout.println(baos.toString("UTF-8")); + sysout.println(new String(baos.toByteArray(), StandardCharsets.UTF_8)); return 0; } @@ -718,7 +718,7 @@ private String getAllValidApplicationStates() { private void listApplicationAttempts(String applicationId) throws YarnException, IOException { PrintWriter writer = new PrintWriter( - new OutputStreamWriter(sysout, Charset.forName("UTF-8"))); + new OutputStreamWriter(sysout, StandardCharsets.UTF_8)); List appAttemptsReport = client .getApplicationAttempts(ApplicationId.fromString(applicationId)); @@ -746,7 +746,7 @@ private void listApplicationAttempts(String applicationId) throws YarnException, private void listContainers(String appAttemptId) throws YarnException, IOException { PrintWriter writer = new PrintWriter( - new OutputStreamWriter(sysout, Charset.forName("UTF-8"))); + new OutputStreamWriter(sysout, StandardCharsets.UTF_8)); List appsReport = client.getContainers( ApplicationAttemptId.fromString(appAttemptId)); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ClusterCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ClusterCLI.java index 78bf93bd56b48..a0df9d6b18bd1 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ClusterCLI.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ClusterCLI.java @@ -23,7 +23,7 @@ import java.io.OutputStreamWriter; import java.io.PrintWriter; import java.io.UnsupportedEncodingException; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.List; @@ -123,12 +123,12 @@ public int run(String[] args) throws Exception { private void printClusterNodeAttributes() throws IOException, YarnException { ByteArrayOutputStream baos = new ByteArrayOutputStream(); PrintWriter pw = new PrintWriter( - new OutputStreamWriter(baos, Charset.forName("UTF-8"))); + new OutputStreamWriter(baos, StandardCharsets.UTF_8)); for (NodeAttributeInfo attribute : client.getClusterAttributes()) { pw.println(attribute.toString()); } pw.close(); - sysout.println(baos.toString("UTF-8")); + sysout.println(new String(baos.toByteArray(), StandardCharsets.UTF_8)); } void printClusterNodeLabels() throws YarnException, IOException { @@ -158,11 +158,11 @@ void printClusterNodeLabels() throws YarnException, IOException { void printUsage(Options opts) throws UnsupportedEncodingException { ByteArrayOutputStream baos = new ByteArrayOutputStream(); PrintWriter pw = - new PrintWriter(new OutputStreamWriter(baos, Charset.forName("UTF-8"))); + new PrintWriter(new OutputStreamWriter(baos, StandardCharsets.UTF_8)); new HelpFormatter().printHelp(pw, HelpFormatter.DEFAULT_WIDTH, TITLE, null, opts, HelpFormatter.DEFAULT_LEFT_PAD, HelpFormatter.DEFAULT_DESC_PAD, null); pw.close(); - sysout.println(baos.toString("UTF-8")); + sysout.println(new String(baos.toByteArray(), StandardCharsets.UTF_8)); } } \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeAttributesCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeAttributesCLI.java index 505eca8dcf3b9..8fe388e71fa8d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeAttributesCLI.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeAttributesCLI.java @@ -58,7 +58,7 @@ import java.io.PrintStream; import java.io.PrintWriter; import java.io.UnsupportedEncodingException; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; @@ -141,10 +141,10 @@ private void print(StringBuilder usageBuilder) throws UnsupportedEncodingException { ByteArrayOutputStream baos = new ByteArrayOutputStream(); PrintWriter pw = - new PrintWriter(new OutputStreamWriter(baos, Charset.forName("UTF-8"))); + new PrintWriter(new OutputStreamWriter(baos, StandardCharsets.UTF_8)); pw.write(usageBuilder.toString()); pw.close(); - errOut.println(baos.toString("UTF-8")); + errOut.println(new String(baos.toByteArray(), StandardCharsets.UTF_8)); } private Options buildOptions(CommandHandler... handlers) { @@ -380,7 +380,7 @@ public int printNodesByAttributes(String[] attrs) protocol.getAttributesToNodes(request); ByteArrayOutputStream baos = new ByteArrayOutputStream(); PrintWriter writer = new PrintWriter( - new OutputStreamWriter(baos, Charset.forName("UTF-8"))); + new OutputStreamWriter(baos, StandardCharsets.UTF_8)); writer.format(HOSTNAMEVAL, "Hostname", "Attribute-value"); response.getAttributesToNodes().forEach((attributeKey, v) -> { writer.println(getKeyString(attributeKey) + " :"); @@ -389,7 +389,7 @@ public int printNodesByAttributes(String[] attrs) attrVal.getAttributeValue())); }); writer.close(); - sysOut.println(baos.toString("UTF-8")); + sysOut.println(new String(baos.toByteArray(), StandardCharsets.UTF_8)); return 0; } @@ -405,7 +405,7 @@ private int printAttributesByNode(String[] nodeArray) response.getNodeToAttributes(); ByteArrayOutputStream baos = new ByteArrayOutputStream(); PrintWriter writer = new PrintWriter( - new OutputStreamWriter(baos, Charset.forName("UTF-8"))); + new OutputStreamWriter(baos, StandardCharsets.UTF_8)); writer.printf(NODEATTRIBUTE, "Attribute", "Type", "Value"); nodeToAttrs.forEach((node, v) -> { // print node header @@ -415,7 +415,7 @@ private int printAttributesByNode(String[] nodeArray) attr.getAttributeType().name(), attr.getAttributeValue())); }); writer.close(); - sysOut.println(baos.toString("UTF-8")); + sysOut.println(new String(baos.toByteArray(), StandardCharsets.UTF_8)); return 0; } @@ -427,14 +427,14 @@ private int printClusterAttributes() throws IOException, YarnException { protocol.getClusterNodeAttributes(request); ByteArrayOutputStream baos = new ByteArrayOutputStream(); PrintWriter writer = new PrintWriter( - new OutputStreamWriter(baos, Charset.forName("UTF-8"))); + new OutputStreamWriter(baos, StandardCharsets.UTF_8)); writer.format(NODEATTRIBUTEINFO, "Attribute", "Type"); for (NodeAttributeInfo attr : response.getNodeAttributes()) { writer.format(NODEATTRIBUTEINFO, getKeyString(attr.getAttributeKey()), attr.getAttributeType().name()); } writer.close(); - sysOut.println(baos.toString("UTF-8")); + sysOut.println(new String(baos.toByteArray(), StandardCharsets.UTF_8)); return 0; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeCLI.java index 6120a8496a611..317f30cdde914 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeCLI.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeCLI.java @@ -21,7 +21,7 @@ import java.io.IOException; import java.io.OutputStreamWriter; import java.io.PrintWriter; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Collections; import java.util.Date; @@ -177,7 +177,7 @@ private void printUsage(Options opts) { private void listClusterNodes(Set nodeStates) throws YarnException, IOException { PrintWriter writer = new PrintWriter( - new OutputStreamWriter(sysout, Charset.forName("UTF-8"))); + new OutputStreamWriter(sysout, StandardCharsets.UTF_8)); List nodesReport = client.getNodeReports( nodeStates.toArray(new NodeState[0])); writer.println("Total Nodes:" + nodesReport.size()); @@ -202,7 +202,7 @@ private void listClusterNodes(Set nodeStates) private void listDetailedClusterNodes(Set nodeStates) throws YarnException, IOException { PrintWriter writer = new PrintWriter(new OutputStreamWriter(sysout, - Charset.forName("UTF-8"))); + StandardCharsets.UTF_8)); List nodesReport = client.getNodeReports(nodeStates .toArray(new NodeState[0])); writer.println("Total Nodes:" + nodesReport.size()); @@ -265,7 +265,7 @@ private void printNodeStatus(String nodeIdStr) throws YarnException, // Use PrintWriter.println, which uses correct platform line ending. ByteArrayOutputStream baos = new ByteArrayOutputStream(); PrintWriter nodeReportStr = new PrintWriter( - new OutputStreamWriter(baos, Charset.forName("UTF-8"))); + new OutputStreamWriter(baos, StandardCharsets.UTF_8)); NodeReport nodeReport = null; for (NodeReport report : nodesReport) { if (!report.getNodeId().equals(nodeId)) { @@ -347,7 +347,7 @@ private void printNodeStatus(String nodeIdStr) throws YarnException, + nodeIdStr); } nodeReportStr.close(); - sysout.println(baos.toString("UTF-8")); + sysout.println(new String(baos.toByteArray(), StandardCharsets.UTF_8)); } private String getAllValidNodeStates() { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/QueueCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/QueueCLI.java index db2b2cdf53f8f..927de284499a0 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/QueueCLI.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/QueueCLI.java @@ -136,7 +136,7 @@ void printUsage(Options opts) { private int listQueue(String queueName) throws YarnException, IOException { int rc; PrintWriter writer = new PrintWriter( - new OutputStreamWriter(sysout, Charset.forName("UTF-8"))); + new OutputStreamWriter(sysout, StandardCharsets.UTF_8)); QueueInfo queueInfo = client.getQueueInfo(queueName); if (queueInfo != null) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RouterCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RouterCLI.java index 2da584f9d61bb..9a68794078e2e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RouterCLI.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RouterCLI.java @@ -18,6 +18,7 @@ package org.apache.hadoop.yarn.client.cli; import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.DefaultParser; import org.apache.commons.cli.GnuParser; import org.apache.commons.cli.MissingArgumentException; import org.apache.commons.cli.Option; @@ -42,6 +43,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.DeregisterSubClusterRequest; import org.apache.hadoop.yarn.server.api.protocolrecords.DeregisterSubClusterResponse; import org.apache.hadoop.yarn.server.api.protocolrecords.DeregisterSubClusters; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationRequest; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationResponse; import org.apache.hadoop.yarn.server.api.protocolrecords.SaveFederationQueuePolicyRequest; import org.apache.hadoop.yarn.server.api.protocolrecords.SaveFederationQueuePolicyResponse; import org.apache.hadoop.yarn.server.api.protocolrecords.BatchSaveFederationQueuePoliciesRequest; @@ -227,12 +230,37 @@ public class RouterCLI extends Configured implements Tool { .addExample(POLICY_LIST_USAGE.args, POLICY_LIST_USAGE_EXAMPLE_1) .addExample(POLICY_LIST_USAGE.args, POLICY_LIST_USAGE_EXAMPLE_2); + // Command3: application + private static final String CMD_APPLICATION = "-application"; + + // Application Delete + protected final static UsageInfo APPLICATION_DELETE_USAGE = new UsageInfo( + "--delete ", + "This command is used to delete the specified application."); + + protected final static String APPLICATION_DELETE_USAGE_EXAMPLE_DESC = + "If we want to delete application_1440536969523_0001."; + + protected final static String APPLICATION_DELETE_USAGE_EXAMPLE_1 = + "yarn routeradmin -application --delete application_1440536969523_0001"; + + protected final static RouterCmdUsageInfos APPLICATION_USAGEINFOS = new RouterCmdUsageInfos() + // application delete + .addUsageInfo(APPLICATION_DELETE_USAGE) + .addExampleDescs(APPLICATION_DELETE_USAGE.args, APPLICATION_DELETE_USAGE_EXAMPLE_DESC) + .addExample(APPLICATION_DELETE_USAGE.args, APPLICATION_DELETE_USAGE_EXAMPLE_1); + + // delete application + private static final String OPTION_DELETE_APP = "delete"; + protected final static Map ADMIN_USAGE = ImmutableMap.builder() // Command1: deregisterSubCluster .put(CMD_DEREGISTERSUBCLUSTER, DEREGISTER_SUBCLUSTER_USAGEINFOS) // Command2: policy .put(CMD_POLICY, POLICY_USAGEINFOS) + // Command3: application + .put(CMD_APPLICATION, APPLICATION_USAGEINFOS) .build(); public RouterCLI() { @@ -814,6 +842,50 @@ protected int handListPolicies(int pageSize, int currentPage, String queue, List } } + private int handleDeleteApplication(String application) { + LOG.info("Delete Application = {}.", application); + try { + DeleteFederationApplicationRequest request = + DeleteFederationApplicationRequest.newInstance(application); + ResourceManagerAdministrationProtocol adminProtocol = createAdminProtocol(); + DeleteFederationApplicationResponse response = + adminProtocol.deleteFederationApplication(request); + System.out.println(response.getMessage()); + return EXIT_SUCCESS; + } catch (Exception e) { + LOG.error("handleSavePolicy error.", e); + return EXIT_ERROR; + } + } + + private int handleApplication(String[] args) + throws IOException, YarnException, ParseException { + // Prepare Options. + Options opts = new Options(); + opts.addOption("application", false, + "We provide a set of commands to query and clean applications."); + Option deleteOpt = new Option(null, OPTION_DELETE_APP, true, + "We will clean up the provided application."); + opts.addOption(deleteOpt); + + // Parse command line arguments. + CommandLine cliParser; + try { + cliParser = new DefaultParser().parse(opts, args); + } catch (MissingArgumentException ex) { + System.out.println("Missing argument for options"); + printUsage(args[0]); + return EXIT_ERROR; + } + + if (cliParser.hasOption(OPTION_DELETE_APP)) { + String application = cliParser.getOptionValue(OPTION_DELETE_APP); + return handleDeleteApplication(application); + } + + return 0; + } + @Override public int run(String[] args) throws Exception { YarnConfiguration yarnConf = getConf() == null ? @@ -839,6 +911,8 @@ public int run(String[] args) throws Exception { return handleDeregisterSubCluster(args); } else if (CMD_POLICY.equals(cmd)) { return handlePolicy(args); + } else if (CMD_APPLICATION.equals(cmd)) { + return handleApplication(args); } else { System.out.println("No related commands found."); printHelp(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestSharedCacheClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestSharedCacheClientImpl.java index 1b179b138a975..8111c7ebdf0f2 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestSharedCacheClientImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestSharedCacheClientImpl.java @@ -27,6 +27,7 @@ import java.io.DataOutputStream; import java.io.FileNotFoundException; import java.io.IOException; +import java.nio.charset.StandardCharsets; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -173,7 +174,7 @@ private Path makeFile(String filename) throws Exception { DataOutputStream out = null; try { out = localFs.create(file); - out.write(input.getBytes("UTF-8")); + out.write(input.getBytes(StandardCharsets.UTF_8)); } finally { if(out != null) { out.close(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestNodeAttributesCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestNodeAttributesCLI.java index cab4bda76c477..12ac21bc98118 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestNodeAttributesCLI.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestNodeAttributesCLI.java @@ -41,6 +41,7 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.PrintStream; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.List; import java.util.Set; @@ -60,7 +61,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; import org.apache.hadoop.thirdparty.com.google.common.base.Joiner; /** @@ -528,8 +528,8 @@ private int runTool(String... args) throws Exception { sysOutBytes.reset(); LOG.info("Running: NodeAttributesCLI " + Joiner.on(" ").join(args)); int ret = nodeAttributesCLI.run(args); - errOutput = new String(errOutBytes.toByteArray(), Charsets.UTF_8); - sysOutput = new String(sysOutBytes.toByteArray(), Charsets.UTF_8); + errOutput = new String(errOutBytes.toByteArray(), StandardCharsets.UTF_8); + sysOutput = new String(sysOutBytes.toByteArray(), StandardCharsets.UTF_8); LOG.info("Err_output:\n" + errOutput); LOG.info("Sys_output:\n" + sysOutput); return ret; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestRMAdminCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestRMAdminCLI.java index 6eb1f2fc986a6..411526db41c7a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestRMAdminCLI.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestRMAdminCLI.java @@ -38,6 +38,7 @@ import java.io.IOException; import java.io.InputStream; import java.io.PrintStream; +import java.nio.charset.StandardCharsets; import java.util.HashSet; import java.util.Iterator; import java.util.Map; @@ -85,7 +86,6 @@ import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet; public class TestRMAdminCLI { @@ -1061,7 +1061,7 @@ public void testRMHAErrorUsage() throws Exception { try { String[] args = {"-transitionToActive"}; assertEquals(-1, rmAdminCLIWithHAEnabled.run(args)); - String errOut = new String(errOutBytes.toByteArray(), Charsets.UTF_8); + String errOut = new String(errOutBytes.toByteArray(), StandardCharsets.UTF_8); errOutBytes.reset(); assertTrue(errOut.contains("Usage: rmadmin")); } finally { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestRouterCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestRouterCLI.java index a86878dac3f81..8e697dd6c01db 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestRouterCLI.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestRouterCLI.java @@ -94,6 +94,7 @@ public void setup() throws Exception { return QueryFederationQueuePoliciesResponse.newInstance(1, 1, 1, 10, weights); }); + Configuration config = new Configuration(); config.setBoolean(YarnConfiguration.FEDERATION_ENABLED, true); @@ -271,7 +272,7 @@ public void testListPolicies() throws Exception { @Test public void testBuildHelpMsg() throws Exception { Map adminUsage = rmAdminCLI.getAdminUsage(); - assertEquals(2, adminUsage.size()); + assertEquals(3, adminUsage.size()); RouterCLI.RouterCmdUsageInfos deregisterSubClusterUsageInfos = adminUsage.get("-deregisterSubCluster"); @@ -291,5 +292,11 @@ public void testBuildHelpMsg() throws Exception { policyExamplesMap.forEach((cmd, cmdExamples) -> { assertEquals(2, cmdExamples.size()); }); + + RouterCLI.RouterCmdUsageInfos applicationUsageInfos = adminUsage.get("-application"); + assertNotNull(applicationUsageInfos); + Map> applicationExamplesMap = applicationUsageInfos.getExamples(); + assertNotNull(applicationExamplesMap); + assertEquals(1, applicationExamplesMap.size()); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java index 26c3e01a45d03..5a752064bce1c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java @@ -31,7 +31,6 @@ import java.io.OutputStream; import java.io.PrintStream; import java.io.Writer; -import java.nio.charset.Charset; import java.nio.charset.StandardCharsets; import java.security.PrivilegedExceptionAction; import java.util.ArrayList; @@ -288,7 +287,7 @@ public void write(DataOutputStream out, Set pendingUploadFiles) this.uploadedFiles.add(logFile); } catch (IOException e) { String message = logErrorMessage(logFile, e); - out.write(message.getBytes(Charset.forName("UTF-8"))); + out.write(message.getBytes(StandardCharsets.UTF_8)); } finally { IOUtils.cleanupWithLogger(LOG, in); } @@ -1067,7 +1066,7 @@ public String nextLog() throws IOException { new BoundedInputStream(valueStream, currentLogLength); currentLogData.setPropagateClose(false); currentLogISR = new InputStreamReader(currentLogData, - Charset.forName("UTF-8")); + StandardCharsets.UTF_8); currentLogType = logType; } catch (EOFException e) { } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogToolUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogToolUtils.java index 3c56b0290d74e..cc137ba696ece 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogToolUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogToolUtils.java @@ -25,7 +25,7 @@ import java.nio.channels.Channels; import java.nio.channels.FileChannel; import java.nio.channels.WritableByteChannel; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Paths; @@ -68,7 +68,7 @@ private static byte[] formatContainerLogHeader(String containerId, .append("LogLastModifiedTime:" + lastModifiedTime + "\n") .append("LogLength:" + fileLength + "\n") .append("LogContents:\n"); - return sb.toString().getBytes(Charset.forName("UTF-8")); + return sb.toString().getBytes(StandardCharsets.UTF_8); } /** diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/LogAggregationHtmlBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/LogAggregationHtmlBlock.java index 4ec8794b14587..b374f5314521d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/LogAggregationHtmlBlock.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/LogAggregationHtmlBlock.java @@ -27,7 +27,7 @@ import java.io.IOException; import java.io.InputStream; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.util.Map; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; @@ -196,7 +196,7 @@ protected void processContainerLog(Block html, long[] range, InputStream in, Hamlet.PRE pre = html.pre(); while (toRead > 0 && (len = in.read(cbuf, 0, currentToRead)) > 0) { - pre.__(new String(cbuf, 0, len, Charset.forName("UTF-8"))); + pre.__(new String(cbuf, 0, len, StandardCharsets.UTF_8)); toRead = toRead - len; currentToRead = toRead > bufferSize ? bufferSize : (int) toRead; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/LogAggregationIndexedFileController.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/LogAggregationIndexedFileController.java index d4431d56b39a4..7c6be1305d09e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/LogAggregationIndexedFileController.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/LogAggregationIndexedFileController.java @@ -26,7 +26,6 @@ import java.io.InputStream; import java.io.OutputStream; import java.io.Serializable; -import java.nio.charset.Charset; import java.nio.charset.StandardCharsets; import java.security.MessageDigest; import java.security.NoSuchAlgorithmException; @@ -220,7 +219,7 @@ public Object run() throws Exception { // append a simple character("\n") to move the writer cursor, so // we could get the correct position when we call // fsOutputStream.getStartPos() - final byte[] dummyBytes = "\n".getBytes(Charset.forName("UTF-8")); + final byte[] dummyBytes = "\n".getBytes(StandardCharsets.UTF_8); fsDataOStream.write(dummyBytes); fsDataOStream.flush(); @@ -286,7 +285,7 @@ private Pair initializeWriterInRolling( int actualLength = b.length; if (actualLength == nameLength) { String recoveredLogFile = new String( - b, Charset.forName("UTF-8")); + b, StandardCharsets.UTF_8); if (recoveredLogFile.equals( currentRemoteLogFile.getName())) { overwriteCheckSum = false; @@ -340,7 +339,7 @@ private Pair initializeWriterInRolling( String fileName = aggregatedLogFile.getName(); checksumFileOutputStream.writeInt(fileName.length()); checksumFileOutputStream.write(fileName.getBytes( - Charset.forName("UTF-8"))); + StandardCharsets.UTF_8)); checksumFileOutputStream.writeLong( currentAggregatedLogFileLength); checksumFileOutputStream.flush(); @@ -403,7 +402,7 @@ public void write(LogKey logKey, LogValue logValue) throws IOException { if (outputStreamState != null && outputStreamState.getOutputStream() != null) { outputStreamState.getOutputStream().write( - message.getBytes(Charset.forName("UTF-8"))); + message.getBytes(StandardCharsets.UTF_8)); } } finally { IOUtils.cleanupWithLogger(LOG, in); @@ -598,7 +597,7 @@ public boolean readAggregatedLogs(ContainerLogsRequest logRequest, Times.format(candidate.getLastModifiedTime()), in, os, buf, ContainerLogAggregationType.AGGREGATED); byte[] b = aggregatedLogSuffix(candidate.getFileName()) - .getBytes(Charset.forName("UTF-8")); + .getBytes(StandardCharsets.UTF_8); os.write(b, 0, b.length); findLogs = true; } catch (IOException e) { @@ -769,7 +768,7 @@ public Map parseCheckSumFiles( checksumFileInputStream.readFully(b); int actualLength = b.length; if (actualLength == nameLength) { - nodeName = new String(b, Charset.forName("UTF-8")); + nodeName = new String(b, StandardCharsets.UTF_8); index = checksumFileInputStream.readLong(); } else { continue; @@ -950,9 +949,9 @@ public IndexedLogsMeta loadIndexedLogsMeta(Path remoteLogPath, long end, if (LOG.isDebugEnabled()) { LOG.debug("the length of loaded UUID:{}", uuidReadLen); LOG.debug("the loaded UUID:{}", new String(uuidRead, - Charset.forName("UTF-8"))); + StandardCharsets.UTF_8)); LOG.debug("the expected UUID:{}", new String(this.uuid, - Charset.forName("UTF-8"))); + StandardCharsets.UTF_8)); } throw new IOException("The UUID from " + remoteLogPath + " is not correct. The offset of loaded UUID is " @@ -1359,7 +1358,7 @@ private byte[] createUUID(ApplicationId appId) throws IOException { try { MessageDigest digest = MessageDigest.getInstance("SHA-256"); return digest.digest(appId.toString().getBytes( - Charset.forName("UTF-8"))); + StandardCharsets.UTF_8)); } catch (NoSuchAlgorithmException ex) { throw new IOException(ex); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/tfile/LogAggregationTFileController.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/tfile/LogAggregationTFileController.java index e1e2c9aeaaff6..4b282488c6d56 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/tfile/LogAggregationTFileController.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/tfile/LogAggregationTFileController.java @@ -22,7 +22,7 @@ import java.io.EOFException; import java.io.IOException; import java.io.OutputStream; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.security.PrivilegedExceptionAction; import java.util.ArrayList; import java.util.HashMap; @@ -217,7 +217,7 @@ public boolean readAggregatedLogs(ContainerLogsRequest logRequest, valueStream, os, buf, ContainerLogAggregationType.AGGREGATED); byte[] b = aggregatedLogSuffix(fileType).getBytes( - Charset.forName("UTF-8")); + StandardCharsets.UTF_8); os.write(b, 0, b.length); findLogs = true; } else { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/ResourceManagerAdministrationProtocolPBClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/ResourceManagerAdministrationProtocolPBClientImpl.java index c7c6dc510424c..9d9c018a46962 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/ResourceManagerAdministrationProtocolPBClientImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/ResourceManagerAdministrationProtocolPBClientImpl.java @@ -48,6 +48,7 @@ import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.SaveFederationQueuePolicyRequestProto; import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.BatchSaveFederationQueuePoliciesRequestProto; import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.QueryFederationQueuePoliciesRequestProto; +import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.DeleteFederationApplicationRequestProto; import org.apache.hadoop.yarn.server.api.ResourceManagerAdministrationProtocol; import org.apache.hadoop.yarn.server.api.ResourceManagerAdministrationProtocolPB; import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsRequest; @@ -86,6 +87,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.BatchSaveFederationQueuePoliciesResponse; import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesRequest; import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesResponse; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationRequest; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationResponse; import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.AddToClusterNodeLabelsRequestPBImpl; import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.AddToClusterNodeLabelsResponsePBImpl; import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.CheckForDecommissioningNodesRequestPBImpl; @@ -122,6 +125,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.BatchSaveFederationQueuePoliciesResponsePBImpl; import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.QueryFederationQueuePoliciesRequestPBImpl; import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.QueryFederationQueuePoliciesResponsePBImpl; +import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.DeleteFederationApplicationRequestPBImpl; +import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.DeleteFederationApplicationResponsePBImpl; import org.apache.hadoop.thirdparty.protobuf.ServiceException; @@ -420,4 +425,18 @@ public QueryFederationQueuePoliciesResponse listFederationQueuePolicies( } return null; } + + @Override + public DeleteFederationApplicationResponse deleteFederationApplication( + DeleteFederationApplicationRequest request) throws YarnException, IOException { + DeleteFederationApplicationRequestProto requestProto = + ((DeleteFederationApplicationRequestPBImpl) request).getProto(); + try { + return new DeleteFederationApplicationResponsePBImpl( + proxy.deleteFederationApplication(null, requestProto)); + } catch (ServiceException e) { + RPCUtil.unwrapAndThrowException(e); + } + return null; + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/service/ResourceManagerAdministrationProtocolPBServiceImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/service/ResourceManagerAdministrationProtocolPBServiceImpl.java index 0eab80b9ad770..1e0b09a619568 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/service/ResourceManagerAdministrationProtocolPBServiceImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/service/ResourceManagerAdministrationProtocolPBServiceImpl.java @@ -60,6 +60,8 @@ import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.DeregisterSubClusterResponseProto; import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.SaveFederationQueuePolicyRequestProto; import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.SaveFederationQueuePolicyResponseProto; +import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.DeleteFederationApplicationResponseProto; +import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.DeleteFederationApplicationRequestProto; import org.apache.hadoop.yarn.server.api.ResourceManagerAdministrationProtocol; import org.apache.hadoop.yarn.server.api.ResourceManagerAdministrationProtocolPB; import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsResponse; @@ -87,6 +89,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.BatchSaveFederationQueuePoliciesResponse; import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesRequest; import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesResponse; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationRequest; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationResponse; import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.AddToClusterNodeLabelsRequestPBImpl; import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.AddToClusterNodeLabelsResponsePBImpl; import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.CheckForDecommissioningNodesRequestPBImpl; @@ -123,6 +127,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.BatchSaveFederationQueuePoliciesResponsePBImpl; import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.QueryFederationQueuePoliciesRequestPBImpl; import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.QueryFederationQueuePoliciesResponsePBImpl; +import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.DeleteFederationApplicationRequestPBImpl; +import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.DeleteFederationApplicationResponsePBImpl; import org.apache.hadoop.thirdparty.protobuf.RpcController; import org.apache.hadoop.thirdparty.protobuf.ServiceException; @@ -445,4 +451,20 @@ public QueryFederationQueuePoliciesResponseProto listFederationQueuePolicies( throw new ServiceException(e); } } + + @Override + public DeleteFederationApplicationResponseProto deleteFederationApplication( + RpcController controller, DeleteFederationApplicationRequestProto proto) + throws ServiceException { + DeleteFederationApplicationRequest request = + new DeleteFederationApplicationRequestPBImpl(proto); + try { + DeleteFederationApplicationResponse response = real.deleteFederationApplication(request); + return ((DeleteFederationApplicationResponsePBImpl) response).getProto(); + } catch (YarnException e) { + throw new ServiceException(e); + } catch (IOException e) { + throw new ServiceException(e); + } + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/DeleteFederationApplicationRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/DeleteFederationApplicationRequestPBImpl.java new file mode 100644 index 0000000000000..0309632c40ac2 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/DeleteFederationApplicationRequestPBImpl.java @@ -0,0 +1,97 @@ +/** + * 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.yarn.server.api.protocolrecords.impl.pb; + +import org.apache.commons.lang3.builder.EqualsBuilder; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.thirdparty.protobuf.TextFormat; +import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.DeleteFederationApplicationRequestProto; +import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.DeleteFederationApplicationRequestProtoOrBuilder; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationRequest; + +@Private +@Unstable +public class DeleteFederationApplicationRequestPBImpl extends DeleteFederationApplicationRequest { + + private DeleteFederationApplicationRequestProto proto = + DeleteFederationApplicationRequestProto.getDefaultInstance(); + private DeleteFederationApplicationRequestProto.Builder builder = null; + private boolean viaProto = false; + + public DeleteFederationApplicationRequestPBImpl() { + builder = DeleteFederationApplicationRequestProto.newBuilder(); + } + + public DeleteFederationApplicationRequestPBImpl(DeleteFederationApplicationRequestProto proto) { + this.proto = proto; + viaProto = true; + } + + private synchronized void maybeInitBuilder() { + if (viaProto || builder == null) { + builder = DeleteFederationApplicationRequestProto.newBuilder(proto); + } + viaProto = false; + } + + public DeleteFederationApplicationRequestProto getProto() { + proto = viaProto ? proto : builder.build(); + viaProto = true; + return proto; + } + + @Override + public int hashCode() { + return getProto().hashCode(); + } + + @Override + public boolean equals(Object other) { + if (!(other instanceof DeleteFederationApplicationRequest)) { + return false; + } + DeleteFederationApplicationRequestPBImpl otherImpl = this.getClass().cast(other); + return new EqualsBuilder().append(this.getProto(), otherImpl.getProto()).isEquals(); + } + + @Override + public String getApplication() { + DeleteFederationApplicationRequestProtoOrBuilder p = viaProto ? proto : builder; + boolean hasApplication = p.hasApplication(); + if (hasApplication) { + return p.getApplication(); + } + return null; + } + + @Override + public void setApplication(String application) { + maybeInitBuilder(); + if (application == null) { + builder.clearApplication(); + return; + } + builder.setApplication(application); + } + + @Override + public String toString() { + return TextFormat.shortDebugString(getProto()); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/DeleteFederationApplicationResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/DeleteFederationApplicationResponsePBImpl.java new file mode 100644 index 0000000000000..f21f6fbde18f0 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/DeleteFederationApplicationResponsePBImpl.java @@ -0,0 +1,97 @@ +/** + * 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.yarn.server.api.protocolrecords.impl.pb; + +import org.apache.commons.lang3.builder.EqualsBuilder; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.thirdparty.protobuf.TextFormat; +import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.DeleteFederationApplicationResponseProto; +import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.DeleteFederationApplicationResponseProtoOrBuilder; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationResponse; + +@Private +@Unstable +public class DeleteFederationApplicationResponsePBImpl extends DeleteFederationApplicationResponse { + + private DeleteFederationApplicationResponseProto proto = + DeleteFederationApplicationResponseProto.getDefaultInstance(); + private DeleteFederationApplicationResponseProto.Builder builder = null; + private boolean viaProto = false; + + public DeleteFederationApplicationResponsePBImpl() { + builder = DeleteFederationApplicationResponseProto.newBuilder(); + } + + public DeleteFederationApplicationResponsePBImpl(DeleteFederationApplicationResponseProto proto) { + this.proto = proto; + viaProto = true; + } + + public DeleteFederationApplicationResponseProto getProto() { + proto = viaProto ? proto : builder.build(); + viaProto = true; + return proto; + } + + @Override + public int hashCode() { + return getProto().hashCode(); + } + + @Override + public boolean equals(Object other) { + if (!(other instanceof DeleteFederationApplicationResponse)) { + return false; + } + DeleteFederationApplicationResponsePBImpl otherImpl = this.getClass().cast(other); + return new EqualsBuilder().append(this.getProto(), otherImpl.getProto()).isEquals(); + } + + @Override + public String toString() { + return TextFormat.shortDebugString(getProto()); + } + + @Override + public String getMessage() { + DeleteFederationApplicationResponseProtoOrBuilder p = viaProto ? proto : builder; + boolean hasMessage = p.hasMessage(); + if (hasMessage) { + return p.getMessage(); + } + return null; + } + + private synchronized void maybeInitBuilder() { + if (viaProto || builder == null) { + builder = DeleteFederationApplicationResponseProto.newBuilder(proto); + } + viaProto = false; + } + + @Override + public void setMessage(String msg) { + maybeInitBuilder(); + if (msg == null) { + builder.clearMessage(); + return; + } + builder.setMessage(msg); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/Graph.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/Graph.java index 66ccafa92a516..1d240347309ca 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/Graph.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/Graph.java @@ -20,7 +20,7 @@ import java.io.FileOutputStream; import java.io.IOException; import java.io.OutputStreamWriter; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.HashSet; import java.util.List; @@ -190,7 +190,7 @@ public String generateGraphViz() { public void save(String filepath) throws IOException { try (OutputStreamWriter fout = new OutputStreamWriter( - new FileOutputStream(filepath), Charset.forName("UTF-8"))) { + new FileOutputStream(filepath), StandardCharsets.UTF_8)) { fout.write(generateGraphViz()); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/DockerClientConfigHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/DockerClientConfigHandler.java index b1a186e8a8767..6351cb69c82e7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/DockerClientConfigHandler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/DockerClientConfigHandler.java @@ -39,7 +39,6 @@ import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; -import java.nio.charset.Charset; import java.nio.charset.StandardCharsets; import java.util.Iterator; @@ -115,7 +114,7 @@ public static Credentials readCredentialsFromConfigFile(Path configFile, new DockerCredentialTokenIdentifier(registryUrl, applicationId); Token token = new Token<>(tokenId.getBytes(), - registryCred.getBytes(Charset.forName("UTF-8")), + registryCred.getBytes(StandardCharsets.UTF_8), tokenId.getKind(), new Text(registryUrl)); credentials.addToken( new Text(registryUrl + "-" + applicationId), token); @@ -173,7 +172,7 @@ public static boolean writeDockerCredentialsToPath(File outConfigFile, ObjectNode registryCredNode = mapper.createObjectNode(); registryUrlNode.set(ti.getRegistryUrl(), registryCredNode); registryCredNode.put(CONFIG_AUTH_KEY, - new String(tk.getPassword(), Charset.forName("UTF-8"))); + new String(tk.getPassword(), StandardCharsets.UTF_8)); LOG.debug("Prepared token for write: {}", tk); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ProcfsBasedProcessTree.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ProcfsBasedProcessTree.java index 5a518dff7e537..ed48a92a04b48 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ProcfsBasedProcessTree.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ProcfsBasedProcessTree.java @@ -26,7 +26,7 @@ import java.io.IOException; import java.io.InputStreamReader; import java.math.BigInteger; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Collections; @@ -519,7 +519,7 @@ private static ProcessInfo constructProcessInfo(ProcessInfo pinfo, File pidDir = new File(procfsDir, pinfo.getPid()); fReader = new InputStreamReader( new FileInputStream( - new File(pidDir, PROCFS_STAT_FILE)), Charset.forName("UTF-8")); + new File(pidDir, PROCFS_STAT_FILE)), StandardCharsets.UTF_8); in = new BufferedReader(fReader); } catch (FileNotFoundException f) { // The process vanished in the interim! @@ -715,7 +715,7 @@ public String getCmdLine(String procfsDir) { fReader = new InputStreamReader( new FileInputStream( new File(new File(procfsDir, pid.toString()), PROCFS_CMDLINE_FILE)), - Charset.forName("UTF-8")); + StandardCharsets.UTF_8); } catch (FileNotFoundException f) { // The process vanished in the interim! return ret; @@ -773,7 +773,7 @@ private static void constructProcessSMAPInfo(ProcessTreeSmapMemInfo pInfo, return; } fReader = new InputStreamReader( - new FileInputStream(file), Charset.forName("UTF-8")); + new FileInputStream(file), StandardCharsets.UTF_8); in = new BufferedReader(fReader); ProcessSmapMemoryInfo memoryMappingInfo = null; List lines = IOUtils.readLines(in); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml index 0e317712f826c..b8892e4fc12ee 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml @@ -742,7 +742,11 @@ - Enable SSL/TLS encryption for the ZooKeeper communication. + + Enable SSL/TLS encryption for the ZooKeeper communication. + Note: if hadoop.zk.ssl.enabled is set to a value, then that central setting has precedence, + and this value will be overridden by the value of hadoop.zk.ssl.enabled. + yarn.resourcemanager.zk-client-ssl.enabled false diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/TestLogAggregationIndexedFileController.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/TestLogAggregationIndexedFileController.java index b7fcb18ff60bf..3e91ab5a54d6f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/TestLogAggregationIndexedFileController.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/TestLogAggregationIndexedFileController.java @@ -25,7 +25,7 @@ import java.io.PrintStream; import java.io.Writer; import java.net.URL; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; @@ -267,7 +267,7 @@ public boolean isRollover(final FileContext fc, final Path candidate) throws IOE fInput = FileSystem.create(fs, checksumFile, LOG_FILE_UMASK); fInput.writeInt(nodeName.length()); fInput.write(nodeName.getBytes( - Charset.forName("UTF-8"))); + StandardCharsets.UTF_8)); fInput.writeLong(0); } finally { IOUtils.closeStream(fInput); @@ -579,7 +579,7 @@ void testGetLogMetaFilesOfNode() throws Exception { fInput = FileSystem.create(fs, checksumFile, LOG_FILE_UMASK); fInput.writeInt(nodeName.length()); fInput.write(nodeName.getBytes( - Charset.forName("UTF-8"))); + StandardCharsets.UTF_8)); fInput.writeLong(0); } finally { IOUtils.closeStream(fInput); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/LeveldbTimelineStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/LeveldbTimelineStore.java index c4d8d4da33d7c..a55aa29332165 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/LeveldbTimelineStore.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/LeveldbTimelineStore.java @@ -48,7 +48,7 @@ import org.slf4j.LoggerFactory; import java.io.IOException; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.util.*; import java.util.Map.Entry; import java.util.concurrent.locks.ReentrantLock; @@ -127,25 +127,25 @@ public class LeveldbTimelineStore extends AbstractService //call LevelDb recovery static final String BACKUP_EXT = ".backup-"; - private static final byte[] START_TIME_LOOKUP_PREFIX = "k".getBytes(Charset.forName("UTF-8")); - private static final byte[] ENTITY_ENTRY_PREFIX = "e".getBytes(Charset.forName("UTF-8")); - private static final byte[] INDEXED_ENTRY_PREFIX = "i".getBytes(Charset.forName("UTF-8")); + private static final byte[] START_TIME_LOOKUP_PREFIX = "k".getBytes(StandardCharsets.UTF_8); + private static final byte[] ENTITY_ENTRY_PREFIX = "e".getBytes(StandardCharsets.UTF_8); + private static final byte[] INDEXED_ENTRY_PREFIX = "i".getBytes(StandardCharsets.UTF_8); - private static final byte[] EVENTS_COLUMN = "e".getBytes(Charset.forName("UTF-8")); - private static final byte[] PRIMARY_FILTERS_COLUMN = "f".getBytes(Charset.forName("UTF-8")); - private static final byte[] OTHER_INFO_COLUMN = "i".getBytes(Charset.forName("UTF-8")); - private static final byte[] RELATED_ENTITIES_COLUMN = "r".getBytes(Charset.forName("UTF-8")); + private static final byte[] EVENTS_COLUMN = "e".getBytes(StandardCharsets.UTF_8); + private static final byte[] PRIMARY_FILTERS_COLUMN = "f".getBytes(StandardCharsets.UTF_8); + private static final byte[] OTHER_INFO_COLUMN = "i".getBytes(StandardCharsets.UTF_8); + private static final byte[] RELATED_ENTITIES_COLUMN = "r".getBytes(StandardCharsets.UTF_8); private static final byte[] INVISIBLE_REVERSE_RELATED_ENTITIES_COLUMN = - "z".getBytes(Charset.forName("UTF-8")); - private static final byte[] DOMAIN_ID_COLUMN = "d".getBytes(Charset.forName("UTF-8")); + "z".getBytes(StandardCharsets.UTF_8); + private static final byte[] DOMAIN_ID_COLUMN = "d".getBytes(StandardCharsets.UTF_8); - private static final byte[] DOMAIN_ENTRY_PREFIX = "d".getBytes(Charset.forName("UTF-8")); - private static final byte[] OWNER_LOOKUP_PREFIX = "o".getBytes(Charset.forName("UTF-8")); - private static final byte[] DESCRIPTION_COLUMN = "d".getBytes(Charset.forName("UTF-8")); - private static final byte[] OWNER_COLUMN = "o".getBytes(Charset.forName("UTF-8")); - private static final byte[] READER_COLUMN = "r".getBytes(Charset.forName("UTF-8")); - private static final byte[] WRITER_COLUMN = "w".getBytes(Charset.forName("UTF-8")); - private static final byte[] TIMESTAMP_COLUMN = "t".getBytes(Charset.forName("UTF-8")); + private static final byte[] DOMAIN_ENTRY_PREFIX = "d".getBytes(StandardCharsets.UTF_8); + private static final byte[] OWNER_LOOKUP_PREFIX = "o".getBytes(StandardCharsets.UTF_8); + private static final byte[] DESCRIPTION_COLUMN = "d".getBytes(StandardCharsets.UTF_8); + private static final byte[] OWNER_COLUMN = "o".getBytes(StandardCharsets.UTF_8); + private static final byte[] READER_COLUMN = "r".getBytes(StandardCharsets.UTF_8); + private static final byte[] WRITER_COLUMN = "w".getBytes(StandardCharsets.UTF_8); + private static final byte[] TIMESTAMP_COLUMN = "t".getBytes(StandardCharsets.UTF_8); private static final byte[] EMPTY_BYTES = new byte[0]; @@ -456,7 +456,7 @@ private static TimelineEntity getEntity(String entityId, String entityType, } } else if (key[prefixlen] == DOMAIN_ID_COLUMN[0]) { byte[] v = iterator.peekNext().getValue(); - String domainId = new String(v, Charset.forName("UTF-8")); + String domainId = new String(v, StandardCharsets.UTF_8); entity.setDomainId(domainId); } else { if (key[prefixlen] != @@ -839,7 +839,7 @@ private void put(TimelineEntity entity, TimelinePutResponse response, if (domainIdBytes == null) { domainId = TimelineDataManager.DEFAULT_DOMAIN_ID; } else { - domainId = new String(domainIdBytes, Charset.forName("UTF-8")); + domainId = new String(domainIdBytes, StandardCharsets.UTF_8); } if (!domainId.equals(entity.getDomainId())) { // in this case the entity will be put, but the relation will be @@ -894,9 +894,9 @@ private void put(TimelineEntity entity, TimelinePutResponse response, return; } } else { - writeBatch.put(key, entity.getDomainId().getBytes(Charset.forName("UTF-8"))); + writeBatch.put(key, entity.getDomainId().getBytes(StandardCharsets.UTF_8)); writePrimaryFilterEntries(writeBatch, primaryFilters, key, - entity.getDomainId().getBytes(Charset.forName("UTF-8"))); + entity.getDomainId().getBytes(StandardCharsets.UTF_8)); } db.write(writeBatch); } catch (DBException de) { @@ -928,7 +928,7 @@ private void put(TimelineEntity entity, TimelinePutResponse response, // This is the new entity, the domain should be the same byte[] key = createDomainIdKey(relatedEntity.getId(), relatedEntity.getType(), relatedEntityStartTime); - db.put(key, entity.getDomainId().getBytes(Charset.forName("UTF-8"))); + db.put(key, entity.getDomainId().getBytes(StandardCharsets.UTF_8)); db.put(createRelatedEntityKey(relatedEntity.getId(), relatedEntity.getType(), relatedEntityStartTime, entity.getEntityId(), entity.getEntityType()), EMPTY_BYTES); @@ -1255,7 +1255,7 @@ private static byte[] createOtherInfoKey(String entityId, String entityType, * to the end of the array (for parsing other info keys). */ private static String parseRemainingKey(byte[] b, int offset) { - return new String(b, offset, b.length - offset, Charset.forName("UTF-8")); + return new String(b, offset, b.length - offset, StandardCharsets.UTF_8); } /** @@ -1629,9 +1629,9 @@ public void put(TimelineDomain domain) throws IOException { domain.getOwner(), domain.getId(), DESCRIPTION_COLUMN); if (domain.getDescription() != null) { writeBatch.put(domainEntryKey, domain.getDescription(). - getBytes(Charset.forName("UTF-8"))); + getBytes(StandardCharsets.UTF_8)); writeBatch.put(ownerLookupEntryKey, domain.getDescription(). - getBytes(Charset.forName("UTF-8"))); + getBytes(StandardCharsets.UTF_8)); } else { writeBatch.put(domainEntryKey, EMPTY_BYTES); writeBatch.put(ownerLookupEntryKey, EMPTY_BYTES); @@ -1642,17 +1642,17 @@ public void put(TimelineDomain domain) throws IOException { ownerLookupEntryKey = createOwnerLookupKey( domain.getOwner(), domain.getId(), OWNER_COLUMN); // Null check for owner is done before - writeBatch.put(domainEntryKey, domain.getOwner().getBytes(Charset.forName("UTF-8"))); - writeBatch.put(ownerLookupEntryKey, domain.getOwner().getBytes(Charset.forName("UTF-8"))); + writeBatch.put(domainEntryKey, domain.getOwner().getBytes(StandardCharsets.UTF_8)); + writeBatch.put(ownerLookupEntryKey, domain.getOwner().getBytes(StandardCharsets.UTF_8)); // Write readers domainEntryKey = createDomainEntryKey(domain.getId(), READER_COLUMN); ownerLookupEntryKey = createOwnerLookupKey( domain.getOwner(), domain.getId(), READER_COLUMN); if (domain.getReaders() != null && domain.getReaders().length() > 0) { - writeBatch.put(domainEntryKey, domain.getReaders().getBytes(Charset.forName("UTF-8"))); + writeBatch.put(domainEntryKey, domain.getReaders().getBytes(StandardCharsets.UTF_8)); writeBatch.put(ownerLookupEntryKey, domain.getReaders(). - getBytes(Charset.forName("UTF-8"))); + getBytes(StandardCharsets.UTF_8)); } else { writeBatch.put(domainEntryKey, EMPTY_BYTES); writeBatch.put(ownerLookupEntryKey, EMPTY_BYTES); @@ -1663,9 +1663,9 @@ public void put(TimelineDomain domain) throws IOException { ownerLookupEntryKey = createOwnerLookupKey( domain.getOwner(), domain.getId(), WRITER_COLUMN); if (domain.getWriters() != null && domain.getWriters().length() > 0) { - writeBatch.put(domainEntryKey, domain.getWriters().getBytes(Charset.forName("UTF-8"))); + writeBatch.put(domainEntryKey, domain.getWriters().getBytes(StandardCharsets.UTF_8)); writeBatch.put(ownerLookupEntryKey, domain.getWriters(). - getBytes(Charset.forName("UTF-8"))); + getBytes(StandardCharsets.UTF_8)); } else { writeBatch.put(domainEntryKey, EMPTY_BYTES); writeBatch.put(ownerLookupEntryKey, EMPTY_BYTES); @@ -1802,13 +1802,13 @@ private static TimelineDomain getTimelineDomain( byte[] value = iterator.peekNext().getValue(); if (value != null && value.length > 0) { if (key[prefix.length] == DESCRIPTION_COLUMN[0]) { - domain.setDescription(new String(value, Charset.forName("UTF-8"))); + domain.setDescription(new String(value, StandardCharsets.UTF_8)); } else if (key[prefix.length] == OWNER_COLUMN[0]) { - domain.setOwner(new String(value, Charset.forName("UTF-8"))); + domain.setOwner(new String(value, StandardCharsets.UTF_8)); } else if (key[prefix.length] == READER_COLUMN[0]) { - domain.setReaders(new String(value, Charset.forName("UTF-8"))); + domain.setReaders(new String(value, StandardCharsets.UTF_8)); } else if (key[prefix.length] == WRITER_COLUMN[0]) { - domain.setWriters(new String(value, Charset.forName("UTF-8"))); + domain.setWriters(new String(value, StandardCharsets.UTF_8)); } else if (key[prefix.length] == TIMESTAMP_COLUMN[0]) { domain.setCreatedTime(readReverseOrderedLong(value, 0)); domain.setModifiedTime(readReverseOrderedLong(value, 8)); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/lib/ZKClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/lib/ZKClient.java index 21cbe20ab486f..c0e877b4c7edb 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/lib/ZKClient.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/lib/ZKClient.java @@ -19,7 +19,7 @@ package org.apache.hadoop.yarn.lib; import java.io.IOException; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.util.List; import org.apache.zookeeper.CreateMode; @@ -57,7 +57,7 @@ public ZKClient(String string) throws IOException { public void registerService(String path, String data) throws IOException, InterruptedException { try { - zkClient.create(path, data.getBytes(Charset.forName("UTF-8")), + zkClient.create(path, data.getBytes(StandardCharsets.UTF_8), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL); } catch(KeeperException ke) { throw new IOException(ke); @@ -114,7 +114,7 @@ public String getServiceData(String path) throws IOException, try { Stat stat = new Stat(); byte[] byteData = zkClient.getData(path, false, stat); - data = new String(byteData, Charset.forName("UTF-8")); + data = new String(byteData, StandardCharsets.UTF_8); } catch(KeeperException ke) { throw new IOException(ke); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/WeightedHomePolicyManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/WeightedHomePolicyManager.java new file mode 100644 index 0000000000000..370594ec32d36 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/WeightedHomePolicyManager.java @@ -0,0 +1,65 @@ +/** + * 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.yarn.server.federation.policies.manager; + +import org.apache.hadoop.classification.VisibleForTesting; +import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.HomeAMRMProxyPolicy; +import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo; +import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException; +import org.apache.hadoop.yarn.server.federation.policies.router.WeightedRandomRouterPolicy; +import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration; + +import java.nio.ByteBuffer; + +/** + * Policy that allows operator to configure "weights" for routing. This picks a + * {@link WeightedRandomRouterPolicy} for the router and a {@link + * HomeAMRMProxyPolicy} for the amrmproxy as they are designed to + * work together. + */ +public class WeightedHomePolicyManager extends AbstractPolicyManager { + + private WeightedPolicyInfo weightedPolicyInfo; + + public WeightedHomePolicyManager() { + // this structurally hard-codes two compatible policies for Router and + // AMRMProxy. + routerFederationPolicy = WeightedRandomRouterPolicy.class; + amrmProxyFederationPolicy = HomeAMRMProxyPolicy.class; + weightedPolicyInfo = new WeightedPolicyInfo(); + } + + @Override + public SubClusterPolicyConfiguration serializeConf() + throws FederationPolicyInitializationException { + ByteBuffer buf = weightedPolicyInfo.toByteBuffer(); + return SubClusterPolicyConfiguration + .newInstance(getQueue(), this.getClass().getCanonicalName(), buf); + } + + @VisibleForTesting + public WeightedPolicyInfo getWeightedPolicyInfo() { + return weightedPolicyInfo; + } + + @VisibleForTesting + public void setWeightedPolicyInfo( + WeightedPolicyInfo weightedPolicyInfo) { + this.weightedPolicyInfo = weightedPolicyInfo; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/LogWebServiceUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/LogWebServiceUtils.java index c4568be8011a7..f396c29a3396e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/LogWebServiceUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/LogWebServiceUtils.java @@ -46,7 +46,7 @@ import java.io.IOException; import java.io.OutputStream; import java.lang.reflect.UndeclaredThrowableException; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.util.HashSet; import java.util.Set; @@ -120,7 +120,7 @@ private static StreamingOutput getStreamingOutput( .readAggregatedLogs(request, os); if (!findLogs) { os.write(("Can not find logs for container:" + containerIdStr) - .getBytes(Charset.forName("UTF-8"))); + .getBytes(StandardCharsets.UTF_8)); } else { if (printEmptyLocalContainerLog) { StringBuilder sb = new StringBuilder(); @@ -129,7 +129,7 @@ private static StreamingOutput getStreamingOutput( + "\n"); sb.append("LogContents:\n"); sb.append(getNoRedirectWarning() + "\n"); - os.write(sb.toString().getBytes(Charset.forName("UTF-8"))); + os.write(sb.toString().getBytes(StandardCharsets.UTF_8)); } } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java index ed66dadefb497..0823660bc0404 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java @@ -181,6 +181,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.BatchSaveFederationQueuePoliciesResponse; import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesRequest; import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesResponse; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationRequest; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationResponse; import org.apache.hadoop.thirdparty.com.google.common.base.Strings; /** @@ -988,6 +990,12 @@ public QueryFederationQueuePoliciesResponse listFederationQueuePolicies( return null; } + @Override + public DeleteFederationApplicationResponse deleteFederationApplication( + DeleteFederationApplicationRequest request) throws YarnException, IOException { + return null; + } + @VisibleForTesting public HashMap> getApplicationContainerIdMap() { return applicationContainerIdMap; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/manager/TestWeightedHomePolicyManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/manager/TestWeightedHomePolicyManager.java new file mode 100644 index 0000000000000..3d0c5895f42d0 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/manager/TestWeightedHomePolicyManager.java @@ -0,0 +1,63 @@ +/** + * 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.yarn.server.federation.policies.manager; + +import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.HomeAMRMProxyPolicy; +import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo; +import org.apache.hadoop.yarn.server.federation.policies.router.WeightedRandomRouterPolicy; +import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId; +import org.apache.hadoop.yarn.server.federation.store.records.SubClusterIdInfo; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.HashMap; +import java.util.Map; + +public class TestWeightedHomePolicyManager extends BasePolicyManagerTest { + private WeightedPolicyInfo policyInfo; + + @Before + public void setup() { + // configure a policy + WeightedHomePolicyManager whpm = new WeightedHomePolicyManager(); + whpm.setQueue("queue1"); + + SubClusterId sc1 = SubClusterId.newInstance("sc1"); + policyInfo = new WeightedPolicyInfo(); + Map routerWeights = new HashMap<>(); + routerWeights.put(new SubClusterIdInfo(sc1), 0.2f); + policyInfo.setRouterPolicyWeights(routerWeights); + + whpm.setWeightedPolicyInfo(policyInfo); + this.wfp = whpm; + + //set expected params that the base test class will use for tests + expectedPolicyManager = WeightedHomePolicyManager.class; + expectedAMRMProxyPolicy = HomeAMRMProxyPolicy.class; + expectedRouterPolicy = WeightedRandomRouterPolicy.class; + } + + @Test + public void testPolicyInfoSetCorrectly() throws Exception { + serializeAndDeserializePolicyManager(wfp, expectedPolicyManager, + expectedAMRMProxyPolicy, expectedRouterPolicy); + // check the policyInfo propagates through ser/der correctly + Assert.assertEquals(((WeightedHomePolicyManager) wfp) + .getWeightedPolicyInfo(), policyInfo); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java index a28a6fc4110e9..19335045c865b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java @@ -50,6 +50,7 @@ import java.io.IOException; import java.io.InterruptedIOException; import java.net.InetSocketAddress; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -1064,7 +1065,7 @@ public synchronized void updateYarnSysFS(Context ctx, String user, if (file.createNewFile()) { FileOutputStream output = new FileOutputStream(file); try { - output.write(spec.getBytes("UTF-8")); + output.write(spec.getBytes(StandardCharsets.UTF_8)); } finally { output.close(); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/WindowsSecureContainerExecutor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/WindowsSecureContainerExecutor.java index 9d57f8fff4fe5..78ba39ef69380 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/WindowsSecureContainerExecutor.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/WindowsSecureContainerExecutor.java @@ -29,7 +29,7 @@ import java.io.PrintStream; import java.net.InetSocketAddress; import java.net.URISyntaxException; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -502,7 +502,7 @@ private Thread startStreamReader(final InputStream stream) @Override public void run() { try (BufferedReader lines = new BufferedReader( - new InputStreamReader(stream, Charset.forName("UTF-8")))) { + new InputStreamReader(stream, StandardCharsets.UTF_8))) { char[] buf = new char[512]; int nRead; while ((nRead = lines.read(buf, 0, buf.length)) > 0) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsResourceCalculator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsResourceCalculator.java index 2267cf50edef9..f5e987deee074 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsResourceCalculator.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsResourceCalculator.java @@ -35,7 +35,7 @@ import java.io.IOException; import java.io.InputStreamReader; import java.math.BigInteger; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.util.function.Function; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -316,7 +316,7 @@ private void processFile(File file, Function processLine) throws YarnException { // Read "procfsDir//stat" file - typically /proc//stat try (InputStreamReader fReader = new InputStreamReader( - new FileInputStream(file), Charset.forName("UTF-8"))) { + new FileInputStream(file), StandardCharsets.UTF_8)) { try (BufferedReader in = new BufferedReader(fReader)) { try { String str; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerLogsPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerLogsPage.java index 2769788569864..d0acc9a0bb4bf 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerLogsPage.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerLogsPage.java @@ -28,7 +28,7 @@ import java.io.FileInputStream; import java.io.IOException; import java.io.InputStreamReader; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.text.ParseException; import java.text.SimpleDateFormat; import java.util.ArrayList; @@ -214,7 +214,7 @@ private void printLocalLogFile(Block html, File logFile) { IOUtils.skipFully(logByteStream, start); InputStreamReader reader = - new InputStreamReader(logByteStream, Charset.forName("UTF-8")); + new InputStreamReader(logByteStream, StandardCharsets.UTF_8); int bufferSize = 65536; char[] cbuf = new char[bufferSize]; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerShellWebSocket.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerShellWebSocket.java index 138f9e0b02ec9..175ee09f8db88 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerShellWebSocket.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerShellWebSocket.java @@ -20,7 +20,7 @@ import java.io.IOException; import java.net.URI; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.util.List; import java.util.Map; @@ -77,7 +77,7 @@ public void onText(Session session, String message) throws IOException { if (!message.equals("1{}")) { // Send keystroke to process input byte[] payload; - payload = message.getBytes(Charset.forName("UTF-8")); + payload = message.getBytes(StandardCharsets.UTF_8); if (payload != null) { pair.out.write(payload); pair.out.flush(); @@ -86,7 +86,7 @@ public void onText(Session session, String message) throws IOException { // Render process output int no = pair.in.available(); pair.in.read(buffer, 0, Math.min(no, buffer.length)); - String formatted = new String(buffer, Charset.forName("UTF-8")) + String formatted = new String(buffer, StandardCharsets.UTF_8) .replaceAll("\n", "\r\n"); session.getRemote().sendString(formatted); } @@ -142,7 +142,7 @@ public void onClose(Session session, int status, String reason) { try { LOG.info(session.getRemoteAddress().getHostString() + " closed!"); String exit = "exit\r\n"; - pair.out.write(exit.getBytes(Charset.forName("UTF-8"))); + pair.out.write(exit.getBytes(StandardCharsets.UTF_8)); pair.out.flush(); pair.in.close(); pair.out.close(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebServices.java index 3cf9f659099d1..5ec584f36d5c6 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebServices.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebServices.java @@ -21,7 +21,7 @@ import java.io.FileInputStream; import java.io.IOException; import java.io.OutputStream; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.security.Principal; import java.util.ArrayList; import java.util.Collection; @@ -490,7 +490,7 @@ public void write(OutputStream os) throws IOException, } sb.append(StringUtils.repeat("*", endOfFile.length() + 50) + "\n\n"); - os.write(sb.toString().getBytes(Charset.forName("UTF-8"))); + os.write(sb.toString().getBytes(StandardCharsets.UTF_8)); // If we have aggregated logs for this container, // output the aggregation logs as well. ApplicationId appId = containerId.getApplicationAttemptId() diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestAuxServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestAuxServices.java index 0d67c13b64734..fd82126b7e830 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestAuxServices.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestAuxServices.java @@ -47,12 +47,12 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; import java.io.File; import java.io.IOException; import java.net.URL; import java.net.URLClassLoader; import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Paths; import java.nio.file.attribute.FileTime; @@ -426,7 +426,7 @@ public void testCustomizedAuxServiceClassPath() throws Exception { Assert.assertTrue(meta.size() == 1); for(Entry i : meta.entrySet()) { auxName = i.getKey(); - String auxClassPath = Charsets.UTF_8.decode(i.getValue()).toString(); + String auxClassPath = StandardCharsets.UTF_8.decode(i.getValue()).toString(); defaultAuxClassPath = new HashSet(Arrays.asList(StringUtils .getTrimmedStrings(auxClassPath))); } @@ -478,7 +478,7 @@ public void testCustomizedAuxServiceClassPath() throws Exception { Set customizedAuxClassPath = null; for(Entry i : meta.entrySet()) { Assert.assertTrue(auxName.equals(i.getKey())); - String classPath = Charsets.UTF_8.decode(i.getValue()).toString(); + String classPath = StandardCharsets.UTF_8.decode(i.getValue()).toString(); customizedAuxClassPath = new HashSet(Arrays.asList(StringUtils .getTrimmedStrings(classPath))); Assert.assertTrue(classPath.contains(testJar.getName())); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java index 6a9dd6b7476bf..7c4815ff94b6f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java @@ -35,7 +35,6 @@ import java.io.PrintStream; import java.io.PrintWriter; import java.nio.ByteBuffer; -import java.nio.charset.Charset; import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Paths; @@ -218,7 +217,7 @@ public void testSpecialCharSymlinks() throws IOException { //Capture output from prelaunch.out List output = Files.readAllLines(Paths.get(localLogDir.getAbsolutePath(), ContainerLaunch.CONTAINER_PRE_LAUNCH_STDOUT), - Charset.forName("UTF-8")); + StandardCharsets.UTF_8); assert(output.contains("hello")); symLinkFile = new File(tmpDir, badSymlink); @@ -549,7 +548,7 @@ public void testInvalidEnvSyntaxDiagnostics() throws IOException { } catch(ExitCodeException e){ //Capture diagnostics from prelaunch.stderr List error = Files.readAllLines(Paths.get(localLogDir.getAbsolutePath(), ContainerLaunch.CONTAINER_PRE_LAUNCH_STDERR), - Charset.forName("UTF-8")); + StandardCharsets.UTF_8); diagnostics = StringUtils.join("\n", error); } Assert.assertTrue(diagnostics.contains(Shell.WINDOWS ? diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestTrafficController.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestTrafficController.java index 63c654463f0b0..33b8434c9a8cb 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestTrafficController.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestTrafficController.java @@ -36,7 +36,7 @@ import java.io.File; import java.io.IOException; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.util.Arrays; import java.util.List; @@ -120,7 +120,7 @@ private void verifyTrafficControlOperation(PrivilegedOperation op, Assert.assertTrue(tcCmdsFile.exists()); List tcCmds = Files.readAllLines(tcCmdsFile.toPath(), - Charset.forName("UTF-8")); + StandardCharsets.UTF_8); //Verify that the number of commands is the same as expected and verify //that each command is the same, in sequence diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java index ea7c213809330..c5f508778f01d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java @@ -76,7 +76,7 @@ import java.io.FileWriter; import java.io.IOException; import java.nio.ByteBuffer; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Paths; import java.nio.file.attribute.FileAttribute; @@ -1992,7 +1992,7 @@ private List getDockerCommandsForDockerStop( PrivilegedOperation.OperationType.RUN_DOCKER_CMD); String dockerCommandFile = op.getArguments().get(0); return Files.readAllLines(Paths.get(dockerCommandFile), - Charset.forName("UTF-8")); + StandardCharsets.UTF_8); } private List getDockerCommandsForSignal( @@ -2471,7 +2471,7 @@ public void testLaunchContainer(ByteBuffer tokens, File dockerConfigFile) String dockerCommandFile = args.get(argsCounter++); List dockerCommands = Files - .readAllLines(Paths.get(dockerCommandFile), Charset.forName("UTF-8")); + .readAllLines(Paths.get(dockerCommandFile), StandardCharsets.UTF_8); int expected = 14; int counter = 0; @@ -2617,7 +2617,7 @@ private List readDockerCommands(int invocations) throws IOException, String dockerCommandFile = args.get((https) ? 14 : 12); List dockerCommands = Files.readAllLines( - Paths.get(dockerCommandFile), Charset.forName("UTF-8")); + Paths.get(dockerCommandFile), StandardCharsets.UTF_8); return dockerCommands; } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/TestDockerCommandExecutor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/TestDockerCommandExecutor.java index dcba179a28cc7..e5737d9246cec 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/TestDockerCommandExecutor.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/TestDockerCommandExecutor.java @@ -37,7 +37,7 @@ import org.junit.Test; import java.io.IOException; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Paths; import java.util.ArrayList; @@ -408,7 +408,7 @@ private List getValidatedDockerCommands( String dockerCommandFile = op.getArguments().get(0); List dockerCommandFileContents = Files .readAllLines(Paths.get(dockerCommandFile), - Charset.forName("UTF-8")); + StandardCharsets.UTF_8); dockerCommands.addAll(dockerCommandFileContents); } return dockerCommands; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/dao/gpu/TestGpuDeviceInformationParser.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/dao/gpu/TestGpuDeviceInformationParser.java index 28e06b72bc4b4..49c4e2cbb1862 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/dao/gpu/TestGpuDeviceInformationParser.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/dao/gpu/TestGpuDeviceInformationParser.java @@ -26,12 +26,12 @@ import java.io.File; import java.io.IOException; +import java.nio.charset.StandardCharsets; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; public class TestGpuDeviceInformationParser { - private static final String UTF_8 = "UTF-8"; private static final double DELTA = 1e-6; @Rule @@ -40,7 +40,7 @@ public class TestGpuDeviceInformationParser { @Test public void testParse() throws IOException, YarnException { File f = new File("src/test/resources/nvidia-smi-sample-output.xml"); - String s = FileUtils.readFileToString(f, UTF_8); + String s = FileUtils.readFileToString(f, StandardCharsets.UTF_8); GpuDeviceInformationParser parser = new GpuDeviceInformationParser(); GpuDeviceInformation info = parser.parseXml(s); @@ -54,7 +54,7 @@ public void testParse() throws IOException, YarnException { @Test public void testParseExcerpt() throws IOException, YarnException { File f = new File("src/test/resources/nvidia-smi-output-excerpt.xml"); - String s = FileUtils.readFileToString(f, UTF_8); + String s = FileUtils.readFileToString(f, StandardCharsets.UTF_8); GpuDeviceInformationParser parser = new GpuDeviceInformationParser(); GpuDeviceInformation info = parser.parseXml(s); @@ -69,7 +69,7 @@ public void testParseExcerpt() throws IOException, YarnException { public void testParseConsecutivelyWithSameParser() throws IOException, YarnException { File f = new File("src/test/resources/nvidia-smi-sample-output.xml"); - String s = FileUtils.readFileToString(f, UTF_8); + String s = FileUtils.readFileToString(f, StandardCharsets.UTF_8); for (int i = 0; i < 3; i++) { GpuDeviceInformationParser parser = new GpuDeviceInformationParser(); @@ -99,7 +99,7 @@ public void testParseInvalidRootElement() throws YarnException { @Test public void testParseMissingTags() throws IOException, YarnException { File f = new File("src/test/resources/nvidia-smi-output-missing-tags.xml"); - String s = FileUtils.readFileToString(f, UTF_8); + String s = FileUtils.readFileToString(f, StandardCharsets.UTF_8); GpuDeviceInformationParser parser = new GpuDeviceInformationParser(); GpuDeviceInformation info = parser.parseXml(s); @@ -119,7 +119,7 @@ public void testParseMissingTags() throws IOException, YarnException { @Test public void testParseMissingInnerTags() throws IOException, YarnException { File f =new File("src/test/resources/nvidia-smi-output-missing-tags2.xml"); - String s = FileUtils.readFileToString(f, UTF_8); + String s = FileUtils.readFileToString(f, StandardCharsets.UTF_8); GpuDeviceInformationParser parser = new GpuDeviceInformationParser(); GpuDeviceInformation info = parser.parseXml(s); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ActiveStandbyElectorBasedElectorService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ActiveStandbyElectorBasedElectorService.java index 989c9c53cd0a9..024bfeffdea2e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ActiveStandbyElectorBasedElectorService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ActiveStandbyElectorBasedElectorService.java @@ -105,8 +105,10 @@ protected void serviceInit(Configuration conf) conf.getInt(YarnConfiguration.RM_HA_FC_ELECTOR_ZK_RETRIES_KEY, conf .getInt(CommonConfigurationKeys.HA_FC_ELECTOR_ZK_OP_RETRIES_KEY, CommonConfigurationKeys.HA_FC_ELECTOR_ZK_OP_RETRIES_DEFAULT)); - boolean isSSLEnabled = conf.getBoolean(YarnConfiguration.RM_ZK_CLIENT_SSL_ENABLED, - YarnConfiguration.DEFAULT_RM_ZK_CLIENT_SSL_ENABLED); + boolean isSSLEnabled = + conf.getBoolean(CommonConfigurationKeys.ZK_CLIENT_SSL_ENABLED, + conf.getBoolean(YarnConfiguration.RM_ZK_CLIENT_SSL_ENABLED, + YarnConfiguration.DEFAULT_RM_ZK_CLIENT_SSL_ENABLED)); SecurityUtil.TruststoreKeystore truststoreKeystore = isSSLEnabled ? new SecurityUtil.TruststoreKeystore(conf) : null; elector = new ActiveStandbyElector(zkQuorum, (int) zkSessionTimeout, diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java index 8f9e8caa53670..c45c34dc85c80 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java @@ -103,6 +103,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.BatchSaveFederationQueuePoliciesResponse; import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesRequest; import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesResponse; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationRequest; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationResponse; import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NodeLabelsUtils; import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSystem; import org.apache.hadoop.yarn.server.resourcemanager.resource.DynamicResourceConfiguration; @@ -1105,6 +1107,14 @@ public QueryFederationQueuePoliciesResponse listFederationQueuePolicies( " Please call Router's listFederationQueuePolicies to list Policies."); } + @Override + public DeleteFederationApplicationResponse deleteFederationApplication( + DeleteFederationApplicationRequest request) throws YarnException, IOException { + throw new YarnException("It is not allowed to call the RM's " + + " deleteFederationApplication. " + + " Please call Router's deleteFederationApplication to delete Application."); + } + private void validateAttributesExists( List nodesToAttributes) throws IOException { NodeAttributesManager nodeAttributesManager = diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAuditLogger.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAuditLogger.java index 1b3a49433fd20..4d81940198372 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAuditLogger.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAuditLogger.java @@ -17,8 +17,8 @@ */ package org.apache.hadoop.yarn.server.resourcemanager; -import java.io.UnsupportedEncodingException; import java.net.InetAddress; +import java.nio.charset.StandardCharsets; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -160,12 +160,8 @@ private static void appendCallerContext(StringBuilder sb, CallerContext callerCo } if (signature != null) { - try { - String sigStr = new String(signature, "UTF-8"); - add(Keys.CALLERSIGNATURE, sigStr, sb); - } catch (UnsupportedEncodingException e) { - // ignore this signature - } + String sigStr = new String(signature, StandardCharsets.UTF_8); + add(Keys.CALLERSIGNATURE, sigStr, sb); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java index f358a22cee715..a3d529c1020a3 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java @@ -19,6 +19,7 @@ package org.apache.hadoop.yarn.server.resourcemanager; import org.apache.commons.lang3.math.NumberUtils; +import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.hadoop.classification.VisibleForTesting; import com.sun.jersey.spi.container.servlet.ServletContainer; @@ -148,7 +149,7 @@ import java.net.InetSocketAddress; import java.net.URI; import java.net.URL; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.security.PrivilegedExceptionAction; import java.security.SecureRandom; import java.util.ArrayList; @@ -421,14 +422,17 @@ public ZKCuratorManager createAndStartZKManager(Configuration String defaultFencingAuth = zkRootNodeUsername + ":" + zkRootNodePassword; byte[] defaultFencingAuthData = - defaultFencingAuth.getBytes(Charset.forName("UTF-8")); + defaultFencingAuth.getBytes(StandardCharsets.UTF_8); String scheme = new DigestAuthenticationProvider().getScheme(); AuthInfo authInfo = new AuthInfo(scheme, defaultFencingAuthData); authInfos.add(authInfo); } - manager.start(authInfos, config.getBoolean(YarnConfiguration.RM_ZK_CLIENT_SSL_ENABLED, - YarnConfiguration.DEFAULT_RM_ZK_CLIENT_SSL_ENABLED)); + boolean isSSLEnabled = + config.getBoolean(CommonConfigurationKeys.ZK_CLIENT_SSL_ENABLED, + config.getBoolean(YarnConfiguration.RM_ZK_CLIENT_SSL_ENABLED, + YarnConfiguration.DEFAULT_RM_ZK_CLIENT_SSL_ENABLED)); + manager.start(authInfos, isSSLEnabled); return manager; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/invariants/MetricsInvariantChecker.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/invariants/MetricsInvariantChecker.java index 2ea44d2aa67fb..756c0b7d27cf7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/invariants/MetricsInvariantChecker.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/invariants/MetricsInvariantChecker.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.yarn.server.resourcemanager.monitor.invariants; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; import org.apache.hadoop.thirdparty.com.google.common.io.Files; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.metrics2.AbstractMetric; @@ -39,6 +38,7 @@ import javax.script.SimpleBindings; import java.io.File; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -113,7 +113,7 @@ public void init(Configuration config, RMContext rmContext, StringBuilder sb = new StringBuilder(); try { List tempInv = - Files.readLines(new File(invariantFile), Charsets.UTF_8); + Files.readLines(new File(invariantFile), StandardCharsets.UTF_8); boolean first = true; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsModification.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsModification.java index be5494a43d804..650e83983b7d9 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsModification.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsModification.java @@ -30,6 +30,7 @@ import java.io.StringReader; import java.io.StringWriter; import java.net.URI; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -925,7 +926,7 @@ public void testAppSubmit(String acceptMedia, String contentMedia) Text key = new Text("secret1"); assertTrue("Secrets missing from credentials object", cs .getAllSecretKeys().contains(key)); - assertEquals("mysecret", new String(cs.getSecretKey(key), "UTF-8")); + assertEquals("mysecret", new String(cs.getSecretKey(key), StandardCharsets.UTF_8)); // Check LogAggregationContext ApplicationSubmissionContext asc = app.getApplicationSubmissionContext(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/RouterMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/RouterMetrics.java index d0e4825fed39b..66157797a936e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/RouterMetrics.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/RouterMetrics.java @@ -155,6 +155,8 @@ public final class RouterMetrics { private MutableGaugeInt numBatchSaveFederationQueuePoliciesFailedRetrieved; @Metric("# of listFederationQueuePolicies failed to be retrieved") private MutableGaugeInt numListFederationQueuePoliciesFailedRetrieved; + @Metric("# of deleteFederationApplication failed to be retrieved") + private MutableGaugeInt numDeleteFederationApplicationFailedRetrieved; @Metric("# of refreshAdminAcls failed to be retrieved") private MutableGaugeInt numRefreshAdminAclsFailedRetrieved; @Metric("# of refreshServiceAcls failed to be retrieved") @@ -307,6 +309,8 @@ public final class RouterMetrics { private MutableRate totalSucceededBatchSaveFederationQueuePoliciesRetrieved; @Metric("Total number of successful Retrieved ListFederationQueuePolicies and latency(ms)") private MutableRate totalSucceededListFederationQueuePoliciesFailedRetrieved; + @Metric("Total number of successful Retrieved DeleteFederationApplication and latency(ms)") + private MutableRate totalSucceededDeleteFederationApplicationFailedRetrieved; @Metric("Total number of successful Retrieved RefreshAdminAcls and latency(ms)") private MutableRate totalSucceededRefreshAdminAclsRetrieved; @Metric("Total number of successful Retrieved RefreshServiceAcls and latency(ms)") @@ -396,6 +400,7 @@ public final class RouterMetrics { private MutableQuantiles saveFederationQueuePolicyLatency; private MutableQuantiles batchSaveFederationQueuePoliciesLatency; private MutableQuantiles listFederationQueuePoliciesLatency; + private MutableQuantiles deleteFederationApplicationLatency; private MutableQuantiles refreshAdminAclsLatency; private MutableQuantiles refreshServiceAclsLatency; private MutableQuantiles replaceLabelsOnNodesLatency; @@ -618,6 +623,10 @@ private RouterMetrics() { "listFederationQueuePoliciesLatency", "latency of list federationqueuepolicies timeouts", "ops", "latency", 10); + deleteFederationApplicationLatency = registry.newQuantiles( + "deleteFederationApplicationLatency", + "latency of delete FederationApplication timeouts", "ops", "latency", 10); + refreshAdminAclsLatency = registry.newQuantiles("refreshAdminAclsLatency", "latency of refresh admin acls timeouts", "ops", "latency", 10); @@ -962,6 +971,11 @@ public long getNumSucceededListFederationQueuePoliciesFailedRetrieved() { return totalSucceededListFederationQueuePoliciesFailedRetrieved.lastStat().numSamples(); } + @VisibleForTesting + public long getNumSucceededDeleteFederationApplicationFailedRetrieved() { + return totalSucceededDeleteFederationApplicationFailedRetrieved.lastStat().numSamples(); + } + @VisibleForTesting public long getNumSucceededRefreshAdminAclsRetrieved() { return totalSucceededRefreshAdminAclsRetrieved.lastStat().numSamples(); @@ -1322,6 +1336,11 @@ public double getLatencySucceededListFederationQueuePoliciesRetrieved() { return totalSucceededListFederationQueuePoliciesFailedRetrieved.lastStat().mean(); } + @VisibleForTesting + public double getLatencySucceededDeleteFederationApplicationFailedRetrieved() { + return totalSucceededDeleteFederationApplicationFailedRetrieved.lastStat().mean(); + } + @VisibleForTesting public double getLatencySucceededRefreshAdminAclsRetrieved() { return totalSucceededRefreshAdminAclsRetrieved.lastStat().mean(); @@ -1629,6 +1648,10 @@ public int getListFederationQueuePoliciesFailedRetrieved() { return numListFederationQueuePoliciesFailedRetrieved.value(); } + public int getDeleteFederationApplicationFailedRetrieved() { + return numDeleteFederationApplicationFailedRetrieved.value(); + } + public int getNumRefreshAdminAclsFailedRetrieved() { return numRefreshAdminAclsFailedRetrieved.value(); } @@ -1996,6 +2019,11 @@ public void succeededListFederationQueuePoliciesRetrieved(long duration) { listFederationQueuePoliciesLatency.add(duration); } + public void succeededDeleteFederationApplicationFailedRetrieved(long duration) { + totalSucceededDeleteFederationApplicationFailedRetrieved.add(duration); + deleteFederationApplicationLatency.add(duration); + } + public void succeededRefreshAdminAclsRetrieved(long duration) { totalSucceededRefreshAdminAclsRetrieved.add(duration); refreshAdminAclsLatency.add(duration); @@ -2286,6 +2314,10 @@ public void incrListFederationQueuePoliciesFailedRetrieved() { numListFederationQueuePoliciesFailedRetrieved.incr(); } + public void incrDeleteFederationApplicationFailedRetrieved() { + numDeleteFederationApplicationFailedRetrieved.incr(); + } + public void incrRefreshAdminAclsFailedRetrieved() { numRefreshAdminAclsFailedRetrieved.incr(); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/RouterServerUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/RouterServerUtil.java index 992636cc023da..744ddc87050d1 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/RouterServerUtil.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/RouterServerUtil.java @@ -105,8 +105,9 @@ public static void logAndThrowException(Throwable t, String errMsgFormat, Object throws YarnException { String msg = String.format(errMsgFormat, args); if (t != null) { - LOG.error(msg, t); - throw new YarnException(msg, t); + String newErrMsg = getErrorMsg(msg, t); + LOG.error(newErrMsg, t); + throw new YarnException(newErrMsg, t); } else { LOG.error(msg); throw new YarnException(msg); @@ -234,8 +235,9 @@ private static List getInterceptorClassNames(Configuration conf, public static void logAndThrowIOException(String errMsg, Throwable t) throws IOException { if (t != null) { - LOG.error(errMsg, t); - throw new IOException(errMsg, t); + String newErrMsg = getErrorMsg(errMsg, t); + LOG.error(newErrMsg, t); + throw new IOException(newErrMsg, t); } else { LOG.error(errMsg); throw new IOException(errMsg); @@ -256,8 +258,9 @@ public static void logAndThrowIOException(Throwable t, String errMsgFormat, Obje throws IOException { String msg = String.format(errMsgFormat, args); if (t != null) { - LOG.error(msg, t); - throw new IOException(msg, t); + String newErrMsg = getErrorMsg(msg, t); + LOG.error(newErrMsg, t); + throw new IOException(newErrMsg, t); } else { LOG.error(msg); throw new IOException(msg); @@ -276,8 +279,9 @@ public static void logAndThrowIOException(Throwable t, String errMsgFormat, Obje public static void logAndThrowRunTimeException(String errMsg, Throwable t) throws RuntimeException { if (t != null) { - LOG.error(errMsg, t); - throw new RuntimeException(errMsg, t); + String newErrMsg = getErrorMsg(errMsg, t); + LOG.error(newErrMsg, t); + throw new RuntimeException(newErrMsg, t); } else { LOG.error(errMsg); throw new RuntimeException(errMsg); @@ -298,8 +302,9 @@ public static void logAndThrowRunTimeException(Throwable t, String errMsgFormat, throws RuntimeException { String msg = String.format(errMsgFormat, args); if (t != null) { - LOG.error(msg, t); - throw new RuntimeException(msg, t); + String newErrMsg = getErrorMsg(msg, t); + LOG.error(newErrMsg, t); + throw new RuntimeException(newErrMsg, t); } else { LOG.error(msg); throw new RuntimeException(msg); @@ -320,8 +325,9 @@ public static RuntimeException logAndReturnRunTimeException( Throwable t, String errMsgFormat, Object... args) { String msg = String.format(errMsgFormat, args); if (t != null) { - LOG.error(msg, t); - return new RuntimeException(msg, t); + String newErrMsg = getErrorMsg(msg, t); + LOG.error(newErrMsg, t); + return new RuntimeException(newErrMsg, t); } else { LOG.error(msg); return new RuntimeException(msg); @@ -356,8 +362,9 @@ public static YarnRuntimeException logAndReturnYarnRunTimeException( Throwable t, String errMsgFormat, Object... args) { String msg = String.format(errMsgFormat, args); if (t != null) { - LOG.error(msg, t); - return new YarnRuntimeException(msg, t); + String newErrMsg = getErrorMsg(msg, t); + LOG.error(newErrMsg, t); + return new YarnRuntimeException(newErrMsg, t); } else { LOG.error(msg); return new YarnRuntimeException(msg); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/DefaultRMAdminRequestInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/DefaultRMAdminRequestInterceptor.java index 23517b97b4e19..2633a90d62cc0 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/DefaultRMAdminRequestInterceptor.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/DefaultRMAdminRequestInterceptor.java @@ -64,6 +64,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.BatchSaveFederationQueuePoliciesResponse; import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesRequest; import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesResponse; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationRequest; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationResponse; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -233,4 +235,11 @@ public QueryFederationQueuePoliciesResponse listFederationQueuePolicies( QueryFederationQueuePoliciesRequest request) throws YarnException, IOException { return rmAdminProxy.listFederationQueuePolicies(request); } + + @Override + public DeleteFederationApplicationResponse deleteFederationApplication( + DeleteFederationApplicationRequest request) + throws YarnException, IOException { + return rmAdminProxy.deleteFederationApplication(request); + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/FederationRMAdminInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/FederationRMAdminInterceptor.java index ee5f6c3cd4e2a..abed384450d7e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/FederationRMAdminInterceptor.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/FederationRMAdminInterceptor.java @@ -28,6 +28,7 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.hadoop.util.Time; +import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.YarnException; @@ -71,6 +72,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.BatchSaveFederationQueuePoliciesResponse; import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesRequest; import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesResponse; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationRequest; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationResponse; import org.apache.hadoop.yarn.server.federation.failover.FederationProxyProviderUtil; import org.apache.hadoop.yarn.server.federation.policies.manager.PriorityBroadcastPolicyManager; import org.apache.hadoop.yarn.server.federation.policies.manager.WeightedLocalityPolicyManager; @@ -1101,6 +1104,41 @@ public QueryFederationQueuePoliciesResponse listFederationQueuePolicies( throw new YarnException("Unable to listFederationQueuePolicies."); } + @Override + public DeleteFederationApplicationResponse deleteFederationApplication( + DeleteFederationApplicationRequest request) throws YarnException, IOException { + + // Parameter validation. + if (request == null) { + routerMetrics.incrDeleteFederationApplicationFailedRetrieved(); + RouterServerUtil.logAndThrowException( + "Missing deleteFederationApplication Request.", null); + } + + String application = request.getApplication(); + if (StringUtils.isBlank(application)) { + routerMetrics.incrDeleteFederationApplicationFailedRetrieved(); + RouterServerUtil.logAndThrowException( + "ApplicationId cannot be null.", null); + } + + // Try calling deleteApplicationHomeSubCluster to delete the application. + try { + long startTime = clock.getTime(); + ApplicationId applicationId = ApplicationId.fromString(application); + federationFacade.deleteApplicationHomeSubCluster(applicationId); + long stopTime = clock.getTime(); + routerMetrics.succeededDeleteFederationApplicationFailedRetrieved(stopTime - startTime); + return DeleteFederationApplicationResponse.newInstance( + "applicationId = " + applicationId + " delete success."); + } catch (Exception e) { + RouterServerUtil.logAndThrowException(e, + "Unable to deleteFederationApplication due to exception. " + e.getMessage()); + } + + throw new YarnException("Unable to deleteFederationApplication."); + } + /** * According to the configuration information of the queue filtering queue, * this part should only return 1 result. diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/RouterRMAdminService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/RouterRMAdminService.java index fc2278d3bb318..718abd2894493 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/RouterRMAdminService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/RouterRMAdminService.java @@ -71,6 +71,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.BatchSaveFederationQueuePoliciesResponse; import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesRequest; import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesResponse; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationRequest; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationResponse; import org.apache.hadoop.yarn.server.router.RouterServerUtil; import org.apache.hadoop.yarn.server.router.security.authorize.RouterPolicyProvider; import org.apache.hadoop.yarn.util.LRUCacheHashMap; @@ -419,4 +421,11 @@ public QueryFederationQueuePoliciesResponse listFederationQueuePolicies( RequestInterceptorChainWrapper pipeline = getInterceptorChain(); return pipeline.getRootInterceptor().listFederationQueuePolicies(request); } + + @Override + public DeleteFederationApplicationResponse deleteFederationApplication( + DeleteFederationApplicationRequest request) throws YarnException, IOException { + RequestInterceptorChainWrapper pipeline = getInterceptorChain(); + return pipeline.getRootInterceptor().deleteFederationApplication(request); + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/FederationInterceptorREST.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/FederationInterceptorREST.java index 7f9446878b3e6..fcfc7fa300e62 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/FederationInterceptorREST.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/FederationInterceptorREST.java @@ -341,6 +341,7 @@ protected DefaultRequestInterceptorREST getOrCreateInterceptorByAppId(String app // Get homeSubCluster By appId SubClusterInfo subClusterInfo = getHomeSubClusterInfoByAppId(appId); + LOG.info("appId = {} : subClusterInfo = {}.", appId, subClusterInfo.getSubClusterId()); return getOrCreateInterceptorForSubCluster(subClusterInfo); } @@ -827,7 +828,7 @@ public AppsInfo getApps(HttpServletRequest hsr, String stateQuery, }); if (apps.getApps().isEmpty()) { - return null; + return new AppsInfo(); } // Merge all the application reports got from all the available YARN RMs @@ -1135,7 +1136,7 @@ public AppState getAppState(HttpServletRequest hsr, String appId) } catch (YarnException | IllegalArgumentException e) { LOG.error("getHomeSubClusterInfoByAppId error, applicationId = {}.", appId, e); } - return null; + return new AppState(); } @Override @@ -3371,17 +3372,19 @@ private Map invokeConcurrent(Collection c } Exception exception = result.getException(); - - // If allowPartialResult=false, it means that if an exception occurs in a subCluster, - // an exception will be thrown directly. - if (!allowPartialResult && exception != null) { + if (exception != null) { throw exception; } } catch (Throwable e) { String subClusterId = subClusterInfo != null ? subClusterInfo.getSubClusterId().getId() : "UNKNOWN"; LOG.error("SubCluster {} failed to {} report.", subClusterId, request.getMethodName(), e); - throw new YarnRuntimeException(e.getCause().getMessage(), e); + // If allowPartialResult=false, it means that if an exception occurs in a subCluster, + // an exception will be thrown directly. + if (!allowPartialResult) { + throw new YarnException("SubCluster " + subClusterId + + " failed to " + request.getMethodName() + " report.", e); + } } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterWebServices.java index c9c56c46c7c33..4e0d97e83e959 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterWebServices.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterWebServices.java @@ -106,7 +106,7 @@ * main difference with AMRMProxyService is the protocol they implement. **/ @Singleton -@Path("/ws/v1/cluster") +@Path(RMWSConsts.RM_WEB_SERVICE_PATH) public class RouterWebServices implements RMWebServiceProtocol { private static final Logger LOG = @@ -424,7 +424,7 @@ public BulkActivitiesInfo getBulkActivities( MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 }) @Override public AppActivitiesInfo getAppActivities(@Context HttpServletRequest hsr, - @QueryParam(RMWSConsts.APP_ID) String appId, + @PathParam(RMWSConsts.APPID) String appId, @QueryParam(RMWSConsts.MAX_TIME) String time, @QueryParam(RMWSConsts.REQUEST_PRIORITIES) Set requestPriorities, @QueryParam(RMWSConsts.ALLOCATION_REQUEST_IDS) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/PassThroughRMAdminRequestInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/PassThroughRMAdminRequestInterceptor.java index 6fe218c7b2a00..0dab931a7ab97 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/PassThroughRMAdminRequestInterceptor.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/PassThroughRMAdminRequestInterceptor.java @@ -58,6 +58,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.BatchSaveFederationQueuePoliciesResponse; import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesRequest; import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesResponse; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationRequest; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationResponse; /** * Mock interceptor that does not do anything other than forwarding it to the @@ -185,4 +187,10 @@ public QueryFederationQueuePoliciesResponse listFederationQueuePolicies( QueryFederationQueuePoliciesRequest request) throws YarnException, IOException { return getNextInterceptor().listFederationQueuePolicies(request); } + + @Override + public DeleteFederationApplicationResponse deleteFederationApplication( + DeleteFederationApplicationRequest request) throws YarnException, IOException { + return getNextInterceptor().deleteFederationApplication(request); + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/TestFederationRMAdminInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/TestFederationRMAdminInterceptor.java index 520c25d22cb09..62f1eee845b56 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/TestFederationRMAdminInterceptor.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/TestFederationRMAdminInterceptor.java @@ -22,6 +22,7 @@ import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; import org.apache.hadoop.test.LambdaTestUtils; +import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.DecommissionType; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.Resource; @@ -64,12 +65,15 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.BatchSaveFederationQueuePoliciesResponse; import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesRequest; import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesResponse; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationRequest; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationResponse; import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo; import org.apache.hadoop.yarn.server.federation.policies.manager.WeightedLocalityPolicyManager; import org.apache.hadoop.yarn.server.federation.store.impl.MemoryFederationStateStore; import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId; import org.apache.hadoop.yarn.server.federation.store.records.SubClusterIdInfo; import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration; +import org.apache.hadoop.yarn.server.federation.store.records.ApplicationHomeSubCluster; import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreFacade; import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreTestUtil; import org.junit.Assert; @@ -989,4 +993,28 @@ public void testFilterPoliciesConfigurationsByQueues() throws Exception { "The index of the records to be retrieved has exceeded the maximum index.", () -> interceptor.listFederationQueuePolicies(request8)); } + + + @Test + public void testDeleteFederationApplication() throws Exception { + ApplicationId applicationId = ApplicationId.newInstance(10, 1); + DeleteFederationApplicationRequest request1 = + DeleteFederationApplicationRequest.newInstance(applicationId.toString()); + LambdaTestUtils.intercept(YarnException.class, + "Application application_10_0001 does not exist.", + () -> interceptor.deleteFederationApplication(request1)); + + ApplicationId applicationId2 = ApplicationId.newInstance(10, 2); + SubClusterId homeSubCluster = SubClusterId.newInstance("SC-1"); + ApplicationHomeSubCluster appHomeSubCluster = + ApplicationHomeSubCluster.newInstance(applicationId2, homeSubCluster); + facade.addApplicationHomeSubCluster(appHomeSubCluster); + DeleteFederationApplicationRequest request2 = + DeleteFederationApplicationRequest.newInstance(applicationId2.toString()); + DeleteFederationApplicationResponse deleteFederationApplicationResponse = + interceptor.deleteFederationApplication(request2); + assertNotNull(deleteFederationApplicationResponse); + assertEquals("applicationId = " + applicationId2 + " delete success.", + deleteFederationApplicationResponse.getMessage()); + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/subcluster/TestFederationSubCluster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/subcluster/TestFederationSubCluster.java index f9cd707821814..71034558687c9 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/subcluster/TestFederationSubCluster.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/subcluster/TestFederationSubCluster.java @@ -32,6 +32,7 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.util.Time; +import org.apache.hadoop.yarn.api.records.NodeLabel; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.server.federation.store.FederationStateStore; @@ -39,6 +40,13 @@ import org.apache.hadoop.yarn.server.federation.store.records.GetSubClustersInfoResponse; import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo; import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreFacade; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ApplicationSubmissionContextInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NewApplication; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodesInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NewReservation; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeLabelsInfo; +import org.apache.hadoop.yarn.server.router.webapp.HTTPMethods; import org.apache.hadoop.yarn.server.router.webapp.JavaProcess; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -48,11 +56,20 @@ import java.security.PrivilegedExceptionAction; import java.util.LinkedList; import java.util.List; +import java.util.ArrayList; import java.util.concurrent.TimeoutException; +import java.util.regex.Matcher; +import java.util.regex.Pattern; import static javax.servlet.http.HttpServletResponse.SC_OK; +import static javax.ws.rs.core.MediaType.APPLICATION_JSON; import static javax.ws.rs.core.MediaType.APPLICATION_XML; import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.RM_WEB_SERVICE_PATH; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.NODES; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.APPS_NEW_APPLICATION; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.APPS; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.RESERVATION_NEW; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.ADD_NODE_LABELS; import static org.apache.hadoop.yarn.server.router.webapp.TestRouterWebServicesREST.waitWebAppRunning; import static org.junit.Assert.assertEquals; @@ -190,6 +207,8 @@ public static T performGetCalls(final String routerAddress, final String pat final String queryValue) throws IOException, InterruptedException { Client clientToRouter = Client.create(); + clientToRouter.setReadTimeout(5000); + clientToRouter.setConnectTimeout(5000); WebResource toRouter = clientToRouter.resource(routerAddress).path(path); final WebResource.Builder toRouterBuilder; @@ -207,4 +226,120 @@ public static T performGetCalls(final String routerAddress, final String pat return response.getEntity(returnType); }); } + + public static ClientResponse performCall(final String routerAddress, final String webAddress, + final String queryKey, final String queryValue, final Object context, + final HTTPMethods method) throws IOException, InterruptedException { + + return UserGroupInformation.createRemoteUser(userName).doAs( + (PrivilegedExceptionAction) () -> { + Client clientToRouter = Client.create(); + WebResource toRouter = clientToRouter.resource(routerAddress).path(webAddress); + + WebResource toRouterWR = toRouter; + if (queryKey != null && queryValue != null) { + toRouterWR = toRouterWR.queryParam(queryKey, queryValue); + } + + WebResource.Builder builder; + if (context != null) { + builder = toRouterWR.entity(context, APPLICATION_JSON); + builder = builder.accept(APPLICATION_JSON); + } else { + builder = toRouterWR.accept(APPLICATION_JSON); + } + + ClientResponse response = null; + + switch (method) { + case DELETE: + response = builder.delete(ClientResponse.class); + break; + case POST: + response = builder.post(ClientResponse.class); + break; + case PUT: + response = builder.put(ClientResponse.class); + break; + default: + break; + } + + return response; + }); + } + + public String getNodeId(String rmAddress) { + Client clientToRM = Client.create(); + clientToRM.setConnectTimeout(3000); + clientToRM.setReadTimeout(3000); + WebResource toRM = clientToRM.resource(rmAddress).path(RM_WEB_SERVICE_PATH + NODES); + ClientResponse response = + toRM.accept(APPLICATION_XML).get(ClientResponse.class); + NodesInfo ci = response.getEntity(NodesInfo.class); + List nodes = ci.getNodes(); + if (nodes.isEmpty()) { + return null; + } + clientToRM.destroy(); + return nodes.get(0).getNodeId(); + } + + public NewApplication getNewApplicationId(String routerAddress) { + Client clientToRM = Client.create(); + clientToRM.setConnectTimeout(3000); + clientToRM.setReadTimeout(3000); + WebResource toRM = clientToRM.resource(routerAddress).path( + RM_WEB_SERVICE_PATH + APPS_NEW_APPLICATION); + ClientResponse response = toRM.accept(APPLICATION_XML).post(ClientResponse.class); + clientToRM.destroy(); + return response.getEntity(NewApplication.class); + } + + public String submitApplication(String routerAddress) { + ApplicationSubmissionContextInfo context = new ApplicationSubmissionContextInfo(); + String appId = getNewApplicationId(routerAddress).getApplicationId(); + context.setApplicationId(appId); + Client clientToRouter = Client.create(); + clientToRouter.setConnectTimeout(3000); + clientToRouter.setReadTimeout(3000); + WebResource toRM = clientToRouter.resource(routerAddress).path( + RM_WEB_SERVICE_PATH + APPS); + toRM.entity(context, APPLICATION_XML).accept(APPLICATION_XML).post(ClientResponse.class); + clientToRouter.destroy(); + return appId; + } + + public NewReservation getNewReservationId(String routerAddress) { + Client clientToRM = Client.create(); + clientToRM.setConnectTimeout(3000); + clientToRM.setReadTimeout(3000); + WebResource toRM = clientToRM.resource(routerAddress). + path(RM_WEB_SERVICE_PATH + RESERVATION_NEW); + ClientResponse response = toRM.accept(APPLICATION_XML).post(ClientResponse.class); + return response.getEntity(NewReservation.class); + } + + public String addNodeLabel(String routerAddress) { + Client clientToRM = Client.create(); + clientToRM.setConnectTimeout(3000); + clientToRM.setReadTimeout(3000); + WebResource toRM = clientToRM.resource(routerAddress) + .path(RM_WEB_SERVICE_PATH + ADD_NODE_LABELS); + List nodeLabels = new ArrayList<>(); + nodeLabels.add(NodeLabel.newInstance("default")); + NodeLabelsInfo context = new NodeLabelsInfo(nodeLabels); + ClientResponse response = toRM + .entity(context, APPLICATION_XML) + .accept(APPLICATION_XML) + .post(ClientResponse.class); + return response.getEntity(String.class); + } + + public static String format(String format, Object... args) { + Pattern p = Pattern.compile("\\{.*?}"); + Matcher m = p.matcher(format); + String newFormat = m.replaceAll("%s"); + return String.format(newFormat, args); + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/subcluster/capacity/TestYarnFederationWithCapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/subcluster/capacity/TestYarnFederationWithCapacityScheduler.java index f37e1245bdcdd..9fc4b5fd03671 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/subcluster/capacity/TestYarnFederationWithCapacityScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/subcluster/capacity/TestYarnFederationWithCapacityScheduler.java @@ -17,21 +17,93 @@ */ package org.apache.hadoop.yarn.server.router.subcluster.capacity; +import static javax.servlet.http.HttpServletResponse.SC_ACCEPTED; +import static javax.servlet.http.HttpServletResponse.SC_BAD_REQUEST; +import static javax.servlet.http.HttpServletResponse.SC_SERVICE_UNAVAILABLE; +import com.sun.jersey.api.client.ClientResponse; +import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.util.Sets; +import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType; +import org.apache.hadoop.yarn.api.records.NodeLabel; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ClusterInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ClusterUserInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ClusterMetricsInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedulerTypeInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.CapacitySchedulerInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodesInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ActivitiesInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppActivitiesInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ApplicationStatisticsInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.StatisticsItemInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NewApplication; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppAttemptsInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppAttemptInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppState; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppPriority; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppQueue; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppTimeoutsInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppTimeoutInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NewReservation; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationSubmissionRequestInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationUpdateRequestInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationDeleteRequestInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeLabelsInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.LabelsToNodesInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsEntryList; import org.apache.hadoop.yarn.server.router.subcluster.TestFederationSubCluster; import org.apache.hadoop.yarn.server.router.webapp.dao.FederationClusterInfo; +import org.apache.hadoop.yarn.server.router.webapp.dao.FederationClusterUserInfo; +import org.apache.hadoop.yarn.server.router.webapp.dao.FederationSchedulerTypeInfo; +import org.apache.hadoop.yarn.server.webapp.dao.AppsInfo; +import org.apache.hadoop.yarn.server.webapp.dao.AppInfo; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; import java.io.IOException; +import java.util.ArrayList; import java.util.List; import java.util.Set; import java.util.concurrent.TimeoutException; +import static javax.servlet.http.HttpServletResponse.SC_OK; import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.RM_WEB_SERVICE_PATH; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.INFO; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.CLUSTER_USER_INFO; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.METRICS; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.SCHEDULER; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.NODES; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.STATES; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.NODES_NODEID; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.SCHEDULER_ACTIVITIES; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.APPS_NEW_APPLICATION; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.APP_STATISTICS; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.APPS; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.APPS_APPID; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.APPS_APPID_APPATTEMPTS; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.APPS_APPID_STATE; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.APPS_APPID_PRIORITY; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.APPS_APPID_QUEUE; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.APPS_TIMEOUTS; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.APPS_TIMEOUT; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.RESERVATION_NEW; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.RESERVATION_SUBMIT; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.RESERVATION_UPDATE; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.RESERVATION_DELETE; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.GET_NODE_LABELS; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.NODES_NODEID_GETLABELS; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.LABEL_MAPPINGS; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.LABELS; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.ADD_NODE_LABELS; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.GET_NODE_TO_LABELS; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.REMOVE_NODE_LABELS; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.REPLACE_NODE_TO_LABELS; +import static org.apache.hadoop.yarn.server.router.subcluster.TestFederationSubCluster.format; +import static org.apache.hadoop.yarn.server.router.webapp.HTTPMethods.POST; +import static org.apache.hadoop.yarn.server.router.webapp.HTTPMethods.PUT; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -41,6 +113,8 @@ public class TestYarnFederationWithCapacityScheduler { private static TestFederationSubCluster testFederationSubCluster; private static Set subClusters; private static final String ROUTER_WEB_ADDRESS = "http://localhost:18089"; + private static final String SC1_RM_WEB_ADDRESS = "http://localhost:18088"; + private static final String SC2_RM_WEB_ADDRESS = "http://localhost:28088"; @BeforeClass public static void setUp() @@ -73,4 +147,469 @@ public void testGetClusterInfo() throws InterruptedException, IOException { assertTrue(subClusters.contains(clusterInfo.getSubClusterId())); } } + + @Test + public void testInfo() throws InterruptedException, IOException { + FederationClusterInfo federationClusterInfo = + TestFederationSubCluster.performGetCalls(ROUTER_WEB_ADDRESS, RM_WEB_SERVICE_PATH + INFO, + FederationClusterInfo.class, null, null); + List clusterInfos = federationClusterInfo.getList(); + assertNotNull(clusterInfos); + assertEquals(2, clusterInfos.size()); + for (ClusterInfo clusterInfo : clusterInfos) { + assertNotNull(clusterInfo); + assertTrue(subClusters.contains(clusterInfo.getSubClusterId())); + } + } + + @Test + public void testClusterUserInfo() throws Exception { + FederationClusterUserInfo federationClusterUserInfo = + TestFederationSubCluster.performGetCalls(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + CLUSTER_USER_INFO, + FederationClusterUserInfo.class, null, null); + List clusterUserInfos = federationClusterUserInfo.getList(); + assertNotNull(clusterUserInfos); + assertEquals(2, clusterUserInfos.size()); + for (ClusterUserInfo clusterUserInfo : clusterUserInfos) { + assertNotNull(clusterUserInfo); + assertTrue(subClusters.contains(clusterUserInfo.getSubClusterId())); + } + } + + @Test + public void testMetricsInfo() throws Exception { + // It takes time to start the sub-cluster. + // We need to wait for the sub-cluster to be completely started, + // so we need to set the waiting time. + // The resources of the two sub-clusters we registered are 24C and 12G, + // so the resources that the Router should collect are 48C and 24G. + GenericTestUtils.waitFor(() -> { + try { + ClusterMetricsInfo clusterMetricsInfo = + TestFederationSubCluster.performGetCalls(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + METRICS, ClusterMetricsInfo.class, null, null); + assertNotNull(clusterMetricsInfo); + return (48 == clusterMetricsInfo.getTotalVirtualCores() && + 24576 == clusterMetricsInfo.getTotalMB()); + } catch (Exception e) { + return false; + } + }, 5000, 50 * 5000); + } + + @Test + public void testSchedulerInfo() throws Exception { + FederationSchedulerTypeInfo schedulerTypeInfo = + TestFederationSubCluster.performGetCalls(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + SCHEDULER, FederationSchedulerTypeInfo.class, null, null); + assertNotNull(schedulerTypeInfo); + List schedulerTypeInfos = schedulerTypeInfo.getList(); + assertNotNull(schedulerTypeInfos); + assertEquals(2, schedulerTypeInfos.size()); + for (SchedulerTypeInfo schedulerTypeInfoItem : schedulerTypeInfos) { + assertNotNull(schedulerTypeInfoItem); + assertTrue(subClusters.contains(schedulerTypeInfoItem.getSubClusterId())); + CapacitySchedulerInfo schedulerInfo = + (CapacitySchedulerInfo) schedulerTypeInfoItem.getSchedulerInfo(); + assertNotNull(schedulerInfo); + assertEquals(3, schedulerInfo.getQueues().getQueueInfoList().size()); + } + } + + @Test + public void testNodesEmpty() throws Exception { + // We are in 2 sub-clusters, each with 3 nodes, so our Router should correspond to 6 nodes. + GenericTestUtils.waitFor(() -> { + try { + NodesInfo nodesInfo = + TestFederationSubCluster.performGetCalls(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + NODES, NodesInfo.class, null, null); + assertNotNull(nodesInfo); + ArrayList nodes = nodesInfo.getNodes(); + assertNotNull(nodes); + return (6 == nodes.size()); + } catch (Exception e) { + return false; + } + }, 5000, 50 * 5000); + } + + @Test + public void testNodesLost() throws Exception { + GenericTestUtils.waitFor(() -> { + try { + NodesInfo nodesInfo = + TestFederationSubCluster.performGetCalls(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + NODES, NodesInfo.class, STATES, "LOST"); + assertNotNull(nodesInfo); + ArrayList nodes = nodesInfo.getNodes(); + assertNotNull(nodes); + return nodes.isEmpty(); + } catch (Exception e) { + return false; + } + }, 5000, 50 * 5000); + } + + @Test + public void testNode() throws Exception { + String rm1NodeId = testFederationSubCluster.getNodeId(SC1_RM_WEB_ADDRESS); + NodeInfo nodeInfo = + TestFederationSubCluster.performGetCalls(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + format(NODES_NODEID, rm1NodeId), + NodeInfo.class, null, null); + assertNotNull(nodeInfo); + assertEquals(rm1NodeId, nodeInfo.getNodeId()); + + String rm2NodeId = testFederationSubCluster.getNodeId(SC2_RM_WEB_ADDRESS); + NodeInfo nodeInfo2 = + TestFederationSubCluster.performGetCalls(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + format(NODES_NODEID, rm2NodeId), + NodeInfo.class, null, null); + assertNotNull(nodeInfo2); + assertEquals(rm2NodeId, nodeInfo2.getNodeId()); + } + + @Test + public void testUpdateNodeResource() throws Exception { + // wait until a node shows up and check the resources + GenericTestUtils.waitFor(() -> testFederationSubCluster.getNodeId(SC1_RM_WEB_ADDRESS) != null, + 100, 5 * 1000); + String rm1NodeId = testFederationSubCluster.getNodeId(SC1_RM_WEB_ADDRESS); + + // assert memory and default vcores + NodeInfo nodeInfo = TestFederationSubCluster.performGetCalls(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + format(NODES_NODEID, rm1NodeId), + NodeInfo.class, null, null); + assertEquals(4096, nodeInfo.getTotalResource().getMemorySize()); + assertEquals(8, nodeInfo.getTotalResource().getvCores()); + } + + @Test + public void testActivies() throws Exception { + // wait until a node shows up and check the resources + GenericTestUtils.waitFor(() -> testFederationSubCluster.getNodeId(SC1_RM_WEB_ADDRESS) != null, + 100, 5 * 1000); + String rm1NodeId = testFederationSubCluster.getNodeId(SC1_RM_WEB_ADDRESS); + + ActivitiesInfo activitiesInfo = TestFederationSubCluster.performGetCalls(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + SCHEDULER_ACTIVITIES, ActivitiesInfo.class, "nodeId", rm1NodeId); + + assertNotNull(activitiesInfo); + assertEquals(rm1NodeId, activitiesInfo.getNodeId()); + } + + @Test + public void testAppActivitiesXML() throws Exception { + String appId = testFederationSubCluster.submitApplication(ROUTER_WEB_ADDRESS); + AppActivitiesInfo appActivitiesInfo = + TestFederationSubCluster.performGetCalls(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + "/scheduler/app-activities/" + appId, + AppActivitiesInfo.class, null, null); + assertNotNull(appActivitiesInfo); + assertEquals(appId, appActivitiesInfo.getApplicationId()); + } + + @Test + public void testAppStatistics() throws Exception { + ApplicationStatisticsInfo applicationStatisticsInfo = + TestFederationSubCluster.performGetCalls(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + APP_STATISTICS, ApplicationStatisticsInfo.class, STATES, "RUNNING"); + assertNotNull(applicationStatisticsInfo); + ArrayList statItems = applicationStatisticsInfo.getStatItems(); + assertNotNull(statItems); + assertEquals(1, statItems.size()); + } + + @Test + public void testNewApplication() throws Exception { + ClientResponse response = TestFederationSubCluster.performCall(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + APPS_NEW_APPLICATION, null, + null, null, POST); + assertEquals(SC_OK, response.getStatus()); + NewApplication ci = response.getEntity(NewApplication.class); + assertNotNull(ci); + } + + @Test + public void testSubmitApplicationXML() throws Exception { + String appId = testFederationSubCluster.submitApplication(ROUTER_WEB_ADDRESS); + assertNotNull(appId); + } + + @Test + public void testApps() throws Exception { + String appId = testFederationSubCluster.submitApplication(ROUTER_WEB_ADDRESS); + assertNotNull(appId); + AppsInfo appsInfo = TestFederationSubCluster.performGetCalls(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + APPS, AppsInfo.class, null, null); + assertNotNull(appsInfo); + assertEquals(1, appsInfo.getApps().size()); + } + + @Test + public void testApp() throws Exception { + String appId = testFederationSubCluster.submitApplication(ROUTER_WEB_ADDRESS); + assertNotNull(appId); + AppInfo appInfo = TestFederationSubCluster.performGetCalls(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + format(APPS_APPID, appId), + AppInfo.class, null, null); + assertNotNull(appInfo); + } + + @Test + public void testAppAttempt() throws Exception { + String appId = testFederationSubCluster.submitApplication(ROUTER_WEB_ADDRESS); + assertNotNull(appId); + AppAttemptsInfo appAttemptsInfo = TestFederationSubCluster.performGetCalls(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + format(APPS_APPID_APPATTEMPTS, appId), + AppAttemptsInfo.class, null, null); + assertNotNull(appAttemptsInfo); + ArrayList attempts = appAttemptsInfo.getAttempts(); + assertNotNull(attempts); + assertEquals(1, attempts.size()); + } + + @Test + public void testAppState() throws Exception { + String appId = testFederationSubCluster.submitApplication(ROUTER_WEB_ADDRESS); + assertNotNull(appId); + AppState appState = TestFederationSubCluster.performGetCalls(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + format(APPS_APPID_STATE, appId), + AppState.class, null, null); + assertNotNull(appState); + String state = appState.getState(); + assertNotNull(state); + assertEquals("ACCEPTED", state); + } + + @Test + public void testUpdateAppState() throws Exception { + String appId = testFederationSubCluster.submitApplication(ROUTER_WEB_ADDRESS); + assertNotNull(appId); + AppState appState = new AppState("KILLED"); + String pathApp = RM_WEB_SERVICE_PATH + format(APPS_APPID_STATE, appId); + ClientResponse response = TestFederationSubCluster.performCall(ROUTER_WEB_ADDRESS, + pathApp, null, null, appState, PUT); + assertNotNull(response); + assertEquals(SC_ACCEPTED, response.getStatus()); + AppState appState1 = response.getEntity(AppState.class); + assertNotNull(appState1); + assertNotNull(appState1.getState()); + assertEquals("KILLING", appState1.getState()); + } + + @Test + public void testAppPriority() throws Exception { + String appId = testFederationSubCluster.submitApplication(ROUTER_WEB_ADDRESS); + assertNotNull(appId); + AppPriority appPriority = TestFederationSubCluster.performGetCalls(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + format(APPS_APPID_PRIORITY, appId), + AppPriority.class, null, null); + assertNotNull(appPriority); + assertEquals(-1, appPriority.getPriority()); + } + + @Test + public void testUpdateAppPriority() throws Exception { + String appId = testFederationSubCluster.submitApplication(ROUTER_WEB_ADDRESS); + AppPriority appPriority = new AppPriority(1); + ClientResponse response = TestFederationSubCluster.performCall(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + format(APPS_APPID_PRIORITY, appId), + null, null, appPriority, PUT); + + assertEquals(SC_OK, response.getStatus()); + AppPriority ci = response.getEntity(AppPriority.class); + assertNotNull(ci); + assertNotNull(ci.getPriority()); + assertEquals(1, ci.getPriority()); + } + + @Test + public void testAppQueue() throws Exception { + String appId = testFederationSubCluster.submitApplication(ROUTER_WEB_ADDRESS); + AppQueue appQueue = TestFederationSubCluster.performGetCalls(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + format(APPS_APPID_QUEUE, appId), + AppQueue.class, null, null); + assertNotNull(appQueue); + String queue = appQueue.getQueue(); + assertEquals("root.default", queue); + } + + @Test + public void testUpdateAppQueue() throws Exception { + String appId = testFederationSubCluster.submitApplication(ROUTER_WEB_ADDRESS); + AppQueue appQueue = new AppQueue("root.default"); + ClientResponse response = TestFederationSubCluster.performCall(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + format(APPS_APPID_QUEUE, appId), + null, null, appQueue, PUT); + assertEquals(SC_OK, response.getStatus()); + AppQueue appQueue1 = response.getEntity(AppQueue.class); + assertNotNull(appQueue1); + String queue1 = appQueue1.getQueue(); + assertEquals("root.default", queue1); + } + + @Test + public void testAppTimeouts() throws Exception { + String appId = testFederationSubCluster.submitApplication(ROUTER_WEB_ADDRESS); + AppTimeoutsInfo appTimeoutsInfo = TestFederationSubCluster.performGetCalls(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + format(APPS_TIMEOUTS, appId), + AppTimeoutsInfo.class, null, null); + assertNotNull(appTimeoutsInfo); + ArrayList appTimeouts = appTimeoutsInfo.getAppTimeouts(); + assertNotNull(appTimeouts); + assertEquals(1, appTimeouts.size()); + AppTimeoutInfo appTimeoutInfo = appTimeouts.get(0); + assertNotNull(appTimeoutInfo); + assertEquals(ApplicationTimeoutType.LIFETIME, appTimeoutInfo.getTimeoutType()); + assertEquals("UNLIMITED", appTimeoutInfo.getExpireTime()); + } + + @Test + public void testAppTimeout() throws Exception { + String appId = testFederationSubCluster.submitApplication(ROUTER_WEB_ADDRESS); + String pathApp = RM_WEB_SERVICE_PATH + format(APPS_TIMEOUTS, appId); + AppTimeoutInfo appTimeoutInfo = TestFederationSubCluster.performGetCalls(ROUTER_WEB_ADDRESS, + pathApp + "/" + "LIFETIME", AppTimeoutInfo.class, null, null); + assertNotNull(appTimeoutInfo); + } + + @Test + public void testUpdateAppTimeouts() throws Exception { + String appId = testFederationSubCluster.submitApplication(ROUTER_WEB_ADDRESS); + AppTimeoutInfo appTimeoutInfo = new AppTimeoutInfo(); + ClientResponse response = TestFederationSubCluster.performCall(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + format(APPS_TIMEOUT, appId), + null, null, appTimeoutInfo, PUT); + assertEquals(SC_BAD_REQUEST, response.getStatus()); + String entity = response.getEntity(String.class); + assertNotNull(entity); + } + + @Test + public void testNewReservation() throws Exception { + ClientResponse response = TestFederationSubCluster.performCall(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + RESERVATION_NEW, + null, null, null, POST); + assertEquals(SC_OK, response.getStatus()); + NewReservation ci = response.getEntity(NewReservation.class); + assertNotNull(ci); + } + + @Test + public void testSubmitReservation() throws Exception { + ReservationSubmissionRequestInfo context = new ReservationSubmissionRequestInfo(); + NewReservation newReservationId = + testFederationSubCluster.getNewReservationId(ROUTER_WEB_ADDRESS); + context.setReservationId(newReservationId.getReservationId()); + ClientResponse response = TestFederationSubCluster.performCall(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + RESERVATION_SUBMIT, null, null, context, POST); + assertEquals(SC_BAD_REQUEST, response.getStatus()); + String entity = response.getEntity(String.class); + assertNotNull(entity); + } + + @Test + public void testUpdateReservation() throws Exception { + NewReservation newReservationId = + testFederationSubCluster.getNewReservationId(ROUTER_WEB_ADDRESS); + String reservationId = newReservationId.getReservationId(); + ReservationUpdateRequestInfo context = new ReservationUpdateRequestInfo(); + context.setReservationId(reservationId); + ClientResponse response = TestFederationSubCluster.performCall(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + RESERVATION_UPDATE, null, null, context, POST); + assertEquals(SC_BAD_REQUEST, response.getStatus()); + String entity = response.getEntity(String.class); + assertNotNull(entity); + } + + @Test + public void testDeleteReservation() throws Exception { + NewReservation newReservationId = + testFederationSubCluster.getNewReservationId(ROUTER_WEB_ADDRESS); + String reservationId = newReservationId.getReservationId(); + ReservationDeleteRequestInfo context = new ReservationDeleteRequestInfo(); + context.setReservationId(reservationId); + ClientResponse response = TestFederationSubCluster.performCall(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + RESERVATION_DELETE, null, null, context, POST); + assertEquals(SC_SERVICE_UNAVAILABLE, response.getStatus()); + String entity = response.getEntity(String.class); + assertNotNull(entity); + } + + @Test + public void testGetClusterNodeLabels() throws Exception { + NodeLabelsInfo nodeLabelsInfo = TestFederationSubCluster.performGetCalls(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + GET_NODE_LABELS, NodeLabelsInfo.class, null, null); + assertNotNull(nodeLabelsInfo); + } + + @Test + public void testGetLabelsOnNode() throws Exception { + String rm1NodeId = testFederationSubCluster.getNodeId(SC1_RM_WEB_ADDRESS); + NodeLabelsInfo nodeLabelsInfo = TestFederationSubCluster.performGetCalls(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + format(NODES_NODEID_GETLABELS, rm1NodeId), + NodeLabelsInfo.class, null, null); + assertNotNull(nodeLabelsInfo); + } + + @Test + public void testGetLabelsMappingEmpty() throws Exception { + LabelsToNodesInfo labelsToNodesInfo = TestFederationSubCluster.performGetCalls( + ROUTER_WEB_ADDRESS, RM_WEB_SERVICE_PATH + LABEL_MAPPINGS, + LabelsToNodesInfo.class, null, null); + assertNotNull(labelsToNodesInfo); + } + + @Test + public void testGetLabelsMapping() throws Exception { + LabelsToNodesInfo labelsToNodesInfo = TestFederationSubCluster.performGetCalls( + ROUTER_WEB_ADDRESS, RM_WEB_SERVICE_PATH + LABEL_MAPPINGS, + LabelsToNodesInfo.class, LABELS, "label1"); + assertNotNull(labelsToNodesInfo); + } + + @Test + public void testAddToClusterNodeLabels() throws Exception { + List nodeLabels = new ArrayList<>(); + nodeLabels.add(NodeLabel.newInstance("default")); + NodeLabelsInfo context = new NodeLabelsInfo(nodeLabels); + ClientResponse response = TestFederationSubCluster.performCall(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + ADD_NODE_LABELS, null, null, context, POST); + assertEquals(SC_OK, response.getStatus()); + String entity = response.getEntity(String.class); + assertNotNull(entity); + } + + @Test + public void testGetNodeToLabels() throws Exception { + NodeToLabelsInfo nodeToLabelsInfo = TestFederationSubCluster.performGetCalls( + ROUTER_WEB_ADDRESS, RM_WEB_SERVICE_PATH + GET_NODE_TO_LABELS, + NodeToLabelsInfo.class, null, null); + assertNotNull(nodeToLabelsInfo); + } + + @Test + public void testRemoveFromClusterNodeLabels() throws Exception { + testFederationSubCluster.addNodeLabel(ROUTER_WEB_ADDRESS); + ClientResponse response = TestFederationSubCluster.performCall(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + REMOVE_NODE_LABELS, + LABELS, "default", null, POST); + assertEquals(SC_OK, response.getStatus()); + String entity = response.getEntity(String.class); + assertNotNull(entity); + } + + @Test + public void testReplaceLabelsOnNodes() throws Exception { + testFederationSubCluster.addNodeLabel(ROUTER_WEB_ADDRESS); + NodeToLabelsEntryList context = new NodeToLabelsEntryList(); + ClientResponse response = TestFederationSubCluster.performCall(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + REPLACE_NODE_TO_LABELS, + null, null, context, POST); + String entity = response.getEntity(String.class); + assertNotNull(entity); + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/subcluster/fair/TestYarnFederationWithFairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/subcluster/fair/TestYarnFederationWithFairScheduler.java index ce27d5a3fc7e6..8af40193149c4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/subcluster/fair/TestYarnFederationWithFairScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/subcluster/fair/TestYarnFederationWithFairScheduler.java @@ -17,21 +17,99 @@ */ package org.apache.hadoop.yarn.server.router.subcluster.fair; +import static javax.servlet.http.HttpServletResponse.SC_ACCEPTED; +import static javax.servlet.http.HttpServletResponse.SC_SERVICE_UNAVAILABLE; +import static javax.servlet.http.HttpServletResponse.SC_BAD_REQUEST; +import com.sun.jersey.api.client.ClientResponse; +import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.util.Sets; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.ResourceOption; +import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType; +import org.apache.hadoop.yarn.api.records.NodeLabel; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ClusterInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ClusterUserInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ClusterMetricsInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedulerTypeInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.FairSchedulerInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.FairSchedulerQueueInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ResourceOptionInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodesInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ActivitiesInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppActivitiesInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ApplicationStatisticsInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.StatisticsItemInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NewApplication; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppAttemptsInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppAttemptInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppState; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppPriority; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppQueue; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppTimeoutsInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppTimeoutInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NewReservation; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationSubmissionRequestInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationUpdateRequestInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationDeleteRequestInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeLabelsInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.LabelsToNodesInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsEntryList; import org.apache.hadoop.yarn.server.router.subcluster.TestFederationSubCluster; import org.apache.hadoop.yarn.server.router.webapp.dao.FederationClusterInfo; +import org.apache.hadoop.yarn.server.router.webapp.dao.FederationClusterUserInfo; +import org.apache.hadoop.yarn.server.router.webapp.dao.FederationSchedulerTypeInfo; +import org.apache.hadoop.yarn.server.webapp.dao.AppsInfo; +import org.codehaus.jettison.json.JSONObject; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; import java.io.IOException; +import java.util.ArrayList; import java.util.List; import java.util.Set; import java.util.concurrent.TimeoutException; import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.RM_WEB_SERVICE_PATH; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.INFO; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.CLUSTER_USER_INFO; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.METRICS; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.SCHEDULER; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.NODES; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.STATES; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.NODES_NODEID; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.NODES_NODEID_REPLACE_LABELS; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.SCHEDULER_ACTIVITIES; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.APPS_NEW_APPLICATION; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.APP_STATISTICS; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.APPS; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.APP_ID; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.APPS_APPID_APPATTEMPTS; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.APPS_APPID_STATE; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.APPS_APPID_PRIORITY; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.APPS_APPID_QUEUE; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.APPS_TIMEOUTS; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.APPS_TIMEOUT; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.RESERVATION_NEW; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.RESERVATION_SUBMIT; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.RESERVATION_UPDATE; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.RESERVATION_DELETE; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.GET_NODE_LABELS; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.NODES_NODEID_GETLABELS; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.LABEL_MAPPINGS; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.LABELS; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.ADD_NODE_LABELS; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.GET_NODE_TO_LABELS; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.REMOVE_NODE_LABELS; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.REPLACE_NODE_TO_LABELS; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts.NODE_RESOURCE; +import static org.apache.hadoop.yarn.server.router.subcluster.TestFederationSubCluster.format; +import static org.apache.hadoop.yarn.server.router.webapp.HTTPMethods.POST; +import static org.apache.hadoop.yarn.server.router.webapp.HTTPMethods.PUT; +import static org.apache.http.HttpStatus.SC_OK; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -40,6 +118,8 @@ public class TestYarnFederationWithFairScheduler { private static TestFederationSubCluster testFederationSubCluster; private static Set subClusters; private static final String ROUTER_WEB_ADDRESS = "http://localhost:28089"; + private static final String SC1_RM_WEB_ADDRESS = "http://localhost:38088"; + private static final String SC2_RM_WEB_ADDRESS = "http://localhost:48088"; @BeforeClass public static void setUp() @@ -72,4 +152,488 @@ public void testGetClusterInfo() throws InterruptedException, IOException { assertTrue(subClusters.contains(clusterInfo.getSubClusterId())); } } + + @Test + public void testInfo() throws InterruptedException, IOException { + FederationClusterInfo federationClusterInfo = + TestFederationSubCluster.performGetCalls(ROUTER_WEB_ADDRESS, RM_WEB_SERVICE_PATH + INFO, + FederationClusterInfo.class, null, null); + List clusterInfos = federationClusterInfo.getList(); + assertNotNull(clusterInfos); + assertEquals(2, clusterInfos.size()); + for (ClusterInfo clusterInfo : clusterInfos) { + assertNotNull(clusterInfo); + assertTrue(subClusters.contains(clusterInfo.getSubClusterId())); + } + } + + @Test + public void testClusterUserInfo() throws Exception { + FederationClusterUserInfo federationClusterUserInfo = + TestFederationSubCluster.performGetCalls(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + CLUSTER_USER_INFO, + FederationClusterUserInfo.class, null, null); + List clusterUserInfos = federationClusterUserInfo.getList(); + assertNotNull(clusterUserInfos); + assertEquals(2, clusterUserInfos.size()); + for (ClusterUserInfo clusterUserInfo : clusterUserInfos) { + assertNotNull(clusterUserInfo); + assertTrue(subClusters.contains(clusterUserInfo.getSubClusterId())); + } + } + + @Test + public void testMetricsInfo() throws Exception { + // It takes time to start the sub-cluster. + // We need to wait for the sub-cluster to be completely started, + // so we need to set the waiting time. + // The resources of the two sub-clusters we registered are 24C and 12G, + // so the resources that the Router should collect are 48C and 24G. + GenericTestUtils.waitFor(() -> { + try { + ClusterMetricsInfo clusterMetricsInfo = + TestFederationSubCluster.performGetCalls(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + METRICS, ClusterMetricsInfo.class, null, null); + assertNotNull(clusterMetricsInfo); + return (48 == clusterMetricsInfo.getTotalVirtualCores() && + 24576 == clusterMetricsInfo.getTotalMB()); + } catch (Exception e) { + return false; + } + }, 5000, 50 * 5000); + } + + @Test + public void testSchedulerInfo() throws Exception { + FederationSchedulerTypeInfo schedulerTypeInfo = + TestFederationSubCluster.performGetCalls(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + SCHEDULER, FederationSchedulerTypeInfo.class, null, null); + assertNotNull(schedulerTypeInfo); + List schedulerTypeInfos = schedulerTypeInfo.getList(); + assertNotNull(schedulerTypeInfos); + assertEquals(2, schedulerTypeInfos.size()); + for (SchedulerTypeInfo schedulerTypeInfoItem : schedulerTypeInfos) { + assertNotNull(schedulerTypeInfoItem); + assertTrue(subClusters.contains(schedulerTypeInfoItem.getSubClusterId())); + FairSchedulerQueueInfo rootQueueInfo = + ((FairSchedulerInfo) schedulerTypeInfoItem.getSchedulerInfo()).getRootQueueInfo(); + assertNotNull(rootQueueInfo); + assertEquals("fair", rootQueueInfo.getSchedulingPolicy()); + } + } + + @Test + public void testNodesEmpty() throws Exception { + // We are in 2 sub-clusters, each with 3 nodes, so our Router should correspond to 6 nodes. + GenericTestUtils.waitFor(() -> { + try { + NodesInfo nodesInfo = + TestFederationSubCluster.performGetCalls(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + NODES, NodesInfo.class, null, null); + assertNotNull(nodesInfo); + ArrayList nodes = nodesInfo.getNodes(); + assertNotNull(nodes); + return (6 == nodes.size()); + } catch (Exception e) { + return false; + } + }, 5000, 50 * 5000); + } + + @Test + public void testNodesLost() throws Exception { + GenericTestUtils.waitFor(() -> { + try { + NodesInfo nodesInfo = + TestFederationSubCluster.performGetCalls(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + NODES, NodesInfo.class, STATES, "LOST"); + assertNotNull(nodesInfo); + ArrayList nodes = nodesInfo.getNodes(); + assertNotNull(nodes); + return nodes.isEmpty(); + } catch (Exception e) { + return false; + } + }, 5000, 50 * 5000); + } + + @Test + public void testNode() throws Exception { + String rm1NodeId = testFederationSubCluster.getNodeId(SC1_RM_WEB_ADDRESS); + NodeInfo nodeInfo = + TestFederationSubCluster.performGetCalls(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + format(NODES_NODEID, rm1NodeId), + NodeInfo.class, null, null); + assertNotNull(nodeInfo); + assertEquals(rm1NodeId, nodeInfo.getNodeId()); + + String rm2NodeId = testFederationSubCluster.getNodeId(SC2_RM_WEB_ADDRESS); + NodeInfo nodeInfo2 = + TestFederationSubCluster.performGetCalls(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + format(NODES_NODEID, rm2NodeId), + NodeInfo.class, null, null); + assertNotNull(nodeInfo2); + assertEquals(rm2NodeId, nodeInfo2.getNodeId()); + } + + @Test + public void testUpdateNodeResource() throws Exception { + // wait until a node shows up and check the resources + GenericTestUtils.waitFor(() -> testFederationSubCluster.getNodeId(SC1_RM_WEB_ADDRESS) != null, + 100, 5 * 1000); + String rm1NodeId = testFederationSubCluster.getNodeId(SC1_RM_WEB_ADDRESS); + + // assert memory and default vcores + NodeInfo nodeInfo = TestFederationSubCluster.performGetCalls(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + format(NODES_NODEID, rm1NodeId), + NodeInfo.class, null, null); + assertEquals(4096, nodeInfo.getTotalResource().getMemorySize()); + assertEquals(8, nodeInfo.getTotalResource().getvCores()); + + Resource resource = Resource.newInstance(4096, 5); + ResourceOptionInfo resourceOption = new ResourceOptionInfo( + ResourceOption.newInstance(resource, 1000)); + ClientResponse routerResponse = TestFederationSubCluster.performCall(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + format(NODE_RESOURCE, rm1NodeId), + null, null, resourceOption, POST); + JSONObject json = routerResponse.getEntity(JSONObject.class); + JSONObject totalResource = json.getJSONObject("resourceInfo"); + assertEquals(resource.getMemorySize(), totalResource.getLong("memory")); + assertEquals(resource.getVirtualCores(), totalResource.getLong("vCores")); + + // assert updated memory and cores + NodeInfo nodeInfo1 = TestFederationSubCluster.performGetCalls(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + format(NODES_NODEID, rm1NodeId), + NodeInfo.class, null, null); + assertEquals(4096, nodeInfo1.getTotalResource().getMemorySize()); + assertEquals(5, nodeInfo1.getTotalResource().getvCores()); + } + + @Test + public void testActivies() throws Exception { + // wait until a node shows up and check the resources + GenericTestUtils.waitFor(() -> testFederationSubCluster.getNodeId(SC1_RM_WEB_ADDRESS) != null, + 100, 5 * 1000); + String rm1NodeId = testFederationSubCluster.getNodeId(SC1_RM_WEB_ADDRESS); + + ActivitiesInfo activitiesInfo = TestFederationSubCluster.performGetCalls(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + SCHEDULER_ACTIVITIES, ActivitiesInfo.class, "nodeId", rm1NodeId); + + assertNotNull(activitiesInfo); + assertEquals(rm1NodeId, activitiesInfo.getNodeId()); + assertEquals("Not Capacity Scheduler", activitiesInfo.getDiagnostic()); + } + + @Test + public void testAppActivities() throws Exception { + String appId = testFederationSubCluster.submitApplication(ROUTER_WEB_ADDRESS); + AppActivitiesInfo appActivitiesInfo = + TestFederationSubCluster.performGetCalls(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + "/scheduler/app-activities/" + appId, + AppActivitiesInfo.class, APP_ID, appId); + assertNotNull(appActivitiesInfo); + assertEquals(appId, appActivitiesInfo.getApplicationId()); + assertEquals("Not Capacity Scheduler", appActivitiesInfo.getDiagnostic()); + } + + @Test + public void testAppStatistics() throws Exception { + ApplicationStatisticsInfo applicationStatisticsInfo = + TestFederationSubCluster.performGetCalls(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + APP_STATISTICS, ApplicationStatisticsInfo.class, STATES, "RUNNING"); + assertNotNull(applicationStatisticsInfo); + ArrayList statItems = applicationStatisticsInfo.getStatItems(); + assertNotNull(statItems); + assertEquals(1, statItems.size()); + } + + @Test + public void testNewApplication() throws Exception { + ClientResponse response = TestFederationSubCluster.performCall(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + APPS_NEW_APPLICATION, null, + null, null, POST); + assertEquals(SC_OK, response.getStatus()); + NewApplication ci = response.getEntity(NewApplication.class); + assertNotNull(ci); + } + + @Test + public void testSubmitApplication() { + String appId = testFederationSubCluster.submitApplication(ROUTER_WEB_ADDRESS); + assertNotNull(appId); + } + + @Test + public void testApps() throws Exception { + String appId = testFederationSubCluster.submitApplication(ROUTER_WEB_ADDRESS); + assertNotNull(appId); + AppsInfo appsInfo = TestFederationSubCluster.performGetCalls(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + APPS, AppsInfo.class, null, null); + assertNotNull(appsInfo); + assertEquals(1, appsInfo.getApps().size()); + } + + @Test + public void testAppAttempt() throws Exception { + String appId = testFederationSubCluster.submitApplication(ROUTER_WEB_ADDRESS); + assertNotNull(appId); + AppAttemptsInfo appAttemptsInfo = TestFederationSubCluster.performGetCalls(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + format(APPS_APPID_APPATTEMPTS, appId), + AppAttemptsInfo.class, null, null); + assertNotNull(appAttemptsInfo); + ArrayList attempts = appAttemptsInfo.getAttempts(); + assertNotNull(attempts); + assertEquals(1, attempts.size()); + } + + @Test + public void testAppState() throws Exception { + String appId = testFederationSubCluster.submitApplication(ROUTER_WEB_ADDRESS); + assertNotNull(appId); + AppState appState = TestFederationSubCluster.performGetCalls(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + format(APPS_APPID_STATE, appId), + AppState.class, null, null); + assertNotNull(appState); + String state = appState.getState(); + assertNotNull(state); + assertEquals("ACCEPTED", state); + } + + @Test + public void testUpdateAppState() throws Exception { + String appId = testFederationSubCluster.submitApplication(ROUTER_WEB_ADDRESS); + assertNotNull(appId); + AppState appState = new AppState("KILLED"); + String pathApp = RM_WEB_SERVICE_PATH + format(APPS_APPID_STATE, appId); + ClientResponse response = TestFederationSubCluster.performCall(ROUTER_WEB_ADDRESS, + pathApp, null, null, appState, PUT); + assertNotNull(response); + assertEquals(SC_ACCEPTED, response.getStatus()); + AppState appState1 = response.getEntity(AppState.class); + assertNotNull(appState1); + assertNotNull(appState1.getState()); + assertEquals("KILLING", appState1.getState()); + } + + @Test + public void testAppPriority() throws Exception { + String appId = testFederationSubCluster.submitApplication(ROUTER_WEB_ADDRESS); + assertNotNull(appId); + AppPriority appPriority = TestFederationSubCluster.performGetCalls(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + format(APPS_APPID_PRIORITY, appId), + AppPriority.class, null, null); + assertNotNull(appPriority); + assertEquals(0, appPriority.getPriority()); + } + + @Test + public void testUpdateAppPriority() throws Exception { + String appId = testFederationSubCluster.submitApplication(ROUTER_WEB_ADDRESS); + AppPriority appPriority = new AppPriority(1); + // FairScheduler does not support Update Application Priority. + ClientResponse response = TestFederationSubCluster.performCall(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + format(APPS_APPID_PRIORITY, appId), + null, null, appPriority, PUT); + assertEquals(SC_SERVICE_UNAVAILABLE, response.getStatus()); + } + + @Test + public void testAppQueue() throws Exception { + String appId = testFederationSubCluster.submitApplication(ROUTER_WEB_ADDRESS); + AppQueue appQueue = TestFederationSubCluster.performGetCalls(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + format(APPS_APPID_QUEUE, appId), + AppQueue.class, null, null); + assertNotNull(appQueue); + String queue = appQueue.getQueue(); + assertEquals("root.dr_dot_who", queue); + } + + @Test + public void testUpdateAppQueue() throws Exception { + String appId = testFederationSubCluster.submitApplication(ROUTER_WEB_ADDRESS); + AppQueue appQueue = new AppQueue("root.a"); + ClientResponse response = TestFederationSubCluster.performCall(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + format(APPS_APPID_QUEUE, appId), + null, null, appQueue, PUT); + assertEquals(SC_OK, response.getStatus()); + AppQueue appQueue1 = response.getEntity(AppQueue.class); + assertNotNull(appQueue1); + String queue1 = appQueue1.getQueue(); + assertEquals("root.a", queue1); + } + + @Test + public void testAppTimeouts() throws Exception { + String appId = testFederationSubCluster.submitApplication(ROUTER_WEB_ADDRESS); + AppTimeoutsInfo appTimeoutsInfo = TestFederationSubCluster.performGetCalls(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + format(APPS_TIMEOUTS, appId), + AppTimeoutsInfo.class, null, null); + assertNotNull(appTimeoutsInfo); + ArrayList appTimeouts = appTimeoutsInfo.getAppTimeouts(); + assertNotNull(appTimeouts); + assertEquals(1, appTimeouts.size()); + AppTimeoutInfo appTimeoutInfo = appTimeouts.get(0); + assertNotNull(appTimeoutInfo); + assertEquals(ApplicationTimeoutType.LIFETIME, appTimeoutInfo.getTimeoutType()); + assertEquals("UNLIMITED", appTimeoutInfo.getExpireTime()); + } + + @Test + public void testAppTimeout() throws Exception { + String appId = testFederationSubCluster.submitApplication(ROUTER_WEB_ADDRESS); + String pathApp = RM_WEB_SERVICE_PATH + format(APPS_TIMEOUTS, appId); + AppTimeoutInfo appTimeoutInfo = TestFederationSubCluster.performGetCalls(ROUTER_WEB_ADDRESS, + pathApp + "/" + "LIFETIME", AppTimeoutInfo.class, null, null); + assertNotNull(appTimeoutInfo); + } + + @Test + public void testUpdateAppTimeouts() throws Exception { + String appId = testFederationSubCluster.submitApplication(ROUTER_WEB_ADDRESS); + AppTimeoutInfo appTimeoutInfo = new AppTimeoutInfo(); + ClientResponse response = TestFederationSubCluster.performCall(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + format(APPS_TIMEOUT, appId), + null, null, appTimeoutInfo, PUT); + assertEquals(SC_BAD_REQUEST, response.getStatus()); + String entity = response.getEntity(String.class); + assertNotNull(entity); + } + + @Test + public void testNewReservation() throws Exception { + ClientResponse response = TestFederationSubCluster.performCall(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + RESERVATION_NEW, + null, null, null, POST); + assertEquals(SC_OK, response.getStatus()); + NewReservation ci = response.getEntity(NewReservation.class); + assertNotNull(ci); + } + + @Test + public void testSubmitReservation() throws Exception { + ReservationSubmissionRequestInfo context = new ReservationSubmissionRequestInfo(); + NewReservation newReservationId = + testFederationSubCluster.getNewReservationId(ROUTER_WEB_ADDRESS); + context.setReservationId(newReservationId.getReservationId()); + ClientResponse response = TestFederationSubCluster.performCall(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + RESERVATION_SUBMIT, null, null, context, POST); + assertEquals(SC_BAD_REQUEST, response.getStatus()); + String entity = response.getEntity(String.class); + assertNotNull(entity); + } + + @Test + public void testUpdateReservation() throws Exception { + NewReservation newReservationId = + testFederationSubCluster.getNewReservationId(ROUTER_WEB_ADDRESS); + String reservationId = newReservationId.getReservationId(); + ReservationUpdateRequestInfo context = new ReservationUpdateRequestInfo(); + context.setReservationId(reservationId); + ClientResponse response = TestFederationSubCluster.performCall(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + RESERVATION_UPDATE, null, null, context, POST); + assertEquals(SC_BAD_REQUEST, response.getStatus()); + String entity = response.getEntity(String.class); + assertNotNull(entity); + } + + @Test + public void testDeleteReservation() throws Exception { + NewReservation newReservationId = + testFederationSubCluster.getNewReservationId(ROUTER_WEB_ADDRESS); + String reservationId = newReservationId.getReservationId(); + ReservationDeleteRequestInfo context = new ReservationDeleteRequestInfo(); + context.setReservationId(reservationId); + ClientResponse response = TestFederationSubCluster.performCall(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + RESERVATION_DELETE, null, null, context, POST); + assertEquals(SC_SERVICE_UNAVAILABLE, response.getStatus()); + String entity = response.getEntity(String.class); + assertNotNull(entity); + } + + @Test + public void testGetClusterNodeLabels() throws Exception { + NodeLabelsInfo nodeLabelsInfo = TestFederationSubCluster.performGetCalls(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + GET_NODE_LABELS, NodeLabelsInfo.class, null, null); + assertNotNull(nodeLabelsInfo); + } + + @Test + public void testGetLabelsOnNode() throws Exception { + String rm1NodeId = testFederationSubCluster.getNodeId(SC1_RM_WEB_ADDRESS); + NodeLabelsInfo nodeLabelsInfo = TestFederationSubCluster.performGetCalls(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + format(NODES_NODEID_GETLABELS, rm1NodeId), + NodeLabelsInfo.class, null, null); + assertNotNull(nodeLabelsInfo); + } + + @Test + public void testGetLabelsMappingEmpty() throws Exception { + LabelsToNodesInfo labelsToNodesInfo = + TestFederationSubCluster.performGetCalls(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + LABEL_MAPPINGS, LabelsToNodesInfo.class, null, null); + assertNotNull(labelsToNodesInfo); + } + + @Test + public void testGetLabelsMapping() throws Exception { + LabelsToNodesInfo labelsToNodesInfo = TestFederationSubCluster.performGetCalls( + ROUTER_WEB_ADDRESS, RM_WEB_SERVICE_PATH + LABEL_MAPPINGS, + LabelsToNodesInfo.class, LABELS, "label1"); + assertNotNull(labelsToNodesInfo); + } + + @Test + public void testAddToClusterNodeLabels() throws Exception { + List nodeLabels = new ArrayList<>(); + nodeLabels.add(NodeLabel.newInstance("default")); + NodeLabelsInfo context = new NodeLabelsInfo(nodeLabels); + ClientResponse response = TestFederationSubCluster.performCall(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + ADD_NODE_LABELS, null, null, context, POST); + assertEquals(SC_OK, response.getStatus()); + String entity = response.getEntity(String.class); + assertNotNull(entity); + } + + @Test + public void testGetNodeToLabels() throws Exception { + NodeToLabelsInfo nodeToLabelsInfo = TestFederationSubCluster.performGetCalls( + ROUTER_WEB_ADDRESS, RM_WEB_SERVICE_PATH + GET_NODE_TO_LABELS, + NodeToLabelsInfo.class, null, null); + assertNotNull(nodeToLabelsInfo); + } + + @Test + public void testRemoveFromClusterNodeLabels() throws Exception { + testFederationSubCluster.addNodeLabel(ROUTER_WEB_ADDRESS); + ClientResponse response = TestFederationSubCluster.performCall(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + REMOVE_NODE_LABELS, + LABELS, "default", null, POST); + assertEquals(SC_OK, response.getStatus()); + String entity = response.getEntity(String.class); + assertNotNull(entity); + } + + @Test + public void testReplaceLabelsOnNodes() throws Exception { + testFederationSubCluster.addNodeLabel(ROUTER_WEB_ADDRESS); + NodeToLabelsEntryList context = new NodeToLabelsEntryList(); + ClientResponse response = TestFederationSubCluster.performCall(ROUTER_WEB_ADDRESS, + RM_WEB_SERVICE_PATH + REPLACE_NODE_TO_LABELS, + null, null, context, POST); + String entity = response.getEntity(String.class); + assertNotNull(entity); + } + + @Test + public void testReplaceLabelsOnNode() throws Exception { + String rm1NodeId = testFederationSubCluster.getNodeId(SC1_RM_WEB_ADDRESS); + String pathNode = RM_WEB_SERVICE_PATH + + format(NODES_NODEID_REPLACE_LABELS, rm1NodeId); + testFederationSubCluster.addNodeLabel(ROUTER_WEB_ADDRESS); + ClientResponse response = TestFederationSubCluster.performCall(ROUTER_WEB_ADDRESS, + pathNode, LABELS, "default", null, POST); + assertEquals(SC_OK, response.getStatus()); + String entity = response.getEntity(String.class); + assertNotNull(entity); + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/resources/yarn-site.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/resources/yarn-site.xml index 4a28627a9a194..94b7972dae764 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/resources/yarn-site.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/resources/yarn-site.xml @@ -47,4 +47,8 @@ yarn.resourcemanager.cluster-id local-cluster + + yarn.router.interceptor.allow-partial-result.enable + true + diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/TestLogInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/TestLogInfo.java index 22495eaf08069..6a4527c13c9d5 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/TestLogInfo.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/TestLogInfo.java @@ -40,7 +40,7 @@ import java.io.IOException; import java.io.OutputStream; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -199,7 +199,7 @@ private void writeBrokenFile(Path logPath) throws IOException { try { String broken = "{ broken { [[]} broken"; out = PluginStoreTestUtils.createLogFile(logPath, fs); - out.write(broken.getBytes(Charset.forName("UTF-8"))); + out.write(broken.getBytes(StandardCharsets.UTF_8)); out.close(); out = null; } finally { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-documentstore/src/test/java/org/apache/hadoop/yarn/server/timelineservice/documentstore/DocumentStoreTestUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-documentstore/src/test/java/org/apache/hadoop/yarn/server/timelineservice/documentstore/DocumentStoreTestUtils.java index 5d442152fe9ba..cdf2e181bdcd9 100755 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-documentstore/src/test/java/org/apache/hadoop/yarn/server/timelineservice/documentstore/DocumentStoreTestUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-documentstore/src/test/java/org/apache/hadoop/yarn/server/timelineservice/documentstore/DocumentStoreTestUtils.java @@ -25,6 +25,7 @@ import org.apache.hadoop.yarn.server.timelineservice.documentstore.collection.document.flowrun.FlowRunDocument; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.List; import com.fasterxml.jackson.core.type.TypeReference; @@ -41,7 +42,7 @@ public static List bakeTimelineEntities() throws IOException { String jsonStr = IOUtils.toString( DocumentStoreTestUtils.class.getClassLoader().getResourceAsStream( - "documents/timeline-entities.json"), "UTF-8"); + "documents/timeline-entities.json"), StandardCharsets.UTF_8); return JsonUtils.fromJson(jsonStr, new TypeReference>(){}); } @@ -50,7 +51,7 @@ public static List bakeYarnAppTimelineEntities() throws IOException { String jsonStr = IOUtils.toString( DocumentStoreTestUtils.class.getClassLoader().getResourceAsStream( - "documents/test-timeline-entities-doc.json"), "UTF-8"); + "documents/test-timeline-entities-doc.json"), StandardCharsets.UTF_8); return JsonUtils.fromJson(jsonStr, new TypeReference>() {}); } @@ -59,7 +60,7 @@ public static TimelineEntityDocument bakeTimelineEntityDoc() throws IOException { String jsonStr = IOUtils.toString( DocumentStoreTestUtils.class.getClassLoader().getResourceAsStream( - "documents/timeline-app-doc.json"), "UTF-8"); + "documents/timeline-app-doc.json"), StandardCharsets.UTF_8); return JsonUtils.fromJson(jsonStr, new TypeReference() {}); } @@ -67,7 +68,7 @@ public static TimelineEntityDocument bakeTimelineEntityDoc() public static FlowActivityDocument bakeFlowActivityDoc() throws IOException { String jsonStr = IOUtils.toString( DocumentStoreTestUtils.class.getClassLoader().getResourceAsStream( - "documents/flowactivity-doc.json"), "UTF-8"); + "documents/flowactivity-doc.json"), StandardCharsets.UTF_8); return JsonUtils.fromJson(jsonStr, new TypeReference() {}); } @@ -75,7 +76,7 @@ public static FlowActivityDocument bakeFlowActivityDoc() throws IOException { public static FlowRunDocument bakeFlowRunDoc() throws IOException { String jsonStr = IOUtils.toString( DocumentStoreTestUtils.class.getClassLoader().getResourceAsStream( - "documents/flowrun-doc.json"), "UTF-8"); + "documents/flowrun-doc.json"), StandardCharsets.UTF_8); return JsonUtils.fromJson(jsonStr, new TypeReference(){}); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FileSystemTimelineReaderImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FileSystemTimelineReaderImpl.java index dff21a31da952..2e771fc77e8f1 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FileSystemTimelineReaderImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FileSystemTimelineReaderImpl.java @@ -23,7 +23,7 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStreamReader; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.util.Comparator; import java.util.EnumSet; import java.util.HashSet; @@ -174,7 +174,7 @@ private String getFlowRunPath(String userId, String clusterId, APP_FLOW_MAPPING_FILE); try (BufferedReader reader = new BufferedReader(new InputStreamReader( - fs.open(appFlowMappingFilePath), Charset.forName("UTF-8"))); + fs.open(appFlowMappingFilePath), StandardCharsets.UTF_8)); CSVParser parser = new CSVParser(reader, csvFormat)) { for (CSVRecord record : parser.getRecords()) { if (record.size() < 4) { @@ -300,7 +300,7 @@ public int compare(Long l1, Long l2) { } try (BufferedReader reader = new BufferedReader( new InputStreamReader(fs.open(entityFile), - Charset.forName("UTF-8")))) { + StandardCharsets.UTF_8))) { TimelineEntity entity = readEntityFromFile(reader); if (!entity.getType().equals(entityType)) { continue; @@ -402,7 +402,7 @@ public TimelineEntity getEntity(TimelineReaderContext context, } try (BufferedReader reader = new BufferedReader(new InputStreamReader( - fs.open(entityFilePath), Charset.forName("UTF-8")))) { + fs.open(entityFilePath), StandardCharsets.UTF_8))) { TimelineEntity entity = readEntityFromFile(reader); return createEntityToBeReturned( entity, dataToRetrieve.getFieldsToRetrieve()); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FileSystemTimelineWriterImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FileSystemTimelineWriterImpl.java index 4b96f4ddd0742..2f7007165a0d2 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FileSystemTimelineWriterImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FileSystemTimelineWriterImpl.java @@ -20,6 +20,7 @@ import java.io.File; import java.io.IOException; +import java.nio.charset.StandardCharsets; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -139,7 +140,7 @@ private synchronized void writeInternal(String clusterId, String userId, byte[] record = new StringBuilder() .append(TimelineUtils.dumpTimelineRecordtoJSON(entity)) - .append("\n").toString().getBytes("UTF-8"); + .append("\n").toString().getBytes(StandardCharsets.UTF_8); writeFileWithRetries(filePath, record); } catch (Exception ioe) { LOG.warn("Interrupted operation:{}", ioe.getMessage()); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServlet.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServlet.java index 56adabe8f30c8..7817362885064 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServlet.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServlet.java @@ -30,6 +30,7 @@ import java.net.URI; import java.net.URISyntaxException; import java.net.URLEncoder; +import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.EnumSet; import java.util.Enumeration; @@ -287,7 +288,7 @@ private void proxyLink(final HttpServletRequest req, StringBuilder sb = new StringBuilder(); BufferedReader reader = new BufferedReader( - new InputStreamReader(req.getInputStream(), "UTF-8")); + new InputStreamReader(req.getInputStream(), StandardCharsets.UTF_8)); String line; while ((line = reader.readLine()) != null) { sb.append(line); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/TestWebAppProxyServlet.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/TestWebAppProxyServlet.java index 1d0ca00e7eff6..49b6a7954ba9d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/TestWebAppProxyServlet.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/TestWebAppProxyServlet.java @@ -30,6 +30,7 @@ import java.net.SocketTimeoutException; import java.net.URI; import java.net.URL; +import java.nio.charset.StandardCharsets; import java.util.Collections; import java.util.Enumeration; import java.util.List; @@ -522,7 +523,7 @@ private String readInputStream(InputStream input) throws Exception { while ((read = input.read(buffer)) >= 0) { data.write(buffer, 0, read); } - return new String(data.toByteArray(), "UTF-8"); + return new String(data.toByteArray(), StandardCharsets.UTF_8); } private boolean isResponseCookiePresent(HttpURLConnection proxyConn, diff --git a/pom.xml b/pom.xml index b86af01172f42..13e3aec63efba 100644 --- a/pom.xml +++ b/pom.xml @@ -118,7 +118,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/x 4.2.0 1.1.1 3.10.1 - 2.7.6 + 2.7.10 bash @@ -225,6 +225,14 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/x org.apache.hadoop.thirdparty.com.google.common.io.BaseEncoding.** + + true + Use java.nio.charset.StandardCharsets rather than Guava provided Charsets + + org.apache.hadoop.thirdparty.com.google.common.base.Charsets + org.apache.hadoop.thirdparty.com.google.common.base.Charsets.** + + true Use alternative to Guava provided Optional