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 */
018package org.apache.hadoop.hbase.backup;
019
020import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_BACKUP_LIST_DESC;
021import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_BANDWIDTH;
022import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_BANDWIDTH_DESC;
023import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_DEBUG;
024import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_DEBUG_DESC;
025import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_KEEP;
026import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_KEEP_DESC;
027import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_LIST;
028import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_PATH;
029import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_PATH_DESC;
030import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_RECORD_NUMBER;
031import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_RECORD_NUMBER_DESC;
032import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_SET;
033import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_SET_DESC;
034import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE;
035import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE_DESC;
036import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_WORKERS;
037import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_WORKERS_DESC;
038import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_YARN_QUEUE_NAME;
039import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_YARN_QUEUE_NAME_DESC;
040
041import java.io.IOException;
042import java.net.URI;
043import java.util.Objects;
044import org.apache.hadoop.conf.Configuration;
045import org.apache.hadoop.fs.Path;
046import org.apache.hadoop.hbase.HBaseConfiguration;
047import org.apache.hadoop.hbase.backup.BackupRestoreConstants.BackupCommand;
048import org.apache.hadoop.hbase.backup.impl.BackupCommands;
049import org.apache.hadoop.hbase.backup.impl.BackupManager;
050import org.apache.hadoop.hbase.logging.Log4jUtils;
051import org.apache.hadoop.hbase.util.AbstractHBaseTool;
052import org.apache.hadoop.hbase.util.CommonFSUtils;
053import org.apache.hadoop.util.ToolRunner;
054import org.apache.yetus.audience.InterfaceAudience;
055import org.slf4j.Logger;
056import org.slf4j.LoggerFactory;
057
058import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
059
060/**
061 * Command-line entry point for backup operation
062 */
063@InterfaceAudience.Private
064public class BackupDriver extends AbstractHBaseTool {
065
066  private static final Logger LOG = LoggerFactory.getLogger(BackupDriver.class);
067  private CommandLine cmd;
068
069  public BackupDriver() throws IOException {
070    init();
071  }
072
073  protected void init() throws IOException {
074    // disable irrelevant loggers to avoid it mess up command output
075    Log4jUtils.disableZkAndClientLoggers();
076  }
077
078  private int parseAndRun(String[] args) throws IOException {
079
080    // Check if backup is enabled
081    if (!BackupManager.isBackupEnabled(getConf())) {
082      System.err.println(BackupRestoreConstants.ENABLE_BACKUP);
083      return -1;
084    }
085
086    String cmd = null;
087    String[] remainArgs = null;
088    if (args == null || args.length == 0) {
089      printToolUsage();
090      return -1;
091    } else {
092      cmd = args[0];
093      remainArgs = new String[args.length - 1];
094      if (args.length > 1) {
095        System.arraycopy(args, 1, remainArgs, 0, args.length - 1);
096      }
097    }
098
099    BackupCommand type = BackupCommand.HELP;
100    if (BackupCommand.CREATE.name().equalsIgnoreCase(cmd)) {
101      type = BackupCommand.CREATE;
102    } else if (BackupCommand.HELP.name().equalsIgnoreCase(cmd)) {
103      type = BackupCommand.HELP;
104    } else if (BackupCommand.DELETE.name().equalsIgnoreCase(cmd)) {
105      type = BackupCommand.DELETE;
106    } else if (BackupCommand.DESCRIBE.name().equalsIgnoreCase(cmd)) {
107      type = BackupCommand.DESCRIBE;
108    } else if (BackupCommand.HISTORY.name().equalsIgnoreCase(cmd)) {
109      type = BackupCommand.HISTORY;
110    } else if (BackupCommand.PROGRESS.name().equalsIgnoreCase(cmd)) {
111      type = BackupCommand.PROGRESS;
112    } else if (BackupCommand.SET.name().equalsIgnoreCase(cmd)) {
113      type = BackupCommand.SET;
114    } else if (BackupCommand.REPAIR.name().equalsIgnoreCase(cmd)) {
115      type = BackupCommand.REPAIR;
116    } else if (BackupCommand.MERGE.name().equalsIgnoreCase(cmd)) {
117      type = BackupCommand.MERGE;
118    } else {
119      System.out.println("Unsupported command for backup: " + cmd);
120      printToolUsage();
121      return -1;
122    }
123
124    // enable debug logging
125    if (this.cmd.hasOption(OPTION_DEBUG)) {
126      Log4jUtils.setLogLevel("org.apache.hadoop.hbase.backup", "DEBUG");
127    }
128
129    BackupCommands.Command command = BackupCommands.createCommand(getConf(), type, this.cmd);
130    if (type == BackupCommand.CREATE && conf != null) {
131      ((BackupCommands.CreateCommand) command).setConf(conf);
132    }
133    try {
134      command.execute();
135    } catch (IOException e) {
136      if (e.getMessage().equals(BackupCommands.INCORRECT_USAGE)) {
137        return -1;
138      }
139      throw e;
140    } finally {
141      command.finish();
142    }
143    return 0;
144  }
145
146  @Override
147  protected void addOptions() {
148    // define supported options
149    addOptNoArg(OPTION_DEBUG, OPTION_DEBUG_DESC);
150    addOptWithArg(OPTION_TABLE, OPTION_TABLE_DESC);
151    addOptWithArg(OPTION_BANDWIDTH, OPTION_BANDWIDTH_DESC);
152    addOptWithArg(OPTION_LIST, OPTION_BACKUP_LIST_DESC);
153    addOptWithArg(OPTION_WORKERS, OPTION_WORKERS_DESC);
154    addOptWithArg(OPTION_RECORD_NUMBER, OPTION_RECORD_NUMBER_DESC);
155    addOptWithArg(OPTION_SET, OPTION_SET_DESC);
156    addOptWithArg(OPTION_PATH, OPTION_PATH_DESC);
157    addOptWithArg(OPTION_KEEP, OPTION_KEEP_DESC);
158    addOptWithArg(OPTION_YARN_QUEUE_NAME, OPTION_YARN_QUEUE_NAME_DESC);
159
160  }
161
162  @Override
163  protected void processOptions(CommandLine cmd) {
164    this.cmd = cmd;
165  }
166
167  @Override
168  protected int doWork() throws Exception {
169    return parseAndRun(cmd.getArgs());
170  }
171
172  public static void main(String[] args) throws Exception {
173    Configuration conf = HBaseConfiguration.create();
174    Path hbasedir = CommonFSUtils.getRootDir(conf);
175    URI defaultFs = hbasedir.getFileSystem(conf).getUri();
176    CommonFSUtils.setFsDefault(conf, new Path(defaultFs));
177    int ret = ToolRunner.run(conf, new BackupDriver(), args);
178    System.exit(ret);
179  }
180
181  @Override
182  public int run(String[] args) throws IOException {
183    Objects.requireNonNull(conf, "Tool configuration is not initialized");
184
185    CommandLine cmd;
186    try {
187      // parse the command line arguments
188      cmd = parseArgs(args);
189      cmdLineArgs = args;
190    } catch (Exception e) {
191      System.err.println("Error when parsing command-line arguments: " + e.getMessage());
192      printToolUsage();
193      return EXIT_FAILURE;
194    }
195    processOptions(cmd);
196
197    int ret = EXIT_FAILURE;
198    try {
199      ret = doWork();
200    } catch (Exception e) {
201      LOG.error("Error running command-line tool", e);
202      return EXIT_FAILURE;
203    }
204    return ret;
205  }
206
207  protected void printToolUsage() throws IOException {
208    System.out.println(BackupCommands.USAGE);
209    System.out.println(BackupRestoreConstants.VERIFY_BACKUP);
210  }
211}