1   /**
2    * Copyright 2007 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.mapreduce;
21  
22  import java.io.File;
23  import java.io.IOException;
24  import java.util.Map;
25  import java.util.NavigableMap;
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.FileUtil;
31  import org.apache.hadoop.fs.Path;
32  import org.apache.hadoop.hbase.HBaseTestingUtility;
33  import org.apache.hadoop.hbase.client.HTable;
34  import org.apache.hadoop.hbase.client.Result;
35  import org.apache.hadoop.hbase.client.Scan;
36  import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
37  import org.apache.hadoop.hbase.util.Bytes;
38  import org.apache.hadoop.io.NullWritable;
39  import org.apache.hadoop.mapreduce.Job;
40  import org.apache.hadoop.mapreduce.Reducer;
41  import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
42  import org.junit.After;
43  import org.junit.AfterClass;
44  import org.junit.Before;
45  import org.junit.BeforeClass;
46  import org.junit.Test;
47  
48  import static org.junit.Assert.assertEquals;
49  import static org.junit.Assert.assertTrue;
50  
51  /**
52   * Tests various scan start and stop row scenarios. This is set in a scan and
53   * tested in a MapReduce job to see if that is handed over and done properly
54   * too.
55   */
56  public class TestTableInputFormatScan {
57  
58    static final Log LOG = LogFactory.getLog(TestTableInputFormatScan.class);
59    static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
60  
61    static final byte[] TABLE_NAME = Bytes.toBytes("scantest");
62    static final byte[] INPUT_FAMILY = Bytes.toBytes("contents");
63    static final String KEY_STARTROW = "startRow";
64    static final String KEY_LASTROW = "stpRow";
65  
66    private static HTable table = null;
67  
68    @BeforeClass
69    public static void setUpBeforeClass() throws Exception {
70      // switch TIF to log at DEBUG level
71      TEST_UTIL.enableDebug(TableInputFormat.class);
72      TEST_UTIL.enableDebug(TableInputFormatBase.class);
73      // start mini hbase cluster
74      TEST_UTIL.startMiniCluster(3);
75      // create and fill table
76      table = TEST_UTIL.createTable(TABLE_NAME, INPUT_FAMILY);
77      TEST_UTIL.createMultiRegions(table, INPUT_FAMILY);
78      TEST_UTIL.loadTable(table, INPUT_FAMILY);
79      // start MR cluster
80      TEST_UTIL.startMiniMapReduceCluster();
81    }
82  
83    @AfterClass
84    public static void tearDownAfterClass() throws Exception {
85      TEST_UTIL.shutdownMiniMapReduceCluster();
86      TEST_UTIL.shutdownMiniCluster();
87    }
88  
89    /**
90     * Pass the key and value to reduce.
91     */
92    public static class ScanMapper
93    extends TableMapper<ImmutableBytesWritable, ImmutableBytesWritable> {
94  
95      /**
96       * Pass the key and value to reduce.
97       *
98       * @param key  The key, here "aaa", "aab" etc.
99       * @param value  The value is the same as the key.
100      * @param context  The task context.
101      * @throws IOException When reading the rows fails.
102      */
103     @Override
104     public void map(ImmutableBytesWritable key, Result value,
105       Context context)
106     throws IOException, InterruptedException {
107       if (value.size() != 1) {
108         throw new IOException("There should only be one input column");
109       }
110       Map<byte[], NavigableMap<byte[], NavigableMap<Long, byte[]>>>
111         cf = value.getMap();
112       if(!cf.containsKey(INPUT_FAMILY)) {
113         throw new IOException("Wrong input columns. Missing: '" +
114           Bytes.toString(INPUT_FAMILY) + "'.");
115       }
116       String val = Bytes.toStringBinary(value.getValue(INPUT_FAMILY, null));
117       LOG.info("map: key -> " + Bytes.toStringBinary(key.get()) +
118         ", value -> " + val);
119       context.write(key, key);
120     }
121 
122   }
123 
124   /**
125    * Checks the last and first key seen against the scanner boundaries.
126    */
127   public static class ScanReducer
128   extends Reducer<ImmutableBytesWritable, ImmutableBytesWritable,
129                   NullWritable, NullWritable> {
130 
131     private String first = null;
132     private String last = null;
133 
134     protected void reduce(ImmutableBytesWritable key,
135         Iterable<ImmutableBytesWritable> values, Context context)
136     throws IOException ,InterruptedException {
137       int count = 0;
138       for (ImmutableBytesWritable value : values) {
139         String val = Bytes.toStringBinary(value.get());
140         LOG.info("reduce: key[" + count + "] -> " +
141           Bytes.toStringBinary(key.get()) + ", value -> " + val);
142         if (first == null) first = val;
143         last = val;
144         count++;
145       }
146     }
147 
148     protected void cleanup(Context context)
149     throws IOException, InterruptedException {
150       Configuration c = context.getConfiguration();
151       String startRow = c.get(KEY_STARTROW);
152       String lastRow = c.get(KEY_LASTROW);
153       LOG.info("cleanup: first -> \"" + first + "\", start row -> \"" + startRow + "\"");
154       LOG.info("cleanup: last -> \"" + last + "\", last row -> \"" + lastRow + "\"");
155       if (startRow != null && startRow.length() > 0) {
156         assertEquals(startRow, first);
157       }
158       if (lastRow != null && lastRow.length() > 0) {
159         assertEquals(lastRow, last);
160       }
161     }
162 
163   }
164 
165   /**
166    * Tests a MR scan using specific start and stop rows.
167    *
168    * @throws IOException
169    * @throws ClassNotFoundException
170    * @throws InterruptedException
171    */
172   @Test
173   public void testScanEmptyToEmpty()
174   throws IOException, InterruptedException, ClassNotFoundException {
175     testScan(null, null, null);
176   }
177 
178   /**
179    * Tests a MR scan using specific start and stop rows.
180    *
181    * @throws IOException
182    * @throws ClassNotFoundException
183    * @throws InterruptedException
184    */
185   @Test
186   public void testScanEmptyToAPP()
187   throws IOException, InterruptedException, ClassNotFoundException {
188     testScan(null, "app", "apo");
189   }
190 
191   /**
192    * Tests a MR scan using specific start and stop rows.
193    *
194    * @throws IOException
195    * @throws ClassNotFoundException
196    * @throws InterruptedException
197    */
198   @Test
199   public void testScanEmptyToBBA()
200   throws IOException, InterruptedException, ClassNotFoundException {
201     testScan(null, "bba", "baz");
202   }
203 
204   /**
205    * Tests a MR scan using specific start and stop rows.
206    *
207    * @throws IOException
208    * @throws ClassNotFoundException
209    * @throws InterruptedException
210    */
211   @Test
212   public void testScanEmptyToBBB()
213   throws IOException, InterruptedException, ClassNotFoundException {
214     testScan(null, "bbb", "bba");
215   }
216 
217   /**
218    * Tests a MR scan using specific start and stop rows.
219    *
220    * @throws IOException
221    * @throws ClassNotFoundException
222    * @throws InterruptedException
223    */
224   @Test
225   public void testScanEmptyToOPP()
226   throws IOException, InterruptedException, ClassNotFoundException {
227     testScan(null, "opp", "opo");
228   }
229 
230   /**
231    * Tests a MR scan using specific start and stop rows.
232    *
233    * @throws IOException
234    * @throws ClassNotFoundException
235    * @throws InterruptedException
236    */
237   @Test
238   public void testScanOBBToOPP()
239   throws IOException, InterruptedException, ClassNotFoundException {
240     testScan("obb", "opp", "opo");
241   }
242 
243   /**
244    * Tests a MR scan using specific start and stop rows.
245    *
246    * @throws IOException
247    * @throws ClassNotFoundException
248    * @throws InterruptedException
249    */
250   @Test
251   public void testScanOBBToQPP()
252   throws IOException, InterruptedException, ClassNotFoundException {
253     testScan("obb", "qpp", "qpo");
254   }
255 
256   /**
257    * Tests a MR scan using specific start and stop rows.
258    *
259    * @throws IOException
260    * @throws ClassNotFoundException
261    * @throws InterruptedException
262    */
263   @Test
264   public void testScanOPPToEmpty()
265   throws IOException, InterruptedException, ClassNotFoundException {
266     testScan("opp", null, "zzz");
267   }
268 
269   /**
270    * Tests a MR scan using specific start and stop rows.
271    *
272    * @throws IOException
273    * @throws ClassNotFoundException
274    * @throws InterruptedException
275    */
276   @Test
277   public void testScanYYXToEmpty()
278   throws IOException, InterruptedException, ClassNotFoundException {
279     testScan("yyx", null, "zzz");
280   }
281 
282   /**
283    * Tests a MR scan using specific start and stop rows.
284    *
285    * @throws IOException
286    * @throws ClassNotFoundException
287    * @throws InterruptedException
288    */
289   @Test
290   public void testScanYYYToEmpty()
291   throws IOException, InterruptedException, ClassNotFoundException {
292     testScan("yyy", null, "zzz");
293   }
294 
295   /**
296    * Tests a MR scan using specific start and stop rows.
297    *
298    * @throws IOException
299    * @throws ClassNotFoundException
300    * @throws InterruptedException
301    */
302   @Test
303   public void testScanYZYToEmpty()
304   throws IOException, InterruptedException, ClassNotFoundException {
305     testScan("yzy", null, "zzz");
306   }
307 
308   /**
309    * Tests a MR scan using specific start and stop rows.
310    *
311    * @throws IOException
312    * @throws ClassNotFoundException
313    * @throws InterruptedException
314    */
315   private void testScan(String start, String stop, String last)
316   throws IOException, InterruptedException, ClassNotFoundException {
317     String jobName = "Scan" + (start != null ? start.toUpperCase() : "Empty") +
318     "To" + (stop != null ? stop.toUpperCase() : "Empty");
319     LOG.info("Before map/reduce startup - job " + jobName);
320     Configuration c = new Configuration(TEST_UTIL.getConfiguration());
321     Scan scan = new Scan();
322     scan.addFamily(INPUT_FAMILY);
323     if (start != null) {
324       scan.setStartRow(Bytes.toBytes(start));
325     }
326     c.set(KEY_STARTROW, start != null ? start : "");
327     if (stop != null) {
328       scan.setStopRow(Bytes.toBytes(stop));
329     }
330     c.set(KEY_LASTROW, last != null ? last : "");
331     LOG.info("scan before: " + scan);
332     Job job = new Job(c, jobName);
333     TableMapReduceUtil.initTableMapperJob(
334       Bytes.toString(TABLE_NAME), scan, ScanMapper.class,
335       ImmutableBytesWritable.class, ImmutableBytesWritable.class, job);
336     job.setReducerClass(ScanReducer.class);
337     job.setNumReduceTasks(1); // one to get final "first" and "last" key
338     FileOutputFormat.setOutputPath(job, new Path(job.getJobName()));
339     LOG.info("Started " + job.getJobName());
340     job.waitForCompletion(true);
341     assertTrue(job.isComplete());
342     LOG.info("After map/reduce completion - job " + jobName);
343   }
344 }