View Javadoc

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   */
18  package org.apache.hadoop.hbase.master.snapshot;
19  
20  import java.io.IOException;
21  import java.util.HashSet;
22  import java.util.List;
23  import java.util.Set;
24  
25  import org.apache.commons.logging.Log;
26  import org.apache.commons.logging.LogFactory;
27  import org.apache.hadoop.classification.InterfaceAudience;
28  import org.apache.hadoop.classification.InterfaceStability;
29  import org.apache.hadoop.fs.Path;
30  import org.apache.hadoop.hbase.HRegionInfo;
31  import org.apache.hadoop.hbase.ServerName;
32  import org.apache.hadoop.hbase.errorhandling.ForeignException;
33  import org.apache.hadoop.hbase.errorhandling.TimeoutExceptionInjector;
34  import org.apache.hadoop.hbase.master.MasterServices;
35  import org.apache.hadoop.hbase.master.metrics.MasterMetrics;
36  import org.apache.hadoop.hbase.monitoring.MonitoredTask;
37  import org.apache.hadoop.hbase.monitoring.TaskMonitor;
38  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
39  import org.apache.hadoop.hbase.regionserver.HRegion;
40  import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
41  import org.apache.hadoop.hbase.snapshot.TableInfoCopyTask;
42  import org.apache.hadoop.hbase.snapshot.TakeSnapshotUtils;
43  import org.apache.hadoop.hbase.util.FSUtils;
44  import org.apache.hadoop.hbase.util.Pair;
45  import org.apache.zookeeper.KeeperException;
46  
47  /**
48   * Take a snapshot of a disabled table.
49   * <p>
50   * Table must exist when taking the snapshot, or results are undefined.
51   */
52  @InterfaceAudience.Private
53  @InterfaceStability.Evolving
54  public class DisabledTableSnapshotHandler extends TakeSnapshotHandler {
55    private static final Log LOG = LogFactory.getLog(DisabledTableSnapshotHandler.class);
56    private final TimeoutExceptionInjector timeoutInjector;
57  
58    /**
59     * @param snapshot descriptor of the snapshot to take
60     * @param masterServices master services provider
61     * @throws IOException on unexpected error
62     */
63    public DisabledTableSnapshotHandler(SnapshotDescription snapshot,
64        final MasterServices masterServices, final MasterMetrics metricsMaster) {
65      super(snapshot, masterServices, metricsMaster);
66  
67      // setup the timer
68      timeoutInjector = TakeSnapshotUtils.getMasterTimerAndBindToMonitor(snapshot, conf, monitor);
69    }
70  
71    // TODO consider parallelizing these operations since they are independent. Right now its just
72    // easier to keep them serial though
73    @Override
74    public void snapshotRegions(List<Pair<HRegionInfo, ServerName>> regionsAndLocations)
75        throws IOException, KeeperException {
76      try {
77        timeoutInjector.start();
78  
79        // 1. get all the regions hosting this table.
80  
81        // extract each pair to separate lists
82        Set<HRegionInfo> regions = new HashSet<HRegionInfo>();
83        for (Pair<HRegionInfo, ServerName> p : regionsAndLocations) {
84          regions.add(p.getFirst());
85        }
86  
87        // 2. for each region, write all the info to disk
88        LOG.info("Starting to write region info and WALs for regions for offline snapshot:"
89            + SnapshotDescriptionUtils.toString(snapshot));
90        for (HRegionInfo regionInfo : regions) {
91          snapshotDisabledRegion(regionInfo);
92        }
93  
94        // 3. write the table info to disk
95        LOG.info("Starting to copy tableinfo for offline snapshot: " +
96        SnapshotDescriptionUtils.toString(snapshot));
97        TableInfoCopyTask tableInfoCopyTask = new TableInfoCopyTask(this.monitor, snapshot, fs,
98            FSUtils.getRootDir(conf));
99        tableInfoCopyTask.call();
100       monitor.rethrowException();
101       status.setStatus("Finished copying tableinfo for snapshot of table: " + snapshot.getTable());
102     } catch (Exception e) {
103       // make sure we capture the exception to propagate back to the client later
104       String reason = "Failed snapshot " + SnapshotDescriptionUtils.toString(snapshot)
105           + " due to exception:" + e.getMessage();
106       ForeignException ee = new ForeignException(reason, e);
107       monitor.receive(ee);
108       status.abort("Snapshot of table: "+ snapshot.getTable() +" failed because " + e.getMessage());
109     } finally {
110       LOG.debug("Marking snapshot" + SnapshotDescriptionUtils.toString(snapshot)
111           + " as finished.");
112 
113       // 6. mark the timer as finished - even if we got an exception, we don't need to time the
114       // operation any further
115       timeoutInjector.complete();
116     }
117   }
118 }