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.mapreduce;
19  
20  import static org.junit.Assert.assertEquals;
21  import static org.junit.Assert.assertFalse;
22  import static org.junit.Assert.assertTrue;
23  
24  import java.util.List;
25  
26  import org.apache.hadoop.conf.Configuration;
27  import org.apache.hadoop.fs.FileStatus;
28  import org.apache.hadoop.fs.FileSystem;
29  import org.apache.hadoop.fs.Path;
30  import org.apache.hadoop.hbase.HBaseTestingUtility;
31  import org.apache.hadoop.hbase.HColumnDescriptor;
32  import org.apache.hadoop.hbase.HConstants;
33  import org.apache.hadoop.hbase.HRegionInfo;
34  import org.apache.hadoop.hbase.HTableDescriptor;
35  import org.apache.hadoop.hbase.KeyValue;
36  import org.apache.hadoop.hbase.MediumTests;
37  import org.apache.hadoop.hbase.mapreduce.HLogInputFormat.HLogRecordReader;
38  import org.apache.hadoop.hbase.regionserver.wal.HLog;
39  import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
40  import org.apache.hadoop.hbase.util.Bytes;
41  import org.apache.hadoop.mapreduce.InputSplit;
42  import org.apache.hadoop.mapreduce.JobContext;
43  import org.apache.hadoop.mapreduce.MapReduceTestUtil;
44  import org.junit.AfterClass;
45  import org.junit.Before;
46  import org.junit.BeforeClass;
47  import org.junit.Test;
48  import org.junit.experimental.categories.Category;
49  
50  /**
51   * JUnit tests for the HLogRecordReader
52   */
53  @Category(MediumTests.class)
54  public class TestHLogRecordReader {
55    private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
56    private static Configuration conf;
57    private static FileSystem fs;
58    private static Path hbaseDir;
59    private static final byte [] tableName = Bytes.toBytes(getName());
60    private static final byte [] rowName = tableName;
61    private static final HRegionInfo info = new HRegionInfo(tableName,
62        Bytes.toBytes(""), Bytes.toBytes(""), false);
63    private static final byte [] family = Bytes.toBytes("column");
64    private static final byte [] value = Bytes.toBytes("value");
65    private static HTableDescriptor htd;
66    private static Path logDir;
67    private static Path oldLogDir;
68  
69    private static String getName() {
70      return "TestHLogRecordReader";
71    }
72  
73    @Before
74    public void setUp() throws Exception {
75      FileStatus[] entries = fs.listStatus(hbaseDir);
76      for (FileStatus dir : entries) {
77        fs.delete(dir.getPath(), true);
78      }
79  
80    }
81    @BeforeClass
82    public static void setUpBeforeClass() throws Exception {
83      // Make block sizes small.
84      conf = TEST_UTIL.getConfiguration();
85      conf.setInt("dfs.blocksize", 1024 * 1024);
86      conf.setInt("dfs.replication", 1);
87      TEST_UTIL.startMiniDFSCluster(1);
88  
89      conf = TEST_UTIL.getConfiguration();
90      fs = TEST_UTIL.getDFSCluster().getFileSystem();
91  
92      hbaseDir = TEST_UTIL.createRootDir();
93      logDir = new Path(hbaseDir, HConstants.HREGION_LOGDIR_NAME);
94      oldLogDir = new Path(hbaseDir, HConstants.HREGION_OLDLOGDIR_NAME);
95      htd = new HTableDescriptor(tableName);
96      htd.addFamily(new HColumnDescriptor(family));
97    }
98  
99    @AfterClass
100   public static void tearDownAfterClass() throws Exception {
101     TEST_UTIL.shutdownMiniCluster();
102   }
103 
104   /**
105    * Test partial reads from the log based on passed time range
106    * @throws Exception
107    */
108   @Test
109   public void testPartialRead() throws Exception {
110     HLog log = new HLog(fs, logDir, oldLogDir, conf);
111     long ts = System.currentTimeMillis();
112     WALEdit edit = new WALEdit();
113     edit.add(new KeyValue(rowName, family, Bytes.toBytes("1"),
114         ts, value));
115     log.append(info, tableName, edit,
116       ts, htd);
117     edit = new WALEdit();
118     edit.add(new KeyValue(rowName, family, Bytes.toBytes("2"),
119         ts+1, value));
120     log.append(info, tableName, edit,
121         ts+1, htd);
122     log.rollWriter();
123 
124     Thread.sleep(1);
125     long ts1 = System.currentTimeMillis();
126 
127     edit = new WALEdit();
128     edit.add(new KeyValue(rowName, family, Bytes.toBytes("3"),
129         ts1+1, value));
130     log.append(info, tableName, edit,
131         ts1+1, htd);
132     edit = new WALEdit();
133     edit.add(new KeyValue(rowName, family, Bytes.toBytes("4"),
134         ts1+2, value));
135     log.append(info, tableName, edit,
136         ts1+2, htd);
137     log.close();
138 
139     HLogInputFormat input = new HLogInputFormat();
140     Configuration jobConf = new Configuration(conf);
141     jobConf.set("mapred.input.dir", logDir.toString());
142     jobConf.setLong(HLogInputFormat.END_TIME_KEY, ts);
143 
144     // only 1st file is considered, and only its 1st entry is used
145     List<InputSplit> splits = input.getSplits(MapreduceTestingShim.createJobContext(jobConf));
146     assertEquals(1, splits.size());
147     testSplit(splits.get(0), Bytes.toBytes("1"));
148 
149     jobConf.setLong(HLogInputFormat.START_TIME_KEY, ts+1);
150     jobConf.setLong(HLogInputFormat.END_TIME_KEY, ts1+1);
151     splits = input.getSplits(MapreduceTestingShim.createJobContext(jobConf));
152     // both files need to be considered
153     assertEquals(2, splits.size());
154     // only the 2nd entry from the 1st file is used
155     testSplit(splits.get(0), Bytes.toBytes("2"));
156     // only the 1nd entry from the 2nd file is used
157     testSplit(splits.get(1), Bytes.toBytes("3"));
158   }
159 
160   /**
161    * Test basic functionality
162    * @throws Exception
163    */
164   @Test
165   public void testHLogRecordReader() throws Exception {
166     HLog log = new HLog(fs, logDir, oldLogDir, conf);
167     byte [] value = Bytes.toBytes("value");
168     WALEdit edit = new WALEdit();
169     edit.add(new KeyValue(rowName, family, Bytes.toBytes("1"),
170         System.currentTimeMillis(), value));
171     log.append(info, tableName, edit,
172       System.currentTimeMillis(), htd);
173 
174     Thread.sleep(1); // make sure 2nd log gets a later timestamp
175     long secondTs = System.currentTimeMillis();
176     log.rollWriter();
177 
178     edit = new WALEdit();
179     edit.add(new KeyValue(rowName, family, Bytes.toBytes("2"),
180         System.currentTimeMillis(), value));
181     log.append(info, tableName, edit,
182       System.currentTimeMillis(), htd);
183     log.close();
184     long thirdTs = System.currentTimeMillis();
185 
186     // should have 2 log files now
187     HLogInputFormat input = new HLogInputFormat();
188     Configuration jobConf = new Configuration(conf);
189     jobConf.set("mapred.input.dir", logDir.toString());
190 
191     // make sure both logs are found
192     List<InputSplit> splits = input.getSplits(MapreduceTestingShim.createJobContext(jobConf));
193     assertEquals(2, splits.size());
194 
195     // should return exactly one KV
196     testSplit(splits.get(0), Bytes.toBytes("1"));
197     // same for the 2nd split
198     testSplit(splits.get(1), Bytes.toBytes("2"));
199 
200     // now test basic time ranges:
201 
202     // set an endtime, the 2nd log file can be ignored completely.
203     jobConf.setLong(HLogInputFormat.END_TIME_KEY, secondTs-1);
204     splits = input.getSplits(MapreduceTestingShim.createJobContext(jobConf));
205     assertEquals(1, splits.size());
206     testSplit(splits.get(0), Bytes.toBytes("1"));
207 
208     // now set a start time
209     jobConf.setLong(HLogInputFormat.END_TIME_KEY, Long.MAX_VALUE);
210     jobConf.setLong(HLogInputFormat.START_TIME_KEY, thirdTs);
211     splits = input.getSplits(MapreduceTestingShim.createJobContext(jobConf));
212     // both logs need to be considered
213     assertEquals(2, splits.size());
214     // but both readers skip all edits
215     testSplit(splits.get(0));
216     testSplit(splits.get(1));
217   }
218 
219   /**
220    * Create a new reader from the split, and match the edits against the passed columns.
221    */
222   private void testSplit(InputSplit split, byte[]... columns) throws Exception {
223     HLogRecordReader reader = new HLogRecordReader();
224     reader.initialize(split, MapReduceTestUtil.createDummyMapTaskAttemptContext(conf));
225 
226     for (byte[] column : columns) {
227       assertTrue(reader.nextKeyValue());
228       assertTrue(Bytes
229           .equals(column, reader.getCurrentValue().getKeyValues().get(0).getQualifier()));
230     }
231     assertFalse(reader.nextKeyValue());
232     reader.close();
233   }
234 
235   @org.junit.Rule
236   public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
237     new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
238 }