فهرست منبع

HADOOP-10567. Shift XAttr value encoding code out for reuse. Contributed by Yi Liu

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2006@1592152 13f79535-47bb-0310-9956-ffa450edef68
Uma Maheswara Rao G 11 سال پیش
والد
کامیت
6bbdf3d4d0

+ 2 - 0
hadoop-common-project/hadoop-common/CHANGES-HDFS-2006.txt

@@ -18,6 +18,8 @@ HDFS-2006 (Unreleased)
 
   HADOOP-10548. Improve FsShell xattr error handling and other fixes. (Charles Lamb via umamahesh)
 
+  HADOOP-10567. Shift XAttr value encoding code out for reuse. (Yi Liu via umamahesh)
+
   OPTIMIZATIONS
 
   BUG FIXES

+ 120 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/XAttrCodec.java

@@ -0,0 +1,120 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import java.io.IOException;
+
+import org.apache.commons.codec.DecoderException;
+import org.apache.commons.codec.binary.Base64;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * The value of <code>XAttr</code> is byte[], this class is to 
+ * covert byte[] to some kind of string representation or convert back.
+ * String representation is convenient for display and input. For example
+ * display in screen as shell response and json response, input as http
+ * or shell parameter. 
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public enum XAttrCodec {
+  /**
+   * Value encoded as text 
+   * string is enclosed in double quotes (\").
+   */
+  TEXT,
+  
+  /**
+   * Value encoded as hexadecimal string 
+   * is prefixed with 0x.
+   */
+  HEX,
+  
+  /**
+   * Value encoded as base64 string 
+   * is prefixed with 0s.
+   */
+  BASE64;
+  
+  private static final String HEX_PREFIX = "0x";
+  private static final String BASE64_PREFIX = "0s";
+  
+  /**
+   * Decode string representation of a value and check whether it's 
+   * encoded. If the given string begins with 0x or 0X, it expresses
+   * a hexadecimal number. If the given string begins with 0s or 0S,
+   * base64 encoding is expected. If the given string is enclosed in 
+   * double quotes, the inner string is treated as text. Otherwise 
+   * the given string is treated as text. 
+   * @param value string representation of the value.
+   * @return byte[] the value
+   * @throws IOException
+   */
+  public static byte[] decodeValue(String value) throws IOException {
+    byte[] result = null;
+    if (value != null) {
+      if (value.length() >= 2) {
+        String en = value.substring(0, 2);
+        if (value.startsWith("\"") && value.endsWith("\"")) {
+          value = value.substring(1, value.length()-1);
+          result = value.getBytes("utf-8");
+        } else if (en.equalsIgnoreCase(HEX_PREFIX)) {
+          value = value.substring(2, value.length());
+          try {
+            result = Hex.decodeHex(value.toCharArray());
+          } catch (DecoderException e) {
+            throw new IOException(e);
+          }
+        } else if (en.equalsIgnoreCase(BASE64_PREFIX)) {
+          value = value.substring(2, value.length());
+          result = Base64.decodeBase64(value);
+        }
+      }
+      if (result == null) {
+        result = value.getBytes("utf-8");
+      }
+    }
+    return result;
+  }
+  
+  /**
+   * Encode byte[] value to string representation with encoding. 
+   * Values encoded as text strings are enclosed in double quotes (\"), 
+   * while strings encoded as hexadecimal and base64 are prefixed with 
+   * 0x and 0s, respectively.
+   * @param value byte[] value
+   * @param encoding
+   * @return String string representation of value
+   * @throws IOException
+   */
+  public static String encodeValue(byte[] value, XAttrCodec encoding) 
+      throws IOException {
+    Preconditions.checkNotNull(value, "Value can not be null.");
+    if (encoding == HEX) {
+      return HEX_PREFIX + Hex.encodeHexString(value);
+    } else if (encoding == BASE64) {
+      return BASE64_PREFIX + Base64.encodeBase64String(value);
+    } else {
+      return "\"" + new String(value, "utf-8") + "\"";
+    }
+  }
+}

+ 26 - 83
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/XAttrCommands.java

@@ -27,12 +27,10 @@ import com.google.common.base.Enums;
 import com.google.common.base.Function;
 import com.google.common.base.Preconditions;
 
