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.mapreduce;
20  
21  import java.io.ByteArrayOutputStream;
22  import java.io.DataInput;
23  import java.io.DataOutput;
24  import java.io.IOException;
25  import java.lang.reflect.Method;
26  import java.util.ArrayList;
27  import java.util.List;
28  import java.util.Set;
29  import java.util.UUID;
30  
31  import com.google.protobuf.HBaseZeroCopyByteString;
32  import org.apache.commons.logging.Log;
33  import org.apache.commons.logging.LogFactory;
34  import org.apache.hadoop.classification.InterfaceAudience;
35  import org.apache.hadoop.classification.InterfaceStability;
36  import org.apache.hadoop.conf.Configuration;
37  import org.apache.hadoop.fs.FileSystem;
38  import org.apache.hadoop.fs.Path;
39  import org.apache.hadoop.hbase.CellUtil;
40  import org.apache.hadoop.hbase.HConstants;
41  import org.apache.hadoop.hbase.HDFSBlocksDistribution;
42  import org.apache.hadoop.hbase.HDFSBlocksDistribution.HostAndWeight;
43  import org.apache.hadoop.hbase.HRegionInfo;
44  import org.apache.hadoop.hbase.HTableDescriptor;
45  import org.apache.hadoop.hbase.client.ClientSideRegionScanner;
46  import org.apache.hadoop.hbase.client.IsolationLevel;
47  import org.apache.hadoop.hbase.client.Result;
48  import org.apache.hadoop.hbase.client.Scan;
49  import org.apache.hadoop.hbase.client.TableSnapshotScanner;
50  import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
51  import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
52  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier;
53  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
54  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
55  import org.apache.hadoop.hbase.protobuf.generated.MapReduceProtos;
56  import org.apache.hadoop.hbase.regionserver.HRegion;
57  import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
58  import org.apache.hadoop.hbase.snapshot.ExportSnapshot;
59  import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
60  import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
61  import org.apache.hadoop.hbase.snapshot.SnapshotReferenceUtil;
62  import org.apache.hadoop.hbase.util.Bytes;
63  import org.apache.hadoop.hbase.util.FSTableDescriptors;
64  import org.apache.hadoop.io.Writable;
65  import org.apache.hadoop.mapreduce.InputFormat;
66  import org.apache.hadoop.mapreduce.InputSplit;
67  import org.apache.hadoop.mapreduce.Job;
68  import org.apache.hadoop.mapreduce.JobContext;
69  import org.apache.hadoop.mapreduce.RecordReader;
70  import org.apache.hadoop.mapreduce.TaskAttemptContext;
71  
72  import com.google.common.annotations.VisibleForTesting;
73  
74  /**
75   * TableSnapshotInputFormat allows a MapReduce job to run over a table snapshot. The job
76   * bypasses HBase servers, and directly accesses the underlying files (hfile, recovered edits,
77   * hlogs, etc) directly to provide maximum performance. The snapshot is not required to be
78   * restored to the live cluster or cloned. This also allows to run the mapreduce job from an
79   * online or offline hbase cluster. The snapshot files can be exported by using the
80   * {@link ExportSnapshot} tool, to a pure-hdfs cluster, and this InputFormat can be used to
81   * run the mapreduce job directly over the snapshot files. The snapshot should not be deleted
82   * while there are jobs reading from snapshot files.
83   * <p>
84   * Usage is similar to TableInputFormat, and
85   * {@link TableMapReduceUtil#initTableSnapshotMapperJob(String, Scan, Class, Class, Class, Job, boolean, Path)}
86   * can be used to configure the job.
87   * <pre>{@code
88   * Job job = new Job(conf);
89   * Scan scan = new Scan();
90   * TableMapReduceUtil.initTableSnapshotMapperJob(snapshotName,
91   *      scan, MyTableMapper.class, MyMapKeyOutput.class,
92   *      MyMapOutputValueWritable.class, job, true);
93   * }
94   * </pre>
95   * <p>
96   * Internally, this input format restores the snapshot into the given tmp directory. Similar to
97   * {@link TableInputFormat} an InputSplit is created per region. The region is opened for reading
98   * from each RecordReader. An internal RegionScanner is used to execute the {@link Scan} obtained
99   * from the user.
100  * <p>
101  * HBase owns all the data and snapshot files on the filesystem. Only the HBase user can read from
102  * snapshot files and data files. HBase also enforces security because all the requests are handled
103  * by the server layer, and the user cannot read from the data files directly. To read from snapshot
104  * files directly from the file system, the user who is running the MR job must have sufficient
105  * permissions to access snapshot and reference files. This means that to run mapreduce over
106  * snapshot files, the MR job has to be run as the HBase user or the user must have group or other
107  * priviledges in the filesystem (See HBASE-8369). Note that, given other users access to read from
108  * snapshot/data files will completely circumvent the access control enforced by HBase.
109  * @see TableSnapshotScanner
110  */
111 @InterfaceAudience.Public
112 @InterfaceStability.Evolving
113 public class TableSnapshotInputFormat extends InputFormat<ImmutableBytesWritable, Result> {
114   // TODO: Snapshots files are owned in fs by the hbase user. There is no
115   // easy way to delegate access.
116 
117   private static final Log LOG = LogFactory.getLog(TableSnapshotInputFormat.class);
118 
119   /** See {@link #getBestLocations(Configuration, HDFSBlocksDistribution)} */
120   private static final String LOCALITY_CUTOFF_MULTIPLIER = "hbase.tablesnapshotinputformat.locality.cutoff.multiplier";
121   private static final float DEFAULT_LOCALITY_CUTOFF_MULTIPLIER = 0.8f;
122 
123   private static final String SNAPSHOT_NAME_KEY = "hbase.TableSnapshotInputFormat.snapshot.name";
124   private static final String TABLE_DIR_KEY = "hbase.TableSnapshotInputFormat.table.dir";
125 
126   public static class TableSnapshotRegionSplit extends InputSplit implements Writable {
127     private String regionName;
128     private String[] locations;
129 
130     // constructor for mapreduce framework / Writable
131     public TableSnapshotRegionSplit() { }
132 
133     TableSnapshotRegionSplit(String regionName, List<String> locations) {
134       this.regionName = regionName;
135       if (locations == null || locations.isEmpty()) {
136         this.locations = new String[0];
137       } else {
138         this.locations = locations.toArray(new String[locations.size()]);
139       }
140     }
141     @Override
142     public long getLength() throws IOException, InterruptedException {
143       //TODO: We can obtain the file sizes of the snapshot here.
144       return 0;
145     }
146 
147     @Override
148     public String[] getLocations() throws IOException, InterruptedException {
149       return locations;
150     }
151 
152     // TODO: We should have ProtobufSerialization in Hadoop, and directly use PB objects instead of
153     // doing this wrapping with Writables.
154     @Override
155     public void write(DataOutput out) throws IOException {
156     MapReduceProtos.TableSnapshotRegionSplit.Builder builder =
157       MapReduceProtos.TableSnapshotRegionSplit.newBuilder()
158         .setRegion(RegionSpecifier.newBuilder()
159           .setType(RegionSpecifierType.ENCODED_REGION_NAME)
160           .setValue(HBaseZeroCopyByteString.wrap(Bytes.toBytes(regionName))).build());
161 
162       for (String location : locations) {
163         builder.addLocations(location);
164       }
165 
166       MapReduceProtos.TableSnapshotRegionSplit split = builder.build();
167 
168       ByteArrayOutputStream baos = new ByteArrayOutputStream();
169       split.writeTo(baos);
170       baos.close();
171       byte[] buf = baos.toByteArray();
172       out.writeInt(buf.length);
173       out.write(buf);
174     }
175     @Override
176     public void readFields(DataInput in) throws IOException {
177       int len = in.readInt();
178       byte[] buf = new byte[len];
179       in.readFully(buf);
180       MapReduceProtos.TableSnapshotRegionSplit split = MapReduceProtos.TableSnapshotRegionSplit.PARSER.parseFrom(buf);
181       this.regionName = Bytes.toString(split.getRegion().getValue().toByteArray());
182       List<String> locationsList = split.getLocationsList();
183       this.locations = locationsList.toArray(new String[locationsList.size()]);
184     }
185   }
186 
187   @VisibleForTesting
188   static class TableSnapshotRegionRecordReader extends RecordReader<ImmutableBytesWritable, Result> {
189     private TableSnapshotRegionSplit split;
190     private Scan scan;
191     private Result result = null;
192     private ImmutableBytesWritable row = null;
193     private ClientSideRegionScanner scanner;
194     private TaskAttemptContext context;
195     private Method getCounter;
196 
197     @Override
198     public void initialize(InputSplit split, TaskAttemptContext context) throws IOException,
199         InterruptedException {
200 
201       Configuration conf = context.getConfiguration();
202       this.split = (TableSnapshotRegionSplit) split;
203       String regionName = this.split.regionName;
204       String snapshotName = getSnapshotName(conf);
205       Path rootDir = new Path(conf.get(HConstants.HBASE_DIR));
206       FileSystem fs = rootDir.getFileSystem(conf);
207 
208       Path tmpRootDir = new Path(conf.get(TABLE_DIR_KEY)); // This is the user specified root
209       // directory where snapshot was restored
210 
211       Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, rootDir);
212 
213       //load table descriptor
214       HTableDescriptor htd = FSTableDescriptors.getTableDescriptorFromFs(fs, snapshotDir);
215 
216       //load region descriptor
217       Path regionDir = new Path(snapshotDir, regionName);
218       HRegionInfo hri = HRegionFileSystem.loadRegionInfoFileContent(fs, regionDir);
219 
220       // create scan
221       String scanStr = conf.get(TableInputFormat.SCAN);
222       if (scanStr == null) {
223         throw new IllegalArgumentException("A Scan is not configured for this job");
224       }
225       scan = TableMapReduceUtil.convertStringToScan(scanStr);
226       // region is immutable, this should be fine,
227       // otherwise we have to set the thread read point
228       scan.setIsolationLevel(IsolationLevel.READ_UNCOMMITTED);
229       // disable caching of data blocks
230       scan.setCacheBlocks(false);
231 
232       scanner = new ClientSideRegionScanner(conf, fs, tmpRootDir, htd, hri, scan, null);
233       if (context != null) {
234         this.context = context;
235         getCounter = TableRecordReaderImpl.retrieveGetCounterWithStringsParams(context);
236       }
237     }
238 
239     @Override
240     public boolean nextKeyValue() throws IOException, InterruptedException {
241       result = scanner.next();
242       if (result == null) {
243         //we are done
244         return false;
245       }
246 
247       if (this.row == null) {
248         this.row = new ImmutableBytesWritable();
249       }
250       this.row.set(result.getRow());
251 
252       ScanMetrics scanMetrics = scanner.getScanMetrics();
253       if (scanMetrics != null && context != null) {
254         TableRecordReaderImpl.updateCounters(scanMetrics, 0, getCounter, context);
255       }
256 
257       return true;
258     }
259 
260     @Override
261     public ImmutableBytesWritable getCurrentKey() throws IOException, InterruptedException {
262       return row;
263     }
264 
265     @Override
266     public Result getCurrentValue() throws IOException, InterruptedException {
267       return result;
268     }
269 
270     @Override
271     public float getProgress() throws IOException, InterruptedException {
272       return 0; // TODO: use total bytes to estimate
273     }
274 
275     @Override
276     public void close() throws IOException {
277       if (this.scanner != null) {
278         this.scanner.close();
279       }
280     }
281   }
282 
283   @Override
284   public RecordReader<ImmutableBytesWritable, Result> createRecordReader(
285       InputSplit split, TaskAttemptContext context) throws IOException {
286     return new TableSnapshotRegionRecordReader();
287   }
288 
289   @Override
290   public List<InputSplit> getSplits(JobContext job) throws IOException, InterruptedException {
291     Configuration conf = job.getConfiguration();
292     String snapshotName = getSnapshotName(conf);
293 
294     Path rootDir = new Path(conf.get(HConstants.HBASE_DIR));
295     FileSystem fs = rootDir.getFileSystem(conf);
296 
297     Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, rootDir);
298     SnapshotDescription snapshotDesc = SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDir);
299 
300     Set<String> snapshotRegionNames
301       = SnapshotReferenceUtil.getSnapshotRegionNames(fs, snapshotDir);
302     if (snapshotRegionNames == null) {
303       throw new IllegalArgumentException("Snapshot seems empty");
304     }
305 
306     // load table descriptor
307     HTableDescriptor htd = FSTableDescriptors.getTableDescriptorFromFs(fs,
308         snapshotDir);
309 
310     Scan scan = TableMapReduceUtil.convertStringToScan(conf
311       .get(TableInputFormat.SCAN));
312     Path tableDir = new Path(conf.get(TABLE_DIR_KEY));
313 
314     List<InputSplit> splits = new ArrayList<InputSplit>();
315     for (String regionName : snapshotRegionNames) {
316       // load region descriptor
317       Path regionDir = new Path(snapshotDir, regionName);
318       HRegionInfo hri = HRegionFileSystem.loadRegionInfoFileContent(fs,
319           regionDir);
320 
321       if (CellUtil.overlappingKeys(scan.getStartRow(), scan.getStopRow(),
322           hri.getStartKey(), hri.getEndKey())) {
323         // compute HDFS locations from snapshot files (which will get the locations for
324         // referred hfiles)
325         List<String> hosts = getBestLocations(conf,
326           HRegion.computeHDFSBlocksDistribution(conf, htd, hri, tableDir));
327 
328         int len = Math.min(3, hosts.size());
329         hosts = hosts.subList(0, len);
330         splits.add(new TableSnapshotRegionSplit(regionName, hosts));
331       }
332     }
333 
334     return splits;
335   }
336 
337   /**
338    * This computes the locations to be passed from the InputSplit. MR/Yarn schedulers does not take
339    * weights into account, thus will treat every location passed from the input split as equal. We
340    * do not want to blindly pass all the locations, since we are creating one split per region, and
341    * the region's blocks are all distributed throughout the cluster unless favorite node assignment
342    * is used. On the expected stable case, only one location will contain most of the blocks as local.
343    * On the other hand, in favored node assignment, 3 nodes will contain highly local blocks. Here
344    * we are doing a simple heuristic, where we will pass all hosts which have at least 80%
345    * (hbase.tablesnapshotinputformat.locality.cutoff.multiplier) as much block locality as the top
346    * host with the best locality.
347    */
348   @VisibleForTesting
349   List<String> getBestLocations(Configuration conf, HDFSBlocksDistribution blockDistribution) {
350     List<String> locations = new ArrayList<String>(3);
351 
352     HostAndWeight[] hostAndWeights = blockDistribution.getTopHostsWithWeights();
353 
354     if (hostAndWeights.length == 0) {
355       return locations;
356     }
357 
358     HostAndWeight topHost = hostAndWeights[0];
359     locations.add(topHost.getHost());
360 
361     // Heuristic: filter all hosts which have at least cutoffMultiplier % of block locality
362     double cutoffMultiplier
363       = conf.getFloat(LOCALITY_CUTOFF_MULTIPLIER, DEFAULT_LOCALITY_CUTOFF_MULTIPLIER);
364 
365     double filterWeight = topHost.getWeight() * cutoffMultiplier;
366 
367     for (int i = 1; i < hostAndWeights.length; i++) {
368       if (hostAndWeights[i].getWeight() >= filterWeight) {
369         locations.add(hostAndWeights[i].getHost());
370       } else {
371         break;
372       }
373     }
374 
375     return locations;
376   }
377 
378   /**
379    * Configures the job to use TableSnapshotInputFormat to read from a snapshot.
380    * @param job the job to configure
381    * @param snapshotName the name of the snapshot to read from
382    * @param restoreDir a temporary directory to restore the snapshot into. Current user should
383    * have write permissions to this directory, and this should not be a subdirectory of rootdir.
384    * After the job is finished, restoreDir can be deleted.
385    * @throws IOException if an error occurs
386    */
387   public static void setInput(Job job, String snapshotName, Path restoreDir) throws IOException {
388     Configuration conf = job.getConfiguration();
389     conf.set(SNAPSHOT_NAME_KEY, snapshotName);
390 
391     Path rootDir = new Path(conf.get(HConstants.HBASE_DIR));
392     FileSystem fs = rootDir.getFileSystem(conf);
393 
394     restoreDir = new Path(restoreDir, UUID.randomUUID().toString());
395 
396     // TODO: restore from record readers to parallelize.
397     RestoreSnapshotHelper.copySnapshotForScanner(conf, fs, rootDir, restoreDir, snapshotName);
398 
399     conf.set(TABLE_DIR_KEY, restoreDir.toString());
400   }
401 
402   private static String getSnapshotName(Configuration conf) {
403     String snapshotName = conf.get(SNAPSHOT_NAME_KEY);
404     if (snapshotName == null) {
405       throw new IllegalArgumentException("Snapshot name must be provided");
406     }
407     return snapshotName;
408   }
409 }