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