You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bigtop.apache.org by ja...@apache.org on 2014/07/13 03:25:40 UTC

[3/3] git commit: BIGTOP-1342. Make TestCLI usable for both HDFS and HCFS

BIGTOP-1342. Make TestCLI usable for both HDFS and HCFS

This change splits TestCLI code into 2 parts:

 * TestCLI using testHCFSConf.xml (cases applicable for any Hadoop filesystem)
 * TestHDFSCLI using testHDFSConf.xml (HDFS only cases)

Signed-off-by: jay@apache.org <jayunit100>


Project: http://git-wip-us.apache.org/repos/asf/bigtop/repo
Commit: http://git-wip-us.apache.org/repos/asf/bigtop/commit/f0bab035
Tree: http://git-wip-us.apache.org/repos/asf/bigtop/tree/f0bab035
Diff: http://git-wip-us.apache.org/repos/asf/bigtop/diff/f0bab035

Branch: refs/heads/master
Commit: f0bab0355ee2184ccf9a23524683caa4e4b227ec
Parents: 3daaf98
Author: Martin Bukatovic <mb...@redhat.com>
Authored: Wed Jun 18 18:14:05 2014 +0200
Committer: jay@apache.org <jayunit100>
Committed: Sat Jul 12 21:24:56 2014 -0400

----------------------------------------------------------------------
 .../bigtop/itest/hadoop/hcfs/FSCmdExecutor.java |    73 +
 .../bigtop/itest/hadoop/hcfs/TestCLI.java       |   160 +
 .../bigtop/itest/hadoop/hdfs/FSCmdExecutor.java |    68 -
 .../bigtop/itest/hadoop/hdfs/TestCLI.java       |   135 -
 .../bigtop/itest/hadoop/hdfs/TestHDFSCLI.java   |    32 +
 .../resources/clitest_data/testHCFSConf.xml     | 12660 +++++++++++++++++
 .../resources/clitest_data/testHDFSConf.xml     | 12585 +---------------
 7 files changed, 13045 insertions(+), 12668 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/f0bab035/bigtop-tests/test-artifacts/hadoop/src/main/groovy/org/apache/bigtop/itest/hadoop/hcfs/FSCmdExecutor.java