-import org.apache.commons.codec.DecoderException;
-import org.apache.commons.codec.binary.Hex;
-import org.apache.commons.codec.binary.Base64;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.XAttrCodec;
 import org.apache.hadoop.util.StringUtils;
 
 /**
@@ -49,60 +47,6 @@ class XAttrCommands extends FsCommand {
     factory.addClass(SetfattrCommand.class, "-" + SET_FATTR);
   }
 
-  private static enum ENCODE {
-    TEXT,
-    HEX,
-    BASE64;
-  }
-
-  private static final String ENCODE_HEX = "0x";
-  private static final String ENCODE_BASE64 = "0s";
-
-  private static String convert(String name, byte[] value, ENCODE encode)
-      throws IOException {
-    final StringBuilder buffer = new StringBuilder();
-    buffer.append(name);
-    if (value != null && value.length != 0) {
-      buffer.append("=");
-      if (encode == ENCODE.TEXT) {
-        buffer.append("\"").append(new String(value, "utf-8")).append("\"");
-      } else if (encode == ENCODE.HEX) {
-        buffer.append(ENCODE_HEX).append(Hex.encodeHexString(value));
-      } else if (encode == ENCODE.BASE64) {
-        buffer.append(ENCODE_BASE64).append(Base64.encodeBase64String(value));
-      }
-    }
-    return buffer.toString();
-  }
-
-  private static byte[] convert(String valueArg) throws IOException {
-    String value = valueArg;
-    byte[] result = null;
-    if (value != null) {
-      if (value.length() >= 2) {
-        final String en = value.substring(0, 2);
-        if (value.startsWith("\"") && value.endsWith("\"")) {
-          value = value.substring(1, value.length()-1);
-          result = value.getBytes("utf-8");
-        } else if (en.equalsIgnoreCase(ENCODE_HEX)) {
-          value = value.substring(2, value.length());
-          try {
-            result = Hex.decodeHex(value.toCharArray());
-          } catch (DecoderException e) {
-            throw new IOException(e);
-          }
-        } else if (en.equalsIgnoreCase(ENCODE_BASE64)) {
-          value = value.substring(2, value.length());
-          result = Base64.decodeBase64(value);
-        }
-      }
-      if (result == null) {
-        result = value.getBytes("utf-8");
-      }
-    }
-    return result;
-  }
-
   /**
    * Implements the '-getfattr' command for the FsShell.
    */
