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.protobuf.generated.HBaseProtos.SnapshotDescription;
36  import org.apache.hadoop.hbase.regionserver.HRegion;
37  import org.apache.hadoop.hbase.snapshot.CopyRecoveredEditsTask;
38  import org.apache.hadoop.hbase.snapshot.ReferenceRegionHFilesTask;
39  import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
40  import org.apache.hadoop.hbase.snapshot.TableInfoCopyTask;
41  import org.apache.hadoop.hbase.snapshot.TakeSnapshotUtils;
42  import org.apache.hadoop.hbase.util.FSUtils;
43  import org.apache.hadoop.hbase.util.Pair;
44  import org.apache.zookeeper.KeeperException;
45  
46  /**
47   * Take a snapshot of a disabled table.
48   * <p>
49   * Table must exist when taking the snapshot, or results are undefined.
50   */
51  @InterfaceAudience.Private
52  @InterfaceStability.Evolving
53  public class DisabledTableSnapshotHandler extends TakeSnapshotHandler {
54    private static final Log LOG = LogFactory.getLog(DisabledTableSnapshotHandler.class);
55    private final TimeoutExceptionInjector timeoutInjector;
56  
57    /**
58     * @param snapshot descriptor of the snapshot to take
59     * @param masterServices master services provider
60     * @throws IOException on unexpected error
61     */
62    public DisabledTableSnapshotHandler(SnapshotDescription snapshot,
63        final MasterServices masterServices) throws IOException {
64      super(snapshot, masterServices);
65  
66      // setup the timer
67      timeoutInjector = TakeSnapshotUtils.getMasterTimerAndBindToMonitor(snapshot, conf, monitor);
68    }
69  
70    // TODO consider parallelizing these operations since they are independent. Right now its just
71    // easier to keep them serial though
72    @Override
73    public void snapshotRegions(List<Pair<HRegionInfo, ServerName>> regionsAndLocations) throws IOException,
74    KeeperException {
75      try {
76        timeoutInjector.start();
77  
78        // 1. get all the regions hosting this table.
79  
80        // extract each pair to separate lists
81        Set<String> serverNames = new HashSet<String>();
82        Set<HRegionInfo> regions = new HashSet<HRegionInfo>();
83        for (Pair<HRegionInfo, ServerName> p : regionsAndLocations) {
84          regions.add(p.getFirst());
85          serverNames.add(p.getSecond().toString());
86        }
87  
88        // 2. for each region, write all the info to disk
89        LOG.info("Starting to write region info and WALs for regions for offline snapshot:"
90            + SnapshotDescriptionUtils.toString(snapshot));
91        for (HRegionInfo regionInfo : regions) {
92          // 2.1 copy the regionInfo files to the snapshot
93          Path snapshotRegionDir = TakeSnapshotUtils.getRegionSnapshotDirectory(snapshot, rootDir,
94            regionInfo.getEncodedName());
95          HRegion.writeRegioninfoOnFilesystem(regionInfo, snapshotRegionDir, fs, conf);
96          // check for error for each region
97          monitor.rethrowException();
98  
99          // 2.2 for each region, copy over its recovered.edits directory
100         Path regionDir = HRegion.getRegionDir(rootDir, regionInfo);
101         new CopyRecoveredEditsTask(snapshot, monitor, fs, regionDir, snapshotRegionDir).call();
102         monitor.rethrowException();
103 
104         // 2.3 reference all the files in the region
105         new ReferenceRegionHFilesTask(snapshot, monitor, regionDir, fs, snapshotRegionDir).call();
106         monitor.rethrowException();
107       }
108 
109       // 3. write the table info to disk
110       LOG.info("Starting to copy tableinfo for offline snapshot: " +
111       SnapshotDescriptionUtils.toString(snapshot));
112       TableInfoCopyTask tableInfoCopyTask = new TableInfoCopyTask(this.monitor, snapshot, fs,
113           FSUtils.getRootDir(conf));
114       tableInfoCopyTask.call();
115       monitor.rethrowException();
116     } catch (Exception e) {
117       // make sure we capture the exception to propagate back to the client later
118       String reason = "Failed snapshot " + SnapshotDescriptionUtils.toString(snapshot)
119           + " due to exception:" + e.getMessage();
120       ForeignException ee = new ForeignException(reason, e);
121       monitor.receive(ee);
122     } finally {
123       LOG.debug("Marking snapshot" + SnapshotDescriptionUtils.toString(snapshot)
124           + " as finished.");
125 
126       // 6. mark the timer as finished - even if we got an exception, we don't need to time the
127       // operation any further
128       timeoutInjector.complete();
129     }
130   }
131 }