----------------------------------------------------------------------
diff --git a/bigtop-tests/test-artifacts/hadoop/src/main/groovy/org/apache/bigtop/itest/hadoop/hcfs/FSCmdExecutor.java b/bigtop-tests/test-artifacts/hadoop/src/main/groovy/org/apache/bigtop/itest/hadoop/hcfs/FSCmdExecutor.java
new file mode 100644
index 0000000..b5eb9c8
--- /dev/null
+++ b/bigtop-tests/test-artifacts/hadoop/src/main/groovy/org/apache/bigtop/itest/hadoop/hcfs/FSCmdExecutor.java
@@ -0,0 +1,73 @@
+/*
+ * 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.bigtop.itest.hadoop.hcfs;
+
+import java.io.File;
+import java.util.StringTokenizer;
+
+import org.apache.hadoop.fs.FsShell;
+import org.apache.hadoop.cli.CLITestHelper;
+import org.apache.hadoop.cli.util.CommandExecutor;
+import org.apache.hadoop.util.ToolRunner;
+
+public class FSCmdExecutor extends CommandExecutor {
+  protected String namenode = null;
+  protected FsShell shell = null;
+
+  public FSCmdExecutor(String namenode, FsShell shell) {
+    this.namenode = namenode;
+    this.shell = shell;
+  }
+
+  /**
+   * Execute given hadoop FsShell command.
+   */
+  protected void execute(final String cmd) throws Exception{
+    String[] args = getCommandAsArgs(cmd, "NAMENODE", this.namenode);
+    ToolRunner.run(shell, args);
+  }
+
+  /**
+   * Prapare given FsShell command for execution by ToolRunner.
+   * This method also expands all variables used in the command.
+   */
+  @Override
+  protected String[] getCommandAsArgs(final String cmd, final String masterKey,
+                                      final String master) {
+    StringTokenizer tokenizer = new StringTokenizer(cmd, " ");
+    String[] args = new String[tokenizer.countTokens()];
+    int i = 0;
+    while (tokenizer.hasMoreTokens()) {
+      args[i] = tokenizer.nextToken();
+      // HCFS fs.default.name Hack (see TestCLI.java)
+      args[i] = args[i].replaceAll("NAMENODETEST_DIR_ABSOLUTE", TestCLI.NAMENODE_TESTDIR_HACK);
+      args[i] = args[i].replaceAll(masterKey, master);
+      args[i] = args[i].replaceAll("CLITEST_DATA",
+          new File(CLITestHelper.TEST_CACHE_DATA_DIR).
+          toURI().toString().replace(' ', '+'));
+      args[i] = args[i].replaceAll("USER_NAME", System.getProperty("user.name"));
+      args[i] = args[i].replaceAll("TEST_DIR_ABSOLUTE", TestCLI.TEST_DIR_ABSOLUTE);
+      args[i] = args[i].replaceAll("HCFS_SCHEME", TestCLI.HCFS_SCHEME);
+      args[i] = args[i].replaceAll("HCFS_DIRSIZE", TestCLI.HCFS_DIRSIZE);
+      args[i] = args[i].replaceAll("HCFS_NNMATCH", TestCLI.HCFS_NNMATCH);
+
+      i++;
+    }
+    return args;
+  }
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/f0bab035/bigtop-tests/test-artifacts/hadoop/src/main/groovy/org/apache/bigtop/itest/hadoop/hcfs/TestCLI.java
----------------------------------------------------------------------
diff --git a/bigtop-tests/test-artifacts/hadoop/src/main/groovy/org/apache/bigtop/itest/hadoop/hcfs/TestCLI.java b/bigtop-tests/test-artifacts/hadoop/src/main/groovy/org/apache/bigtop/itest/hadoop/hcfs/TestCLI.java
new file mode 100644
index 0000000..1b4dcd7
--- /dev/null
+++ b/bigtop-tests/test-artifacts/hadoop/src/main/groovy/org/apache/bigtop/itest/hadoop/hcfs/TestCLI.java
@@ -0,0 +1,160 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.bigtop.itest.hadoop.hcfs;
+
+import java.io.File;
+
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.fs.FsShell;
+import org.apache.hadoop.cli.CLITestHelper;
+import org.apache.hadoop.cli.util.CLICommand;
+import org.apache.hadoop.cli.util.CLICommandFS;
+import org.apache.hadoop.cli.util.CommandExecutor;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.bigtop.itest.shell.Shell;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.Assert;
+
+import org.apache.commons.lang.StringUtils;
+
+/**
+ * Tests for the Command Line Interface (CLI)
+ */
+public class TestCLI extends CLITestHelper {
+  public static final String TEST_DIR_ABSOLUTE = "/tmp/testcli_" + Long.valueOf(System.currentTimeMillis());
+  public static String HCFS_SCHEME;
+  public static String HCFS_DIRSIZE;
+  public static String HCFS_NNMATCH;
+  public static String NAMENODE_TESTDIR_HACK;
+  private String supergroup;
+  private String namenode;
+  private static Shell shHDFS = new Shell("/bin/bash");
+
+  @Before
+  @Override
+  public void setUp() throws Exception {
+    readTestConfigFile();
+
+    // Configuration of real Hadoop cluster
+    conf = new HdfsConfiguration();
+    supergroup = System.getProperty("hcfs.root.groupname",
+        conf.get(DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROUP_KEY));
+    namenode = conf.get(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY);
+
+    conf.setBoolean(CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION, true);
+    // Many of the tests expect a replication value of 1 in the output
+    conf.setInt("dfs.replication", 1);
+
+    clitestDataDir = new File(TEST_CACHE_DATA_DIR).toURI().toString().replace(' ', '+');
+
+    String[] createTestcliDirCmds = {
+        "hadoop fs -mkdir -p "  + TEST_DIR_ABSOLUTE,
+        "hadoop fs -chmod 777 " + TEST_DIR_ABSOLUTE
+    };
+    shHDFS.exec(createTestcliDirCmds);
+
+    // Check assumptions which would make some cases fail if not met
+    Assert.assertEquals("Creation of testcli dir should succeed and return 0"
+        + " (but it failed with the following error message: "
+        + StringUtils.join(shHDFS.getErr().toArray(), "\\n") + ")",
+        0, shHDFS.getRet());
+    // We can't just use conf.setInt(fs.trash.interval",0) because if trash is
+    // enabled on the server, client configuration value is ignored.
+    Assert.assertEquals("HDFS trash should be disabled via fs.trash.interval",
+        0, conf.getInt("fs.trash.interval",0));
+    Assert.assertEquals("This test needs to be run under root user of hcfs",
+        System.getProperty("hcfs.root.username", "hdfs"),
+        System.getProperty("user.name"));
+
+    // Initialize variables from test config file
+    HCFS_SCHEME = System.getProperty("hcfs.scheme", "hdfs:");
+    HCFS_DIRSIZE = System.getProperty("hcfs.dirsize.pattern", "0");
+    HCFS_NNMATCH = System.getProperty("hcfs.namenode.pattern", "\\\\w+[-.a-z0-9]*(:[0-9]+)?");
+
+    // HCFS fs.default.name Hack
+    // Hadoop property 'fs.default.name' usually has value like this one:
+    // "hdfs://namenode_hostname:port". But for other hadoop filesystems, the
+    // value may just end with 3 slashes in a row (eg. 'glusterfs:///' or
+    // 'maprfs:///'). This leads to file paths with 4 slashes in it (eg.
+    // 'glusterfs:////tmp/testcli_sth') which are shortened back to
+    // 'glusterfs:///tmp/...' if the file actually exists. To fix this we just
+    // replace 4 slashes with 3 to prevent this from happening.
+    String namenode_testdir = namenode + TEST_DIR_ABSOLUTE;
+    NAMENODE_TESTDIR_HACK = namenode_testdir.replace(":////", ":///");
+  }
+
+  @After
+  @Override
+  public void tearDown() throws Exception {
+    super.tearDown();
+
+    String removeTestcliDirCmd = "hadoop fs -rm -r " + TEST_DIR_ABSOLUTE;
+    shHDFS.exec(removeTestcliDirCmd);
+  }
+
+  @Override
+  protected String getTestFile() {
+    return "testHCFSConf.xml";
+  }
+
+  @Test
+  @Override
+  public void testAll() {
+    super.testAll();
+  }
+
+  /**
+   * Expand commands from the test config file.
+   * This method is used in displayResults() and compareTestOutput() only,
+   * so it doesn't have any effect on the test execution itself.
+   *
+   * @param cmd
+   * @return String expanded command
+   */
+  @Override
+  protected String expandCommand(final String cmd) {
+    String expCmd = super.expandCommand(cmd);
+    // note: super.expandCommand() expands CLITEST_DATA and USERNAME
+    expCmd = expCmd.replaceAll("NAMENODETEST_DIR_ABSOLUTE", NAMENODE_TESTDIR_HACK);
+    expCmd = expCmd.replaceAll("TEST_DIR_ABSOLUTE", TEST_DIR_ABSOLUTE);
+    expCmd = expCmd.replaceAll("supergroup", supergroup);
+    expCmd = expCmd.replaceAll("NAMENODE", namenode);
+    expCmd = expCmd.replaceAll("USER_NAME", System.getProperty("user.name"));
+    expCmd = expCmd.replaceAll("HCFS_SCHEME", HCFS_SCHEME);
+    expCmd = expCmd.replaceAll("HCFS_DIRSIZE", HCFS_DIRSIZE);
+    expCmd = expCmd.replaceAll("HCFS_NNMATCH", HCFS_NNMATCH);
+    return expCmd;
+  }
+
+  /**
+   * Execute given hadoop FsShell command (via Toolrunner).
+   */
+  @Override
+  protected CommandExecutor.Result execute(CLICommand cmd) throws Exception {
+    if (cmd.getType() instanceof CLICommandFS) {
+      CommandExecutor cmdExecutor = new FSCmdExecutor(namenode, new FsShell(conf));
+      return cmdExecutor.executeCommand(cmd.getCmd());
+    } else {
+      throw new IllegalArgumentException("Unknown type of test command: " + cmd.getType());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/f0bab035/bigtop-tests/test-artifacts/hadoop/src/main/groovy/org/apache/bigtop/itest/hadoop/hdfs/FSCmdExecutor.java
----------------------------------------------------------------------
diff --git a/bigtop-tests/test-artifacts/hadoop/src/main/groovy/org/apache/bigtop/itest/hadoop/hdfs/FSCmdExecutor.java b/bigtop-tests/test-artifacts/hadoop/src/main/groovy/org/apache/bigtop/itest/hadoop/hdfs/FSCmdExecutor.java
deleted file mode 100644
index 04f0d57..0000000
--- a/bigtop-tests/test-artifacts/hadoop/src/main/groovy/org/apache/bigtop/itest/hadoop/hdfs/FSCmdExecutor.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.bigtop.itest.hadoop.hdfs;
-
-import java.io.File;
-import java.util.StringTokenizer;
-
-import org.apache.hadoop.fs.FsShell;
-import org.apache.hadoop.cli.CLITestHelper;
-import org.apache.hadoop.cli.util.CommandExecutor;
-import org.apache.hadoop.util.ToolRunner;
-
-public class FSCmdExecutor extends CommandExecutor {
-  protected String namenode = null;
-  protected FsShell shell = null;
-
-  public FSCmdExecutor(String namenode, FsShell shell) {
-    this.namenode = namenode;
-    this.shell = shell;
-  }
-
-  /**
-   * Execute given hadoop FsShell command.
-   */
-  protected void execute(final String cmd) throws Exception{
-    String[] args = getCommandAsArgs(cmd, "NAMENODE", this.namenode);
-    ToolRunner.run(shell, args);
-  }
-
-  /**
-   * Prapare given FsShell command for execution by ToolRunner.
-   * This method also expands all variables used in the command.
-   */
-  @Override
-  protected String[] getCommandAsArgs(final String cmd, final String masterKey,
-                                      final String master) {
-    StringTokenizer tokenizer = new StringTokenizer(cmd, " ");
-    String[] args = new String[tokenizer.countTokens()];
-    int i = 0;
-    while (tokenizer.hasMoreTokens()) {
-      args[i] = tokenizer.nextToken();
-      args[i] = args[i].replaceAll(masterKey, master);
-      args[i] = args[i].replaceAll("CLITEST_DATA", 
-          new File(CLITestHelper.TEST_CACHE_DATA_DIR).
-          toURI().toString().replace(' ', '+'));
-      args[i] = args[i].replaceAll("USER_NAME", System.getProperty("user.name"));
-      args[i] = args[i].replaceAll("TEST_DIR_ABSOLUTE", TestCLI.TEST_DIR_ABSOLUTE);
-
-      i++;
-    }
-    return args;
-  }
-}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/f0bab035/bigtop-tests/test-artifacts/hadoop/src/main/groovy/org/apache/bigtop/itest/hadoop/hdfs/TestCLI.java
----------------------------------------------------------------------
diff --git a/bigtop-tests/test-artifacts/hadoop/src/main/groovy/org/apache/bigtop/itest/hadoop/hdfs/TestCLI.java b/bigtop-tests/test-artifacts/hadoop/src/main/groovy/org/apache/bigtop/itest/hadoop/hdfs/TestCLI.java
deleted file mode 100644
index 8214209..0000000
--- a/bigtop-tests/test-artifacts/hadoop/src/main/groovy/org/apache/bigtop/itest/hadoop/hdfs/TestCLI.java
+++ /dev/null
@@ -1,135 +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
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * 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.bigtop.itest.hadoop.hdfs;
-
-import java.io.File;
-
-import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.apache.hadoop.fs.FsShell;
-import org.apache.hadoop.cli.CLITestHelper;
-import org.apache.hadoop.cli.util.CLICommand;
-import org.apache.hadoop.cli.util.CLICommandFS;
-import org.apache.hadoop.cli.util.CommandExecutor;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.bigtop.itest.shell.Shell;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.Assert;
-
-import org.apache.commons.lang.StringUtils;
-
-/**
- * Tests for the Command Line Interface (CLI)
- */
-public class TestCLI extends CLITestHelper {
-  public static final String TEST_DIR_ABSOLUTE = "/tmp/testcli_" + Long.valueOf(System.currentTimeMillis());
-  private String supergroup;
-  private String namenode;
-  private static Shell shHDFS = new Shell("/bin/bash");
-
-  @Before
-  @Override
-  public void setUp() throws Exception {
-    readTestConfigFile();
-
-    // Configuration of real Hadoop cluster
-    conf = new HdfsConfiguration();
-    supergroup = conf.get(DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROUP_KEY);
-    namenode = conf.get(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY);
-
-    conf.setBoolean(CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION, true);
-    // Many of the tests expect a replication value of 1 in the output
-    conf.setInt("dfs.replication", 1);
-
-    clitestDataDir = new File(TEST_CACHE_DATA_DIR).toURI().toString().replace(' ', '+');
-
-    String[] createTestcliDirCmds = {
-        "hadoop fs -mkdir -p "  + TEST_DIR_ABSOLUTE,
-        "hadoop fs -chmod 777 " + TEST_DIR_ABSOLUTE
-    };
-    shHDFS.exec(createTestcliDirCmds);
-
-    // Check assumptions which would make some cases fail if not met
-    Assert.assertEquals("Creation of testcli dir should succeed and return 0"
-        + " (but it failed with the following error message: "
-        + StringUtils.join(shHDFS.getErr().toArray(), "\\n") + ")",
-        0, shHDFS.getRet());
-    // We can't just use conf.setInt(fs.trash.interval",0) because if trash is
-    // enabled on the server, client configuration value is ignored.
-    Assert.assertEquals("HDFS trash should be disabled via fs.trash.interval",
-        0, conf.getInt("fs.trash.interval",0));
-    Assert.assertEquals("This test needs to be run under root user of hcfs",
-        System.getProperty("hcfs.root.username", "hdfs"),
-        System.getProperty("user.name"));
-  }
-
-  @After
-  @Override
-  public void tearDown() throws Exception {
-    super.tearDown();
-
-    String removeTestcliDirCmd = "hadoop fs -rm -r " + TEST_DIR_ABSOLUTE;
-    shHDFS.exec(removeTestcliDirCmd);
-  }
-
-  @Override
-  protected String getTestFile() {
-    return "testHDFSConf.xml";
-  }
-
-  @Test
-  @Override
-  public void testAll() {
-    super.testAll();
-  }
-
-  /**
-   * Expand commands from the test config file.
-   * This method is used in displayResults() and compareTestOutput() only,
-   * so it doesn't have any effect on the test execution itself.
-   *
-   * @param cmd
-   * @return String expanded command
-   */
-  @Override
-  protected String expandCommand(final String cmd) {
-    String expCmd = super.expandCommand(cmd);
-    // note: super.expandCommand() expands CLITEST_DATA and USERNAME
-    expCmd = expCmd.replaceAll("TEST_DIR_ABSOLUTE", TEST_DIR_ABSOLUTE);
-    expCmd = expCmd.replaceAll("supergroup", supergroup);
-    expCmd = expCmd.replaceAll("NAMENODE", namenode);
-    expCmd = expCmd.replaceAll("USER_NAME", System.getProperty("user.name"));
-    return expCmd;
-  }
-
-  /**
-   * Execute given hadoop FsShell command (via Toolrunner).
-   */
-  @Override
-  protected CommandExecutor.Result execute(CLICommand cmd) throws Exception {
-    if (cmd.getType() instanceof CLICommandFS) {
-      CommandExecutor cmdExecutor = new FSCmdExecutor(namenode, new FsShell(conf));
-      return cmdExecutor.executeCommand(cmd.getCmd());
-    } else {
-      throw new IllegalArgumentException("Unknown type of test command: " + cmd.getType());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/f0bab035/bigtop-tests/test-artifacts/hadoop/src/main/groovy/org/apache/bigtop/itest/hadoop/hdfs/TestHDFSCLI.java
----------------------------------------------------------------------
diff --git a/bigtop-tests/test-artifacts/hadoop/src/main/groovy/org/apache/bigtop/itest/hadoop/hdfs/TestHDFSCLI.java b/bigtop-tests/test-artifacts/hadoop/src/main/groovy/org/apache/bigtop/itest/hadoop/hdfs/TestHDFSCLI.java
new file mode 100644
index 0000000..53901e0
--- /dev/null
+++ b/bigtop-tests/test-artifacts/hadoop/src/main/groovy/org/apache/bigtop/itest/hadoop/hdfs/TestHDFSCLI.java
@@ -0,0 +1,32 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.bigtop.itest.hadoop.hdfs;
+
+import org.apache.bigtop.itest.hadoop.hcfs.TestCLI;
+
+/**
+ * HDFS-only tests for the Command Line Interface (CLI)
+ */
+public class TestHDFSCLI extends TestCLI {
+
+  @Override
+  protected String getTestFile() {
+    return "testHDFSConf.xml";
+  }
+
+}