001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018
019package org.apache.hadoop.fs.azure;
020
021import java.io.IOException;
022import java.util.Arrays;
023import java.util.List;
024
025import org.apache.hadoop.classification.InterfaceAudience;
026import org.apache.hadoop.classification.InterfaceStability;
027import org.apache.hadoop.conf.Configuration;
028import org.apache.hadoop.conf.Configured;
029import org.apache.hadoop.fs.FileStatus;
030import org.apache.hadoop.fs.FileSystem;
031import org.apache.hadoop.fs.Path;
032import org.apache.hadoop.util.Tool;
033import org.apache.hadoop.util.ToolRunner;
034
035import com.google.common.annotations.VisibleForTesting;
036
037/**
038 * An fsck tool implementation for WASB that does various admin/cleanup/recovery
039 * tasks on the WASB file system.
040 */
041@InterfaceAudience.Public
042@InterfaceStability.Evolving
043public class WasbFsck extends Configured implements Tool {
044  private FileSystem mockFileSystemForTesting = null;
045  private static final String LOST_AND_FOUND_PATH = "/lost+found";
046  private boolean pathNameWarning = false;
047
048  public WasbFsck(Configuration conf) {
049    super(conf);
050  }
051
052  /**
053   * For testing purposes, set the file system to use here instead of relying on
054   * getting it from the FileSystem class based on the URI.
055   * 
056   * @param fileSystem
057   *          The file system to use.
058   */
059  @VisibleForTesting
060  public void setMockFileSystemForTesting(FileSystem fileSystem) {
061    this.mockFileSystemForTesting = fileSystem;
062  }
063
064  @Override
065  public int run(String[] args) throws Exception {
066    if (doPrintUsage(Arrays.asList(args))) {
067      printUsage();
068      return -1;
069    }
070    Path pathToCheck = null;
071    boolean doRecover = false;
072    boolean doDelete = false;
073    for (String arg : args) {
074      if (!arg.startsWith("-")) {
075        if (pathToCheck != null) {
076          System.err
077              .println("Can't specify multiple paths to check on the command-line");
078          return 1;
079        }
080        pathToCheck = new Path(arg);
081      } else if (arg.equals("-move")) {
082        doRecover = true;
083      } else if (arg.equals("-delete")) {
084        doDelete = true;
085      }
086    }
087    if (doRecover && doDelete) {
088      System.err
089          .println("Conflicting options: can't specify both -move and -delete.");
090      return 1;
091    }
092    if (pathToCheck == null) {
093      pathToCheck = new Path("/"); // Check everything.
094    }
095    FileSystem fs;
096    if (mockFileSystemForTesting == null) {
097      fs = FileSystem.get(pathToCheck.toUri(), getConf());
098    } else {
099      fs = mockFileSystemForTesting;
100    }
101
102    if (!recursiveCheckChildPathName(fs, fs.makeQualified(pathToCheck))) {
103      pathNameWarning = true;
104    }
105
106    if (!(fs instanceof NativeAzureFileSystem)) {
107      System.err
108          .println("Can only check WASB file system. Instead I'm asked to"
109              + " check: " + fs.getUri());
110      return 2;
111    }
112    NativeAzureFileSystem wasbFs = (NativeAzureFileSystem) fs;
113    if (doRecover) {
114      System.out.println("Recovering files with dangling data under: "
115          + pathToCheck);
116      wasbFs.recoverFilesWithDanglingTempData(pathToCheck, new Path(
117          LOST_AND_FOUND_PATH));
118    } else if (doDelete) {
119      System.out.println("Deleting temp files with dangling data under: "
120          + pathToCheck);
121      wasbFs.deleteFilesWithDanglingTempData(pathToCheck);
122    } else {
123      System.out.println("Please specify -move or -delete");
124    }
125    return 0;
126  }
127
128  public boolean getPathNameWarning() {
129    return pathNameWarning;
130  }
131
132  /**
133   * Recursively check if a given path and its child paths have colons in their
134   * names. It returns true if none of them has a colon or this path does not
135   * exist, and false otherwise.
136   */
137  private boolean recursiveCheckChildPathName(FileSystem fs, Path p)
138      throws IOException {
139    if (p == null) {
140      return true;
141    }
142    if (!fs.exists(p)) {
143      System.out.println("Path " + p + " does not exist!");
144      return true;
145    }
146
147    if (fs.isFile(p)) {
148      if (containsColon(p)) {
149        System.out.println("Warning: file " + p + " has a colon in its name.");
150        return false;
151      } else {
152        return true;
153      }
154    } else {
155      boolean flag;
156      if (containsColon(p)) {
157        System.out.println("Warning: directory " + p
158            + " has a colon in its name.");
159        flag = false;
160      } else {
161        flag = true;
162      }
163      FileStatus[] listed = fs.listStatus(p);
164      for (FileStatus l : listed) {
165        if (!recursiveCheckChildPathName(fs, l.getPath())) {
166          flag = false;
167        }
168      }
169      return flag;
170    }
171  }
172
173  private boolean containsColon(Path p) {
174    return p.toUri().getPath().toString().contains(":");
175  }
176
177  private static void printUsage() {
178    System.out.println("Usage: WasbFSck [<path>] [-move | -delete]");
179    System.out.println("\t<path>\tstart checking from this path");
180    System.out.println("\t-move\tmove any files whose upload was interrupted"
181        + " mid-stream to " + LOST_AND_FOUND_PATH);
182    System.out
183        .println("\t-delete\tdelete any files whose upload was interrupted"
184            + " mid-stream");
185    ToolRunner.printGenericCommandUsage(System.out);
186  }
187
188  private boolean doPrintUsage(List<String> args) {
189    return args.contains("-H");
190  }
191
192  public static void main(String[] args) throws Exception {
193    int res = ToolRunner.run(new WasbFsck(new Configuration()), args);
194    System.exit(res);
195  }
196}