source: proiecte/HadoopJUnit/hadoop-0.20.1/src/core/org/apache/hadoop/fs/FsShell.java @ 120

Last change on this file since 120 was 120, checked in by (none), 14 years ago

Added the mail files for the Hadoop JUNit Project

  • Property svn:executable set to *
File size: 67.2 KB
Line 
1/**
2 * Licensed to the Apache Software Foundation (ASF) under one
3 * or more contributor license agreements.  See the NOTICE file
4 * distributed with this work for additional information
5 * regarding copyright ownership.  The ASF licenses this file
6 * to you under the Apache License, Version 2.0 (the
7 * "License"); you may not use this file except in compliance
8 * with the License.  You may obtain a copy of the License at
9 *
10 *     http://www.apache.org/licenses/LICENSE-2.0
11 *
12 * Unless required by applicable law or agreed to in writing, software
13 * distributed under the License is distributed on an "AS IS" BASIS,
14 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 * See the License for the specific language governing permissions and
16 * limitations under the License.
17 */
18package org.apache.hadoop.fs;
19
20import java.io.File;
21import java.io.FileNotFoundException;
22import java.io.IOException;
23import java.io.InputStream;
24import java.net.URI;
25import java.text.DecimalFormat;
26import java.text.NumberFormat;
27import java.text.SimpleDateFormat;
28import java.util.*;
29import java.util.zip.GZIPInputStream;
30
31import org.apache.hadoop.conf.Configuration;
32import org.apache.hadoop.conf.Configured;
33import org.apache.hadoop.fs.shell.CommandFormat;
34import org.apache.hadoop.fs.shell.Count;
35import org.apache.hadoop.io.DataInputBuffer;
36import org.apache.hadoop.io.DataOutputBuffer;
37import org.apache.hadoop.io.IOUtils;
38import org.apache.hadoop.io.SequenceFile;
39import org.apache.hadoop.io.Writable;
40import org.apache.hadoop.io.WritableComparable;
41import org.apache.hadoop.ipc.RPC;
42import org.apache.hadoop.ipc.RemoteException;
43import org.apache.hadoop.util.ReflectionUtils;
44import org.apache.hadoop.util.Tool;
45import org.apache.hadoop.util.ToolRunner;
46import org.apache.hadoop.util.StringUtils;
47
48/** Provide command line access to a FileSystem. */
49public class FsShell extends Configured implements Tool {
50
51  protected FileSystem fs;
52  private Trash trash;
53  public static final SimpleDateFormat dateForm = 
54    new SimpleDateFormat("yyyy-MM-dd HH:mm");
55  protected static final SimpleDateFormat modifFmt =
56    new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
57  static final int BORDER = 2;
58  static {
59    modifFmt.setTimeZone(TimeZone.getTimeZone("UTC"));
60  }
61  static final String SETREP_SHORT_USAGE="-setrep [-R] [-w] <rep> <path/file>";
62  static final String GET_SHORT_USAGE = "-get [-ignoreCrc] [-crc] <src> <localdst>";
63  static final String COPYTOLOCAL_SHORT_USAGE = GET_SHORT_USAGE.replace(
64      "-get", "-copyToLocal");
65  static final String TAIL_USAGE="-tail [-f] <file>";
66
67  /**
68   */
69  public FsShell() {
70    this(null);
71  }
72
73  public FsShell(Configuration conf) {
74    super(conf);
75    fs = null;
76    trash = null;
77  }
78 
79  protected void init() throws IOException {
80    getConf().setQuietMode(true);
81    if (this.fs == null) {
82     this.fs = FileSystem.get(getConf());
83    }
84    if (this.trash == null) {
85      this.trash = new Trash(getConf());
86    }
87  }
88
89 
90  /**
91   * Copies from stdin to the indicated file.
92   */
93  private void copyFromStdin(Path dst, FileSystem dstFs) throws IOException {
94    if (dstFs.isDirectory(dst)) {
95      throw new IOException("When source is stdin, destination must be a file.");
96    }
97    if (dstFs.exists(dst)) {
98      throw new IOException("Target " + dst.toString() + " already exists.");
99    }
100    FSDataOutputStream out = dstFs.create(dst); 
101    try {
102      IOUtils.copyBytes(System.in, out, getConf(), false);
103    } 
104    finally {
105      out.close();
106    }
107  }
108
109  /**
110   * Print from src to stdout.
111   */
112  private void printToStdout(InputStream in) throws IOException {
113    try {
114      IOUtils.copyBytes(in, System.out, getConf(), false);
115    } finally {
116      in.close();
117    }
118  }
119
120 
121  /**
122   * Add local files to the indicated FileSystem name. src is kept.
123   */
124  void copyFromLocal(Path[] srcs, String dstf) throws IOException {
125    Path dstPath = new Path(dstf);
126    FileSystem dstFs = dstPath.getFileSystem(getConf());
127    if (srcs.length == 1 && srcs[0].toString().equals("-"))
128      copyFromStdin(dstPath, dstFs);
129    else
130      dstFs.copyFromLocalFile(false, false, srcs, dstPath);
131  }
132 
133  /**
134   * Add local files to the indicated FileSystem name. src is removed.
135   */
136  void moveFromLocal(Path[] srcs, String dstf) throws IOException {
137    Path dstPath = new Path(dstf);
138    FileSystem dstFs = dstPath.getFileSystem(getConf());
139    dstFs.moveFromLocalFile(srcs, dstPath);
140  }
141
142  /**
143   * Add a local file to the indicated FileSystem name. src is removed.
144   */
145  void moveFromLocal(Path src, String dstf) throws IOException {
146    moveFromLocal((new Path[]{src}), dstf);
147  }
148
149  /**
150   * Obtain the indicated files that match the file pattern <i>srcf</i>
151   * and copy them to the local name. srcf is kept.
152   * When copying multiple files, the destination must be a directory.
153   * Otherwise, IOException is thrown.
154   * @param argv: arguments
155   * @param pos: Ignore everything before argv[pos] 
156   * @exception: IOException 
157   * @see org.apache.hadoop.fs.FileSystem.globStatus
158   */
159  void copyToLocal(String[]argv, int pos) throws IOException {
160    CommandFormat cf = new CommandFormat("copyToLocal", 2,2,"crc","ignoreCrc");
161   
162    String srcstr = null;
163    String dststr = null;
164    try {
165      List<String> parameters = cf.parse(argv, pos);
166      srcstr = parameters.get(0);
167      dststr = parameters.get(1);
168    }
169    catch(IllegalArgumentException iae) {
170      System.err.println("Usage: java FsShell " + GET_SHORT_USAGE);
171      throw iae;
172    }
173    boolean copyCrc = cf.getOpt("crc");
174    final boolean verifyChecksum = !cf.getOpt("ignoreCrc");
175
176    if (dststr.equals("-")) {
177      if (copyCrc) {
178        System.err.println("-crc option is not valid when destination is stdout.");
179      }
180      cat(srcstr, verifyChecksum);
181    } else {
182      File dst = new File(dststr);     
183      Path srcpath = new Path(srcstr);
184      FileSystem srcFS = getSrcFileSystem(srcpath, verifyChecksum);
185      if (copyCrc && !(srcFS instanceof ChecksumFileSystem)) {
186        System.err.println("-crc option is not valid when source file system " +
187            "does not have crc files. Automatically turn the option off.");
188        copyCrc = false;
189      }
190      FileStatus[] srcs = srcFS.globStatus(srcpath);
191      boolean dstIsDir = dst.isDirectory(); 
192      if (srcs.length > 1 && !dstIsDir) {
193        throw new IOException("When copying multiple files, "
194                              + "destination should be a directory.");
195      }
196      for (FileStatus status : srcs) {
197        Path p = status.getPath();
198        File f = dstIsDir? new File(dst, p.getName()): dst;
199        copyToLocal(srcFS, p, f, copyCrc);
200      }
201    }
202  }
203
204  /**
205   * Return the {@link FileSystem} specified by src and the conf.
206   * It the {@link FileSystem} supports checksum, set verifyChecksum.
207   */
208  private FileSystem getSrcFileSystem(Path src, boolean verifyChecksum
209      ) throws IOException { 
210    FileSystem srcFs = src.getFileSystem(getConf());
211    srcFs.setVerifyChecksum(verifyChecksum);
212    return srcFs;
213  }
214
215  /**
216   * The prefix for the tmp file used in copyToLocal.
217   * It must be at least three characters long, required by
218   * {@link java.io.File#createTempFile(String, String, File)}.
219   */
220  static final String COPYTOLOCAL_PREFIX = "_copyToLocal_";
221
222  /**
223   * Copy a source file from a given file system to local destination.
224   * @param srcFS source file system
225   * @param src source path
226   * @param dst destination
227   * @param copyCrc copy CRC files?
228   * @exception IOException If some IO failed
229   */
230  private void copyToLocal(final FileSystem srcFS, final Path src,
231                           final File dst, final boolean copyCrc)
232    throws IOException {
233    /* Keep the structure similar to ChecksumFileSystem.copyToLocal().
234     * Ideal these two should just invoke FileUtil.copy() and not repeat
235     * recursion here. Of course, copy() should support two more options :
236     * copyCrc and useTmpFile (may be useTmpFile need not be an option).
237     */
238   
239    if (!srcFS.getFileStatus(src).isDir()) {
240      if (dst.exists()) {
241        // match the error message in FileUtil.checkDest():
242        throw new IOException("Target " + dst + " already exists");
243      }
244     
245      // use absolute name so that tmp file is always created under dest dir
246      File tmp = FileUtil.createLocalTempFile(dst.getAbsoluteFile(),
247                                              COPYTOLOCAL_PREFIX, true);
248      if (!FileUtil.copy(srcFS, src, tmp, false, srcFS.getConf())) {
249        throw new IOException("Failed to copy " + src + " to " + dst); 
250      }
251     
252      if (!tmp.renameTo(dst)) {
253        throw new IOException("Failed to rename tmp file " + tmp + 
254                              " to local destination \"" + dst + "\".");
255      }
256
257      if (copyCrc) {
258        if (!(srcFS instanceof ChecksumFileSystem)) {
259          throw new IOException("Source file system does not have crc files");
260        }
261       
262        ChecksumFileSystem csfs = (ChecksumFileSystem) srcFS;
263        File dstcs = FileSystem.getLocal(srcFS.getConf())
264          .pathToFile(csfs.getChecksumFile(new Path(dst.getCanonicalPath())));
265        copyToLocal(csfs.getRawFileSystem(), csfs.getChecksumFile(src),
266                    dstcs, false);
267      } 
268    } else {
269      // once FileUtil.copy() supports tmp file, we don't need to mkdirs().
270      dst.mkdirs();
271      for(FileStatus path : srcFS.listStatus(src)) {
272        copyToLocal(srcFS, path.getPath(), 
273                    new File(dst, path.getPath().getName()), copyCrc);
274      }
275    }
276  }
277
278  /**
279   * Get all the files in the directories that match the source file
280   * pattern and merge and sort them to only one file on local fs
281   * srcf is kept.
282   * @param srcf: a file pattern specifying source files
283   * @param dstf: a destination local file/directory
284   * @exception: IOException 
285   * @see org.apache.hadoop.fs.FileSystem.globStatus
286   */
287  void copyMergeToLocal(String srcf, Path dst) throws IOException {
288    copyMergeToLocal(srcf, dst, false);
289  }   
290   
291
292  /**
293   * Get all the files in the directories that match the source file pattern
294   * and merge and sort them to only one file on local fs
295   * srcf is kept.
296   *
297   * Also adds a string between the files (useful for adding \n
298   * to a text file)
299   * @param srcf: a file pattern specifying source files
300   * @param dstf: a destination local file/directory
301   * @param endline: if an end of line character is added to a text file
302   * @exception: IOException 
303   * @see org.apache.hadoop.fs.FileSystem.globStatus
304   */
305  void copyMergeToLocal(String srcf, Path dst, boolean endline) throws IOException {
306    Path srcPath = new Path(srcf);
307    FileSystem srcFs = srcPath.getFileSystem(getConf());
308    Path [] srcs = FileUtil.stat2Paths(srcFs.globStatus(srcPath), 
309                                       srcPath);
310    for(int i=0; i<srcs.length; i++) {
311      if (endline) {
312        FileUtil.copyMerge(srcFs, srcs[i], 
313                           FileSystem.getLocal(getConf()), dst, false, getConf(), "\n");
314      } else {
315        FileUtil.copyMerge(srcFs, srcs[i], 
316                           FileSystem.getLocal(getConf()), dst, false, getConf(), null);
317      }
318    }
319  }     
320
321  /**
322   * Obtain the indicated file and copy to the local name.
323   * srcf is removed.
324   */
325  void moveToLocal(String srcf, Path dst) throws IOException {
326    System.err.println("Option '-moveToLocal' is not implemented yet.");
327  }
328
329  /**
330   * Fetch all files that match the file pattern <i>srcf</i> and display
331   * their content on stdout.
332   * @param srcf: a file pattern specifying source files
333   * @exception: IOException
334   * @see org.apache.hadoop.fs.FileSystem.globStatus
335   */
336  void cat(String src, boolean verifyChecksum) throws IOException {
337    //cat behavior in Linux
338    //  [~/1207]$ ls ?.txt
339    //  x.txt  z.txt
340    //  [~/1207]$ cat x.txt y.txt z.txt
341    //  xxx
342    //  cat: y.txt: No such file or directory
343    //  zzz
344
345    Path srcPattern = new Path(src);
346    new DelayedExceptionThrowing() {
347      @Override
348      void process(Path p, FileSystem srcFs) throws IOException {
349        if (srcFs.getFileStatus(p).isDir()) {
350          throw new IOException("Source must be a file.");
351        }
352        printToStdout(srcFs.open(p));
353      }
354    }.globAndProcess(srcPattern, getSrcFileSystem(srcPattern, verifyChecksum));
355  }
356
357  private class TextRecordInputStream extends InputStream {
358    SequenceFile.Reader r;
359    WritableComparable key;
360    Writable val;
361
362    DataInputBuffer inbuf;
363    DataOutputBuffer outbuf;
364
365    public TextRecordInputStream(FileStatus f) throws IOException {
366      r = new SequenceFile.Reader(fs, f.getPath(), getConf());
367      key = ReflectionUtils.newInstance(r.getKeyClass().asSubclass(WritableComparable.class),
368                                        getConf());
369      val = ReflectionUtils.newInstance(r.getValueClass().asSubclass(Writable.class),
370                                        getConf());
371      inbuf = new DataInputBuffer();
372      outbuf = new DataOutputBuffer();
373    }
374
375    public int read() throws IOException {
376      int ret;
377      if (null == inbuf || -1 == (ret = inbuf.read())) {
378        if (!r.next(key, val)) {
379          return -1;
380        }
381        byte[] tmp = key.toString().getBytes();
382        outbuf.write(tmp, 0, tmp.length);
383        outbuf.write('\t');
384        tmp = val.toString().getBytes();
385        outbuf.write(tmp, 0, tmp.length);
386        outbuf.write('\n');
387        inbuf.reset(outbuf.getData(), outbuf.getLength());
388        outbuf.reset();
389        ret = inbuf.read();
390      }
391      return ret;
392    }
393  }
394
395  private InputStream forMagic(Path p, FileSystem srcFs) throws IOException {
396    FSDataInputStream i = srcFs.open(p);
397    switch(i.readShort()) {
398      case 0x1f8b: // RFC 1952
399        i.seek(0);
400        return new GZIPInputStream(i);
401      case 0x5345: // 'S' 'E'
402        if (i.readByte() == 'Q') {
403          i.close();
404          return new TextRecordInputStream(srcFs.getFileStatus(p));
405        }
406        break;
407    }
408    i.seek(0);
409    return i;
410  }
411
412  void text(String srcf) throws IOException {
413    Path srcPattern = new Path(srcf);
414    new DelayedExceptionThrowing() {
415      @Override
416      void process(Path p, FileSystem srcFs) throws IOException {
417        if (srcFs.isDirectory(p)) {
418          throw new IOException("Source must be a file.");
419        }
420        printToStdout(forMagic(p, srcFs));
421      }
422    }.globAndProcess(srcPattern, srcPattern.getFileSystem(getConf()));
423  }
424
425  /**
426   * Parse the incoming command string
427   * @param cmd
428   * @param pos ignore anything before this pos in cmd
429   * @throws IOException
430   */
431  private void setReplication(String[] cmd, int pos) throws IOException {
432    CommandFormat c = new CommandFormat("setrep", 2, 2, "R", "w");
433    String dst = null;
434    short rep = 0;
435
436    try {
437      List<String> parameters = c.parse(cmd, pos);
438      rep = Short.parseShort(parameters.get(0));
439      dst = parameters.get(1);
440    } catch (NumberFormatException nfe) {
441      System.err.println("Illegal replication, a positive integer expected");
442      throw nfe;
443    }
444    catch(IllegalArgumentException iae) {
445      System.err.println("Usage: java FsShell " + SETREP_SHORT_USAGE);
446      throw iae;
447    }
448
449    if (rep < 1) {
450      System.err.println("Cannot set replication to: " + rep);
451      throw new IllegalArgumentException("replication must be >= 1");
452    }
453
454    List<Path> waitList = c.getOpt("w")? new ArrayList<Path>(): null;
455    setReplication(rep, dst, c.getOpt("R"), waitList);
456
457    if (waitList != null) {
458      waitForReplication(waitList, rep);
459    }
460  }
461   
462  /**
463   * Wait for all files in waitList to have replication number equal to rep.
464   * @param waitList The files are waited for.
465   * @param rep The new replication number.
466   * @throws IOException IOException
467   */
468  void waitForReplication(List<Path> waitList, int rep) throws IOException {
469    for(Path f : waitList) {
470      System.out.print("Waiting for " + f + " ...");
471      System.out.flush();
472
473      boolean printWarning = false;
474      FileStatus status = fs.getFileStatus(f);
475      long len = status.getLen();
476
477      for(boolean done = false; !done; ) {
478        BlockLocation[] locations = fs.getFileBlockLocations(status, 0, len);
479        int i = 0;
480        for(; i < locations.length && 
481          locations[i].getHosts().length == rep; i++)
482          if (!printWarning && locations[i].getHosts().length > rep) {
483            System.out.println("\nWARNING: the waiting time may be long for "
484                + "DECREASING the number of replication.");
485            printWarning = true;
486          }
487        done = i == locations.length;
488
489        if (!done) {
490          System.out.print(".");
491          System.out.flush();
492          try {Thread.sleep(10000);} catch (InterruptedException e) {}
493        }
494      }
495
496      System.out.println(" done");
497    }
498  }
499
500  /**
501   * Set the replication for files that match file pattern <i>srcf</i>
502   * if it's a directory and recursive is true,
503   * set replication for all the subdirs and those files too.
504   * @param newRep new replication factor
505   * @param srcf a file pattern specifying source files
506   * @param recursive if need to set replication factor for files in subdirs
507   * @throws IOException 
508   * @see org.apache.hadoop.fs.FileSystem#globStatus(Path)
509   */
510  void setReplication(short newRep, String srcf, boolean recursive,
511                      List<Path> waitingList)
512    throws IOException {
513    Path srcPath = new Path(srcf);
514    FileSystem srcFs = srcPath.getFileSystem(getConf());
515    Path[] srcs = FileUtil.stat2Paths(srcFs.globStatus(srcPath),
516                                      srcPath);
517    for(int i=0; i<srcs.length; i++) {
518      setReplication(newRep, srcFs, srcs[i], recursive, waitingList);
519    }
520  }
521
522  private void setReplication(short newRep, FileSystem srcFs, 
523                              Path src, boolean recursive,
524                              List<Path> waitingList)
525    throws IOException {
526    if (!srcFs.getFileStatus(src).isDir()) {
527      setFileReplication(src, srcFs, newRep, waitingList);
528      return;
529    }
530    FileStatus items[] = srcFs.listStatus(src);
531    if (items == null) {
532      throw new IOException("Could not get listing for " + src);
533    } else {
534
535      for (int i = 0; i < items.length; i++) {
536        if (!items[i].isDir()) {
537          setFileReplication(items[i].getPath(), srcFs, newRep, waitingList);
538        } else if (recursive) {
539          setReplication(newRep, srcFs, items[i].getPath(), recursive, 
540                         waitingList);
541        }
542      }
543    }
544  }
545   
546  /**
547   * Actually set the replication for this file
548   * If it fails either throw IOException or print an error msg
549   * @param file: a file/directory
550   * @param newRep: new replication factor
551   * @throws IOException
552   */
553  private void setFileReplication(Path file, FileSystem srcFs, short newRep, List<Path> waitList)
554    throws IOException {
555    if (srcFs.setReplication(file, newRep)) {
556      if (waitList != null) {
557        waitList.add(file);
558      }
559      System.out.println("Replication " + newRep + " set: " + file);
560    } else {
561      System.err.println("Could not set replication for: " + file);
562    }
563  }
564   
565   
566  /**
567   * Get a listing of all files in that match the file pattern <i>srcf</i>.
568   * @param srcf a file pattern specifying source files
569   * @param recursive if need to list files in subdirs
570   * @throws IOException 
571   * @see org.apache.hadoop.fs.FileSystem#globStatus(Path)
572   */
573  private int ls(String srcf, boolean recursive) throws IOException {
574    Path srcPath = new Path(srcf);
575    FileSystem srcFs = srcPath.getFileSystem(this.getConf());
576    FileStatus[] srcs = srcFs.globStatus(srcPath);
577    if (srcs==null || srcs.length==0) {
578      throw new FileNotFoundException("Cannot access " + srcf + 
579          ": No such file or directory.");
580    }
581 
582    boolean printHeader = (srcs.length == 1) ? true: false;
583    int numOfErrors = 0;
584    for(int i=0; i<srcs.length; i++) {
585      numOfErrors += ls(srcs[i], srcFs, recursive, printHeader);
586    }
587    return numOfErrors == 0 ? 0 : -1;
588  }
589
590  /* list all files under the directory <i>src</i>
591   * ideally we should provide "-l" option, that lists like "ls -l".
592   */
593  private int ls(FileStatus src, FileSystem srcFs, boolean recursive,
594      boolean printHeader) throws IOException {
595    final String cmd = recursive? "lsr": "ls";
596    final FileStatus[] items = shellListStatus(cmd, srcFs, src);
597    if (items == null) {
598      return 1;
599    } else {
600      int numOfErrors = 0;
601      if (!recursive && printHeader) {
602        if (items.length != 0) {
603          System.out.println("Found " + items.length + " items");
604        }
605      }
606     
607      int maxReplication = 3, maxLen = 10, maxOwner = 0,maxGroup = 0;
608
609      for(int i = 0; i < items.length; i++) {
610        FileStatus stat = items[i];
611        int replication = String.valueOf(stat.getReplication()).length();
612        int len = String.valueOf(stat.getLen()).length();
613        int owner = String.valueOf(stat.getOwner()).length();
614        int group = String.valueOf(stat.getGroup()).length();
615       
616        if (replication > maxReplication) maxReplication = replication;
617        if (len > maxLen) maxLen = len;
618        if (owner > maxOwner)  maxOwner = owner;
619        if (group > maxGroup)  maxGroup = group;
620      }
621     
622      for (int i = 0; i < items.length; i++) {
623        FileStatus stat = items[i];
624        Path cur = stat.getPath();
625        String mdate = dateForm.format(new Date(stat.getModificationTime()));
626       
627        System.out.print((stat.isDir() ? "d" : "-") + 
628          stat.getPermission() + " ");
629        System.out.printf("%"+ maxReplication + 
630          "s ", (!stat.isDir() ? stat.getReplication() : "-"));
631        if (maxOwner > 0)
632          System.out.printf("%-"+ maxOwner + "s ", stat.getOwner());
633        if (maxGroup > 0)
634          System.out.printf("%-"+ maxGroup + "s ", stat.getGroup());
635        System.out.printf("%"+ maxLen + "d ", stat.getLen());
636        System.out.print(mdate + " ");
637        System.out.println(cur.toUri().getPath());
638        if (recursive && stat.isDir()) {
639          numOfErrors += ls(stat,srcFs, recursive, printHeader);
640        }
641      }
642      return numOfErrors;
643    }
644  }
645
646  /**
647   * Show the size of all files that match the file pattern <i>src</i>
648   * @param src a file pattern specifying source files
649   * @throws IOException 
650   * @see org.apache.hadoop.fs.FileSystem#globStatus(Path)
651   */
652  void du(String src) throws IOException {
653    Path srcPath = new Path(src);
654    FileSystem srcFs = srcPath.getFileSystem(getConf());
655    Path[] pathItems = FileUtil.stat2Paths(srcFs.globStatus(srcPath), 
656                                           srcPath);
657    FileStatus items[] = srcFs.listStatus(pathItems);
658    if ((items == null) || ((items.length == 0) && 
659        (!srcFs.exists(srcPath)))){
660      throw new FileNotFoundException("Cannot access " + src
661            + ": No such file or directory.");
662    } else {
663      System.out.println("Found " + items.length + " items");
664      int maxLength = 10;
665     
666      long length[] = new long[items.length];
667      for (int i = 0; i < items.length; i++) {
668        length[i] = items[i].isDir() ?
669          srcFs.getContentSummary(items[i].getPath()).getLength() :
670          items[i].getLen();
671        int len = String.valueOf(length[i]).length();
672        if (len > maxLength) maxLength = len;
673      }
674      for(int i = 0; i < items.length; i++) {
675        System.out.printf("%-"+ (maxLength + BORDER) +"d", length[i]);
676        System.out.println(items[i].getPath());
677      }
678    }
679  }
680   
681  /**
682   * Show the summary disk usage of each dir/file
683   * that matches the file pattern <i>src</i>
684   * @param src a file pattern specifying source files
685   * @throws IOException 
686   * @see org.apache.hadoop.fs.FileSystem#globStatus(Path)
687   */
688  void dus(String src) throws IOException {
689    Path srcPath = new Path(src);
690    FileSystem srcFs = srcPath.getFileSystem(getConf());
691    FileStatus status[] = srcFs.globStatus(new Path(src));
692    if (status==null || status.length==0) {
693      throw new FileNotFoundException("Cannot access " + src + 
694          ": No such file or directory.");
695    }
696    for(int i=0; i<status.length; i++) {
697      long totalSize = srcFs.getContentSummary(status[i].getPath()).getLength();
698      String pathStr = status[i].getPath().toString();
699      System.out.println(("".equals(pathStr)?".":pathStr) + "\t" + totalSize);
700    }
701  }
702
703  /**
704   * Create the given dir
705   */
706  void mkdir(String src) throws IOException {
707    Path f = new Path(src);
708    FileSystem srcFs = f.getFileSystem(getConf());
709    FileStatus fstatus = null;
710    try {
711      fstatus = srcFs.getFileStatus(f);
712      if (fstatus.isDir()) {
713        throw new IOException("cannot create directory " 
714            + src + ": File exists");
715      }
716      else {
717        throw new IOException(src + " exists but " +
718            "is not a directory");
719      }
720    } catch(FileNotFoundException e) {
721        if (!srcFs.mkdirs(f)) {
722          throw new IOException("failed to create " + src);
723        }
724    }
725  }
726
727  /**
728   * (Re)create zero-length file at the specified path.
729   * This will be replaced by a more UNIX-like touch when files may be
730   * modified.
731   */
732  void touchz(String src) throws IOException {
733    Path f = new Path(src);
734    FileSystem srcFs = f.getFileSystem(getConf());
735    FileStatus st;
736    if (srcFs.exists(f)) {
737      st = srcFs.getFileStatus(f);
738      if (st.isDir()) {
739        // TODO: handle this
740        throw new IOException(src + " is a directory");
741      } else if (st.getLen() != 0)
742        throw new IOException(src + " must be a zero-length file");
743    }
744    FSDataOutputStream out = srcFs.create(f);
745    out.close();
746  }
747
748  /**
749   * Check file types.
750   */
751  int test(String argv[], int i) throws IOException {
752    if (!argv[i].startsWith("-") || argv[i].length() > 2)
753      throw new IOException("Not a flag: " + argv[i]);
754    char flag = argv[i].toCharArray()[1];
755    Path f = new Path(argv[++i]);
756    FileSystem srcFs = f.getFileSystem(getConf());
757    switch(flag) {
758      case 'e':
759        return srcFs.exists(f) ? 0 : 1;
760      case 'z':
761        return srcFs.getFileStatus(f).getLen() == 0 ? 0 : 1;
762      case 'd':
763        return srcFs.getFileStatus(f).isDir() ? 0 : 1;
764      default:
765        throw new IOException("Unknown flag: " + flag);
766    }
767  }
768
769  /**
770   * Print statistics about path in specified format.
771   * Format sequences:
772   *   %b: Size of file in blocks
773   *   %n: Filename
774   *   %o: Block size
775   *   %r: replication
776   *   %y: UTC date as &quot;yyyy-MM-dd HH:mm:ss&quot;
777   *   %Y: Milliseconds since January 1, 1970 UTC
778   */
779  void stat(char[] fmt, String src) throws IOException {
780    Path srcPath = new Path(src);
781    FileSystem srcFs = srcPath.getFileSystem(getConf());
782    FileStatus glob[] = srcFs.globStatus(srcPath);
783    if (null == glob)
784      throw new IOException("cannot stat `" + src + "': No such file or directory");
785    for (FileStatus f : glob) {
786      StringBuilder buf = new StringBuilder();
787      for (int i = 0; i < fmt.length; ++i) {
788        if (fmt[i] != '%') {
789          buf.append(fmt[i]);
790        } else {
791          if (i + 1 == fmt.length) break;
792          switch(fmt[++i]) {
793            case 'b':
794              buf.append(f.getLen());
795              break;
796            case 'F':
797              buf.append(f.isDir() ? "directory" : "regular file");
798              break;
799            case 'n':
800              buf.append(f.getPath().getName());
801              break;
802            case 'o':
803              buf.append(f.getBlockSize());
804              break;
805            case 'r':
806              buf.append(f.getReplication());
807              break;
808            case 'y':
809              buf.append(modifFmt.format(new Date(f.getModificationTime())));
810              break;
811            case 'Y':
812              buf.append(f.getModificationTime());
813              break;
814            default:
815              buf.append(fmt[i]);
816              break;
817          }
818        }
819      }
820      System.out.println(buf.toString());
821    }
822  }
823
824  /**
825   * Move files that match the file pattern <i>srcf</i>
826   * to a destination file.
827   * When moving mutiple files, the destination must be a directory.
828   * Otherwise, IOException is thrown.
829   * @param srcf a file pattern specifying source files
830   * @param dstf a destination local file/directory
831   * @throws IOException 
832   * @see org.apache.hadoop.fs.FileSystem#globStatus(Path)
833   */
834  void rename(String srcf, String dstf) throws IOException {
835    Path srcPath = new Path(srcf);
836    Path dstPath = new Path(dstf);
837    FileSystem srcFs = srcPath.getFileSystem(getConf());
838    FileSystem dstFs = dstPath.getFileSystem(getConf());
839    URI srcURI = srcFs.getUri();
840    URI dstURI = dstFs.getUri();
841    if (srcURI.compareTo(dstURI) != 0) {
842      throw new IOException("src and destination filesystems do not match.");
843    }
844    Path[] srcs = FileUtil.stat2Paths(srcFs.globStatus(srcPath), srcPath);
845    Path dst = new Path(dstf);
846    if (srcs.length > 1 && !srcFs.isDirectory(dst)) {
847      throw new IOException("When moving multiple files, " 
848                            + "destination should be a directory.");
849    }
850    for(int i=0; i<srcs.length; i++) {
851      if (!srcFs.rename(srcs[i], dst)) {
852        FileStatus srcFstatus = null;
853        FileStatus dstFstatus = null;
854        try {
855          srcFstatus = srcFs.getFileStatus(srcs[i]);
856        } catch(FileNotFoundException e) {
857          throw new FileNotFoundException(srcs[i] + 
858          ": No such file or directory");
859        }
860        try {
861          dstFstatus = dstFs.getFileStatus(dst);
862        } catch(IOException e) {
863        }
864        if((srcFstatus!= null) && (dstFstatus!= null)) {
865          if (srcFstatus.isDir()  && !dstFstatus.isDir()) {
866            throw new IOException("cannot overwrite non directory "
867                + dst + " with directory " + srcs[i]);
868          }
869        }
870        throw new IOException("Failed to rename " + srcs[i] + " to " + dst);
871      }
872    }
873  }
874
875  /**
876   * Move/rename file(s) to a destination file. Multiple source
877   * files can be specified. The destination is the last element of
878   * the argvp[] array.
879   * If multiple source files are specified, then the destination
880   * must be a directory. Otherwise, IOException is thrown.
881   * @exception: IOException 
882   */
883  private int rename(String argv[], Configuration conf) throws IOException {
884    int i = 0;
885    int exitCode = 0;
886    String cmd = argv[i++]; 
887    String dest = argv[argv.length-1];
888    //
889    // If the user has specified multiple source files, then
890    // the destination has to be a directory
891    //
892    if (argv.length > 3) {
893      Path dst = new Path(dest);
894      FileSystem dstFs = dst.getFileSystem(getConf());
895      if (!dstFs.isDirectory(dst)) {
896        throw new IOException("When moving multiple files, " 
897                              + "destination " + dest + " should be a directory.");
898      }
899    }
900    //
901    // for each source file, issue the rename
902    //
903    for (; i < argv.length - 1; i++) {
904      try {
905        //
906        // issue the rename to the fs
907        //
908        rename(argv[i], dest);
909      } catch (RemoteException e) {
910        //
911        // This is a error returned by hadoop server. Print
912        // out the first line of the error mesage.
913        //
914        exitCode = -1;
915        try {
916          String[] content;
917          content = e.getLocalizedMessage().split("\n");
918          System.err.println(cmd.substring(1) + ": " + content[0]);
919        } catch (Exception ex) {
920          System.err.println(cmd.substring(1) + ": " +
921                             ex.getLocalizedMessage());
922        }
923      } catch (IOException e) {
924        //
925        // IO exception encountered locally.
926        //
927        exitCode = -1;
928        System.err.println(cmd.substring(1) + ": " +
929                           e.getLocalizedMessage());
930      }
931    }
932    return exitCode;
933  }
934
935  /**
936   * Copy files that match the file pattern <i>srcf</i>
937   * to a destination file.
938   * When copying mutiple files, the destination must be a directory.
939   * Otherwise, IOException is thrown.
940   * @param srcf a file pattern specifying source files
941   * @param dstf a destination local file/directory
942   * @throws IOException 
943   * @see org.apache.hadoop.fs.FileSystem#globStatus(Path)
944   */
945  void copy(String srcf, String dstf, Configuration conf) throws IOException {
946    Path srcPath = new Path(srcf);
947    FileSystem srcFs = srcPath.getFileSystem(getConf());
948    Path dstPath = new Path(dstf);
949    FileSystem dstFs = dstPath.getFileSystem(getConf());
950    Path [] srcs = FileUtil.stat2Paths(srcFs.globStatus(srcPath), srcPath);
951    if (srcs.length > 1 && !dstFs.isDirectory(dstPath)) {
952      throw new IOException("When copying multiple files, " 
953                            + "destination should be a directory.");
954    }
955    for(int i=0; i<srcs.length; i++) {
956      FileUtil.copy(srcFs, srcs[i], dstFs, dstPath, false, conf);
957    }
958  }
959
960  /**
961   * Copy file(s) to a destination file. Multiple source
962   * files can be specified. The destination is the last element of
963   * the argvp[] array.
964   * If multiple source files are specified, then the destination
965   * must be a directory. Otherwise, IOException is thrown.
966   * @exception: IOException 
967   */
968  private int copy(String argv[], Configuration conf) throws IOException {
969    int i = 0;
970    int exitCode = 0;
971    String cmd = argv[i++]; 
972    String dest = argv[argv.length-1];
973    //
974    // If the user has specified multiple source files, then
975    // the destination has to be a directory
976    //
977    if (argv.length > 3) {
978      Path dst = new Path(dest);
979      if (!fs.isDirectory(dst)) {
980        throw new IOException("When copying multiple files, " 
981                              + "destination " + dest + " should be a directory.");
982      }
983    }
984    //
985    // for each source file, issue the copy
986    //
987    for (; i < argv.length - 1; i++) {
988      try {
989        //
990        // issue the copy to the fs
991        //
992        copy(argv[i], dest, conf);
993      } catch (RemoteException e) {
994        //
995        // This is a error returned by hadoop server. Print
996        // out the first line of the error mesage.
997        //
998        exitCode = -1;
999        try {
1000          String[] content;
1001          content = e.getLocalizedMessage().split("\n");
1002          System.err.println(cmd.substring(1) + ": " +
1003                             content[0]);
1004        } catch (Exception ex) {
1005          System.err.println(cmd.substring(1) + ": " +
1006                             ex.getLocalizedMessage());
1007        }
1008      } catch (IOException e) {
1009        //
1010        // IO exception encountered locally.
1011        //
1012        exitCode = -1;
1013        System.err.println(cmd.substring(1) + ": " +
1014                           e.getLocalizedMessage());
1015      }
1016    }
1017    return exitCode;
1018  }
1019
1020  /**
1021   * Delete all files that match the file pattern <i>srcf</i>.
1022   * @param srcf a file pattern specifying source files
1023   * @param recursive if need to delete subdirs
1024   * @param skipTrash Should we skip the trash, if it's enabled?
1025   * @throws IOException 
1026   * @see org.apache.hadoop.fs.FileSystem#globStatus(Path)
1027   */
1028  void delete(String srcf, final boolean recursive, final boolean skipTrash) 
1029                                                            throws IOException {
1030    //rm behavior in Linux
1031    //  [~/1207]$ ls ?.txt
1032    //  x.txt  z.txt
1033    //  [~/1207]$ rm x.txt y.txt z.txt
1034    //  rm: cannot remove `y.txt': No such file or directory
1035
1036    Path srcPattern = new Path(srcf);
1037    new DelayedExceptionThrowing() {
1038      @Override
1039      void process(Path p, FileSystem srcFs) throws IOException {
1040        delete(p, srcFs, recursive, skipTrash);
1041      }
1042    }.globAndProcess(srcPattern, srcPattern.getFileSystem(getConf()));
1043  }
1044   
1045  /* delete a file */
1046  private void delete(Path src, FileSystem srcFs, boolean recursive, 
1047                      boolean skipTrash) throws IOException {
1048    FileStatus fs = null;
1049    try {
1050      fs = srcFs.getFileStatus(src);
1051    } catch (FileNotFoundException fnfe) {
1052      // Have to re-throw so that console output is as expected
1053      throw new FileNotFoundException("cannot remove "
1054          + src + ": No such file or directory.");
1055    }
1056
1057    if (fs.isDir() && !recursive) {
1058      throw new IOException("Cannot remove directory \"" + src +
1059                            "\", use -rmr instead");
1060    }
1061   
1062    if(!skipTrash) {
1063      Trash trashTmp = new Trash(srcFs, getConf());
1064      if (trashTmp.moveToTrash(src)) {
1065        System.out.println("Moved to trash: " + src);
1066        return;
1067      }
1068    }
1069   
1070    if (srcFs.delete(src, true)) {
1071      System.out.println("Deleted " + src);
1072    } else {
1073      throw new IOException("Delete failed " + src);
1074    }
1075  }
1076  private void expunge() throws IOException {
1077    trash.expunge();
1078    trash.checkpoint();
1079  }
1080
1081  /**
1082   * Returns the Trash object associated with this shell.
1083   */
1084  public Path getCurrentTrashDir() {
1085    return trash.getCurrentTrashDir();
1086  }
1087
1088  /**
1089   * Parse the incoming command string
1090   * @param cmd
1091   * @param pos ignore anything before this pos in cmd
1092   * @throws IOException
1093   */
1094  private void tail(String[] cmd, int pos) throws IOException {
1095    CommandFormat c = new CommandFormat("tail", 1, 1, "f");
1096    String src = null;
1097    Path path = null;
1098
1099    try {
1100      List<String> parameters = c.parse(cmd, pos);
1101      src = parameters.get(0);
1102    } catch(IllegalArgumentException iae) {
1103      System.err.println("Usage: java FsShell " + TAIL_USAGE);
1104      throw iae;
1105    }
1106    boolean foption = c.getOpt("f") ? true: false;
1107    path = new Path(src);
1108    FileSystem srcFs = path.getFileSystem(getConf());
1109    if (srcFs.isDirectory(path)) {
1110      throw new IOException("Source must be a file.");
1111    }
1112
1113    long fileSize = srcFs.getFileStatus(path).getLen();
1114    long offset = (fileSize > 1024) ? fileSize - 1024: 0;
1115
1116    while (true) {
1117      FSDataInputStream in = srcFs.open(path);
1118      in.seek(offset);
1119      IOUtils.copyBytes(in, System.out, 1024, false);
1120      offset = in.getPos();
1121      in.close();
1122      if (!foption) {
1123        break;
1124      }
1125      fileSize = srcFs.getFileStatus(path).getLen();
1126      offset = (fileSize > offset) ? offset: fileSize;
1127      try {
1128        Thread.sleep(5000);
1129      } catch (InterruptedException e) {
1130        break;
1131      }
1132    }
1133  }
1134
1135  /**
1136   * This class runs a command on a given FileStatus. This can be used for
1137   * running various commands like chmod, chown etc.
1138   */
1139  static abstract class CmdHandler {
1140   
1141    protected int errorCode = 0;
1142    protected boolean okToContinue = true;
1143    protected String cmdName;
1144   
1145    int getErrorCode() { return errorCode; }
1146    boolean okToContinue() { return okToContinue; }
1147    String getName() { return cmdName; }
1148   
1149    protected CmdHandler(String cmdName, FileSystem fs) {
1150      this.cmdName = cmdName;
1151    }
1152   
1153    public abstract void run(FileStatus file, FileSystem fs) throws IOException;
1154  }
1155 
1156  /** helper returns listStatus() */
1157  private static FileStatus[] shellListStatus(String cmd, 
1158                                                   FileSystem srcFs,
1159                                                   FileStatus src) {
1160    if (!src.isDir()) {
1161      FileStatus[] files = { src };
1162      return files;
1163    }
1164    Path path = src.getPath();
1165    try {
1166      FileStatus[] files = srcFs.listStatus(path);
1167      if ( files == null ) {
1168        System.err.println(cmd + 
1169                           ": could not get listing for '" + path + "'");
1170      }
1171      return files;
1172    } catch (IOException e) {
1173      System.err.println(cmd + 
1174                         ": could not get get listing for '" + path + "' : " +
1175                         e.getMessage().split("\n")[0]);
1176    }
1177    return null;
1178  }
1179 
1180 
1181  /**
1182   * Runs the command on a given file with the command handler.
1183   * If recursive is set, command is run recursively.
1184   */                                       
1185  private static int runCmdHandler(CmdHandler handler, FileStatus stat, 
1186                                   FileSystem srcFs, 
1187                                   boolean recursive) throws IOException {
1188    int errors = 0;
1189    handler.run(stat, srcFs);
1190    if (recursive && stat.isDir() && handler.okToContinue()) {
1191      FileStatus[] files = shellListStatus(handler.getName(), srcFs, stat);
1192      if (files == null) {
1193        return 1;
1194      }
1195      for(FileStatus file : files ) {
1196        errors += runCmdHandler(handler, file, srcFs, recursive);
1197      }
1198    }
1199    return errors;
1200  }
1201
1202  ///top level runCmdHandler
1203  int runCmdHandler(CmdHandler handler, String[] args,
1204                                   int startIndex, boolean recursive) 
1205                                   throws IOException {
1206    int errors = 0;
1207   
1208    for (int i=startIndex; i<args.length; i++) {
1209      Path srcPath = new Path(args[i]);
1210      FileSystem srcFs = srcPath.getFileSystem(getConf());
1211      Path[] paths = FileUtil.stat2Paths(srcFs.globStatus(srcPath), srcPath);
1212      for(Path path : paths) {
1213        try {
1214          FileStatus file = srcFs.getFileStatus(path);
1215          if (file == null) {
1216            System.err.println(handler.getName() + 
1217                               ": could not get status for '" + path + "'");
1218            errors++;
1219          } else {
1220            errors += runCmdHandler(handler, file, srcFs, recursive);
1221          }
1222        } catch (IOException e) {
1223          String msg = (e.getMessage() != null ? e.getLocalizedMessage() :
1224            (e.getCause().getMessage() != null ? 
1225                e.getCause().getLocalizedMessage() : "null"));
1226          System.err.println(handler.getName() + ": could not get status for '"
1227                                        + path + "': " + msg.split("\n")[0]);       
1228        }
1229      }
1230    }
1231   
1232    return (errors > 0 || handler.getErrorCode() != 0) ? 1 : 0;
1233  }
1234 
1235  /**
1236   * Return an abbreviated English-language desc of the byte length
1237   * @deprecated Consider using {@link org.apache.hadoop.util.StringUtils#byteDesc} instead.
1238   */
1239  @Deprecated
1240  public static String byteDesc(long len) {
1241    return StringUtils.byteDesc(len);
1242  }
1243
1244  /**
1245   * @deprecated Consider using {@link org.apache.hadoop.util.StringUtils#limitDecimalTo2} instead.
1246   */
1247  @Deprecated
1248  public static synchronized String limitDecimalTo2(double d) {
1249    return StringUtils.limitDecimalTo2(d);
1250  }
1251
1252  private void printHelp(String cmd) {
1253    String summary = "hadoop fs is the command to execute fs commands. " +
1254      "The full syntax is: \n\n" +
1255      "hadoop fs [-fs <local | file system URI>] [-conf <configuration file>]\n\t" +
1256      "[-D <property=value>] [-ls <path>] [-lsr <path>] [-du <path>]\n\t" + 
1257      "[-dus <path>] [-mv <src> <dst>] [-cp <src> <dst>] [-rm [-skipTrash] <src>]\n\t" + 
1258      "[-rmr [-skipTrash] <src>] [-put <localsrc> ... <dst>] [-copyFromLocal <localsrc> ... <dst>]\n\t" +
1259      "[-moveFromLocal <localsrc> ... <dst>] [" + 
1260      GET_SHORT_USAGE + "\n\t" +
1261      "[-getmerge <src> <localdst> [addnl]] [-cat <src>]\n\t" +
1262      "[" + COPYTOLOCAL_SHORT_USAGE + "] [-moveToLocal <src> <localdst>]\n\t" +
1263      "[-mkdir <path>] [-report] [" + SETREP_SHORT_USAGE + "]\n\t" +
1264      "[-touchz <path>] [-test -[ezd] <path>] [-stat [format] <path>]\n\t" +
1265      "[-tail [-f] <path>] [-text <path>]\n\t" +
1266      "[" + FsShellPermissions.CHMOD_USAGE + "]\n\t" +
1267      "[" + FsShellPermissions.CHOWN_USAGE + "]\n\t" +
1268      "[" + FsShellPermissions.CHGRP_USAGE + "]\n\t" +     
1269      "[" + Count.USAGE + "]\n\t" +     
1270      "[-help [cmd]]\n";
1271
1272    String conf ="-conf <configuration file>:  Specify an application configuration file.";
1273 
1274    String D = "-D <property=value>:  Use value for given property.";
1275 
1276    String fs = "-fs [local | <file system URI>]: \tSpecify the file system to use.\n" + 
1277      "\t\tIf not specified, the current configuration is used, \n" +
1278      "\t\ttaken from the following, in increasing precedence: \n" + 
1279      "\t\t\tcore-default.xml inside the hadoop jar file \n" +
1280      "\t\t\tcore-site.xml in $HADOOP_CONF_DIR \n" +
1281      "\t\t'local' means use the local file system as your DFS. \n" +
1282      "\t\t<file system URI> specifies a particular file system to \n" +
1283      "\t\tcontact. This argument is optional but if used must appear\n" +
1284      "\t\tappear first on the command line.  Exactly one additional\n" +
1285      "\t\targument must be specified. \n";
1286
1287       
1288    String ls = "-ls <path>: \tList the contents that match the specified file pattern. If\n" + 
1289      "\t\tpath is not specified, the contents of /user/<currentUser>\n" +
1290      "\t\twill be listed. Directory entries are of the form \n" +
1291      "\t\t\tdirName (full path) <dir> \n" +
1292      "\t\tand file entries are of the form \n" + 
1293      "\t\t\tfileName(full path) <r n> size \n" +
1294      "\t\twhere n is the number of replicas specified for the file \n" + 
1295      "\t\tand size is the size of the file, in bytes.\n";
1296
1297    String lsr = "-lsr <path>: \tRecursively list the contents that match the specified\n" +
1298      "\t\tfile pattern.  Behaves very similarly to hadoop fs -ls,\n" + 
1299      "\t\texcept that the data is shown for all the entries in the\n" +
1300      "\t\tsubtree.\n";
1301
1302    String du = "-du <path>: \tShow the amount of space, in bytes, used by the files that \n" +
1303      "\t\tmatch the specified file pattern.  Equivalent to the unix\n" + 
1304      "\t\tcommand \"du -sb <path>/*\" in case of a directory, \n" +
1305      "\t\tand to \"du -b <path>\" in case of a file.\n" +
1306      "\t\tThe output is in the form \n" + 
1307      "\t\t\tname(full path) size (in bytes)\n"; 
1308
1309    String dus = "-dus <path>: \tShow the amount of space, in bytes, used by the files that \n" +
1310      "\t\tmatch the specified file pattern.  Equivalent to the unix\n" + 
1311      "\t\tcommand \"du -sb\"  The output is in the form \n" + 
1312      "\t\t\tname(full path) size (in bytes)\n"; 
1313   
1314    String mv = "-mv <src> <dst>:   Move files that match the specified file pattern <src>\n" +
1315      "\t\tto a destination <dst>.  When moving multiple files, the \n" +
1316      "\t\tdestination must be a directory. \n";
1317
1318    String cp = "-cp <src> <dst>:   Copy files that match the file pattern <src> to a \n" +
1319      "\t\tdestination.  When copying multiple files, the destination\n" +
1320      "\t\tmust be a directory. \n";
1321
1322    String rm = "-rm [-skipTrash] <src>: \tDelete all files that match the specified file pattern.\n" +
1323      "\t\tEquivalent to the Unix command \"rm <src>\"\n" +
1324      "\t\t-skipTrash option bypasses trash, if enabled, and immediately\n" +
1325      "deletes <src>";
1326
1327    String rmr = "-rmr [-skipTrash] <src>: \tRemove all directories which match the specified file \n" +
1328      "\t\tpattern. Equivalent to the Unix command \"rm -rf <src>\"\n" +
1329      "\t\t-skipTrash option bypasses trash, if enabled, and immediately\n" +
1330      "deletes <src>";
1331
1332    String put = "-put <localsrc> ... <dst>: \tCopy files " + 
1333    "from the local file system \n\t\tinto fs. \n";
1334
1335    String copyFromLocal = "-copyFromLocal <localsrc> ... <dst>:" +
1336    " Identical to the -put command.\n";
1337
1338    String moveFromLocal = "-moveFromLocal <localsrc> ... <dst>:" +
1339    " Same as -put, except that the source is\n\t\tdeleted after it's copied.\n"; 
1340
1341    String get = GET_SHORT_USAGE
1342      + ":  Copy files that match the file pattern <src> \n" +
1343      "\t\tto the local name.  <src> is kept.  When copying mutiple, \n" +
1344      "\t\tfiles, the destination must be a directory. \n";
1345
1346    String getmerge = "-getmerge <src> <localdst>:  Get all the files in the directories that \n" +
1347      "\t\tmatch the source file pattern and merge and sort them to only\n" +
1348      "\t\tone file on local fs. <src> is kept.\n";
1349
1350    String cat = "-cat <src>: \tFetch all files that match the file pattern <src> \n" +
1351      "\t\tand display their content on stdout.\n";
1352
1353   
1354    String text = "-text <src>: \tTakes a source file and outputs the file in text format.\n" +
1355      "\t\tThe allowed formats are zip and TextRecordInputStream.\n";
1356         
1357   
1358    String copyToLocal = COPYTOLOCAL_SHORT_USAGE
1359                         + ":  Identical to the -get command.\n";
1360
1361    String moveToLocal = "-moveToLocal <src> <localdst>:  Not implemented yet \n";
1362       
1363    String mkdir = "-mkdir <path>: \tCreate a directory in specified location. \n";
1364
1365    String setrep = SETREP_SHORT_USAGE
1366      + ":  Set the replication level of a file. \n"
1367      + "\t\tThe -R flag requests a recursive change of replication level \n"
1368      + "\t\tfor an entire tree.\n";
1369
1370    String touchz = "-touchz <path>: Write a timestamp in yyyy-MM-dd HH:mm:ss format\n" +
1371      "\t\tin a file at <path>. An error is returned if the file exists with non-zero length\n";
1372
1373    String test = "-test -[ezd] <path>: If file { exists, has zero length, is a directory\n" +
1374      "\t\tthen return 0, else return 1.\n";
1375
1376    String stat = "-stat [format] <path>: Print statistics about the file/directory at <path>\n" +
1377      "\t\tin the specified format. Format accepts filesize in blocks (%b), filename (%n),\n" +
1378      "\t\tblock size (%o), replication (%r), modification date (%y, %Y)\n";
1379
1380    String tail = TAIL_USAGE
1381      + ":  Show the last 1KB of the file. \n"
1382      + "\t\tThe -f option shows apended data as the file grows. \n";
1383
1384    String chmod = FsShellPermissions.CHMOD_USAGE + "\n" +
1385      "\t\tChanges permissions of a file.\n" +
1386      "\t\tThis works similar to shell's chmod with a few exceptions.\n\n" +
1387      "\t-R\tmodifies the files recursively. This is the only option\n" +
1388      "\t\tcurrently supported.\n\n" +
1389      "\tMODE\tMode is same as mode used for chmod shell command.\n" +
1390      "\t\tOnly letters recognized are 'rwxX'. E.g. a+r,g-w,+rwx,o=r\n\n" +
1391      "\tOCTALMODE Mode specifed in 3 digits. Unlike shell command,\n" +
1392      "\t\tthis requires all three digits.\n" +
1393      "\t\tE.g. 754 is same as u=rwx,g=rx,o=r\n\n" +
1394      "\t\tIf none of 'augo' is specified, 'a' is assumed and unlike\n" +
1395      "\t\tshell command, no umask is applied.\n";
1396   
1397    String chown = FsShellPermissions.CHOWN_USAGE + "\n" +
1398      "\t\tChanges owner and group of a file.\n" +
1399      "\t\tThis is similar to shell's chown with a few exceptions.\n\n" +
1400      "\t-R\tmodifies the files recursively. This is the only option\n" +
1401      "\t\tcurrently supported.\n\n" +
1402      "\t\tIf only owner or group is specified then only owner or\n" +
1403      "\t\tgroup is modified.\n\n" +
1404      "\t\tThe owner and group names may only cosists of digits, alphabet,\n"+
1405      "\t\tand any of '-_.@/' i.e. [-_.@/a-zA-Z0-9]. The names are case\n" +
1406      "\t\tsensitive.\n\n" +
1407      "\t\tWARNING: Avoid using '.' to separate user name and group though\n" +
1408      "\t\tLinux allows it. If user names have dots in them and you are\n" +
1409      "\t\tusing local file system, you might see surprising results since\n" +
1410      "\t\tshell command 'chown' is used for local files.\n";
1411   
1412    String chgrp = FsShellPermissions.CHGRP_USAGE + "\n" +
1413      "\t\tThis is equivalent to -chown ... :GROUP ...\n";
1414   
1415    String help = "-help [cmd]: \tDisplays help for given command or all commands if none\n" +
1416      "\t\tis specified.\n";
1417
1418    if ("fs".equals(cmd)) {
1419      System.out.println(fs);
1420    } else if ("conf".equals(cmd)) {
1421      System.out.println(conf);
1422    } else if ("D".equals(cmd)) {
1423      System.out.println(D);
1424    } else if ("ls".equals(cmd)) {
1425      System.out.println(ls);
1426    } else if ("lsr".equals(cmd)) {
1427      System.out.println(lsr);
1428    } else if ("du".equals(cmd)) {
1429      System.out.println(du);
1430    } else if ("dus".equals(cmd)) {
1431      System.out.println(dus);
1432    } else if ("rm".equals(cmd)) {
1433      System.out.println(rm);
1434    } else if ("rmr".equals(cmd)) {
1435      System.out.println(rmr);
1436    } else if ("mkdir".equals(cmd)) {
1437      System.out.println(mkdir);
1438    } else if ("mv".equals(cmd)) {
1439      System.out.println(mv);
1440    } else if ("cp".equals(cmd)) {
1441      System.out.println(cp);
1442    } else if ("put".equals(cmd)) {
1443      System.out.println(put);
1444    } else if ("copyFromLocal".equals(cmd)) {
1445      System.out.println(copyFromLocal);
1446    } else if ("moveFromLocal".equals(cmd)) {
1447      System.out.println(moveFromLocal);
1448    } else if ("get".equals(cmd)) {
1449      System.out.println(get);
1450    } else if ("getmerge".equals(cmd)) {
1451      System.out.println(getmerge);
1452    } else if ("copyToLocal".equals(cmd)) {
1453      System.out.println(copyToLocal);
1454    } else if ("moveToLocal".equals(cmd)) {
1455      System.out.println(moveToLocal);
1456    } else if ("cat".equals(cmd)) {
1457      System.out.println(cat);
1458    } else if ("get".equals(cmd)) {
1459      System.out.println(get);
1460    } else if ("setrep".equals(cmd)) {
1461      System.out.println(setrep);
1462    } else if ("touchz".equals(cmd)) {
1463      System.out.println(touchz);
1464    } else if ("test".equals(cmd)) {
1465      System.out.println(test);
1466    } else if ("text".equals(cmd)) {
1467      System.out.println(text);
1468    } else if ("stat".equals(cmd)) {
1469      System.out.println(stat);
1470    } else if ("tail".equals(cmd)) {
1471      System.out.println(tail);
1472    } else if ("chmod".equals(cmd)) {
1473      System.out.println(chmod);
1474    } else if ("chown".equals(cmd)) {
1475      System.out.println(chown);
1476    } else if ("chgrp".equals(cmd)) {
1477      System.out.println(chgrp);
1478    } else if (Count.matches(cmd)) {
1479      System.out.println(Count.DESCRIPTION);
1480    } else if ("help".equals(cmd)) {
1481      System.out.println(help);
1482    } else {
1483      System.out.println(summary);
1484      System.out.println(fs);
1485      System.out.println(ls);
1486      System.out.println(lsr);
1487      System.out.println(du);
1488      System.out.println(dus);
1489      System.out.println(mv);
1490      System.out.println(cp);
1491      System.out.println(rm);
1492      System.out.println(rmr);
1493      System.out.println(put);
1494      System.out.println(copyFromLocal);
1495      System.out.println(moveFromLocal);
1496      System.out.println(get);
1497      System.out.println(getmerge);
1498      System.out.println(cat);
1499      System.out.println(copyToLocal);
1500      System.out.println(moveToLocal);
1501      System.out.println(mkdir);
1502      System.out.println(setrep);
1503      System.out.println(tail);
1504      System.out.println(touchz);
1505      System.out.println(test);
1506      System.out.println(text);
1507      System.out.println(stat);
1508      System.out.println(chmod);
1509      System.out.println(chown);     
1510      System.out.println(chgrp);
1511      System.out.println(Count.DESCRIPTION);
1512      System.out.println(help);
1513    }       
1514
1515                           
1516  }
1517
1518  /**
1519   * Apply operation specified by 'cmd' on all parameters
1520   * starting from argv[startindex].
1521   */
1522  private int doall(String cmd, String argv[], int startindex) {
1523    int exitCode = 0;
1524    int i = startindex;
1525    boolean rmSkipTrash = false;
1526   
1527    // Check for -skipTrash option in rm/rmr
1528    if(("-rm".equals(cmd) || "-rmr".equals(cmd)) 
1529        && "-skipTrash".equals(argv[i])) {
1530      rmSkipTrash = true;
1531      i++;
1532    }
1533
1534    //
1535    // for each source file, issue the command
1536    //
1537    for (; i < argv.length; i++) {
1538      try {
1539        //
1540        // issue the command to the fs
1541        //
1542        if ("-cat".equals(cmd)) {
1543          cat(argv[i], true);
1544        } else if ("-mkdir".equals(cmd)) {
1545          mkdir(argv[i]);
1546        } else if ("-rm".equals(cmd)) {
1547          delete(argv[i], false, rmSkipTrash);
1548        } else if ("-rmr".equals(cmd)) {
1549          delete(argv[i], true, rmSkipTrash);
1550        } else if ("-du".equals(cmd)) {
1551          du(argv[i]);
1552        } else if ("-dus".equals(cmd)) {
1553          dus(argv[i]);
1554        } else if (Count.matches(cmd)) {
1555          new Count(argv, i, getConf()).runAll();
1556        } else if ("-ls".equals(cmd)) {
1557          exitCode = ls(argv[i], false);
1558        } else if ("-lsr".equals(cmd)) {
1559          exitCode = ls(argv[i], true);
1560        } else if ("-touchz".equals(cmd)) {
1561          touchz(argv[i]);
1562        } else if ("-text".equals(cmd)) {
1563          text(argv[i]);
1564        }
1565      } catch (RemoteException e) {
1566        //
1567        // This is a error returned by hadoop server. Print
1568        // out the first line of the error message.
1569        //
1570        exitCode = -1;
1571        try {
1572          String[] content;
1573          content = e.getLocalizedMessage().split("\n");
1574          System.err.println(cmd.substring(1) + ": " +
1575                             content[0]);
1576        } catch (Exception ex) {
1577          System.err.println(cmd.substring(1) + ": " +
1578                             ex.getLocalizedMessage());
1579        }
1580      } catch (IOException e) {
1581        //
1582        // IO exception encountered locally.
1583        //
1584        exitCode = -1;
1585        String content = e.getLocalizedMessage();
1586        if (content != null) {
1587          content = content.split("\n")[0];
1588        }
1589        System.err.println(cmd.substring(1) + ": " +
1590                          content);
1591      }
1592    }
1593    return exitCode;
1594  }
1595
1596  /**
1597   * Displays format of commands.
1598   *
1599   */
1600  private static void printUsage(String cmd) {
1601    String prefix = "Usage: java " + FsShell.class.getSimpleName();
1602    if ("-fs".equals(cmd)) {
1603      System.err.println("Usage: java FsShell" + 
1604                         " [-fs <local | file system URI>]");
1605    } else if ("-conf".equals(cmd)) {
1606      System.err.println("Usage: java FsShell" + 
1607                         " [-conf <configuration file>]");
1608    } else if ("-D".equals(cmd)) {
1609      System.err.println("Usage: java FsShell" + 
1610                         " [-D <[property=value>]");
1611    } else if ("-ls".equals(cmd) || "-lsr".equals(cmd) ||
1612               "-du".equals(cmd) || "-dus".equals(cmd) ||
1613               "-touchz".equals(cmd) || "-mkdir".equals(cmd) ||
1614               "-text".equals(cmd)) {
1615      System.err.println("Usage: java FsShell" + 
1616                         " [" + cmd + " <path>]");
1617    } else if (Count.matches(cmd)) {
1618      System.err.println(prefix + " [" + Count.USAGE + "]");
1619    } else if ("-rm".equals(cmd) || "-rmr".equals(cmd)) {
1620      System.err.println("Usage: java FsShell [" + cmd + 
1621                           " [-skipTrash] <src>]");
1622    } else if ("-mv".equals(cmd) || "-cp".equals(cmd)) {
1623      System.err.println("Usage: java FsShell" + 
1624                         " [" + cmd + " <src> <dst>]");
1625    } else if ("-put".equals(cmd) || "-copyFromLocal".equals(cmd) ||
1626               "-moveFromLocal".equals(cmd)) {
1627      System.err.println("Usage: java FsShell" + 
1628                         " [" + cmd + " <localsrc> ... <dst>]");
1629    } else if ("-get".equals(cmd)) {
1630      System.err.println("Usage: java FsShell [" + GET_SHORT_USAGE + "]"); 
1631    } else if ("-copyToLocal".equals(cmd)) {
1632      System.err.println("Usage: java FsShell [" + COPYTOLOCAL_SHORT_USAGE+ "]"); 
1633    } else if ("-moveToLocal".equals(cmd)) {
1634      System.err.println("Usage: java FsShell" + 
1635                         " [" + cmd + " [-crc] <src> <localdst>]");
1636    } else if ("-cat".equals(cmd)) {
1637      System.err.println("Usage: java FsShell" + 
1638                         " [" + cmd + " <src>]");
1639    } else if ("-setrep".equals(cmd)) {
1640      System.err.println("Usage: java FsShell [" + SETREP_SHORT_USAGE + "]");
1641    } else if ("-test".equals(cmd)) {
1642      System.err.println("Usage: java FsShell" +
1643                         " [-test -[ezd] <path>]");
1644    } else if ("-stat".equals(cmd)) {
1645      System.err.println("Usage: java FsShell" +
1646                         " [-stat [format] <path>]");
1647    } else if ("-tail".equals(cmd)) {
1648      System.err.println("Usage: java FsShell [" + TAIL_USAGE + "]");
1649    } else {
1650      System.err.println("Usage: java FsShell");
1651      System.err.println("           [-ls <path>]");
1652      System.err.println("           [-lsr <path>]");
1653      System.err.println("           [-du <path>]");
1654      System.err.println("           [-dus <path>]");
1655      System.err.println("           [" + Count.USAGE + "]");
1656      System.err.println("           [-mv <src> <dst>]");
1657      System.err.println("           [-cp <src> <dst>]");
1658      System.err.println("           [-rm [-skipTrash] <path>]");
1659      System.err.println("           [-rmr [-skipTrash] <path>]");
1660      System.err.println("           [-expunge]");
1661      System.err.println("           [-put <localsrc> ... <dst>]");
1662      System.err.println("           [-copyFromLocal <localsrc> ... <dst>]");
1663      System.err.println("           [-moveFromLocal <localsrc> ... <dst>]");
1664      System.err.println("           [" + GET_SHORT_USAGE + "]");
1665      System.err.println("           [-getmerge <src> <localdst> [addnl]]");
1666      System.err.println("           [-cat <src>]");
1667      System.err.println("           [-text <src>]");
1668      System.err.println("           [" + COPYTOLOCAL_SHORT_USAGE + "]");
1669      System.err.println("           [-moveToLocal [-crc] <src> <localdst>]");
1670      System.err.println("           [-mkdir <path>]");
1671      System.err.println("           [" + SETREP_SHORT_USAGE + "]");
1672      System.err.println("           [-touchz <path>]");
1673      System.err.println("           [-test -[ezd] <path>]");
1674      System.err.println("           [-stat [format] <path>]");
1675      System.err.println("           [" + TAIL_USAGE + "]");
1676      System.err.println("           [" + FsShellPermissions.CHMOD_USAGE + "]");     
1677      System.err.println("           [" + FsShellPermissions.CHOWN_USAGE + "]");
1678      System.err.println("           [" + FsShellPermissions.CHGRP_USAGE + "]");
1679      System.err.println("           [-help [cmd]]");
1680      System.err.println();
1681      ToolRunner.printGenericCommandUsage(System.err);
1682    }
1683  }
1684
1685  /**
1686   * run
1687   */
1688  public int run(String argv[]) throws Exception {
1689
1690    if (argv.length < 1) {
1691      printUsage(""); 
1692      return -1;
1693    }
1694
1695    int exitCode = -1;
1696    int i = 0;
1697    String cmd = argv[i++];
1698
1699    //
1700    // verify that we have enough command line parameters
1701    //
1702    if ("-put".equals(cmd) || "-test".equals(cmd) ||
1703        "-copyFromLocal".equals(cmd) || "-moveFromLocal".equals(cmd)) {
1704      if (argv.length < 3) {
1705        printUsage(cmd);
1706        return exitCode;
1707      }
1708    } else if ("-get".equals(cmd) || 
1709               "-copyToLocal".equals(cmd) || "-moveToLocal".equals(cmd)) {
1710      if (argv.length < 3) {
1711        printUsage(cmd);
1712        return exitCode;
1713      }
1714    } else if ("-mv".equals(cmd) || "-cp".equals(cmd)) {
1715      if (argv.length < 3) {
1716        printUsage(cmd);
1717        return exitCode;
1718      }
1719    } else if ("-rm".equals(cmd) || "-rmr".equals(cmd) ||
1720               "-cat".equals(cmd) || "-mkdir".equals(cmd) ||
1721               "-touchz".equals(cmd) || "-stat".equals(cmd) ||
1722               "-text".equals(cmd)) {
1723      if (argv.length < 2) {
1724        printUsage(cmd);
1725        return exitCode;
1726      }
1727    }
1728
1729    // initialize FsShell
1730    try {
1731      init();
1732    } catch (RPC.VersionMismatch v) { 
1733      System.err.println("Version Mismatch between client and server" +
1734                         "... command aborted.");
1735      return exitCode;
1736    } catch (IOException e) {
1737      System.err.println("Bad connection to FS. command aborted.");
1738      return exitCode;
1739    }
1740
1741    exitCode = 0;
1742    try {
1743      if ("-put".equals(cmd) || "-copyFromLocal".equals(cmd)) {
1744        Path[] srcs = new Path[argv.length-2];
1745        for (int j=0 ; i < argv.length-1 ;) 
1746          srcs[j++] = new Path(argv[i++]);
1747        copyFromLocal(srcs, argv[i++]);
1748      } else if ("-moveFromLocal".equals(cmd)) {
1749        Path[] srcs = new Path[argv.length-2];
1750        for (int j=0 ; i < argv.length-1 ;) 
1751          srcs[j++] = new Path(argv[i++]);
1752        moveFromLocal(srcs, argv[i++]);
1753      } else if ("-get".equals(cmd) || "-copyToLocal".equals(cmd)) {
1754        copyToLocal(argv, i);
1755      } else if ("-getmerge".equals(cmd)) {
1756        if (argv.length>i+2)
1757          copyMergeToLocal(argv[i++], new Path(argv[i++]), Boolean.parseBoolean(argv[i++]));
1758        else
1759          copyMergeToLocal(argv[i++], new Path(argv[i++]));
1760      } else if ("-cat".equals(cmd)) {
1761        exitCode = doall(cmd, argv, i);
1762      } else if ("-text".equals(cmd)) {
1763        exitCode = doall(cmd, argv, i);
1764      } else if ("-moveToLocal".equals(cmd)) {
1765        moveToLocal(argv[i++], new Path(argv[i++]));
1766      } else if ("-setrep".equals(cmd)) {
1767        setReplication(argv, i);           
1768      } else if ("-chmod".equals(cmd) || 
1769                 "-chown".equals(cmd) ||
1770                 "-chgrp".equals(cmd)) {
1771        FsShellPermissions.changePermissions(fs, cmd, argv, i, this);
1772      } else if ("-ls".equals(cmd)) {
1773        if (i < argv.length) {
1774          exitCode = doall(cmd, argv, i);
1775        } else {
1776          exitCode = ls(Path.CUR_DIR, false);
1777        } 
1778      } else if ("-lsr".equals(cmd)) {
1779        if (i < argv.length) {
1780          exitCode = doall(cmd, argv, i);
1781        } else {
1782          exitCode = ls(Path.CUR_DIR, true);
1783        } 
1784      } else if ("-mv".equals(cmd)) {
1785        exitCode = rename(argv, getConf());
1786      } else if ("-cp".equals(cmd)) {
1787        exitCode = copy(argv, getConf());
1788      } else if ("-rm".equals(cmd)) {
1789        exitCode = doall(cmd, argv, i);
1790      } else if ("-rmr".equals(cmd)) {
1791        exitCode = doall(cmd, argv, i);
1792      } else if ("-expunge".equals(cmd)) {
1793        expunge();
1794      } else if ("-du".equals(cmd)) {
1795        if (i < argv.length) {
1796          exitCode = doall(cmd, argv, i);
1797        } else {
1798          du(".");
1799        }
1800      } else if ("-dus".equals(cmd)) {
1801        if (i < argv.length) {
1802          exitCode = doall(cmd, argv, i);
1803        } else {
1804          dus(".");
1805        }         
1806      } else if (Count.matches(cmd)) {
1807        exitCode = new Count(argv, i, getConf()).runAll();
1808      } else if ("-mkdir".equals(cmd)) {
1809        exitCode = doall(cmd, argv, i);
1810      } else if ("-touchz".equals(cmd)) {
1811        exitCode = doall(cmd, argv, i);
1812      } else if ("-test".equals(cmd)) {
1813        exitCode = test(argv, i);
1814      } else if ("-stat".equals(cmd)) {
1815        if (i + 1 < argv.length) {
1816          stat(argv[i++].toCharArray(), argv[i++]);
1817        } else {
1818          stat("%y".toCharArray(), argv[i]);
1819        }
1820      } else if ("-help".equals(cmd)) {
1821        if (i < argv.length) {
1822          printHelp(argv[i]);
1823        } else {
1824          printHelp("");
1825        }
1826      } else if ("-tail".equals(cmd)) {
1827        tail(argv, i);           
1828      } else {
1829        exitCode = -1;
1830        System.err.println(cmd.substring(1) + ": Unknown command");
1831        printUsage("");
1832      }
1833    } catch (IllegalArgumentException arge) {
1834      exitCode = -1;
1835      System.err.println(cmd.substring(1) + ": " + arge.getLocalizedMessage());
1836      printUsage(cmd);
1837    } catch (RemoteException e) {
1838      //
1839      // This is a error returned by hadoop server. Print
1840      // out the first line of the error mesage, ignore the stack trace.
1841      exitCode = -1;
1842      try {
1843        String[] content;
1844        content = e.getLocalizedMessage().split("\n");
1845        System.err.println(cmd.substring(1) + ": " + 
1846                           content[0]);
1847      } catch (Exception ex) {
1848        System.err.println(cmd.substring(1) + ": " + 
1849                           ex.getLocalizedMessage()); 
1850      }
1851    } catch (IOException e) {
1852      //
1853      // IO exception encountered locally.
1854      //
1855      exitCode = -1;
1856      System.err.println(cmd.substring(1) + ": " + 
1857                         e.getLocalizedMessage()); 
1858    } catch (Exception re) {
1859      exitCode = -1;
1860      System.err.println(cmd.substring(1) + ": " + re.getLocalizedMessage()); 
1861    } finally {
1862    }
1863    return exitCode;
1864  }
1865
1866  public void close() throws IOException {
1867    if (fs != null) {
1868      fs.close();
1869      fs = null;
1870    }
1871  }
1872
1873  /**
1874   * main() has some simple utility methods
1875   */
1876  public static void main(String argv[]) throws Exception {
1877    FsShell shell = new FsShell();
1878    int res;
1879    try {
1880      res = ToolRunner.run(shell, argv);
1881    } finally {
1882      shell.close();
1883    }
1884    System.exit(res);
1885  }
1886
1887  /**
1888   * Accumulate exceptions if there is any.  Throw them at last.
1889   */
1890  private abstract class DelayedExceptionThrowing {
1891    abstract void process(Path p, FileSystem srcFs) throws IOException;
1892
1893    final void globAndProcess(Path srcPattern, FileSystem srcFs
1894        ) throws IOException {
1895      List<IOException> exceptions = new ArrayList<IOException>();
1896      for(Path p : FileUtil.stat2Paths(srcFs.globStatus(srcPattern), 
1897                                       srcPattern))
1898        try { process(p, srcFs); } 
1899        catch(IOException ioe) { exceptions.add(ioe); }
1900   
1901      if (!exceptions.isEmpty())
1902        if (exceptions.size() == 1)
1903          throw exceptions.get(0);
1904        else 
1905          throw new IOException("Multiple IOExceptions: " + exceptions);
1906    }
1907  }
1908}
Note: See TracBrowser for help on using the repository browser.