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  import java.util.concurrent.ThreadPoolExecutor;
25  
26  import org.apache.commons.logging.Log;
27  import org.apache.commons.logging.LogFactory;
28  import org.apache.hadoop.conf.Configuration;
29  import org.apache.hadoop.classification.InterfaceAudience;
30  import org.apache.hadoop.classification.InterfaceStability;
31  import org.apache.hadoop.hbase.HRegionInfo;
32  import org.apache.hadoop.hbase.ServerName;
33  import org.apache.hadoop.hbase.errorhandling.ForeignException;
34  import org.apache.hadoop.hbase.errorhandling.ForeignExceptionListener;
35  import org.apache.hadoop.hbase.errorhandling.TimeoutExceptionInjector;
36  import org.apache.hadoop.hbase.master.MasterServices;
37  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
38  import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
39  import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
40  import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
41  import org.apache.hadoop.hbase.util.FSUtils;
42  import org.apache.hadoop.hbase.util.ModifyRegionUtils;
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     */
61    public DisabledTableSnapshotHandler(SnapshotDescription snapshot,
62        final MasterServices masterServices) {
63      super(snapshot, masterServices);
64  
65      // setup the timer
66      timeoutInjector = getMasterTimerAndBindToMonitor(snapshot, conf, monitor);
67    }
68  
69    @Override
70    public DisabledTableSnapshotHandler prepare() throws Exception {
71      return (DisabledTableSnapshotHandler) super.prepare();
72    }
73  
74    // TODO consider parallelizing these operations since they are independent. Right now its just
75    // easier to keep them serial though
76    @Override
77    public void snapshotRegions(List<Pair<HRegionInfo, ServerName>> regionsAndLocations)
78        throws IOException, KeeperException {
79      try {
80        timeoutInjector.start();
81  
82        // 1. get all the regions hosting this table.
83  
84        // extract each pair to separate lists
85        Set<HRegionInfo> regions = new HashSet<HRegionInfo>();
86        for (Pair<HRegionInfo, ServerName> p : regionsAndLocations) {
87          regions.add(p.getFirst());
88        }
89  
90        // 2. for each region, write all the info to disk
91        String msg = "Starting to write region info and WALs for regions for offline snapshot:"
92            + ClientSnapshotDescriptionUtils.toString(snapshot);
93        LOG.info(msg);
94        status.setStatus(msg);
95  
96        ThreadPoolExecutor exec = SnapshotManifest.createExecutor(conf, "DisabledTableSnapshot");
97        try {
98          ModifyRegionUtils.editRegions(exec, regions, new ModifyRegionUtils.RegionEditTask() {
99            @Override
100           public void editRegion(final HRegionInfo regionInfo) throws IOException {
101             snapshotManifest.addRegion(FSUtils.getTableDir(rootDir, snapshotTable), regionInfo);
102           }
103         });
104       } finally {
105         exec.shutdown();
106       }
107     } catch (Exception e) {
108       // make sure we capture the exception to propagate back to the client later
109       String reason = "Failed snapshot " + ClientSnapshotDescriptionUtils.toString(snapshot)
110           + " due to exception:" + e.getMessage();
111       ForeignException ee = new ForeignException(reason, e);
112       monitor.receive(ee);
113       status.abort("Snapshot of table: "+ snapshotTable + " failed because " + e.getMessage());
114     } finally {
115       LOG.debug("Marking snapshot" + ClientSnapshotDescriptionUtils.toString(snapshot)
116           + " as finished.");
117 
118       // 3. mark the timer as finished - even if we got an exception, we don't need to time the
119       // operation any further
120       timeoutInjector.complete();
121     }
122   }
123 
124 
125   /**
126    * Create a snapshot timer for the master which notifies the monitor when an error occurs
127    * @param snapshot snapshot to monitor
128    * @param conf configuration to use when getting the max snapshot life
129    * @param monitor monitor to notify when the snapshot life expires
130    * @return the timer to use update to signal the start and end of the snapshot
131    */
132   private TimeoutExceptionInjector getMasterTimerAndBindToMonitor(SnapshotDescription snapshot,
133       Configuration conf, ForeignExceptionListener monitor) {
134     long maxTime = SnapshotDescriptionUtils.getMaxMasterTimeout(conf, snapshot.getType(),
135       SnapshotDescriptionUtils.DEFAULT_MAX_WAIT_TIME);
136     return new TimeoutExceptionInjector(monitor, maxTime);
137   }
138 }