@@ -121,30 +65,26 @@ class XAttrCommands extends FsCommand {
       "strings are enclosed in double quotes (\"), and values encoded" +
       " as hexadecimal and base64 are prefixed with 0x and 0s, respectively.\n" +
       "<path>: The file or directory.\n";
-    private static final String NAME_OPT = "-n";
-    private static final String ENCODE_OPT = "-e";
-    final CommandFormat cf = new CommandFormat(0, Integer.MAX_VALUE, "d", "R");
+    private final static Function<String, XAttrCodec> enValueOfFunc =
+        Enums.valueOfFunction(XAttrCodec.class);
 
     private String name = null;
     private boolean dump = false;
-    private ENCODE encode = ENCODE.TEXT;
-
-    private final static Function<String, ENCODE> encodeValueOfFunc =
-      Enums.valueOfFunction(ENCODE.class);
+    private XAttrCodec encoding = XAttrCodec.TEXT;
 
     @Override
     protected void processOptions(LinkedList<String> args) throws IOException {
-      name = StringUtils.popOptionWithArgument(NAME_OPT, args);
-      String en = StringUtils.popOptionWithArgument(ENCODE_OPT, args);
+      name = StringUtils.popOptionWithArgument("-n", args);
+      String en = StringUtils.popOptionWithArgument("-e", args);
       if (en != null) {
-        encode = encodeValueOfFunc.apply(en.toUpperCase());
+        encoding = enValueOfFunc.apply(en.toUpperCase());
+        Preconditions.checkArgument(encoding != null,
+          "Invalid/unsupported encoding option specified: " + en);
       }
-      Preconditions.checkArgument(encode != null,
-        "Invalid/unsupported encoding option specified: " + en);
 
-      cf.parse(args);
-      setRecursive(cf.getOpt("R"));
-      dump = cf.getOpt("d");
+      boolean r = StringUtils.popOption("-R", args);
+      setRecursive(r);
+      dump = StringUtils.popOption("-d", args);
 
       if (!dump && name == null) {
         throw new HadoopIllegalArgumentException(
@@ -168,14 +108,20 @@ class XAttrCommands extends FsCommand {
           Iterator<Entry<String, byte[]>> iter = xattrs.entrySet().iterator();
           while(iter.hasNext()) {
             Entry<String, byte[]> entry = iter.next();
-            out.println(convert(entry.getKey(), entry.getValue(), encode));
+            printXAttr(entry.getKey(), entry.getValue());
           }
         }
       } else {
         byte[] value = item.fs.getXAttr(item.path, name);
-        if (value != null) {
-          out.println(convert(name, value, encode));
-        }
+        printXAttr(name, value);
+      }
+    }
+    
+    private void printXAttr(String name, byte[] value) throws IOException{
+      if (value != null && value.length != 0) {
+        out.println(name + "=" + XAttrCodec.encodeValue(value, encoding));
+      } else {
+        out.println(name);
       }
     }
   }
@@ -197,9 +143,6 @@ class XAttrCommands extends FsCommand {
       "base64 encoding.\n" +
       "-x name: Remove the extended attribute.\n" +
       "<path>: The file or directory.\n";
-    private static final String NAME_OPT = "-n";
-    private static final String VALUE_OPT = "-v";
-    private static final String REMOVE_OPT = "-x";
 
     private String name = null;
     private byte[] value = null;
@@ -207,12 +150,12 @@ class XAttrCommands extends FsCommand {
 
     @Override
     protected void processOptions(LinkedList<String> args) throws IOException {
-      name = StringUtils.popOptionWithArgument(NAME_OPT, args);
-      String v = StringUtils.popOptionWithArgument(VALUE_OPT, args);
+      name = StringUtils.popOptionWithArgument("-n", args);
+      String v = StringUtils.popOptionWithArgument("-v", args);
       if (v != null) {
-        value = convert(v);
+        value = XAttrCodec.decodeValue(v);
       }
-      xname = StringUtils.popOptionWithArgument(REMOVE_OPT, args);
+      xname = StringUtils.popOptionWithArgument("-x", args);
 
       if (name != null && xname != null) {
         throw new HadoopIllegalArgumentException(

+ 28 - 22
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestXAttrCommands.java

@@ -31,49 +31,55 @@ import org.junit.Before;
 import org.junit.Test;
 
 public class TestXAttrCommands {
-
+  private final ByteArrayOutputStream errContent = 
+      new ByteArrayOutputStream();
   private Configuration conf = null;
 
   @Before
   public void setup() throws IOException {
+    System.setErr(new PrintStream(errContent));
     conf = new Configuration();
   }
 
   @Test
   public void testGetfattrValidations() throws Exception {
+    errContent.reset();
     assertFalse("getfattr should fail without path",
-        0 == runCommand(new String[] { "-getfattr" }));
+        0 == runCommand(new String[] { "-getfattr", "-d"}));
+    assertTrue(errContent.toString().contains("<path> is missing"));
+
+    errContent.reset();
     assertFalse("getfattr should fail with extra argument",
-        0 == runCommand(new String[] { "-getfattr", "extra", "/test"}));
+        0 == runCommand(new String[] { "-getfattr", "extra", "-d", "/test"}));
+    assertTrue(errContent.toString().contains("Too many arguments"));
+    
+    errContent.reset();
     assertFalse("getfattr should fail without \"-n name\" or \"-d\"",
         0 == runCommand(new String[] { "-getfattr", "/test"}));
-  }
-
-  @Test
-  public void testGetfattrWithInvalidEncoding() throws Exception {
-    final PrintStream backup = System.err;
-    final ByteArrayOutputStream out = new ByteArrayOutputStream();
-    System.setErr(new PrintStream(out));
-    try {
-      runCommand(new String[] { "-getfattr", "-e", "invalid", "-n",
-                                "xattrname", "/file1" });
-      assertTrue("getfattr should fail with \"-getfattr: Invalid/unsupported "
-        + "econding option specified: invalid\". But the output is: "
-        + out.toString(), out.toString().contains("-getfattr: "
-          + "Invalid/unsupported encoding option specified: invalid"));
-    } finally {
-      System.setErr(backup);
-    }
+    assertTrue(errContent.toString().contains("Must specify '-n name' or '-d' option"));
+    
+    errContent.reset();
+    assertFalse("getfattr should fail with invalid encoding",
+        0 == runCommand(new String[] { "-getfattr", "-d", "-e", "aaa", "/test"}));
+    assertTrue(errContent.toString().contains("Invalid/unsupported encoding option specified: aaa"));
   }
 
   @Test
   public void testSetfattrValidations() throws Exception {
+    errContent.reset();
     assertFalse("setfattr should fail without path",
-        0 == runCommand(new String[] { "-setfattr" }));
+        0 == runCommand(new String[] { "-setfattr", "-n", "user.a1" }));
+    assertTrue(errContent.toString().contains("<path> is missing"));
+    
+    errContent.reset();
     assertFalse("setfattr should fail with extra arguments",
-        0 == runCommand(new String[] { "-setfattr", "extra", "/test"}));
+        0 == runCommand(new String[] { "-setfattr", "extra", "-n", "user.a1", "/test"}));
+    assertTrue(errContent.toString().contains("Too many arguments"));
+    
+    errContent.reset();
     assertFalse("setfattr should fail without \"-n name\" or \"-x name\"",
         0 == runCommand(new String[] { "-setfattr", "/test"}));
+    assertTrue(errContent.toString().contains("Must specify '-n name' or '-x name' option"));
   }
 
   private int runCommand(String[] commands) throws Exception {