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  
19  package org.apache.hadoop.hbase.client;
20  
21  import java.io.IOException;
22  import java.util.ArrayList;
23  import java.util.Collections;
24  import java.util.List;
25  import java.util.UUID;
26  
27  import org.apache.commons.logging.Log;
28  import org.apache.commons.logging.LogFactory;
29  import org.apache.hadoop.conf.Configuration;
30  import org.apache.hadoop.fs.FileSystem;
31  import org.apache.hadoop.fs.Path;
32  import org.apache.hadoop.hbase.CellUtil;
33  import org.apache.hadoop.hbase.HRegionInfo;
34  import org.apache.hadoop.hbase.HTableDescriptor;
35  import org.apache.hadoop.hbase.classification.InterfaceAudience;
36  import org.apache.hadoop.hbase.classification.InterfaceStability;
37  import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
38  import org.apache.hadoop.hbase.util.FSUtils;
39  
40  /**
41   * A Scanner which performs a scan over snapshot files. Using this class requires copying the
42   * snapshot to a temporary empty directory, which will copy the snapshot reference files into that
43   * directory. Actual data files are not copied.
44   *
45   * <p>
46   * This also allows one to run the scan from an
47   * online or offline hbase cluster. The snapshot files can be exported by using the
48   * {@link org.apache.hadoop.hbase.snapshot.ExportSnapshot} tool,
49   * to a pure-hdfs cluster, and this scanner can be used to
50   * run the scan directly over the snapshot files. The snapshot should not be deleted while there
51   * are open scanners reading from snapshot files.
52   *
53   * <p>
54   * An internal RegionScanner is used to execute the {@link Scan} obtained
55   * from the user for each region in the snapshot.
56   * <p>
57   * HBase owns all the data and snapshot files on the filesystem. Only the HBase user can read from
58   * snapshot files and data files. HBase also enforces security because all the requests are handled
59   * by the server layer, and the user cannot read from the data files directly. To read from snapshot
60   * files directly from the file system, the user who is running the MR job must have sufficient
61   * permissions to access snapshot and reference files. This means that to run mapreduce over
62   * snapshot files, the job has to be run as the HBase user or the user must have group or other
63   * priviledges in the filesystem (See HBASE-8369). Note that, given other users access to read from
64   * snapshot/data files will completely circumvent the access control enforced by HBase.
65   * @see org.apache.hadoop.hbase.mapreduce.TableSnapshotInputFormat
66   */
67  @InterfaceAudience.Public
68  @InterfaceStability.Evolving
69  public class TableSnapshotScanner extends AbstractClientScanner {
70  
71    private static final Log LOG = LogFactory.getLog(TableSnapshotScanner.class);
72  
73    private Configuration conf;
74    private String snapshotName;
75    private FileSystem fs;
76    private Path rootDir;
77    private Path restoreDir;
78    private Scan scan;
79    private ArrayList<HRegionInfo> regions;
80    private HTableDescriptor htd;
81  
82    private ClientSideRegionScanner currentRegionScanner  = null;
83    private int currentRegion = -1;
84  
85    /**
86     * Creates a TableSnapshotScanner.
87     * @param conf the configuration
88     * @param restoreDir a temporary directory to copy the snapshot files into. Current user should
89     * have write permissions to this directory, and this should not be a subdirectory of rootdir.
90     * The scanner deletes the contents of the directory once the scanner is closed.
91     * @param snapshotName the name of the snapshot to read from
92     * @param scan a Scan representing scan parameters
93     * @throws IOException in case of error
94     */
95    public TableSnapshotScanner(Configuration conf, Path restoreDir,
96        String snapshotName, Scan scan) throws IOException {
97      this(conf, FSUtils.getRootDir(conf), restoreDir, snapshotName, scan);
98    }
99  
100   /**
101    * Creates a TableSnapshotScanner.
102    * @param conf the configuration
103    * @param rootDir root directory for HBase.
104    * @param restoreDir a temporary directory to copy the snapshot files into. Current user should
105    * have write permissions to this directory, and this should not be a subdirectory of rootdir.
106    * The scanner deletes the contents of the directory once the scanner is closed.
107    * @param snapshotName the name of the snapshot to read from
108    * @param scan a Scan representing scan parameters
109    * @throws IOException in case of error
110    */
111   public TableSnapshotScanner(Configuration conf, Path rootDir,
112       Path restoreDir, String snapshotName, Scan scan) throws IOException {
113     this.conf = conf;
114     this.snapshotName = snapshotName;
115     this.rootDir = rootDir;
116     // restoreDir will be deleted in close(), use a unique sub directory
117     this.restoreDir = new Path(restoreDir, UUID.randomUUID().toString());
118     this.scan = scan;
119     this.fs = rootDir.getFileSystem(conf);
120     init();
121   }
122 
123   private void init() throws IOException {
124     final RestoreSnapshotHelper.RestoreMetaChanges meta =
125       RestoreSnapshotHelper.copySnapshotForScanner(
126         conf, fs, rootDir, restoreDir, snapshotName);
127     final List<HRegionInfo> restoredRegions = meta.getRegionsToAdd();
128 
129     htd = meta.getTableDescriptor();
130     regions = new ArrayList<HRegionInfo>(restoredRegions.size());
131     for (HRegionInfo hri: restoredRegions) {
132       if (hri.isOffline() && (hri.isSplit() || hri.isSplitParent())) {
133         continue;
134       }
135       if (CellUtil.overlappingKeys(scan.getStartRow(), scan.getStopRow(),
136           hri.getStartKey(), hri.getEndKey())) {
137         regions.add(hri);
138       }
139     }
140 
141     // sort for regions according to startKey.
142     Collections.sort(regions);
143     initScanMetrics(scan);
144   }
145 
146   @Override
147   public Result next() throws IOException {
148     Result result = null;
149     while (true) {
150       if (currentRegionScanner == null) {
151         currentRegion++;
152         if (currentRegion >= regions.size()) {
153           return null;
154         }
155 
156         HRegionInfo hri = regions.get(currentRegion);
157         currentRegionScanner = new ClientSideRegionScanner(conf, fs,
158           restoreDir, htd, hri, scan, scanMetrics);
159         if (this.scanMetrics != null) {
160           this.scanMetrics.countOfRegions.incrementAndGet();
161         }
162       }
163 
164       try {
165         result = currentRegionScanner.next();
166         if (result != null) {
167           return result;
168         }
169       } finally {
170         if (result == null) {
171           currentRegionScanner.close();
172           currentRegionScanner = null;
173         }
174       }
175     }
176   }
177 
178   @Override
179   public void close() {
180     if (currentRegionScanner != null) {
181       currentRegionScanner.close();
182     }
183     try {
184       fs.delete(this.restoreDir, true);
185     } catch (IOException ex) {
186       LOG.warn("Could not delete restore directory for the snapshot:" + ex);
187     }
188   }
189 
190   @Override
191   public boolean renewLease() {
192     throw new UnsupportedOperationException();
193   }
194 
195 }