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.mapred;
19  
20  import org.apache.hadoop.hbase.classification.InterfaceAudience;
21  import org.apache.hadoop.hbase.classification.InterfaceStability;
22  import org.apache.hadoop.fs.Path;
23  import org.apache.hadoop.hbase.HRegionInfo;
24  import org.apache.hadoop.hbase.HTableDescriptor;
25  import org.apache.hadoop.hbase.client.Result;
26  import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
27  import org.apache.hadoop.hbase.mapreduce.TableSnapshotInputFormatImpl;
28  import org.apache.hadoop.mapred.InputFormat;
29  import org.apache.hadoop.mapred.InputSplit;
30  import org.apache.hadoop.mapred.JobConf;
31  import org.apache.hadoop.mapred.RecordReader;
32  import org.apache.hadoop.mapred.Reporter;
33  
34  import java.io.DataInput;
35  import java.io.DataOutput;
36  import java.io.IOException;
37  import java.util.List;
38  
39  /**
40   * TableSnapshotInputFormat allows a MapReduce job to run over a table snapshot. Further
41   * documentation available on {@link org.apache.hadoop.hbase.mapreduce.TableSnapshotInputFormat}.
42   *
43   * @see org.apache.hadoop.hbase.mapreduce.TableSnapshotInputFormat
44   */
45  @InterfaceAudience.Public
46  @InterfaceStability.Evolving
47  public class TableSnapshotInputFormat implements InputFormat<ImmutableBytesWritable, Result> {
48  
49    public static class TableSnapshotRegionSplit implements InputSplit {
50      private TableSnapshotInputFormatImpl.InputSplit delegate;
51  
52      // constructor for mapreduce framework / Writable
53      public TableSnapshotRegionSplit() {
54        this.delegate = new TableSnapshotInputFormatImpl.InputSplit();
55      }
56  
57      public TableSnapshotRegionSplit(TableSnapshotInputFormatImpl.InputSplit delegate) {
58        this.delegate = delegate;
59      }
60  
61      public TableSnapshotRegionSplit(HTableDescriptor htd, HRegionInfo regionInfo,
62          List<String> locations) {
63        this.delegate = new TableSnapshotInputFormatImpl.InputSplit(htd, regionInfo, locations);
64      }
65  
66      @Override
67      public long getLength() throws IOException {
68        return delegate.getLength();
69      }
70  
71      @Override
72      public String[] getLocations() throws IOException {
73        return delegate.getLocations();
74      }
75  
76      @Override
77      public void write(DataOutput out) throws IOException {
78        delegate.write(out);
79      }
80  
81      @Override
82      public void readFields(DataInput in) throws IOException {
83        delegate.readFields(in);
84      }
85    }
86  
87    static class TableSnapshotRecordReader
88        implements RecordReader<ImmutableBytesWritable, Result> {
89  
90      private TableSnapshotInputFormatImpl.RecordReader delegate;
91  
92      public TableSnapshotRecordReader(TableSnapshotRegionSplit split, JobConf job)
93          throws IOException {
94        delegate = new TableSnapshotInputFormatImpl.RecordReader();
95        delegate.initialize(split.delegate, job);
96      }
97  
98      @Override
99      public boolean next(ImmutableBytesWritable key, Result value) throws IOException {
100       if (!delegate.nextKeyValue()) {
101         return false;
102       }
103       ImmutableBytesWritable currentKey = delegate.getCurrentKey();
104       key.set(currentKey.get(), currentKey.getOffset(), currentKey.getLength());
105       value.copyFrom(delegate.getCurrentValue());
106       return true;
107     }
108 
109     @Override
110     public ImmutableBytesWritable createKey() {
111       return new ImmutableBytesWritable();
112     }
113 
114     @Override
115     public Result createValue() {
116       return new Result();
117     }
118 
119     @Override
120     public long getPos() throws IOException {
121       return delegate.getPos();
122     }
123 
124     @Override
125     public void close() throws IOException {
126       delegate.close();
127     }
128 
129     @Override
130     public float getProgress() throws IOException {
131       return delegate.getProgress();
132     }
133   }
134 
135   @Override
136   public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
137     List<TableSnapshotInputFormatImpl.InputSplit> splits =
138       TableSnapshotInputFormatImpl.getSplits(job);
139     InputSplit[] results = new InputSplit[splits.size()];
140     for (int i = 0; i < splits.size(); i++) {
141       results[i] = new TableSnapshotRegionSplit(splits.get(i));
142     }
143     return results;
144   }
145 
146   @Override
147   public RecordReader<ImmutableBytesWritable, Result>
148   getRecordReader(InputSplit split, JobConf job, Reporter reporter) throws IOException {
149     return new TableSnapshotRecordReader((TableSnapshotRegionSplit) split, job);
150   }
151 
152   /**
153    * Configures the job to use TableSnapshotInputFormat to read from a snapshot.
154    * @param job the job to configure
155    * @param snapshotName the name of the snapshot to read from
156    * @param restoreDir a temporary directory to restore the snapshot into. Current user should
157    * have write permissions to this directory, and this should not be a subdirectory of rootdir.
158    * After the job is finished, restoreDir can be deleted.
159    * @throws IOException if an error occurs
160    */
161   public static void setInput(JobConf job, String snapshotName, Path restoreDir) throws IOException {
162     TableSnapshotInputFormatImpl.setInput(job, snapshotName, restoreDir);
163   }
164 }