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