1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
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 import java.util.concurrent.atomic.AtomicLong;
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.FileStatus;
31 import org.apache.hadoop.fs.FileSystem;
32 import org.apache.hadoop.fs.Path;
33 import org.apache.hadoop.hbase.Cell;
34 import org.apache.hadoop.hbase.TableName;
35 import org.apache.hadoop.hbase.HBaseTestingUtility;
36 import org.apache.hadoop.hbase.HColumnDescriptor;
37 import org.apache.hadoop.hbase.HConstants;
38 import org.apache.hadoop.hbase.HRegionInfo;
39 import org.apache.hadoop.hbase.HTableDescriptor;
40 import org.apache.hadoop.hbase.KeyValue;
41 import org.apache.hadoop.hbase.testclassification.MediumTests;
42 import org.apache.hadoop.hbase.mapreduce.WALInputFormat.WALKeyRecordReader;
43 import org.apache.hadoop.hbase.mapreduce.WALInputFormat.WALRecordReader;
44 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
45 import org.apache.hadoop.hbase.wal.WAL;
46 import org.apache.hadoop.hbase.wal.WALFactory;
47 import org.apache.hadoop.hbase.wal.WALKey;
48 import org.apache.hadoop.hbase.util.Bytes;
49 import org.apache.hadoop.mapreduce.InputSplit;
50 import org.apache.hadoop.mapreduce.MapReduceTestUtil;
51 import org.junit.AfterClass;
52 import org.junit.Before;
53 import org.junit.BeforeClass;
54 import org.junit.Test;
55 import org.junit.experimental.categories.Category;
56
57
58
59
60 @Category(MediumTests.class)
61 public class TestWALRecordReader {
62 private final Log LOG = LogFactory.getLog(getClass());
63 private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
64 private static Configuration conf;
65 private static FileSystem fs;
66 private static Path hbaseDir;
67
68 static final TableName tableName = TableName.valueOf(getName());
69 private static final byte [] rowName = tableName.getName();
70
71 static final HRegionInfo info = new HRegionInfo(tableName,
72 Bytes.toBytes(""), Bytes.toBytes(""), false);
73 private static final byte [] family = Bytes.toBytes("column");
74 private static final byte [] value = Bytes.toBytes("value");
75 private static HTableDescriptor htd;
76 private static Path logDir;
77
78 private static String getName() {
79 return "TestWALRecordReader";
80 }
81
82 @Before
83 public void setUp() throws Exception {
84 FileStatus[] entries = fs.listStatus(hbaseDir);
85 for (FileStatus dir : entries) {
86 fs.delete(dir.getPath(), true);
87 }
88
89 }
90 @BeforeClass
91 public static void setUpBeforeClass() throws Exception {
92
93 conf = TEST_UTIL.getConfiguration();
94 conf.setInt("dfs.blocksize", 1024 * 1024);
95 conf.setInt("dfs.replication", 1);
96 TEST_UTIL.startMiniDFSCluster(1);
97
98 conf = TEST_UTIL.getConfiguration();
99 fs = TEST_UTIL.getDFSCluster().getFileSystem();
100
101 hbaseDir = TEST_UTIL.createRootDir();
102
103 logDir = new Path(hbaseDir, HConstants.HREGION_LOGDIR_NAME);
104
105 htd = new HTableDescriptor(tableName);
106 htd.addFamily(new HColumnDescriptor(family));
107 }
108
109 @AfterClass
110 public static void tearDownAfterClass() throws Exception {
111 TEST_UTIL.shutdownMiniCluster();
112 }
113
114
115
116
117
118 @Test
119 public void testPartialRead() throws Exception {
120 final WALFactory walfactory = new WALFactory(conf, null, getName());
121 WAL log = walfactory.getWAL(info.getEncodedNameAsBytes());
122
123
124 long ts = System.currentTimeMillis();
125 WALEdit edit = new WALEdit();
126 final AtomicLong sequenceId = new AtomicLong(0);
127 edit.add(new KeyValue(rowName, family, Bytes.toBytes("1"), ts, value));
128 log.append(htd, info, getWalKey(ts), edit, sequenceId, true, null);
129 edit = new WALEdit();
130 edit.add(new KeyValue(rowName, family, Bytes.toBytes("2"), ts+1, value));
131 log.append(htd, info, getWalKey(ts+1), edit, sequenceId,
132 true, null);
133 log.sync();
134 LOG.info("Before 1st WAL roll " + log.toString());
135 log.rollWriter();
136 LOG.info("Past 1st WAL roll " + log.toString());
137
138 Thread.sleep(1);
139 long ts1 = System.currentTimeMillis();
140
141 edit = new WALEdit();
142 edit.add(new KeyValue(rowName, family, Bytes.toBytes("3"), ts1+1, value));
143 log.append(htd, info, getWalKey(ts1+1), edit, sequenceId,
144 true, null);
145 edit = new WALEdit();
146 edit.add(new KeyValue(rowName, family, Bytes.toBytes("4"), ts1+2, value));
147 log.append(htd, info, getWalKey(ts1+2), edit, sequenceId,
148 true, null);
149 log.sync();
150 log.shutdown();
151 walfactory.shutdown();
152 LOG.info("Closed WAL " + log.toString());
153
154
155 WALInputFormat input = new WALInputFormat();
156 Configuration jobConf = new Configuration(conf);
157 jobConf.set("mapreduce.input.fileinputformat.inputdir", logDir.toString());
158 jobConf.setLong(WALInputFormat.END_TIME_KEY, ts);
159
160
161 List<InputSplit> splits = input.getSplits(MapreduceTestingShim.createJobContext(jobConf));
162
163 assertEquals(1, splits.size());
164 testSplit(splits.get(0), Bytes.toBytes("1"));
165
166 jobConf.setLong(WALInputFormat.START_TIME_KEY, ts+1);
167 jobConf.setLong(WALInputFormat.END_TIME_KEY, ts1+1);
168 splits = input.getSplits(MapreduceTestingShim.createJobContext(jobConf));
169
170 assertEquals(2, splits.size());
171
172 testSplit(splits.get(0), Bytes.toBytes("2"));
173
174 testSplit(splits.get(1), Bytes.toBytes("3"));
175 }
176
177
178
179
180
181 @Test
182 public void testWALRecordReader() throws Exception {
183 final WALFactory walfactory = new WALFactory(conf, null, getName());
184 WAL log = walfactory.getWAL(info.getEncodedNameAsBytes());
185 byte [] value = Bytes.toBytes("value");
186 final AtomicLong sequenceId = new AtomicLong(0);
187 WALEdit edit = new WALEdit();
188 edit.add(new KeyValue(rowName, family, Bytes.toBytes("1"),
189 System.currentTimeMillis(), value));
190 long txid = log.append(htd, info, getWalKey(System.currentTimeMillis()), edit, sequenceId, true,
191 null);
192 log.sync(txid);
193
194 Thread.sleep(1);
195 long secondTs = System.currentTimeMillis();
196 log.rollWriter();
197
198 edit = new WALEdit();
199 edit.add(new KeyValue(rowName, family, Bytes.toBytes("2"),
200 System.currentTimeMillis(), value));
201 txid = log.append(htd, info, getWalKey(System.currentTimeMillis()), edit, sequenceId, true,
202 null);
203 log.sync(txid);
204 log.shutdown();
205 walfactory.shutdown();
206 long thirdTs = System.currentTimeMillis();
207
208
209 WALInputFormat input = new WALInputFormat();
210 Configuration jobConf = new Configuration(conf);
211 jobConf.set("mapreduce.input.fileinputformat.inputdir", logDir.toString());
212
213
214 List<InputSplit> splits = input.getSplits(MapreduceTestingShim.createJobContext(jobConf));
215 assertEquals(2, splits.size());
216
217
218 testSplit(splits.get(0), Bytes.toBytes("1"));
219
220 testSplit(splits.get(1), Bytes.toBytes("2"));
221
222
223
224
225 jobConf.setLong(WALInputFormat.END_TIME_KEY, secondTs-1);
226 splits = input.getSplits(MapreduceTestingShim.createJobContext(jobConf));
227 assertEquals(1, splits.size());
228 testSplit(splits.get(0), Bytes.toBytes("1"));
229
230
231 jobConf.setLong(WALInputFormat.END_TIME_KEY, Long.MAX_VALUE);
232 jobConf.setLong(WALInputFormat.START_TIME_KEY, thirdTs);
233 splits = input.getSplits(MapreduceTestingShim.createJobContext(jobConf));
234
235 assertEquals(2, splits.size());
236
237 testSplit(splits.get(0));
238 testSplit(splits.get(1));
239 }
240
241 protected WALKey getWalKey(final long sequenceid) {
242 return new WALKey(info.getEncodedNameAsBytes(), tableName, sequenceid);
243 }
244
245 protected WALRecordReader getReader() {
246 return new WALKeyRecordReader();
247 }
248
249
250
251
252 private void testSplit(InputSplit split, byte[]... columns) throws Exception {
253 final WALRecordReader reader = getReader();
254 reader.initialize(split, MapReduceTestUtil.createDummyMapTaskAttemptContext(conf));
255
256 for (byte[] column : columns) {
257 assertTrue(reader.nextKeyValue());
258 Cell cell = reader.getCurrentValue().getCells().get(0);
259 if (!Bytes.equals(column, cell.getQualifier())) {
260 assertTrue("expected [" + Bytes.toString(column) + "], actual ["
261 + Bytes.toString(cell.getQualifier()) + "]", false);
262 }
263 }
264 assertFalse(reader.nextKeyValue());
265 reader.close();
266 }
267
268 }