Browse Source

HDFS-10892. Add unit tests for HDFS command 'dfs -tail' and 'dfs -stat'. Contributed by Mingliang Liu

Mingliang Liu 8 years ago
parent
commit
84c6264079

+ 182 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java

@@ -20,18 +20,28 @@ package org.apache.hadoop.hdfs;
 import java.io.*;
 import java.security.Permission;
 import java.security.PrivilegedExceptionAction;
+import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Date;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
 import java.util.Scanner;
+import java.util.TimeZone;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.zip.DeflaterOutputStream;
 import java.util.zip.GZIPOutputStream;
 
+import com.google.common.base.Supplier;
+import com.google.common.collect.Lists;
+import org.apache.commons.lang.RandomStringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.log4j.Level;
+import org.junit.Test;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.fs.permission.AclEntry;
@@ -51,13 +61,11 @@ import org.apache.hadoop.io.compress.BZip2Codec;
 import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.hadoop.net.ServerSocketUtil;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.ToolRunner;
-import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.log4j.Level;
-import org.junit.Test;
 
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY;
 import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
@@ -70,8 +78,6 @@ import static org.hamcrest.CoreMatchers.not;
 import static org.junit.Assert.*;
 import static org.hamcrest.core.StringContains.containsString;
 
-import com.google.common.collect.Lists;
-
 /**
  * This class tests commands from DFSShell.
  */
@@ -118,6 +124,7 @@ public class TestDFSShell {
     assertFalse(fs.exists(p));
   }
 
+  /** Create a local file whose content contains its full path. */
   static File createLocalFile(File f) throws IOException {
     assertTrue(!f.exists());
     PrintWriter out = new PrintWriter(f);
@@ -914,6 +921,97 @@ public class TestDFSShell {
     }
   }
 
+  /**
+   * Test that -tail displays last kilobyte of the file to stdout.
+   */
+  @Test (timeout = 30000)
+  public void testTail() throws Exception {
+    final int blockSize = 1024;
+    final int fileLen = 5 * blockSize;
+    final Configuration conf = new Configuration();
+    conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
+
+    try (MiniDFSCluster cluster =
+             new MiniDFSCluster.Builder(conf).numDataNodes(3).build()) {
+      cluster.waitActive();
+      final DistributedFileSystem dfs = cluster.getFileSystem();
+
+      // create a text file with multiple KB bytes (and multiple blocks)
+      final Path testFile = new Path("testTail", "file1");
+      final String text = RandomStringUtils.randomAscii(fileLen);
+      try (OutputStream pout = dfs.create(testFile)) {
+        pout.write(text.getBytes());
+      }
+      final ByteArrayOutputStream out = new ByteArrayOutputStream();
+      System.setOut(new PrintStream(out));
+      final String[] argv = new String[]{"-tail", testFile.toString()};
+      final int ret = ToolRunner.run(new FsShell(conf), argv);
+
+      assertEquals(Arrays.toString(argv) + " returned " + ret, 0, ret);
+      assertEquals("-tail returned " + out.size() + " bytes data, expected 1KB",
+          1024, out.size());
+      // tailed out last 1KB of the file content
+      assertArrayEquals("Tail output doesn't match input",
+          text.substring(fileLen - 1024).getBytes(), out.toByteArray());
+      out.reset();
+    }
+  }
+
+  /**
+   * Test that -tail -f outputs appended data as the file grows.
+   */
+  @Test(timeout = 30000)
+  public void testTailWithFresh() throws Exception {
+    final int blockSize = 1024;
+    final Configuration conf = new Configuration();
+    conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
+
+    try (MiniDFSCluster cluster =
+             new MiniDFSCluster.Builder(conf).numDataNodes(3).build()) {
+      cluster.waitActive();
+      final DistributedFileSystem dfs = cluster.getFileSystem();
+      final Path testFile = new Path("testTailWithFresh", "file1");
+      dfs.create(testFile);
+
+      final ByteArrayOutputStream out = new ByteArrayOutputStream();
+      System.setOut(new PrintStream(out));
+      final Thread tailer = new Thread() {
+        @Override
+        public void run() {
+          final String[] argv = new String[]{"-tail", "-f",
+              testFile.toString()};
+          try {
+            ToolRunner.run(new FsShell(conf), argv);
+          } catch (Exception e) {
+            LOG.error("Client that tails the test file fails", e);
+          } finally {
+            out.reset();
+          }
+        }
+      };
+      tailer.start();
+      // wait till the tailer is sleeping
+      GenericTestUtils.waitFor(new Supplier<Boolean>() {
+        @Override
+        public Boolean get() {
+          return tailer.getState() == Thread.State.TIMED_WAITING;
+        }
+      }, 100, 10000);
+
+      final String text = RandomStringUtils.randomAscii(blockSize / 2);
+      try (OutputStream pout = dfs.create(testFile)) {
+        pout.write(text.getBytes());
+      }
+      // The tailer should eventually show the file contents
+      GenericTestUtils.waitFor(new Supplier<Boolean>() {
+        @Override
+        public Boolean get() {
+          return Arrays.equals(text.getBytes(), out.toByteArray());
+        }
+      }, 100, 10000);
+    }
+  }
+
   @Test (timeout = 30000)
   public void testText() throws Exception {
     Configuration conf = new HdfsConfiguration();
@@ -2010,6 +2108,85 @@ public class TestDFSShell {
     }
   }
 
