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 extends FederationNamenodeContext>> cacheNS;
+ private Map, List extends FederationNamenodeContext>> 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 extends FederationNamenodeContext> 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 extends FederationNamenodeContext> 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 extends FederationNamenodeContext> 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 extends ReservedSpaceCalculator> 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