You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by zj...@apache.org on 2015/04/10 06:25:24 UTC

[28/47] hadoop git commit: HDFS-8089. Move o.a.h.hdfs.web.resources.* to the client jars. Contributed by Haohui Mai.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21655165/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/PutOpParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/PutOpParam.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/PutOpParam.java
new file mode 100644
index 0000000..dede6a5
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/PutOpParam.java
@@ -0,0 +1,114 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.web.resources;
+
+import java.net.HttpURLConnection;
+
+/** Http POST operation parameter. */
+public class PutOpParam extends HttpOpParam<PutOpParam.Op> {
+  /** Put operations. */
+  public static enum Op implements HttpOpParam.Op {
+    CREATE(true, HttpURLConnection.HTTP_CREATED),
+
+    MKDIRS(false, HttpURLConnection.HTTP_OK),
+    CREATESYMLINK(false, HttpURLConnection.HTTP_OK),
+    RENAME(false, HttpURLConnection.HTTP_OK),
+    SETREPLICATION(false, HttpURLConnection.HTTP_OK),
+
+    SETOWNER(false, HttpURLConnection.HTTP_OK),
+    SETPERMISSION(false, HttpURLConnection.HTTP_OK),
+    SETTIMES(false, HttpURLConnection.HTTP_OK),
+
+    RENEWDELEGATIONTOKEN(false, HttpURLConnection.HTTP_OK, true),
+    CANCELDELEGATIONTOKEN(false, HttpURLConnection.HTTP_OK, true),
+
+    MODIFYACLENTRIES(false, HttpURLConnection.HTTP_OK),
+    REMOVEACLENTRIES(false, HttpURLConnection.HTTP_OK),
+    REMOVEDEFAULTACL(false, HttpURLConnection.HTTP_OK),
+    REMOVEACL(false, HttpURLConnection.HTTP_OK),
+    SETACL(false, HttpURLConnection.HTTP_OK),
+
+    SETXATTR(false, HttpURLConnection.HTTP_OK),
+    REMOVEXATTR(false, HttpURLConnection.HTTP_OK),
+
+    CREATESNAPSHOT(false, HttpURLConnection.HTTP_OK),
+    RENAMESNAPSHOT(false, HttpURLConnection.HTTP_OK),
+
+    NULL(false, HttpURLConnection.HTTP_NOT_IMPLEMENTED);
+
+    final boolean doOutputAndRedirect;
+    final int expectedHttpResponseCode;
+    final boolean requireAuth;
+
+    Op(final boolean doOutputAndRedirect, final int expectedHttpResponseCode) {
+      this(doOutputAndRedirect, expectedHttpResponseCode, false);
+    }
+
+    Op(final boolean doOutputAndRedirect, final int expectedHttpResponseCode,
+       final boolean requireAuth) {
+      this.doOutputAndRedirect = doOutputAndRedirect;
+      this.expectedHttpResponseCode = expectedHttpResponseCode;
+      this.requireAuth = requireAuth;
+    }
+
+    @Override
+    public HttpOpParam.Type getType() {
+      return HttpOpParam.Type.PUT;
+    }
+
+    @Override
+    public boolean getRequireAuth() {
+      return requireAuth;
+    }
+
+    @Override
+    public boolean getDoOutput() {
+      return doOutputAndRedirect;
+    }
+
+    @Override
+    public boolean getRedirect() {
+      return doOutputAndRedirect;
+    }
+
+    @Override
+    public int getExpectedHttpResponseCode() {
+      return expectedHttpResponseCode;
+    }
+
+    @Override
+    public String toQueryString() {
+      return NAME + "=" + this;
+    }
+  }
+
+  private static final Domain<Op> DOMAIN = new Domain<Op>(NAME, Op.class);
+
+  /**
+   * Constructor.
+   * @param str a string representation of the parameter value.
+   */
+  public PutOpParam(final String str) {
+    super(DOMAIN, DOMAIN.parse(str));
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21655165/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/RecursiveParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/RecursiveParam.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/RecursiveParam.java
new file mode 100644
index 0000000..4890a61
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/RecursiveParam.java
@@ -0,0 +1,49 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.web.resources;
+
+/** Recursive parameter. */
+public class RecursiveParam extends BooleanParam {
+  /** Parameter name. */
+  public static final String NAME = "recursive";
+  /** Default parameter value. */
+  public static final String DEFAULT = FALSE;
+
+  private static final Domain DOMAIN = new Domain(NAME);
+
+  /**
+   * Constructor.
+   * @param value the parameter value.
+   */
+  public RecursiveParam(final Boolean value) {
+    super(DOMAIN, value);
+  }
+
+  /**
+   * Constructor.
+   * @param str a string representation of the parameter value.
+   */
+  public RecursiveParam(final String str) {
+    this(DOMAIN.parse(str));
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21655165/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/RenameOptionSetParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/RenameOptionSetParam.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/RenameOptionSetParam.java
new file mode 100644
index 0000000..d7c157d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/RenameOptionSetParam.java
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.web.resources;
+
+import org.apache.hadoop.fs.Options;
+
+/** Rename option set parameter. */
+public class RenameOptionSetParam extends EnumSetParam<Options.Rename> {
+  /** Parameter name. */
+  public static final String NAME = "renameoptions";
+  /** Default parameter value. */
+  public static final String DEFAULT = "";
+
+  private static final Domain<Options.Rename> DOMAIN = new Domain<Options.Rename>(
+      NAME, Options.Rename.class);
+
+  /**
+   * Constructor.
+   * @param options rename options.
+   */
+  public RenameOptionSetParam(final Options.Rename... options) {
+    super(DOMAIN, toEnumSet(Options.Rename.class, options));
+  }
+
+  /**
+   * Constructor.
+   * @param str a string representation of the parameter value.
+   */
+  public RenameOptionSetParam(final String str) {
+    super(DOMAIN, DOMAIN.parse(str));
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21655165/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/RenewerParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/RenewerParam.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/RenewerParam.java
new file mode 100644
index 0000000..750e8bc
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/RenewerParam.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.web.resources;
+
+/** Renewer parameter. */
+public class RenewerParam extends StringParam {
+  /** Parameter name. */
+  public static final String NAME = "renewer";
+  /** Default parameter value. */
+  public static final String DEFAULT = NULL;
+
+  private static final Domain DOMAIN = new Domain(NAME, null);
+
+  /**
+   * Constructor.
+   * @param str a string representation of the parameter value.
+   */
+  public RenewerParam(final String str) {
+    super(DOMAIN, str == null || str.equals(DEFAULT)? null: str);
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21655165/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/ReplicationParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/ReplicationParam.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/ReplicationParam.java
new file mode 100644
index 0000000..af2ca23
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/ReplicationParam.java
@@ -0,0 +1,60 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.web.resources;
+
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_REPLICATION_DEFAULT;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_REPLICATION_KEY;
+
+import org.apache.hadoop.conf.Configuration;
+
+/** Replication parameter. */
+public class ReplicationParam extends ShortParam {
+  /** Parameter name. */
+  public static final String NAME = "replication";
+  /** Default parameter value. */
+  public static final String DEFAULT = NULL;
+
+  private static final Domain DOMAIN = new Domain(NAME);
+
+  /**
+   * Constructor.
+   * @param value the parameter value.
+   */
+  public ReplicationParam(final Short value) {
+    super(DOMAIN, value, (short)1, null);
+  }
+
+  /**
+   * Constructor.
+   * @param str a string representation of the parameter value.
+   */
+  public ReplicationParam(final String str) {
+    this(DOMAIN.parse(str));
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+
+  /** @return the value or, if it is null, return the default from conf. */
+  public short getValue(final Configuration conf) {
+    return getValue() != null? getValue()
+        : (short)conf.getInt(DFS_REPLICATION_KEY, DFS_REPLICATION_DEFAULT);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21655165/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/ShortParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/ShortParam.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/ShortParam.java
new file mode 100644
index 0000000..3f9af85
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/ShortParam.java
@@ -0,0 +1,88 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.web.resources;
+
+/** Short parameter. */
+abstract class ShortParam extends Param<Short, ShortParam.Domain> {
+  ShortParam(final Domain domain, final Short value,
+      final Short min, final Short max) {
+    super(domain, value);
+    checkRange(min, max);
+  }
+
+  private void checkRange(final Short min, final Short max) {
+    if (value == null) {
+      return;
+    }
+    if (min != null && value < min) {
+      throw new IllegalArgumentException("Invalid parameter range: " + getName()
+          + " = " + domain.toString(value) + " < " + domain.toString(min));
+    }
+    if (max != null && value > max) {
+      throw new IllegalArgumentException("Invalid parameter range: " + getName()
+          + " = " + domain.toString(value) + " > " + domain.toString(max));
+    }
+  }
+
+  @Override
+  public String toString() {
+    return getName() + "=" + domain.toString(getValue());
+  }
+
+  /** @return the parameter value as a string */
+  @Override
+  public final String getValueString() {
+    return domain.toString(getValue());
+  }
+
+  /** The domain of the parameter. */
+  static final class Domain extends Param.Domain<Short> {
+    /** The radix of the number. */
+    final int radix;
+
+    Domain(final String paramName) {
+      this(paramName, 10);
+    }
+
+    Domain(final String paramName, final int radix) {
+      super(paramName);
+      this.radix = radix;
+    }
+
+    @Override
+    public String getDomain() {
+      return "<" + NULL + " | short in radix " + radix + ">";
+    }
+
+    @Override
+    Short parse(final String str) {
+      try {
+        return NULL.equals(str) || str == null ? null : Short.parseShort(str,
+          radix);
+      } catch(NumberFormatException e) {
+        throw new IllegalArgumentException("Failed to parse \"" + str
+            + "\" as a radix-" + radix + " short integer.", e);
+      }
+    }
+
+    /** Convert a Short to a String. */
+    String toString(final Short n) {
+      return n == null? NULL: Integer.toString(n, radix);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21655165/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/SnapshotNameParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/SnapshotNameParam.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/SnapshotNameParam.java
new file mode 100644
index 0000000..ad3bcd6
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/SnapshotNameParam.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.web.resources;
+
+/**
+ * The snapshot name parameter for createSnapshot and deleteSnapshot operation.
+ * Also used to indicate the new snapshot name for renameSnapshot operation.
+ */
+public class SnapshotNameParam extends StringParam {
+  /** Parameter name. */
+  public static final String NAME = "snapshotname";
+
+  /** Default parameter value. */
+  public static final String DEFAULT = "";
+
+  private static final Domain DOMAIN = new Domain(NAME, null);
+
+  public SnapshotNameParam(final String str) {
+    super(DOMAIN, str != null && !str.equals(DEFAULT) ? str : null);
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21655165/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/StringParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/StringParam.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/StringParam.java
new file mode 100644
index 0000000..f063120
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/StringParam.java
@@ -0,0 +1,60 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.web.resources;
+
+import java.util.regex.Pattern;
+
+/** String parameter. */
+abstract class StringParam extends Param<String, StringParam.Domain> {
+  StringParam(final Domain domain, String str) {
+    super(domain, domain.parse(str));
+  }
+
+  /** @return the parameter value as a string */
+  @Override
+  public String getValueString() {
+    return value;
+  }
+
+  /** The domain of the parameter. */
+  static final class Domain extends Param.Domain<String> {
+    /** The pattern defining the domain; null . */
+    private final Pattern pattern;
+
+    Domain(final String paramName, final Pattern pattern) {
+      super(paramName);
+      this.pattern = pattern;
+    }
+
+    @Override
+    public final String getDomain() {
+      return pattern == null ? "<String>" : pattern.pattern();
+    }
+
+    @Override
+    final String parse(final String str) {
+      if (str != null && pattern != null) {
+        if (!pattern.matcher(str).matches()) {
+          throw new IllegalArgumentException("Invalid value: \"" + str
+              + "\" does not belong to the domain " + getDomain());
+        }
+      }
+      return str;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21655165/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/TokenArgumentParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/TokenArgumentParam.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/TokenArgumentParam.java
new file mode 100644
index 0000000..53b38ac
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/TokenArgumentParam.java
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.web.resources;
+
+/**
+ * Represents delegation token parameter as method arguments. This is
+ * different from {@link DelegationParam}.
+ */
+public class TokenArgumentParam extends StringParam {
+  /** Parameter name. */
+  public static final String NAME = "token";
+  /** Default parameter value. */
+  public static final String DEFAULT = "";
+
+  private static final Domain DOMAIN = new Domain(NAME, null);
+
+  /**
+   * Constructor.
+   * @param str A string representation of the parameter value.
+   */
+  public TokenArgumentParam(final String str) {
+    super(DOMAIN, str != null && !str.equals(DEFAULT) ? str : null);
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21655165/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/UserParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/UserParam.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/UserParam.java
new file mode 100644
index 0000000..46402fd
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/UserParam.java
@@ -0,0 +1,82 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.web.resources;
+
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_WEBHDFS_USER_PATTERN_DEFAULT;
+import org.apache.hadoop.security.UserGroupInformation;
+import com.google.common.annotations.VisibleForTesting;
+
+import java.text.MessageFormat;
+import java.util.regex.Pattern;
+
+/** User parameter. */
+public class UserParam extends StringParam {
+  /** Parameter name. */
+  public static final String NAME = "user.name";
+  /** Default parameter value. */
+  public static final String DEFAULT = "";
+
+  private static Domain domain = new Domain(NAME, Pattern.compile(DFS_WEBHDFS_USER_PATTERN_DEFAULT));
+
+  @VisibleForTesting
+  public static Domain getUserPatternDomain() {
+    return domain;
+  }
+
+  @VisibleForTesting
+  public static void setUserPatternDomain(Domain dm) {
+    domain = dm;
+  }
+
+  public static void setUserPattern(String pattern) {
+    domain = new Domain(NAME, Pattern.compile(pattern));
+  }
+
+  private static String validateLength(String str) {
+    if (str == null) {
+      throw new IllegalArgumentException(
+        MessageFormat.format("Parameter [{0}], cannot be NULL", NAME));
+    }
+    int len = str.length();
+    if (len < 1) {
+      throw new IllegalArgumentException(MessageFormat.format(
+        "Parameter [{0}], it's length must be at least 1", NAME));
+    }
+    return str;
+  }
+
+  /**
+   * Constructor.
+   * @param str a string representation of the parameter value.
+   */
+  public UserParam(final String str) {
+    super(domain, str == null || str.equals(DEFAULT)? null : validateLength(str));
+  }
+
+  /**
+   * Construct an object from a UGI.
+   */
+  public UserParam(final UserGroupInformation ugi) {
+    this(ugi.getShortUserName());
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21655165/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/XAttrEncodingParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/XAttrEncodingParam.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/XAttrEncodingParam.java
new file mode 100644
index 0000000..dd6eda1
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/XAttrEncodingParam.java
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.web.resources;
+
+import org.apache.hadoop.fs.XAttrCodec;
+
+public class XAttrEncodingParam extends EnumParam<XAttrCodec> {
+  /** Parameter name. */
+  public static final String NAME = "encoding";
+  /** Default parameter value. */
+  public static final String DEFAULT = "";
+
+  private static final Domain<XAttrCodec> DOMAIN =
+      new Domain<XAttrCodec>(NAME, XAttrCodec.class);
+
+  public XAttrEncodingParam(final XAttrCodec encoding) {
+    super(DOMAIN, encoding);
+  }
+
+  /**
+   * Constructor.
+   * @param str a string representation of the parameter value.
+   */
+  public XAttrEncodingParam(final String str) {
+    super(DOMAIN, str != null && !str.isEmpty() ? DOMAIN.parse(str) : null);
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+
+  @Override
+  public String getValueString() {
+    return value.toString();
+  }
+
+  public XAttrCodec getEncoding() {
+    return getValue();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21655165/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/XAttrNameParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/XAttrNameParam.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/XAttrNameParam.java
new file mode 100644
index 0000000..b64666a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/XAttrNameParam.java
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.web.resources;
+
+import java.util.regex.Pattern;
+
+public class XAttrNameParam extends StringParam {
+  /** Parameter name. **/
+  public static final String NAME = "xattr.name";
+  /** Default parameter value. **/
+  public static final String DEFAULT = "";
+
+  private static Domain DOMAIN = new Domain(NAME,
+      Pattern.compile(".*"));
+
+  public XAttrNameParam(final String str) {
+    super(DOMAIN, str == null || str.equals(DEFAULT) ? null : str);
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+
+  public String getXAttrName() {
+    final String v = getValue();
+    return v;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21655165/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/XAttrSetFlagParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/XAttrSetFlagParam.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/XAttrSetFlagParam.java
new file mode 100644
index 0000000..7cd5419
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/XAttrSetFlagParam.java
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.web.resources;
+
+import java.util.EnumSet;
+
+import org.apache.hadoop.fs.XAttrSetFlag;
+
+public class XAttrSetFlagParam extends EnumSetParam<XAttrSetFlag> {
+  /** Parameter name. */
+  public static final String NAME = "flag";
+  /** Default parameter value. */
+  public static final String DEFAULT = "";
+
+  private static final Domain<XAttrSetFlag> DOMAIN = new Domain<XAttrSetFlag>(
+      NAME, XAttrSetFlag.class);
+
+  public XAttrSetFlagParam(final EnumSet<XAttrSetFlag> flag) {
+    super(DOMAIN, flag);
+  }
+
+  /**
+   * Constructor.
+   * @param str a string representation of the parameter value.
+   */
+  public XAttrSetFlagParam(final String str) {
+    super(DOMAIN, DOMAIN.parse(str));
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+
+  public EnumSet<XAttrSetFlag> getFlag() {
+    return getValue();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21655165/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/XAttrValueParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/XAttrValueParam.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/XAttrValueParam.java
new file mode 100644
index 0000000..b08f1b3
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/XAttrValueParam.java
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.web.resources;
+
+import java.io.IOException;
+
+import org.apache.hadoop.fs.XAttrCodec;
+
+public class XAttrValueParam extends StringParam {
+  /** Parameter name. **/
+  public static final String NAME = "xattr.value";
+  /** Default parameter value. **/
+  public static final String DEFAULT = "";
+
+  private static Domain DOMAIN = new Domain(NAME, null);
+
+  public XAttrValueParam(final String str) {
+    super(DOMAIN, str == null || str.equals(DEFAULT) ? null : str);
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+
+  public byte[] getXAttrValue() throws IOException {
+    final String v = getValue();
+    return XAttrCodec.decodeValue(v);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21655165/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 1fdf6aa..dacdb3f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -402,6 +402,8 @@ Release 2.8.0 - UNRELEASED
     HDFS-8025. Addendum fix for HDFS-3087 Decomissioning on NN restart can
     complete without blocks being replicated. (Ming Ma via wang)
 
+    HDFS-8089. Move o.a.h.hdfs.web.resources.* to the client jars. (wheat9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21655165/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index 869df32..3bb2ae6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -34,10 +34,15 @@ import org.apache.hadoop.http.HttpConfig;
  */
 @InterfaceAudience.Private
 public class DFSConfigKeys extends CommonConfigurationKeys {
-  public static final String  DFS_BLOCK_SIZE_KEY = "dfs.blocksize";
-  public static final long    DFS_BLOCK_SIZE_DEFAULT = 128*1024*1024;
-  public static final String  DFS_REPLICATION_KEY = "dfs.replication";
-  public static final short   DFS_REPLICATION_DEFAULT = 3;
+  public static final String  DFS_BLOCK_SIZE_KEY =
+      HdfsClientConfigKeys.DFS_BLOCK_SIZE_KEY;
+  public static final long    DFS_BLOCK_SIZE_DEFAULT =
+      HdfsClientConfigKeys.DFS_BLOCK_SIZE_DEFAULT;
+  public static final String  DFS_REPLICATION_KEY =
+      HdfsClientConfigKeys.DFS_REPLICATION_KEY;
+  public static final short   DFS_REPLICATION_DEFAULT =
+      HdfsClientConfigKeys.DFS_REPLICATION_DEFAULT;
+
   public static final String  DFS_STREAM_BUFFER_SIZE_KEY = "dfs.stream-buffer-size";
   public static final int     DFS_STREAM_BUFFER_SIZE_DEFAULT = 4096;
   public static final String  DFS_BYTES_PER_CHECKSUM_KEY = "dfs.bytes-per-checksum";
@@ -49,7 +54,7 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_HDFS_BLOCKS_METADATA_ENABLED = "dfs.datanode.hdfs-blocks-metadata.enabled";
   public static final boolean DFS_HDFS_BLOCKS_METADATA_ENABLED_DEFAULT = false;
   public static final String DFS_WEBHDFS_ACL_PERMISSION_PATTERN_DEFAULT =
-      "^(default:)?(user|group|mask|other):[[A-Za-z_][A-Za-z0-9._-]]*:([rwx-]{3})?(,(default:)?(user|group|mask|other):[[A-Za-z_][A-Za-z0-9._-]]*:([rwx-]{3})?)*$";
+      HdfsClientConfigKeys.DFS_WEBHDFS_ACL_PERMISSION_PATTERN_DEFAULT;
 
   // HA related configuration
   public static final String  DFS_DATANODE_RESTART_REPLICA_EXPIRY_KEY = "dfs.datanode.restart.replica.expiration";
@@ -154,7 +159,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_WEBHDFS_AUTHENTICATION_FILTER_KEY = "dfs.web.authentication.filter";
   public static final String  DFS_WEBHDFS_AUTHENTICATION_FILTER_DEFAULT = AuthFilter.class.getName();
   public static final String  DFS_WEBHDFS_USER_PATTERN_KEY = "dfs.webhdfs.user.provider.user.pattern";
-  public static final String  DFS_WEBHDFS_USER_PATTERN_DEFAULT = "^[A-Za-z_][A-Za-z0-9._-]*[$]?$";
+  public static final String  DFS_WEBHDFS_USER_PATTERN_DEFAULT =
+      HdfsClientConfigKeys.DFS_WEBHDFS_USER_PATTERN_DEFAULT;
   public static final String  DFS_PERMISSIONS_ENABLED_KEY = "dfs.permissions.enabled";
   public static final boolean DFS_PERMISSIONS_ENABLED_DEFAULT = true;
   public static final String  DFS_PERMISSIONS_SUPERUSERGROUP_KEY = "dfs.permissions.superusergroup";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21655165/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/AccessTimeParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/AccessTimeParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/AccessTimeParam.java
deleted file mode 100644
index 9bc938d..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/AccessTimeParam.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.web.resources;
-
-/** Access time parameter. */
-public class AccessTimeParam extends LongParam {
-  /** Parameter name. */
-  public static final String NAME = "accesstime";
-  /** Default parameter value. */
-  public static final String DEFAULT = "-1";
-
-  private static final Domain DOMAIN = new Domain(NAME);
-
-  /**
-   * Constructor.
-   * @param value the parameter value.
-   */
-  public AccessTimeParam(final Long value) {
-    super(DOMAIN, value, -1L, null);
-  }
-
-  /**
-   * Constructor.
-   * @param str a string representation of the parameter value.
-   */
-  public AccessTimeParam(final String str) {
-    this(DOMAIN.parse(str));
-  }
-
-  @Override
-  public String getName() {
-    return NAME;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21655165/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/AclPermissionParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/AclPermissionParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/AclPermissionParam.java
deleted file mode 100644
index 9ed0a30..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/AclPermissionParam.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.web.resources;
-
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_WEBHDFS_ACL_PERMISSION_PATTERN_DEFAULT;
-
-import java.util.List;
-import java.util.regex.Pattern;
-
-import org.apache.hadoop.fs.permission.AclEntry;
-import org.apache.commons.lang.StringUtils;
-
-/** AclPermission parameter. */
-public class AclPermissionParam extends StringParam {
-  /** Parameter name. */
-  public static final String NAME = "aclspec";
-  /** Default parameter value. */
-  public static final String DEFAULT = "";
-
-  private static final Domain DOMAIN = new Domain(NAME,
-      Pattern.compile(DFS_WEBHDFS_ACL_PERMISSION_PATTERN_DEFAULT));
-
-  /**
-   * Constructor.
-   * 
-   * @param str a string representation of the parameter value.
-   */
-  public AclPermissionParam(final String str) {
-    super(DOMAIN, str == null || str.equals(DEFAULT) ? null : str);
-  }
-
-  public AclPermissionParam(List<AclEntry> acl) {
-    super(DOMAIN,parseAclSpec(acl).equals(DEFAULT) ? null : parseAclSpec(acl));
-  }
-
-  @Override
-  public String getName() {
-    return NAME;
-  }
-
-  public List<AclEntry> getAclPermission(boolean includePermission) {
-    final String v = getValue();
-    return (v != null ? AclEntry.parseAclSpec(v, includePermission) : AclEntry
-        .parseAclSpec(DEFAULT, includePermission));
-  }
-
-  /**
-   * @return parse {@code aclEntry} and return aclspec
-   */
-  private static String parseAclSpec(List<AclEntry> aclEntry) {
-    return StringUtils.join(aclEntry, ",");
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21655165/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/BlockSizeParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/BlockSizeParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/BlockSizeParam.java
deleted file mode 100644
index b6d82c2..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/BlockSizeParam.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.web.resources;
-
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY;
-
-import org.apache.hadoop.conf.Configuration;
-
-/** Block size parameter. */
-public class BlockSizeParam extends LongParam {
-  /** Parameter name. */
-  public static final String NAME = "blocksize";
-  /** Default parameter value. */
-  public static final String DEFAULT = NULL;
-
-  private static final Domain DOMAIN = new Domain(NAME);
-
-  /**
-   * Constructor.
-   * @param value the parameter value.
-   */
-  public BlockSizeParam(final Long value) {
-    super(DOMAIN, value, 1L, null);
-  }
-
-  /**
-   * Constructor.
-   * @param str a string representation of the parameter value.
-   */
-  public BlockSizeParam(final String str) {
-    this(DOMAIN.parse(str));
-  }
-
-  @Override
-  public String getName() {
-    return NAME;
-  }
-
-  /** @return the value or, if it is null, return the default from conf. */
-  public long getValue(final Configuration conf) {
-    return getValue() != null? getValue()
-        : conf.getLongBytes(DFS_BLOCK_SIZE_KEY, DFS_BLOCK_SIZE_DEFAULT);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21655165/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/BooleanParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/BooleanParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/BooleanParam.java
deleted file mode 100644
index 3437a0c..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/BooleanParam.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.web.resources;
-
-/** Boolean parameter. */
-abstract class BooleanParam extends Param<Boolean, BooleanParam.Domain> {
-  static final String TRUE = "true";
-  static final String FALSE = "false";
-
-  /** @return the parameter value as a string */
-  @Override
-  public String getValueString() {
-    return value.toString();
-  }
-
-  BooleanParam(final Domain domain, final Boolean value) {
-    super(domain, value);
-  }
-
-  /** The domain of the parameter. */
-  static final class Domain extends Param.Domain<Boolean> {
-    Domain(final String paramName) {
-      super(paramName);
-    }
-
-    @Override
-    public String getDomain() {
-      return "<" + NULL + " | boolean>";
-    }
-
-    @Override
-    Boolean parse(final String str) {
-      if (TRUE.equalsIgnoreCase(str)) {
-        return true;
-      } else if (FALSE.equalsIgnoreCase(str)) {
-        return false;
-      }
-      throw new IllegalArgumentException("Failed to parse \"" + str
-          + "\" to Boolean.");
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21655165/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/ConcatSourcesParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/ConcatSourcesParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/ConcatSourcesParam.java
deleted file mode 100644
index b68c5f5..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/ConcatSourcesParam.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hdfs.web.resources;
-
-import org.apache.hadoop.fs.Path;
-
-/** The concat source paths parameter. */
-public class ConcatSourcesParam extends StringParam {
-  /** Parameter name. */
-  public static final String NAME = "sources";
-
-  public static final String DEFAULT = "";
-
-  private static final Domain DOMAIN = new Domain(NAME, null);
-
-  private static String paths2String(Path[] paths) {
-    if (paths == null || paths.length == 0) {
-      return "";
-    }
-    final StringBuilder b = new StringBuilder(paths[0].toUri().getPath());
-    for(int i = 1; i < paths.length; i++) {
-      b.append(',').append(paths[i].toUri().getPath());
-    }
-    return b.toString();
-  }
-
-  /**
-   * Constructor.
-   * @param str a string representation of the parameter value.
-   */
-  public ConcatSourcesParam(String str) {
-    super(DOMAIN, str);
-  }
-
-  public ConcatSourcesParam(Path[] paths) {
-    this(paths2String(paths));
-  }
-
-  @Override
-  public String getName() {
-    return NAME;
-  }
-
-  /** @return the absolute path. */
-  public final String[] getAbsolutePaths() {
-    final String[] paths = getValue().split(",");
-    return paths;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21655165/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/CreateParentParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/CreateParentParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/CreateParentParam.java
deleted file mode 100644
index 8152515..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/CreateParentParam.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.web.resources;
-
-/** Create Parent parameter. */
-public class CreateParentParam extends BooleanParam {
-  /** Parameter name. */
-  public static final String NAME = "createparent";
-  /** Default parameter value. */
-  public static final String DEFAULT = FALSE;
-
-  private static final Domain DOMAIN = new Domain(NAME);
-
-  /**
-   * Constructor.
-   * @param value the parameter value.
-   */
-  public CreateParentParam(final Boolean value) {
-    super(DOMAIN, value);
-  }
-
-  /**
-   * Constructor.
-   * @param str a string representation of the parameter value.
-   */
-  public CreateParentParam(final String str) {
-    this(DOMAIN.parse(str));
-  }
-
-  @Override
-  public String getName() {
-    return NAME;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21655165/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/DelegationParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/DelegationParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/DelegationParam.java
deleted file mode 100644
index 57be43e..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/DelegationParam.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.web.resources;
-
-import org.apache.hadoop.security.UserGroupInformation;
-
-/** Represents delegation token used for authentication. */
-public class DelegationParam extends StringParam {
-  /** Parameter name. */
-  public static final String NAME = "delegation";
-  /** Default parameter value. */
-  public static final String DEFAULT = "";
-
-  private static final Domain DOMAIN = new Domain(NAME, null);
-
-  /**
-   * Constructor.
-   * @param str a string representation of the parameter value.
-   */
-  public DelegationParam(final String str) {
-    super(DOMAIN, UserGroupInformation.isSecurityEnabled()
-        && str != null && !str.equals(DEFAULT)? str: null);
-  }
-
-  @Override
-  public String getName() {
-    return NAME;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21655165/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/DeleteOpParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/DeleteOpParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/DeleteOpParam.java
deleted file mode 100644
index 65275e0..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/DeleteOpParam.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.web.resources;
-
-import java.net.HttpURLConnection;
-
-/** Http DELETE operation parameter. */
-public class DeleteOpParam extends HttpOpParam<DeleteOpParam.Op> {
-  /** Delete operations. */
-  public static enum Op implements HttpOpParam.Op {
-    DELETE(HttpURLConnection.HTTP_OK),
-    DELETESNAPSHOT(HttpURLConnection.HTTP_OK),
-
-    NULL(HttpURLConnection.HTTP_NOT_IMPLEMENTED);
-
-    final int expectedHttpResponseCode;
-
-    Op(final int expectedHttpResponseCode) {
-      this.expectedHttpResponseCode = expectedHttpResponseCode;
-    }
-
-    @Override
-    public HttpOpParam.Type getType() {
-      return HttpOpParam.Type.DELETE;
-    }
-
-    @Override
-    public boolean getRequireAuth() {
-      return false;
-    }
-
-    @Override
-    public boolean getDoOutput() {
-      return false;
-    }
-
-    @Override
-    public boolean getRedirect() {
-      return false;
-    }
-
-    @Override
-    public int getExpectedHttpResponseCode() {
-      return expectedHttpResponseCode;
-    }
-
-    @Override
-    public String toQueryString() {
-      return NAME + "=" + this;
-    }
-  }
-
-  private static final Domain<Op> DOMAIN = new Domain<Op>(NAME, Op.class);
-
-  /**
-   * Constructor.
-   * @param str a string representation of the parameter value.
-   */
-  public DeleteOpParam(final String str) {
-    super(DOMAIN, DOMAIN.parse(str));
-  }
-
-  @Override
-  public String getName() {
-    return NAME;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21655165/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/DestinationParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/DestinationParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/DestinationParam.java
deleted file mode 100644
index 6759738..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/DestinationParam.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.web.resources;
-
-import org.apache.hadoop.fs.Path;
-
-/** Destination path parameter. */
-public class DestinationParam extends StringParam {
-  /** Parameter name. */
-  public static final String NAME = "destination";
-  /** Default parameter value. */
-  public static final String DEFAULT = "";
-
-  private static final Domain DOMAIN = new Domain(NAME, null);
-
-  private static String validate(final String str) {
-    if (str == null || str.equals(DEFAULT)) {
-      return null;
-    }
-    if (!str.startsWith(Path.SEPARATOR)) {
-      throw new IllegalArgumentException("Invalid parameter value: " + NAME
-          + " = \"" + str + "\" is not an absolute path.");
-    }
-    return new Path(str).toUri().getPath();
-  }
-
-  /**
-   * Constructor.
-   * @param str a string representation of the parameter value.
-   */
-  public DestinationParam(final String str) {
-    super(DOMAIN, validate(str));
-  }
-
-  @Override
-  public String getName() {
-    return NAME;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21655165/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/DoAsParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/DoAsParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/DoAsParam.java
deleted file mode 100644
index 13d188c..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/DoAsParam.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.web.resources;
-
-/** DoAs parameter for proxy user. */
-public class DoAsParam extends StringParam {
-  /** Parameter name. */
-  public static final String NAME = "doas";
-  /** Default parameter value. */
-  public static final String DEFAULT = "";
-
-  private static final Domain DOMAIN = new Domain(NAME, null);
-
-  /**
-   * Constructor.
-   * @param str a string representation of the parameter value.
-   */
-  public DoAsParam(final String str) {
-    super(DOMAIN, str == null || str.equals(DEFAULT)? null: str);
-  }
-
-  @Override
-  public String getName() {
-    return NAME;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21655165/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/EnumParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/EnumParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/EnumParam.java
deleted file mode 100644
index 60d201b..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/EnumParam.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.web.resources;
-
-import java.util.Arrays;
-import org.apache.hadoop.util.StringUtils;
-
-abstract class EnumParam<E extends Enum<E>> extends Param<E, EnumParam.Domain<E>> {
-  EnumParam(final Domain<E> domain, final E value) {
-    super(domain, value);
-  }
-
-  /** The domain of the parameter. */
-  static final class Domain<E extends Enum<E>> extends Param.Domain<E> {
-    private final Class<E> enumClass;
-
-    Domain(String name, final Class<E> enumClass) {
-      super(name);
-      this.enumClass = enumClass;
-    }
-
-    @Override
-    public final String getDomain() {
-      return Arrays.asList(enumClass.getEnumConstants()).toString();
-    }
-
-    @Override
-    final E parse(final String str) {
-      return Enum.valueOf(enumClass, StringUtils.toUpperCase(str));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21655165/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/EnumSetParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/EnumSetParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/EnumSetParam.java
deleted file mode 100644
index c2dfadf..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/EnumSetParam.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.web.resources;
-
-import java.util.Arrays;
-import java.util.EnumSet;
-import java.util.Iterator;
-import org.apache.hadoop.util.StringUtils;
-
-abstract class EnumSetParam<E extends Enum<E>> extends Param<EnumSet<E>, EnumSetParam.Domain<E>> {
-  /** Convert an EnumSet to a string of comma separated values. */
-  static <E extends Enum<E>> String toString(EnumSet<E> set) {
-    if (set == null || set.isEmpty()) {
-      return "";
-    } else {
-      final StringBuilder b = new StringBuilder();
-      final Iterator<E> i = set.iterator();
-      b.append(i.next());
-      for(; i.hasNext(); ) {
-        b.append(',').append(i.next());
-      }
-      return b.toString();
-    }
-  }
-
-  static <E extends Enum<E>> EnumSet<E> toEnumSet(final Class<E> clazz,
-      final E... values) {
-    final EnumSet<E> set = EnumSet.noneOf(clazz);
-    set.addAll(Arrays.asList(values));
-    return set;
-  }
-
-  EnumSetParam(final Domain<E> domain, final EnumSet<E> value) {
-    super(domain, value);
-  }
-
-  @Override
-  public String toString() {
-    return getName() + "=" + toString(value);
-  }
-
-  /** @return the parameter value as a string */
-  @Override
-  public String getValueString() {
-    return toString(value);
-  }
-  
-  /** The domain of the parameter. */
-  static final class Domain<E extends Enum<E>> extends Param.Domain<EnumSet<E>> {
-    private final Class<E> enumClass;
-
-    Domain(String name, final Class<E> enumClass) {
-      super(name);
-      this.enumClass = enumClass;
-    }
-
-    @Override
-    public final String getDomain() {
-      return Arrays.asList(enumClass.getEnumConstants()).toString();
-    }
-
-    /** The string contains a comma separated values. */
-    @Override
-    final EnumSet<E> parse(final String str) {
-      final EnumSet<E> set = EnumSet.noneOf(enumClass);
-      if (!str.isEmpty()) {
-        for(int i, j = 0; j >= 0; ) {
-          i = j > 0 ? j + 1 : 0;
-          j = str.indexOf(',', i);
-          final String sub = j >= 0? str.substring(i, j): str.substring(i);
-          set.add(Enum.valueOf(enumClass, StringUtils.toUpperCase(sub.trim())));
-        }
-      }
-      return set;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21655165/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/ExcludeDatanodesParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/ExcludeDatanodesParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/ExcludeDatanodesParam.java
deleted file mode 100644
index 3f44fae..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/ExcludeDatanodesParam.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.web.resources;
-
-
-/** Exclude datanodes param */
-public class ExcludeDatanodesParam extends StringParam {
-  /** Parameter name. */
-  public static final String NAME = "excludedatanodes";
-  /** Default parameter value. */
-  public static final String DEFAULT = "";
-
-  private static final Domain DOMAIN = new Domain(NAME, null);
-
-  /**
-   * Constructor.
-   * @param str a string representation of the parameter value.
-   */
-  public ExcludeDatanodesParam(final String str) {
-    super(DOMAIN, str == null || str.equals(DEFAULT)? null: DOMAIN.parse(str));
-  }
-
-  @Override
-  public String getName() {
-    return NAME;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21655165/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/FsActionParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/FsActionParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/FsActionParam.java
deleted file mode 100644
index c840196..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/FsActionParam.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.web.resources;
-
-import org.apache.hadoop.fs.permission.FsAction;
-
-import java.util.regex.Pattern;
-
-/** {@link FsAction} Parameter */
-public class FsActionParam extends StringParam {
-
-  /** Parameter name. */
-  public static final String NAME = "fsaction";
-
-  /** Default parameter value. */
-  public static final String DEFAULT = NULL;
-
-  private static String FS_ACTION_PATTERN = "[rwx-]{3}";
-
-  private static final Domain DOMAIN = new Domain(NAME,
-      Pattern.compile(FS_ACTION_PATTERN));
-
-  /**
-   * Constructor.
-   * @param str a string representation of the parameter value.
-   */
-  public FsActionParam(final String str) {
-    super(DOMAIN, str == null || str.equals(DEFAULT)? null: str);
-  }
-
-  /**
-   * Constructor.
-   * @param value the parameter value.
-   */
-  public FsActionParam(final FsAction value) {
-    super(DOMAIN, value == null? null: value.SYMBOL);
-  }
-
-  @Override
-  public String getName() {
-    return NAME;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21655165/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java
deleted file mode 100644
index f63ed44..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java
+++ /dev/null
@@ -1,106 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.web.resources;
-
-import java.net.HttpURLConnection;
-
-/** Http GET operation parameter. */
-public class GetOpParam extends HttpOpParam<GetOpParam.Op> {
-  /** Get operations. */
-  public static enum Op implements HttpOpParam.Op {
-    OPEN(true, HttpURLConnection.HTTP_OK),
-
-    GETFILESTATUS(false, HttpURLConnection.HTTP_OK),
-    LISTSTATUS(false, HttpURLConnection.HTTP_OK),
-    GETCONTENTSUMMARY(false, HttpURLConnection.HTTP_OK),
-    GETFILECHECKSUM(true, HttpURLConnection.HTTP_OK),
-
-    GETHOMEDIRECTORY(false, HttpURLConnection.HTTP_OK),
-    GETDELEGATIONTOKEN(false, HttpURLConnection.HTTP_OK, true),
-
-    /** GET_BLOCK_LOCATIONS is a private unstable op. */
-    GET_BLOCK_LOCATIONS(false, HttpURLConnection.HTTP_OK),
-    GETACLSTATUS(false, HttpURLConnection.HTTP_OK),
-    GETXATTRS(false, HttpURLConnection.HTTP_OK),
-    LISTXATTRS(false, HttpURLConnection.HTTP_OK),
-
-    NULL(false, HttpURLConnection.HTTP_NOT_IMPLEMENTED),
-
-    CHECKACCESS(false, HttpURLConnection.HTTP_OK);
-
-    final boolean redirect;
-    final int expectedHttpResponseCode;
-    final boolean requireAuth;
-
-    Op(final boolean redirect, final int expectedHttpResponseCode) {
-      this(redirect, expectedHttpResponseCode, false);
-    }
-    
-    Op(final boolean redirect, final int expectedHttpResponseCode,
-       final boolean requireAuth) {
-      this.redirect = redirect;
-      this.expectedHttpResponseCode = expectedHttpResponseCode;
-      this.requireAuth = requireAuth;
-    }
-
-    @Override
-    public HttpOpParam.Type getType() {
-      return HttpOpParam.Type.GET;
-    }
-    
-    @Override
-    public boolean getRequireAuth() {
-      return requireAuth;
-    }
-
-    @Override
-    public boolean getDoOutput() {
-      return false;
-    }
-
-    @Override
-    public boolean getRedirect() {
-      return redirect;
-    }
-
-    @Override
-    public int getExpectedHttpResponseCode() {
-      return expectedHttpResponseCode;
-    }
-
-    @Override
-    public String toQueryString() {
-      return NAME + "=" + this;
-    }
-  }
-
-  private static final Domain<Op> DOMAIN = new Domain<Op>(NAME, Op.class);
-
-  /**
-   * Constructor.
-   * @param str a string representation of the parameter value.
-   */
-  public GetOpParam(final String str) {
-    super(DOMAIN, DOMAIN.parse(str));
-  }
-
-  @Override
-  public String getName() {
-    return NAME;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21655165/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/GroupParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/GroupParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/GroupParam.java
deleted file mode 100644
index c0429cc..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/GroupParam.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.web.resources;
-
-/** Group parameter. */
-public class GroupParam extends StringParam {
-  /** Parameter name. */
-  public static final String NAME = "group";
-  /** Default parameter value. */
-  public static final String DEFAULT = "";
-
-  private static final Domain DOMAIN = new Domain(NAME, null);
-
-  /**
-   * Constructor.
-   * @param str a string representation of the parameter value.
-   */
-  public GroupParam(final String str) {
-    super(DOMAIN, str == null || str.equals(DEFAULT)? null: str);
-  }
-
-  @Override
-  public String getName() {
-    return NAME;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21655165/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/HttpOpParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/HttpOpParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/HttpOpParam.java
deleted file mode 100644
index f4c24ff..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/HttpOpParam.java
+++ /dev/null
@@ -1,134 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.web.resources;
-
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-
-import javax.ws.rs.core.Response;
-
-
-/** Http operation parameter. */
-public abstract class HttpOpParam<E extends Enum<E> & HttpOpParam.Op>
-    extends EnumParam<E> {
-  /** Parameter name. */
-  public static final String NAME = "op";
-
-  /** Default parameter value. */
-  public static final String DEFAULT = NULL;
-
-  /** Http operation types */
-  public static enum Type {
-    GET, PUT, POST, DELETE;
-  }
-
-  /** Http operation interface. */
-  public static interface Op {
-    /** @return the Http operation type. */
-    public Type getType();
-
-    /** @return true if the operation cannot use a token */
-    public boolean getRequireAuth();
-    
-    /** @return true if the operation will do output. */
-    public boolean getDoOutput();
-
-    /** @return true if the operation will be redirected. */
-    public boolean getRedirect();
-
-    /** @return true the expected http response code. */
-    public int getExpectedHttpResponseCode();
-
-    /** @return a URI query string. */
-    public String toQueryString();
-  }
-
-  /** Expects HTTP response 307 "Temporary Redirect". */
-  public static class TemporaryRedirectOp implements Op {
-    static final TemporaryRedirectOp CREATE = new TemporaryRedirectOp(
-        PutOpParam.Op.CREATE);
-    static final TemporaryRedirectOp APPEND = new TemporaryRedirectOp(
-        PostOpParam.Op.APPEND);
-    static final TemporaryRedirectOp OPEN = new TemporaryRedirectOp(
-        GetOpParam.Op.OPEN);
-    static final TemporaryRedirectOp GETFILECHECKSUM = new TemporaryRedirectOp(
-        GetOpParam.Op.GETFILECHECKSUM);
-    
-    static final List<TemporaryRedirectOp> values
-        = Collections.unmodifiableList(Arrays.asList(CREATE, APPEND, OPEN,
-                                       GETFILECHECKSUM));
-
-    /** Get an object for the given op. */
-    public static TemporaryRedirectOp valueOf(final Op op) {
-      for(TemporaryRedirectOp t : values) {
-        if (op == t.op) {
-          return t;
-        }
-      }
-      throw new IllegalArgumentException(op + " not found.");
-    }
-
-    private final Op op;
-
-    private TemporaryRedirectOp(final Op op) {
-      this.op = op;
-    }
-
-    @Override
-    public Type getType() {
-      return op.getType();
-    }
-
-    @Override
-    public boolean getRequireAuth() {
-      return op.getRequireAuth();
-    }
-
-    @Override
-    public boolean getDoOutput() {
-      return false;
-    }
-
-    @Override
-    public boolean getRedirect() {
-      return false;
-    }
-
-    /** Override the original expected response with "Temporary Redirect". */
-    @Override
-    public int getExpectedHttpResponseCode() {
-      return Response.Status.TEMPORARY_REDIRECT.getStatusCode();
-    }
-
-    @Override
-    public String toQueryString() {
-      return op.toQueryString();
-    }
-  }
-
-  /** @return the parameter value as a string */
-  @Override
-  public String getValueString() {
-    return value.toString();
-  }
-
-  HttpOpParam(final Domain<E> domain, final E value) {
-    super(domain, value);
-  }
-}
\ No newline at end of file