소스 검색

HDFS-10707. Replace org.apache.commons.io.Charsets with java.nio.charset.StandardCharsets. Contributed by Vincent Poon.

(cherry picked from commit c0166b074382576da2aaa0eb76965abe91ed4115)

Conflicts:
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/webhdfs/WebHdfsHandler.java
Kihwal Lee 8 년 전
부모
커밋
804db5b277

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java

@@ -29,6 +29,7 @@ import java.net.Socket;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.nio.channels.SocketChannel;
+import java.nio.charset.StandardCharsets;
 import java.text.SimpleDateFormat;
 import java.util.Collection;
 import java.util.Collections;
@@ -40,7 +41,6 @@ import java.util.Map;
 
 import javax.net.SocketFactory;
 
-import org.apache.commons.io.Charsets;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.fs.BlockLocation;
@@ -84,7 +84,7 @@ public class DFSUtilClient {
    * Converts a string to a byte array using UTF8 encoding.
    */
   public static byte[] string2Bytes(String str) {
-    return str.getBytes(Charsets.UTF_8);
+    return str.getBytes(StandardCharsets.UTF_8);
   }
 
   /**

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSUtils.java

@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.fs.http.client;
 
-import org.apache.commons.io.Charsets;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.Path;
 import org.json.simple.parser.JSONParser;
@@ -29,6 +28,7 @@ import java.net.HttpURLConnection;
 import java.net.URI;
 import java.net.URL;
 import java.net.URLEncoder;
+import java.nio.charset.StandardCharsets;
 import java.text.MessageFormat;
 import java.util.List;
 import java.util.Map;
@@ -128,7 +128,8 @@ public class HttpFSUtils {
   static Object jsonParse(HttpURLConnection conn) throws IOException {
     try {
       JSONParser parser = new JSONParser();
-      return parser.parse(new InputStreamReader(conn.getInputStream(), Charsets.UTF_8));
+      return parser.parse(
+          new InputStreamReader(conn.getInputStream(), StandardCharsets.UTF_8));
     } catch (ParseException ex) {
       throw new IOException("JSON parser error, " + ex.getMessage(), ex);
     }

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSAuthenticationFilter.java

@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.fs.http.server;
 
-import org.apache.commons.io.Charsets;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.web.WebHdfsConstants;
@@ -32,6 +31,7 @@ import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.io.Reader;
+import java.nio.charset.StandardCharsets;
 import java.util.Map;
 import java.util.Properties;
 
@@ -83,7 +83,7 @@ public class HttpFSAuthenticationFilter
     try {
       StringBuilder secret = new StringBuilder();
       Reader reader = new InputStreamReader(new FileInputStream(
-          signatureSecretFile), Charsets.UTF_8);
+          signatureSecretFile), StandardCharsets.UTF_8);
       int c = reader.read();
       while (c > -1) {
         secret.append((char)c);

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/JSONMapProvider.java

@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.lib.wsrs;
 
-import org.apache.commons.io.Charsets;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.json.simple.JSONObject;
 
@@ -34,6 +33,7 @@ import java.io.OutputStreamWriter;
 import java.io.Writer;
 import java.lang.annotation.Annotation;
 import java.lang.reflect.Type;
+import java.nio.charset.StandardCharsets;
 import java.util.Map;
 
 @Provider
@@ -56,7 +56,8 @@ public class JSONMapProvider implements MessageBodyWriter<Map> {
   public void writeTo(Map map, Class<?> aClass, Type type, Annotation[] annotations,
                       MediaType mediaType, MultivaluedMap<String, Object> stringObjectMultivaluedMap,
                       OutputStream outputStream) throws IOException, WebApplicationException {
-    Writer writer = new OutputStreamWriter(outputStream,  Charsets.UTF_8);
+    Writer writer =
+        new OutputStreamWriter(outputStream, StandardCharsets.UTF_8);
     JSONObject.writeJSONString(map, writer);
     writer.write(ENTER);
     writer.flush();

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/JSONProvider.java

@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.lib.wsrs;
 
-import org.apache.commons.io.Charsets;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.json.simple.JSONStreamAware;
 
@@ -34,6 +33,7 @@ import java.io.OutputStreamWriter;
 import java.io.Writer;
 import java.lang.annotation.Annotation;
 import java.lang.reflect.Type;
+import java.nio.charset.StandardCharsets;
 
 @Provider
 @Produces(MediaType.APPLICATION_JSON)
@@ -56,7 +56,8 @@ public class JSONProvider implements MessageBodyWriter<JSONStreamAware> {
   public void writeTo(JSONStreamAware jsonStreamAware, Class<?> aClass, Type type, Annotation[] annotations,
                       MediaType mediaType, MultivaluedMap<String, Object> stringObjectMultivaluedMap,
                       OutputStream outputStream) throws IOException, WebApplicationException {
-    Writer writer = new OutputStreamWriter(outputStream, Charsets.UTF_8);
+    Writer writer =
+        new OutputStreamWriter(outputStream, StandardCharsets.UTF_8);
     jsonStreamAware.writeJSONString(writer);
     writer.write(ENTER);
     writer.flush();

+ 5 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/webhdfs/ParameterParser.java

@@ -18,7 +18,6 @@
 package org.apache.hadoop.hdfs.server.datanode.web.webhdfs;
 
 import io.netty.handler.codec.http.QueryStringDecoder;
-import org.apache.commons.io.Charsets;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.permission.FsPermission;
@@ -44,6 +43,7 @@ import org.apache.hadoop.security.token.Token;
 import java.io.IOException;
 import java.net.URI;
 import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
 import java.util.EnumSet;
 import java.util.List;
 import java.util.Map;
@@ -57,8 +57,8 @@ class ParameterParser {
   private final Map<String, List<String>> params;
 
   ParameterParser(QueryStringDecoder decoder, Configuration conf) {
-    this.path = decodeComponent(decoder.path().substring
-        (WEBHDFS_PREFIX_LENGTH), Charsets.UTF_8);
+    this.path = decodeComponent(decoder.path().substring(WEBHDFS_PREFIX_LENGTH),
+        StandardCharsets.UTF_8);
     this.params = decoder.parameters();
     this.conf = conf;
   }
@@ -131,7 +131,8 @@ class ParameterParser {
   }
 
   public EnumSet<CreateFlag> createFlag() {
-    String cf = decodeComponent(param(CreateFlagParam.NAME), Charsets.UTF_8);
+    String cf =
+        decodeComponent(param(CreateFlagParam.NAME), StandardCharsets.UTF_8);
 
     return new CreateFlagParam(cf).getValue();
   }

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/webhdfs/WebHdfsHandler.java

@@ -50,10 +50,10 @@ import java.io.InputStream;
 import java.io.OutputStream;
 import java.net.URI;
 import java.net.URISyntaxException;
+import java.nio.charset.StandardCharsets;
 import java.security.PrivilegedExceptionAction;
 import java.util.EnumSet;
 
-import org.apache.commons.io.Charsets;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -252,7 +252,8 @@ public class WebHdfsHandler extends SimpleChannelInboundHandler<HttpRequest> {
     } finally {
       IOUtils.cleanup(LOG, dfsclient);
     }
-    final byte[] js = JsonUtil.toJsonString(checksum).getBytes(Charsets.UTF_8);
+    final byte[] js =
+        JsonUtil.toJsonString(checksum).getBytes(StandardCharsets.UTF_8);
     DefaultFullHttpResponse resp =
       new DefaultFullHttpResponse(HTTP_1_1, OK, Unpooled.wrappedBuffer(js));
 

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirMkdirOp.java

@@ -18,7 +18,6 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import com.google.common.base.Preconditions;
-import org.apache.commons.io.Charsets;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.InvalidPathException;
 import org.apache.hadoop.fs.UnresolvedLinkException;
@@ -33,6 +32,7 @@ import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 
 import java.io.IOException;
+import java.nio.charset.StandardCharsets;
 import java.util.AbstractMap;
 import java.util.List;
 import java.util.Map;
@@ -121,7 +121,8 @@ class FSDirMkdirOp {
   static Map.Entry<INodesInPath, String> createAncestorDirectories(
       FSDirectory fsd, INodesInPath iip, PermissionStatus permission)
       throws IOException {
-    final String last = new String(iip.getLastLocalName(), Charsets.UTF_8);
+    final String last =
+        new String(iip.getLastLocalName(), StandardCharsets.UTF_8);
     INodesInPath existing = iip.getExistingINodes();
     List<String> children = iip.getPath(existing.length(),
         iip.length() - existing.length());
@@ -189,7 +190,7 @@ class FSDirMkdirOp {
       throws IOException {
     assert fsd.hasWriteLock();
     existing = unprotectedMkdir(fsd, fsd.allocateNewInodeId(), existing,
-        localName.getBytes(Charsets.UTF_8), perm, null, now());
+        localName.getBytes(StandardCharsets.UTF_8), perm, null, now());
     if (existing == null) {
       return null;
     }

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java

@@ -19,7 +19,6 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import com.google.common.base.Preconditions;
-import org.apache.commons.io.Charsets;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.DirectoryListingStartAfterNotFoundException;
 import org.apache.hadoop.fs.FileEncryptionInfo;
@@ -44,6 +43,7 @@ import org.apache.hadoop.hdfs.util.ReadOnlyList;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.nio.charset.StandardCharsets;
 import java.util.Arrays;
 
 import static org.apache.hadoop.util.Time.now;
@@ -53,7 +53,8 @@ class FSDirStatAndListingOp {
       byte[] startAfter, boolean needLocation) throws IOException {
     byte[][] pathComponents = FSDirectory
         .getPathComponentsForReservedPath(srcArg);
-    final String startAfterString = new String(startAfter, Charsets.UTF_8);
+    final String startAfterString =
+        new String(startAfter, StandardCharsets.UTF_8);
     String src = null;
 
     if (fsd.isPermissionEnabled()) {

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java

@@ -18,7 +18,6 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import com.google.common.base.Preconditions;
-import org.apache.commons.io.Charsets;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.crypto.CipherSuite;
 import org.apache.hadoop.crypto.CryptoProtocolVersion;
@@ -57,6 +56,7 @@ import org.apache.hadoop.util.ChunkedArrayList;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -564,7 +564,7 @@ class FSDirWriteFileOp {
     long modTime = now();
     INodeFile newNode = newINodeFile(fsd.allocateNewInodeId(), permissions,
                                      modTime, modTime, replication, preferredBlockSize);
-    newNode.setLocalName(localName.getBytes(Charsets.UTF_8));
+    newNode.setLocalName(localName.getBytes(StandardCharsets.UTF_8));
     newNode.toUnderConstruction(clientName, clientMachine);
 
     INodesInPath newiip;

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageUtil.java

@@ -21,9 +21,9 @@ import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.RandomAccessFile;
+import java.nio.charset.StandardCharsets;
 import java.util.Arrays;
 
-import org.apache.commons.io.Charsets;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
@@ -34,7 +34,7 @@ import org.apache.hadoop.io.compress.CompressionCodec;
 @InterfaceAudience.Private
 public final class FSImageUtil {
   public static final byte[] MAGIC_HEADER =
-      "HDFSIMG1".getBytes(Charsets.UTF_8);
+      "HDFSIMG1".getBytes(StandardCharsets.UTF_8);
   public static final int FILE_VERSION = 1;
 
   public static boolean checkFileFormat(RandomAccessFile file)

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGetBlockLocations.java

@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import org.apache.commons.io.Charsets;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
@@ -27,6 +26,7 @@ import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
 import java.io.IOException;
+import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_DEFAULT;
@@ -124,7 +124,7 @@ public class TestGetBlockLocations {
         "hdfs", "supergroup",
         FsPermission.createImmutable((short) 0x1ff));
     final INodeFile file = new INodeFile(
-        MOCK_INODE_ID, FILE_NAME.getBytes(Charsets.UTF_8),
+        MOCK_INODE_ID, FILE_NAME.getBytes(StandardCharsets.UTF_8),
         perm, 1, 1, new BlockInfo[] {}, (short) 1,
         DFS_BLOCK_SIZE_DEFAULT);
     fsn.getFSDirectory().addINode(iip, file);