+  /**
+   * Test -stat [format] <path>... prints statistics about the file/directory
+   * at <path> in the specified format.
+   */
+  @Test (timeout = 30000)
+  public void testStat() throws Exception {
+    final int blockSize = 1024;
+    final Configuration conf = new HdfsConfiguration();
+    conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
+
+    try (MiniDFSCluster cluster =
+             new MiniDFSCluster.Builder(conf).numDataNodes(3).build()) {
+      cluster.waitActive();
+      final DistributedFileSystem dfs = cluster.getFileSystem();
+
+      final SimpleDateFormat fmt = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+      fmt.setTimeZone(TimeZone.getTimeZone("UTC"));
+      final Path testDir1 = new Path("testStat", "dir1");
+      dfs.mkdirs(testDir1);
+      final FileStatus status1 = dfs.getFileStatus(testDir1);
+      final String mtime1 = fmt.format(new Date(status1.getModificationTime()));
+      final Path testFile2 = new Path(testDir1, "file2");
+      DFSTestUtil.createFile(dfs, testFile2, 2 * blockSize, (short) 3, 0);
+      final FileStatus status2 = dfs.getFileStatus(testDir1);
+      final String mtime2 = fmt.format(new Date(status2.getModificationTime()));
+
+      final ByteArrayOutputStream out = new ByteArrayOutputStream();
+      System.setOut(new PrintStream(out));
+
+      doFsStat(conf, null);
+
+      out.reset();
+      doFsStat(conf, null, testDir1);
+      assertEquals("Unexpected -stat output: " + out,
+          out.toString(), String.format("%s%n", mtime1));
+
+      out.reset();
+      doFsStat(conf, null, testDir1, testFile2);
+      assertEquals("Unexpected -stat output: " + out,
+          out.toString(), String.format("%s%n%s%n", mtime1, mtime2));
+
+      doFsStat(conf, "%F %u:%g %b %y %n");
+
+      out.reset();
+      doFsStat(conf, "%F %u:%g %b %y %n", testDir1);
+      assertTrue(out.toString(), out.toString().contains(mtime1));
+      assertTrue(out.toString(), out.toString().contains("directory"));
+      assertTrue(out.toString(), out.toString().contains(status1.getGroup()));
+
+      out.reset();
+      doFsStat(conf, "%F %u:%g %b %y %n", testDir1, testFile2);
+      assertTrue(out.toString(), out.toString().contains(mtime1));
+      assertTrue(out.toString(), out.toString().contains("regular file"));
+      assertTrue(out.toString(), out.toString().contains(mtime2));
+    }
+  }
+
+  private static void doFsStat(Configuration conf, String format, Path... files)
+      throws Exception {
+    if (files == null || files.length == 0) {
+      final String[] argv = (format == null ? new String[] {"-stat"} :
+          new String[] {"-stat", format});
+      assertEquals("Should have failed with missing arguments",
+          -1, ToolRunner.run(new FsShell(conf), argv));
+    } else {
+      List<String> argv = new LinkedList<>();
+      argv.add("-stat");
+      if (format != null) {
+        argv.add(format);
+      }
+      for (Path f : files) {
+        argv.add(f.toString());
+      }
+
+      int ret = ToolRunner.run(new FsShell(conf), argv.toArray(new String[0]));
+      assertEquals(argv + " returned non-zero status " + ret, 0, ret);
+    }
+  }
+
   @Test (timeout = 30000)
   public void testLsr() throws Exception {
     final Configuration conf = new HdfsConfiguration();