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 com.google.common.collect.Lists; 22 import org.apache.hadoop.conf.Configuration; 23 import org.apache.hadoop.fs.Path; 24 import org.apache.hadoop.hbase.classification.InterfaceAudience; 25 import org.apache.hadoop.hbase.classification.InterfaceStability; 26 import org.apache.hadoop.hbase.client.Scan; 27 import org.apache.hadoop.mapreduce.InputSplit; 28 import org.apache.hadoop.mapreduce.JobContext; 29 30 import java.io.IOException; 31 import java.util.Collection; 32 import java.util.List; 33 import java.util.Map; 34 35 /** 36 * MultiTableSnapshotInputFormat generalizes 37 * {@link org.apache.hadoop.hbase.mapreduce.TableSnapshotInputFormat} 38 * allowing a MapReduce job to run over one or more table snapshots, with one or more scans 39 * configured for each. 40 * Internally, the input format delegates to 41 * {@link org.apache.hadoop.hbase.mapreduce.TableSnapshotInputFormat} 42 * and thus has the same performance advantages; 43 * see {@link org.apache.hadoop.hbase.mapreduce.TableSnapshotInputFormat} for 44 * more details. 45 * Usage is similar to TableSnapshotInputFormat, with the following exception: 46 * initMultiTableSnapshotMapperJob takes in a map 47 * from snapshot name to a collection of scans. For each snapshot in the map, each corresponding 48 * scan will be applied; 49 * the overall dataset for the job is defined by the concatenation of the regions and tables 50 * included in each snapshot/scan 51 * pair. 52 * {@link org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil#initMultiTableSnapshotMapperJob 53 * (java.util.Map, Class, Class, Class, org.apache.hadoop.mapreduce.Job, boolean, org.apache 54 * .hadoop.fs.Path)} 55 * can be used to configure the job. 56 * <pre>{@code 57 * Job job = new Job(conf); 58 * Map<String, Collection<Scan>> snapshotScans = ImmutableMap.of( 59 * "snapshot1", ImmutableList.of(new Scan(Bytes.toBytes("a"), Bytes.toBytes("b"))), 60 * "snapshot2", ImmutableList.of(new Scan(Bytes.toBytes("1"), Bytes.toBytes("2"))) 61 * ); 62 * Path restoreDir = new Path("/tmp/snapshot_restore_dir") 63 * TableMapReduceUtil.initTableSnapshotMapperJob( 64 * snapshotScans, MyTableMapper.class, MyMapKeyOutput.class, 65 * MyMapOutputValueWritable.class, job, true, restoreDir); 66 * } 67 * </pre> 68 * Internally, this input format restores each snapshot into a subdirectory of the given tmp 69 * directory. Input splits and 70 * record readers are created as described in {@link org.apache.hadoop.hbase.mapreduce 71 * .TableSnapshotInputFormat} 72 * (one per region). 73 * See {@link org.apache.hadoop.hbase.mapreduce.TableSnapshotInputFormat} for more notes on 74 * permissioning; the 75 * same caveats apply here. 76 * 77 * @see org.apache.hadoop.hbase.mapreduce.TableSnapshotInputFormat 78 * @see org.apache.hadoop.hbase.client.TableSnapshotScanner 79 */ 80 @InterfaceAudience.Public 81 @InterfaceStability.Evolving 82 public class MultiTableSnapshotInputFormat extends TableSnapshotInputFormat { 83 84 private final MultiTableSnapshotInputFormatImpl delegate; 85 86 public MultiTableSnapshotInputFormat() { 87 this.delegate = new MultiTableSnapshotInputFormatImpl(); 88 } 89 90 @Override 91 public List<InputSplit> getSplits(JobContext jobContext) 92 throws IOException, InterruptedException { 93 List<TableSnapshotInputFormatImpl.InputSplit> splits = 94 delegate.getSplits(jobContext.getConfiguration()); 95 List<InputSplit> rtn = Lists.newArrayListWithCapacity(splits.size()); 96 97 for (TableSnapshotInputFormatImpl.InputSplit split : splits) { 98 rtn.add(new TableSnapshotInputFormat.TableSnapshotRegionSplit(split)); 99 } 100 101 return rtn; 102 } 103 104 public static void setInput(Configuration configuration, 105 Map<String, Collection<Scan>> snapshotScans, Path tmpRestoreDir) throws IOException { 106 new MultiTableSnapshotInputFormatImpl().setInput(configuration, snapshotScans, tmpRestoreDir); 107 } 108 }