001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.mapreduce;
019
020import java.io.IOException;
021import java.util.Collection;
022import java.util.List;
023import java.util.Map;
024import org.apache.hadoop.conf.Configuration;
025import org.apache.hadoop.fs.Path;
026import org.apache.hadoop.hbase.client.Scan;
027import org.apache.hadoop.mapreduce.InputSplit;
028import org.apache.hadoop.mapreduce.JobContext;
029import org.apache.yetus.audience.InterfaceAudience;
030
031import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
032
033/**
034 * MultiTableSnapshotInputFormat generalizes {@link TableSnapshotInputFormat} allowing a MapReduce
035 * job to run over one or more table snapshots, with one or more scans configured for each.
036 * Internally, the input format delegates to {@link TableSnapshotInputFormat} and thus has the same
037 * performance advantages; see {@link TableSnapshotInputFormat} for more details. Usage is similar
038 * to TableSnapshotInputFormat, with the following exception: initMultiTableSnapshotMapperJob takes
039 * in a map from snapshot name to a collection of scans. For each snapshot in the map, each
040 * corresponding scan will be applied; the overall dataset for the job is defined by the
041 * concatenation of the regions and tables included in each snapshot/scan pair.
042 * {@link TableMapReduceUtil#initMultiTableSnapshotMapperJob (Map, Class, Class, Class, org.apache.hadoop.mapreduce.Job, boolean, Path)}
043 * can be used to configure the job.
044 *
045 * <pre>
046 * {@code
047 * Job job = new Job(conf);
048 * Map<String, Collection<Scan>> snapshotScans = ImmutableMap.of(
049 *    "snapshot1", ImmutableList.of(new Scan(Bytes.toBytes("a"), Bytes.toBytes("b"))),
050 *    "snapshot2", ImmutableList.of(new Scan(Bytes.toBytes("1"), Bytes.toBytes("2")))
051 * );
052 * Path restoreDir = new Path("/tmp/snapshot_restore_dir")
053 * TableMapReduceUtil.initTableSnapshotMapperJob(
054 *     snapshotScans, MyTableMapper.class, MyMapKeyOutput.class,
055 *      MyMapOutputValueWritable.class, job, true, restoreDir);
056 * }
057 * </pre>
058 *
059 * Internally, this input format restores each snapshot into a subdirectory of the given tmp
060 * directory. Input splits and record readers are created as described in
061 * {@link org.apache.hadoop.hbase.mapreduce.TableSnapshotInputFormat} (one per region). See
062 * {@link TableSnapshotInputFormat} for more notes on permissioning; the same caveats apply here.
063 * @see TableSnapshotInputFormat
064 * @see org.apache.hadoop.hbase.client.TableSnapshotScanner
065 */
066@InterfaceAudience.Public
067public class MultiTableSnapshotInputFormat extends TableSnapshotInputFormat {
068
069  private final MultiTableSnapshotInputFormatImpl delegate;
070
071  public MultiTableSnapshotInputFormat() {
072    this.delegate = new MultiTableSnapshotInputFormatImpl();
073  }
074
075  @Override
076  public List<InputSplit> getSplits(JobContext jobContext)
077    throws IOException, InterruptedException {
078    List<TableSnapshotInputFormatImpl.InputSplit> splits =
079      delegate.getSplits(jobContext.getConfiguration());
080    List<InputSplit> rtn = Lists.newArrayListWithCapacity(splits.size());
081
082    for (TableSnapshotInputFormatImpl.InputSplit split : splits) {
083      rtn.add(new TableSnapshotInputFormat.TableSnapshotRegionSplit(split));
084    }
085
086    return rtn;
087  }
088
089  public static void setInput(Configuration configuration,
090    Map<String, Collection<Scan>> snapshotScans, Path tmpRestoreDir) throws IOException {
091    new MultiTableSnapshotInputFormatImpl().setInput(configuration, snapshotScans, tmpRestoreDir);
092  }
093}