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_CHECK;
021import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_CHECK_DESC;
022import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_DEBUG;
023import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_DEBUG_DESC;
024import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_OVERWRITE;
025import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_OVERWRITE_DESC;
026import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_SET;
027import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_SET_RESTORE_DESC;
028import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE;
029import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE_LIST_DESC;
030import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE_MAPPING;
031import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE_MAPPING_DESC;
032import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_YARN_QUEUE_NAME;
033import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_YARN_QUEUE_NAME_RESTORE_DESC;
034
035import java.io.IOException;
036import java.net.URI;
037import java.util.List;
038import java.util.Objects;
039import org.apache.commons.lang3.StringUtils;
040import org.apache.hadoop.conf.Configuration;
041import org.apache.hadoop.fs.Path;
042import org.apache.hadoop.hbase.HBaseConfiguration;
043import org.apache.hadoop.hbase.TableName;
044import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl;
045import org.apache.hadoop.hbase.backup.impl.BackupManager;
046import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
047import org.apache.hadoop.hbase.backup.util.BackupUtils;
048import org.apache.hadoop.hbase.client.Connection;
049import org.apache.hadoop.hbase.client.ConnectionFactory;
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;
059import org.apache.hbase.thirdparty.org.apache.commons.cli.HelpFormatter;
060
061/**
062 * Command-line entry point for restore operation
063 */
064@InterfaceAudience.Private
065public class RestoreDriver extends AbstractHBaseTool {
066  private static final Logger LOG = LoggerFactory.getLogger(RestoreDriver.class);
067  private CommandLine cmd;
068
069  private static final String USAGE_STRING =
070    "Usage: hbase restore <backup_path> <backup_id> [options]\n"
071      + "  backup_path     Path to a backup destination root\n"
072      + "  backup_id       Backup image ID to restore\n"
073      + "  table(s)        Comma-separated list of tables to restore\n";
074
075  private static final String USAGE_FOOTER = "";
076
077  protected RestoreDriver() throws IOException {
078    init();
079  }
080
081  protected void init() {
082    // disable irrelevant loggers to avoid it mess up command output
083    Log4jUtils.disableZkAndClientLoggers();
084  }
085
086  private int parseAndRun() throws IOException {
087    // Check if backup is enabled
088    if (!BackupManager.isBackupEnabled(getConf())) {
089      System.err.println(BackupRestoreConstants.ENABLE_BACKUP);
090      return -1;
091    }
092
093    // enable debug logging
094    if (cmd.hasOption(OPTION_DEBUG)) {
095      Log4jUtils.setLogLevel("org.apache.hadoop.hbase.backup", "DEBUG");
096    }
097
098    // whether to overwrite to existing table if any, false by default
099    boolean overwrite = cmd.hasOption(OPTION_OVERWRITE);
100    if (overwrite) {
101      LOG.debug("Found -overwrite option in restore command, "
102        + "will overwrite to existing table if any in the restore target");
103    }
104
105    // whether to only check the dependencies, false by default
106    boolean check = cmd.hasOption(OPTION_CHECK);
107    if (check) {
108      LOG.debug(
109        "Found -check option in restore command, " + "will check and verify the dependencies");
110    }
111
112    if (cmd.hasOption(OPTION_SET) && cmd.hasOption(OPTION_TABLE)) {
113      System.err.println(
114        "Options -s and -t are mutaully exclusive," + " you can not specify both of them.");
115      printToolUsage();
116      return -1;
117    }
118
119    if (!cmd.hasOption(OPTION_SET) && !cmd.hasOption(OPTION_TABLE)) {
120      System.err.println("You have to specify either set name or table list to restore");
121      printToolUsage();
122      return -1;
123    }
124
125    if (cmd.hasOption(OPTION_YARN_QUEUE_NAME)) {
126      String queueName = cmd.getOptionValue(OPTION_YARN_QUEUE_NAME);
127      // Set system property value for MR job
128      System.setProperty("mapreduce.job.queuename", queueName);
129    }
130
131    // parse main restore command options
132    String[] remainArgs = cmd.getArgs();
133    if (remainArgs.length != 2) {
134      printToolUsage();
135      return -1;
136    }
137
138    String backupRootDir = remainArgs[0];
139    String backupId = remainArgs[1];
140    String tables;
141    String tableMapping =
142      cmd.hasOption(OPTION_TABLE_MAPPING) ? cmd.getOptionValue(OPTION_TABLE_MAPPING) : null;
143    try (final Connection conn = ConnectionFactory.createConnection(conf);
144      BackupAdmin client = new BackupAdminImpl(conn)) {
145      // Check backup set
146      if (cmd.hasOption(OPTION_SET)) {
147        String setName = cmd.getOptionValue(OPTION_SET);
148        try {
149          tables = getTablesForSet(conn, setName);
150        } catch (IOException e) {
151          System.out.println("ERROR: " + e.getMessage() + " for setName=" + setName);
152          printToolUsage();
153          return -2;
154        }
155        if (tables == null) {
156          System.out
157            .println("ERROR: Backup set '" + setName + "' is either empty or does not exist");
158          printToolUsage();
159          return -3;
160        }
161      } else {
162        tables = cmd.getOptionValue(OPTION_TABLE);
163      }
164
165      TableName[] sTableArray = BackupUtils.parseTableNames(tables);
166      TableName[] tTableArray = BackupUtils.parseTableNames(tableMapping);
167
168      if (
169        sTableArray != null && tTableArray != null && (sTableArray.length != tTableArray.length)
170      ) {
171        System.out.println("ERROR: table mapping mismatch: " + tables + " : " + tableMapping);
172        printToolUsage();
173        return -4;
174      }
175
176      client.restore(BackupUtils.createRestoreRequest(backupRootDir, backupId, check, sTableArray,
177        tTableArray, overwrite));
178    } catch (Exception e) {
179      LOG.error("Error while running restore backup", e);
180      return -5;
181    }
182    return 0;
183  }
184
185  private String getTablesForSet(Connection conn, String name) throws IOException {
186    try (final BackupSystemTable table = new BackupSystemTable(conn)) {
187      List<TableName> tables = table.describeBackupSet(name);
188
189      if (tables == null) {
190        return null;
191      }
192
193      return StringUtils.join(tables, BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND);
194    }
195  }
196
197  @Override
198  protected void addOptions() {
199    // define supported options
200    addOptNoArg(OPTION_OVERWRITE, OPTION_OVERWRITE_DESC);
201    addOptNoArg(OPTION_CHECK, OPTION_CHECK_DESC);
202    addOptNoArg(OPTION_DEBUG, OPTION_DEBUG_DESC);
203    addOptWithArg(OPTION_SET, OPTION_SET_RESTORE_DESC);
204    addOptWithArg(OPTION_TABLE, OPTION_TABLE_LIST_DESC);
205    addOptWithArg(OPTION_TABLE_MAPPING, OPTION_TABLE_MAPPING_DESC);
206    addOptWithArg(OPTION_YARN_QUEUE_NAME, OPTION_YARN_QUEUE_NAME_RESTORE_DESC);
207  }
208
209  @Override
210  protected void processOptions(CommandLine cmd) {
211    this.cmd = cmd;
212  }
213
214  @Override
215  protected int doWork() throws Exception {
216    return parseAndRun();
217  }
218
219  public static void main(String[] args) throws Exception {
220    Configuration conf = HBaseConfiguration.create();
221    Path hbasedir = CommonFSUtils.getRootDir(conf);
222    URI defaultFs = hbasedir.getFileSystem(conf).getUri();
223    CommonFSUtils.setFsDefault(conf, new Path(defaultFs));
224    int ret = ToolRunner.run(conf, new RestoreDriver(), args);
225    System.exit(ret);
226  }
227
228  @Override
229  public int run(String[] args) {
230    Objects.requireNonNull(conf, "Tool configuration is not initialized");
231
232    CommandLine cmd;
233    try {
234      // parse the command line arguments
235      cmd = parseArgs(args);
236      cmdLineArgs = args;
237    } catch (Exception e) {
238      System.out.println("Error when parsing command-line arguments: " + e.getMessage());
239      printToolUsage();
240      return EXIT_FAILURE;
241    }
242
243    if (cmd.hasOption(SHORT_HELP_OPTION) || cmd.hasOption(LONG_HELP_OPTION)) {
244      printToolUsage();
245      return EXIT_FAILURE;
246    }
247
248    processOptions(cmd);
249
250    int ret = EXIT_FAILURE;
251    try {
252      ret = doWork();
253    } catch (Exception e) {
254      LOG.error("Error running command-line tool", e);
255      return EXIT_FAILURE;
256    }
257    return ret;
258  }
259
260  protected void printToolUsage() {
261    System.out.println(USAGE_STRING);
262    HelpFormatter helpFormatter = new HelpFormatter();
263    helpFormatter.setLeftPadding(2);
264    helpFormatter.setDescPadding(8);
265    helpFormatter.setWidth(100);
266    helpFormatter.setSyntaxPrefix("Options:");
267    helpFormatter.printHelp(" ", null, options, USAGE_FOOTER);
268    System.out.println(BackupRestoreConstants.VERIFY_BACKUP);
269  }
270}