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.regionserver.wal;
019
020import static org.junit.Assert.assertEquals;
021import static org.junit.Assert.assertNotNull;
022import static org.junit.Assert.assertTrue;
023import static org.junit.Assert.fail;
024import static org.mockito.ArgumentMatchers.any;
025import static org.mockito.ArgumentMatchers.anyInt;
026import static org.mockito.ArgumentMatchers.eq;
027import static org.mockito.Mockito.doAnswer;
028import static org.mockito.Mockito.spy;
029import static org.mockito.Mockito.when;
030
031import java.io.FilterInputStream;
032import java.io.IOException;
033import java.lang.reflect.Field;
034import java.security.PrivilegedExceptionAction;
035import java.util.ArrayList;
036import java.util.Arrays;
037import java.util.Collection;
038import java.util.HashSet;
039import java.util.List;
040import java.util.NavigableMap;
041import java.util.Set;
042import java.util.TreeMap;
043import java.util.concurrent.atomic.AtomicBoolean;
044import java.util.concurrent.atomic.AtomicInteger;
045import java.util.function.Consumer;
046import org.apache.hadoop.conf.Configuration;
047import org.apache.hadoop.fs.FSDataInputStream;
048import org.apache.hadoop.fs.FileStatus;
049import org.apache.hadoop.fs.FileSystem;
050import org.apache.hadoop.fs.Path;
051import org.apache.hadoop.fs.PathFilter;
052import org.apache.hadoop.hbase.Cell;
053import org.apache.hadoop.hbase.ExtendedCell;
054import org.apache.hadoop.hbase.HBaseConfiguration;
055import org.apache.hadoop.hbase.HBaseTestingUtil;
056import org.apache.hadoop.hbase.HConstants;
057import org.apache.hadoop.hbase.KeyValue;
058import org.apache.hadoop.hbase.ServerName;
059import org.apache.hadoop.hbase.SingleProcessHBaseCluster;
060import org.apache.hadoop.hbase.TableName;
061import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
062import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
063import org.apache.hadoop.hbase.client.Delete;
064import org.apache.hadoop.hbase.client.Get;
065import org.apache.hadoop.hbase.client.Put;
066import org.apache.hadoop.hbase.client.RegionInfo;
067import org.apache.hadoop.hbase.client.RegionInfoBuilder;
068import org.apache.hadoop.hbase.client.Result;
069import org.apache.hadoop.hbase.client.ResultScanner;
070import org.apache.hadoop.hbase.client.Scan;
071import org.apache.hadoop.hbase.client.Table;
072import org.apache.hadoop.hbase.client.TableDescriptor;
073import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
074import org.apache.hadoop.hbase.io.asyncfs.monitor.StreamSlowMonitor;
075import org.apache.hadoop.hbase.monitoring.MonitoredTask;
076import org.apache.hadoop.hbase.regionserver.DefaultStoreEngine;
077import org.apache.hadoop.hbase.regionserver.DefaultStoreFlusher;
078import org.apache.hadoop.hbase.regionserver.FlushLifeCycleTracker;
079import org.apache.hadoop.hbase.regionserver.FlushRequestListener;
080import org.apache.hadoop.hbase.regionserver.FlushRequester;
081import org.apache.hadoop.hbase.regionserver.HRegion;
082import org.apache.hadoop.hbase.regionserver.HRegionServer;
083import org.apache.hadoop.hbase.regionserver.HStore;
084import org.apache.hadoop.hbase.regionserver.MemStoreSizing;
085import org.apache.hadoop.hbase.regionserver.MemStoreSnapshot;
086import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
087import org.apache.hadoop.hbase.regionserver.Region;
088import org.apache.hadoop.hbase.regionserver.RegionScanner;
089import org.apache.hadoop.hbase.regionserver.RegionServerServices;
090import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
091import org.apache.hadoop.hbase.security.User;
092import org.apache.hadoop.hbase.util.Bytes;
093import org.apache.hadoop.hbase.util.CommonFSUtils;
094import org.apache.hadoop.hbase.util.CommonFSUtils.StreamLacksCapabilityException;
095import org.apache.hadoop.hbase.util.EnvironmentEdge;
096import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
097import org.apache.hadoop.hbase.util.HFileTestUtil;
098import org.apache.hadoop.hbase.util.Pair;
099import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
100import org.apache.hadoop.hbase.wal.WAL;
101import org.apache.hadoop.hbase.wal.WALEdit;
102import org.apache.hadoop.hbase.wal.WALEditInternalHelper;
103import org.apache.hadoop.hbase.wal.WALFactory;
104import org.apache.hadoop.hbase.wal.WALKeyImpl;
105import org.apache.hadoop.hbase.wal.WALSplitUtil;
106import org.apache.hadoop.hbase.wal.WALSplitter;
107import org.apache.hadoop.hbase.wal.WALStreamReader;
108import org.apache.hadoop.hdfs.DFSInputStream;
109import org.junit.After;
110import org.junit.AfterClass;
111import org.junit.Before;
112import org.junit.BeforeClass;
113import org.junit.Rule;
114import org.junit.Test;
115import org.junit.rules.TestName;
116import org.mockito.Mockito;
117import org.mockito.invocation.InvocationOnMock;
118import org.mockito.stubbing.Answer;
119import org.slf4j.Logger;
120import org.slf4j.LoggerFactory;
121
122/**
123 * Test replay of edits out of a WAL split.
124 */
125public abstract class AbstractTestWALReplay {
126  private static final Logger LOG = LoggerFactory.getLogger(AbstractTestWALReplay.class);
127  static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil();
128  private final EnvironmentEdge ee = EnvironmentEdgeManager.getDelegate();
129  private Path hbaseRootDir = null;
130  private String logName;
131  private Path oldLogDir;
132  private Path logDir;
133  private FileSystem fs;
134  private Configuration conf;
135  private WALFactory wals;
136
137  @Rule
138  public final TestName currentTest = new TestName();
139
140  @BeforeClass
141  public static void setUpBeforeClass() throws Exception {
142    Configuration conf = TEST_UTIL.getConfiguration();
143    // The below config supported by 0.20-append and CDH3b2
144    conf.setInt("dfs.client.block.recovery.retries", 2);
145    TEST_UTIL.startMiniCluster(3);
146    Path hbaseRootDir = TEST_UTIL.getDFSCluster().getFileSystem().makeQualified(new Path("/hbase"));
147    LOG.info("hbase.rootdir=" + hbaseRootDir);
148    CommonFSUtils.setRootDir(conf, hbaseRootDir);
149  }
150
151  @AfterClass
152  public static void tearDownAfterClass() throws Exception {
153    TEST_UTIL.shutdownMiniCluster();
154  }
155
156  @Before
157  public void setUp() throws Exception {
158    this.conf = HBaseConfiguration.create(TEST_UTIL.getConfiguration());
159    this.fs = TEST_UTIL.getDFSCluster().getFileSystem();
160    this.hbaseRootDir = CommonFSUtils.getRootDir(this.conf);
161    this.oldLogDir = new Path(this.hbaseRootDir, HConstants.HREGION_OLDLOGDIR_NAME);
162    String serverName = ServerName
163      .valueOf(currentTest.getMethodName() + "-manual", 16010, EnvironmentEdgeManager.currentTime())
164      .toString();
165    this.logName = AbstractFSWALProvider.getWALDirectoryName(serverName);
166    this.logDir = new Path(this.hbaseRootDir, logName);
167    if (TEST_UTIL.getDFSCluster().getFileSystem().exists(this.hbaseRootDir)) {
168      TEST_UTIL.getDFSCluster().getFileSystem().delete(this.hbaseRootDir, true);
169    }
170    this.wals = new WALFactory(conf, currentTest.getMethodName());
171  }
172
173  @After
174  public void tearDown() throws Exception {
175    this.wals.close();
176    TEST_UTIL.getDFSCluster().getFileSystem().delete(this.hbaseRootDir, true);
177  }
178
179  /*
180   * @param p Directory to cleanup
181   */
182  private void deleteDir(final Path p) throws IOException {
183    if (this.fs.exists(p)) {
184      if (!this.fs.delete(p, true)) {
185        throw new IOException("Failed remove of " + p);
186      }
187    }
188  }
189
190  /**
191   *   */
192  @Test
193  public void testReplayEditsAfterRegionMovedWithMultiCF() throws Exception {
194    final TableName tableName = TableName.valueOf("testReplayEditsAfterRegionMovedWithMultiCF");
195    byte[] family1 = Bytes.toBytes("cf1");
196    byte[] family2 = Bytes.toBytes("cf2");
197    byte[] qualifier = Bytes.toBytes("q");
198    byte[] value = Bytes.toBytes("testV");
199    byte[][] familys = { family1, family2 };
200    TEST_UTIL.createTable(tableName, familys);
201    Table htable = TEST_UTIL.getConnection().getTable(tableName);
202    Put put = new Put(Bytes.toBytes("r1"));
203    put.addColumn(family1, qualifier, value);
204    htable.put(put);
205    ResultScanner resultScanner = htable.getScanner(new Scan());
206    int count = 0;
207    while (resultScanner.next() != null) {
208      count++;
209    }
210    resultScanner.close();
211    assertEquals(1, count);
212
213    SingleProcessHBaseCluster hbaseCluster = TEST_UTIL.getMiniHBaseCluster();
214    List<HRegion> regions = hbaseCluster.getRegions(tableName);
215    assertEquals(1, regions.size());
216
217    // move region to another regionserver
218    Region destRegion = regions.get(0);
219    int originServerNum = hbaseCluster.getServerWith(destRegion.getRegionInfo().getRegionName());
220    assertTrue("Please start more than 1 regionserver",
221      hbaseCluster.getRegionServerThreads().size() > 1);
222    int destServerNum = 0;
223    while (destServerNum == originServerNum) {
224      destServerNum++;
225    }
226    HRegionServer originServer = hbaseCluster.getRegionServer(originServerNum);
227    HRegionServer destServer = hbaseCluster.getRegionServer(destServerNum);
228    // move region to destination regionserver
229    TEST_UTIL.moveRegionAndWait(destRegion.getRegionInfo(), destServer.getServerName());
230
231    // delete the row
232    Delete del = new Delete(Bytes.toBytes("r1"));
233    htable.delete(del);
234    resultScanner = htable.getScanner(new Scan());
235    count = 0;
236    while (resultScanner.next() != null) {
237      count++;
238    }
239    resultScanner.close();
240    assertEquals(0, count);
241
242    // flush region and make major compaction
243    HRegion region =
244      (HRegion) destServer.getOnlineRegion(destRegion.getRegionInfo().getRegionName());
245    region.flush(true);
246    // wait to complete major compaction
247    for (HStore store : region.getStores()) {
248      store.triggerMajorCompaction();
249    }
250    region.compact(true);
251
252    // move region to origin regionserver
253    TEST_UTIL.moveRegionAndWait(destRegion.getRegionInfo(), originServer.getServerName());
254    // abort the origin regionserver
255    originServer.abort("testing");
256
257    // see what we get
258    Result result = htable.get(new Get(Bytes.toBytes("r1")));
259    if (result != null) {
260      assertTrue("Row is deleted, but we get" + result.toString(),
261        (result == null) || result.isEmpty());
262    }
263    resultScanner.close();
264  }
265
266  /**
267   * Tests for hbase-2727.
268   * @see <a href="https://issues.apache.org/jira/browse/HBASE-2727">HBASE-2727</a>
269   */
270  @Test
271  public void test2727() throws Exception {
272    // Test being able to have > 1 set of edits in the recovered.edits directory.
273    // Ensure edits are replayed properly.
274    final TableName tableName = TableName.valueOf("test2727");
275
276    MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl();
277    RegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
278    Path basedir = CommonFSUtils.getTableDir(hbaseRootDir, tableName);
279    deleteDir(basedir);
280
281    TableDescriptor tableDescriptor = createBasic3FamilyHTD(tableName);
282    Region region2 =
283      HBaseTestingUtil.createRegionAndWAL(hri, hbaseRootDir, this.conf, tableDescriptor);
284    HBaseTestingUtil.closeRegionAndWAL(region2);
285    final byte[] rowName = tableName.getName();
286
287    WAL wal1 = createWAL(this.conf, hbaseRootDir, logName);
288    // Add 1k to each family.
289    final int countPerFamily = 1000;
290
291    NavigableMap<byte[], Integer> scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR);
292    for (byte[] fam : tableDescriptor.getColumnFamilyNames()) {
293      scopes.put(fam, 0);
294    }
295    for (ColumnFamilyDescriptor familyDescriptor : tableDescriptor.getColumnFamilies()) {
296      addWALEdits(tableName, hri, rowName, familyDescriptor.getName(), countPerFamily, ee, wal1,
297        mvcc, scopes);
298    }
299    wal1.shutdown();
300    runWALSplit(this.conf);
301
302    WAL wal2 = createWAL(this.conf, hbaseRootDir, logName);
303    // Add 1k to each family.
304    for (ColumnFamilyDescriptor familyDescriptor : tableDescriptor.getColumnFamilies()) {
305      addWALEdits(tableName, hri, rowName, familyDescriptor.getName(), countPerFamily, ee, wal2,
306        mvcc, scopes);
307    }
308    wal2.shutdown();
309    runWALSplit(this.conf);
310
311    WAL wal3 = createWAL(this.conf, hbaseRootDir, logName);
312    try {
313      HRegion region =
314        HRegion.openHRegion(this.conf, this.fs, hbaseRootDir, hri, tableDescriptor, wal3);
315      long seqid = region.getOpenSeqNum();
316      // The regions opens with sequenceId as 1. With 6k edits, its sequence number reaches 6k + 1.
317      // When opened, this region would apply 6k edits, and increment the sequenceId by 1
318      assertTrue(seqid > mvcc.getWritePoint());
319      assertEquals(seqid - 1, mvcc.getWritePoint());
320      LOG.debug(
321        "region.getOpenSeqNum(): " + region.getOpenSeqNum() + ", wal3.id: " + mvcc.getReadPoint());
322
323      // TODO: Scan all.
324      region.close();
325    } finally {
326      wal3.close();
327    }
328  }
329
330  /**
331   * Test case of HRegion that is only made out of bulk loaded files. Assert that we don't 'crash'.
332   */
333  @Test
334  public void testRegionMadeOfBulkLoadedFilesOnly() throws IOException, SecurityException,
335    IllegalArgumentException, NoSuchFieldException, IllegalAccessException, InterruptedException {
336    final TableName tableName = TableName.valueOf("testRegionMadeOfBulkLoadedFilesOnly");
337    final RegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
338    final Path basedir = new Path(this.hbaseRootDir, tableName.getNameAsString());
339    deleteDir(basedir);
340    final TableDescriptor htd = createBasic3FamilyHTD(tableName);
341    Region region2 = HBaseTestingUtil.createRegionAndWAL(hri, hbaseRootDir, this.conf, htd);
342    HBaseTestingUtil.closeRegionAndWAL(region2);
343    WAL wal = createWAL(this.conf, hbaseRootDir, logName);
344    HRegion region = HRegion.openHRegion(hri, htd, wal, this.conf);
345
346    byte[] family = htd.getColumnFamilies()[0].getName();
347    Path f = new Path(basedir, "hfile");
348    HFileTestUtil.createHFile(this.conf, fs, f, family, family, Bytes.toBytes(""),
349      Bytes.toBytes("z"), 10);
350    List<Pair<byte[], String>> hfs = new ArrayList<>(1);
351    hfs.add(Pair.newPair(family, f.toString()));
352    region.bulkLoadHFiles(hfs, true, null);
353
354    // Add an edit so something in the WAL
355    byte[] row = tableName.getName();
356    region.put((new Put(row)).addColumn(family, family, family));
357    wal.sync();
358    final int rowsInsertedCount = 11;
359
360    assertEquals(rowsInsertedCount, getScannedCount(region.getScanner(new Scan())));
361
362    // Now 'crash' the region by stealing its wal
363    final Configuration newConf = HBaseConfiguration.create(this.conf);
364    User user = HBaseTestingUtil.getDifferentUser(newConf, tableName.getNameAsString());
365    user.runAs(new PrivilegedExceptionAction() {
366      @Override
367      public Object run() throws Exception {
368        runWALSplit(newConf);
369        WAL wal2 = createWAL(newConf, hbaseRootDir, logName);
370
371        HRegion region2 =
372          HRegion.openHRegion(newConf, FileSystem.get(newConf), hbaseRootDir, hri, htd, wal2);
373        long seqid2 = region2.getOpenSeqNum();
374        assertTrue(seqid2 > -1);
375        assertEquals(rowsInsertedCount, getScannedCount(region2.getScanner(new Scan())));
376
377        // I can't close wal1. Its been appropriated when we split.
378        region2.close();
379        wal2.close();
380        return null;
381      }
382    });
383  }
384
385  /**
386   * HRegion test case that is made of a major compacted HFile (created with three bulk loaded
387   * files) and an edit in the memstore.
388   * <p/>
389   * This is for HBASE-10958 "[dataloss] Bulk loading with seqids can prevent some log entries from
390   * being replayed"
391   */
392  @Test
393  public void testCompactedBulkLoadedFiles() throws IOException, SecurityException,
394    IllegalArgumentException, NoSuchFieldException, IllegalAccessException, InterruptedException {
395    final TableName tableName = TableName.valueOf("testCompactedBulkLoadedFiles");
396    final RegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
397    final Path basedir = new Path(this.hbaseRootDir, tableName.getNameAsString());
398    deleteDir(basedir);
399    final TableDescriptor htd = createBasic3FamilyHTD(tableName);
400    HRegion region2 = HBaseTestingUtil.createRegionAndWAL(hri, hbaseRootDir, this.conf, htd);
401    HBaseTestingUtil.closeRegionAndWAL(region2);
402    WAL wal = createWAL(this.conf, hbaseRootDir, logName);
403    HRegion region = HRegion.openHRegion(hri, htd, wal, this.conf);
404
405    // Add an edit so something in the WAL
406    byte[] row = tableName.getName();
407    byte[] family = htd.getColumnFamilies()[0].getName();
408    region.put((new Put(row)).addColumn(family, family, family));
409    wal.sync();
410
411    List<Pair<byte[], String>> hfs = new ArrayList<>(1);
412    for (int i = 0; i < 3; i++) {
413      Path f = new Path(basedir, "hfile" + i);
414      HFileTestUtil.createHFile(this.conf, fs, f, family, family, Bytes.toBytes(i + "00"),
415        Bytes.toBytes(i + "50"), 10);
416      hfs.add(Pair.newPair(family, f.toString()));
417    }
418    region.bulkLoadHFiles(hfs, true, null);
419    final int rowsInsertedCount = 31;
420    assertEquals(rowsInsertedCount, getScannedCount(region.getScanner(new Scan())));
421
422    // major compact to turn all the bulk loaded files into one normal file
423    region.compact(true);
424    assertEquals(rowsInsertedCount, getScannedCount(region.getScanner(new Scan())));
425
426    // Now 'crash' the region by stealing its wal
427    final Configuration newConf = HBaseConfiguration.create(this.conf);
428    User user = HBaseTestingUtil.getDifferentUser(newConf, tableName.getNameAsString());
429    user.runAs(new PrivilegedExceptionAction() {
430      @Override
431      public Object run() throws Exception {
432        runWALSplit(newConf);
433        WAL wal2 = createWAL(newConf, hbaseRootDir, logName);
434
435        HRegion region2 =
436          HRegion.openHRegion(newConf, FileSystem.get(newConf), hbaseRootDir, hri, htd, wal2);
437        long seqid2 = region2.getOpenSeqNum();
438        assertTrue(seqid2 > -1);
439        assertEquals(rowsInsertedCount, getScannedCount(region2.getScanner(new Scan())));
440
441        // I can't close wal1. Its been appropriated when we split.
442        region2.close();
443        wal2.close();
444        return null;
445      }
446    });
447  }
448
449  /**
450   * Test writing edits into an HRegion, closing it, splitting logs, opening Region again. Verify
451   * seqids.
452   */
453  @Test
454  public void testReplayEditsWrittenViaHRegion() throws IOException, SecurityException,
455    IllegalArgumentException, NoSuchFieldException, IllegalAccessException, InterruptedException {
456    final TableName tableName = TableName.valueOf("testReplayEditsWrittenViaHRegion");
457    final RegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
458    final Path basedir = CommonFSUtils.getTableDir(this.hbaseRootDir, tableName);
459    deleteDir(basedir);
460    final byte[] rowName = tableName.getName();
461    final int countPerFamily = 10;
462    final TableDescriptor htd = createBasic3FamilyHTD(tableName);
463    HRegion region3 = HBaseTestingUtil.createRegionAndWAL(hri, hbaseRootDir, this.conf, htd);
464    HBaseTestingUtil.closeRegionAndWAL(region3);
465    // Write countPerFamily edits into the three families. Do a flush on one
466    // of the families during the load of edits so its seqid is not same as
467    // others to test we do right thing when different seqids.
468    WAL wal = createWAL(this.conf, hbaseRootDir, logName);
469    HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseRootDir, hri, htd, wal);
470    long seqid = region.getOpenSeqNum();
471    boolean first = true;
472    for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) {
473      addRegionEdits(rowName, hcd.getName(), countPerFamily, this.ee, region, "x");
474      if (first) {
475        // If first, so we have at least one family w/ different seqid to rest.
476        region.flush(true);
477        first = false;
478      }
479    }
480    // Now assert edits made it in.
481    final Get g = new Get(rowName);
482    Result result = region.get(g);
483    assertEquals(countPerFamily * htd.getColumnFamilies().length, result.size());
484    // Now close the region (without flush), split the log, reopen the region and assert that
485    // replay of log has the correct effect, that our seqids are calculated correctly so
486    // all edits in logs are seen as 'stale'/old.
487    region.close(true);
488    wal.shutdown();
489    runWALSplit(this.conf);
490    WAL wal2 = createWAL(this.conf, hbaseRootDir, logName);
491    HRegion region2 = HRegion.openHRegion(conf, this.fs, hbaseRootDir, hri, htd, wal2);
492    long seqid2 = region2.getOpenSeqNum();
493    assertTrue(seqid + result.size() < seqid2);
494    final Result result1b = region2.get(g);
495    assertEquals(result.size(), result1b.size());
496
497    // Next test. Add more edits, then 'crash' this region by stealing its wal
498    // out from under it and assert that replay of the log adds the edits back
499    // correctly when region is opened again.
500    for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) {
501      addRegionEdits(rowName, hcd.getName(), countPerFamily, this.ee, region2, "y");
502    }
503    // Get count of edits.
504    final Result result2 = region2.get(g);
505    assertEquals(2 * result.size(), result2.size());
506    wal2.sync();
507    final Configuration newConf = HBaseConfiguration.create(this.conf);
508    User user = HBaseTestingUtil.getDifferentUser(newConf, tableName.getNameAsString());
509    user.runAs(new PrivilegedExceptionAction<Object>() {
510      @Override
511      public Object run() throws Exception {
512        runWALSplit(newConf);
513        FileSystem newFS = FileSystem.get(newConf);
514        // Make a new wal for new region open.
515        WAL wal3 = createWAL(newConf, hbaseRootDir, logName);
516        final AtomicInteger countOfRestoredEdits = new AtomicInteger(0);
517        HRegion region3 = new HRegion(basedir, wal3, newFS, newConf, hri, htd, null) {
518          @Override
519          protected void restoreEdit(HStore s, ExtendedCell cell, MemStoreSizing memstoreSizing) {
520            super.restoreEdit(s, cell, memstoreSizing);
521            countOfRestoredEdits.incrementAndGet();
522          }
523        };
524        long seqid3 = region3.initialize();
525        Result result3 = region3.get(g);
526        // Assert that count of cells is same as before crash.
527        assertEquals(result2.size(), result3.size());
528        assertEquals(htd.getColumnFamilies().length * countPerFamily, countOfRestoredEdits.get());
529
530        // I can't close wal1. Its been appropriated when we split.
531        region3.close();
532        wal3.close();
533        return null;
534      }
535    });
536  }
537
538  /**
539   * Test that we recover correctly when there is a failure in between the flushes. i.e. Some stores
540   * got flushed but others did not.
541   * <p/>
542   * Unfortunately, there is no easy hook to flush at a store level. The way we get around this is
543   * by flushing at the region level, and then deleting the recently flushed store file for one of
544   * the Stores. This would put us back in the situation where all but that store got flushed and
545   * the region died.
546   * <p/>
547   * We restart Region again, and verify that the edits were replayed.
548   */
549  @Test
550  public void testReplayEditsAfterPartialFlush() throws IOException, SecurityException,
551    IllegalArgumentException, NoSuchFieldException, IllegalAccessException, InterruptedException {
552    final TableName tableName = TableName.valueOf("testReplayEditsWrittenViaHRegion");
553    final RegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
554    final Path basedir = CommonFSUtils.getTableDir(this.hbaseRootDir, tableName);
555    deleteDir(basedir);
556    final byte[] rowName = tableName.getName();
557    final int countPerFamily = 10;
558    final TableDescriptor htd = createBasic3FamilyHTD(tableName);
559    HRegion region3 = HBaseTestingUtil.createRegionAndWAL(hri, hbaseRootDir, this.conf, htd);
560    HBaseTestingUtil.closeRegionAndWAL(region3);
561    // Write countPerFamily edits into the three families. Do a flush on one
562    // of the families during the load of edits so its seqid is not same as
563    // others to test we do right thing when different seqids.
564    WAL wal = createWAL(this.conf, hbaseRootDir, logName);
565    HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseRootDir, hri, htd, wal);
566    long seqid = region.getOpenSeqNum();
567    for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) {
568      addRegionEdits(rowName, hcd.getName(), countPerFamily, this.ee, region, "x");
569    }
570
571    // Now assert edits made it in.
572    final Get g = new Get(rowName);
573    Result result = region.get(g);
574    assertEquals(countPerFamily * htd.getColumnFamilies().length, result.size());
575
576    // Let us flush the region
577    region.flush(true);
578    region.close(true);
579    wal.shutdown();
580
581    // delete the store files in the second column family to simulate a failure
582    // in between the flushcache();
583    // we have 3 families. killing the middle one ensures that taking the maximum
584    // will make us fail.
585    int cf_count = 0;
586    for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) {
587      cf_count++;
588      if (cf_count == 2) {
589        region.getRegionFileSystem().deleteFamily(hcd.getNameAsString());
590      }
591    }
592
593    // Let us try to split and recover
594    runWALSplit(this.conf);
595    WAL wal2 = createWAL(this.conf, hbaseRootDir, logName);
596    HRegion region2 = HRegion.openHRegion(this.conf, this.fs, hbaseRootDir, hri, htd, wal2);
597    long seqid2 = region2.getOpenSeqNum();
598    assertTrue(seqid + result.size() < seqid2);
599
600    final Result result1b = region2.get(g);
601    assertEquals(result.size(), result1b.size());
602  }
603
604  // StoreFlusher implementation used in testReplayEditsAfterAbortingFlush.
605  // Only throws exception if throwExceptionWhenFlushing is set true.
606  public static class CustomStoreFlusher extends DefaultStoreFlusher {
607    // Switch between throw and not throw exception in flush
608    public static final AtomicBoolean throwExceptionWhenFlushing = new AtomicBoolean(false);
609
610    public CustomStoreFlusher(Configuration conf, HStore store) {
611      super(conf, store);
612    }
613
614    @Override
615    public List<Path> flushSnapshot(MemStoreSnapshot snapshot, long cacheFlushId,
616      MonitoredTask status, ThroughputController throughputController,
617      FlushLifeCycleTracker tracker, Consumer<Path> writerCreationTracker) throws IOException {
618      if (throwExceptionWhenFlushing.get()) {
619        throw new IOException("Simulated exception by tests");
620      }
621      return super.flushSnapshot(snapshot, cacheFlushId, status, throughputController, tracker,
622        writerCreationTracker);
623    }
624  }
625
626  /**
627   * Test that we could recover the data correctly after aborting flush. In the test, first we abort
628   * flush after writing some data, then writing more data and flush again, at last verify the data.
629   */
630  @Test
631  public void testReplayEditsAfterAbortingFlush() throws IOException {
632    final TableName tableName = TableName.valueOf("testReplayEditsAfterAbortingFlush");
633    final RegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
634    final Path basedir = CommonFSUtils.getTableDir(this.hbaseRootDir, tableName);
635    deleteDir(basedir);
636    final TableDescriptor htd = createBasic3FamilyHTD(tableName);
637    HRegion region3 = HBaseTestingUtil.createRegionAndWAL(hri, hbaseRootDir, this.conf, htd);
638    HBaseTestingUtil.closeRegionAndWAL(region3);
639    // Write countPerFamily edits into the three families. Do a flush on one
640    // of the families during the load of edits so its seqid is not same as
641    // others to test we do right thing when different seqids.
642    WAL wal = createWAL(this.conf, hbaseRootDir, logName);
643    RegionServerServices rsServices = Mockito.mock(RegionServerServices.class);
644    Mockito.doReturn(false).when(rsServices).isAborted();
645    when(rsServices.getServerName()).thenReturn(ServerName.valueOf("foo", 10, 10));
646    when(rsServices.getConfiguration()).thenReturn(conf);
647    Configuration customConf = new Configuration(this.conf);
648    customConf.set(DefaultStoreEngine.DEFAULT_STORE_FLUSHER_CLASS_KEY,
649      CustomStoreFlusher.class.getName());
650    HRegion region =
651      HRegion.openHRegion(this.hbaseRootDir, hri, htd, wal, customConf, rsServices, null);
652    int writtenRowCount = 10;
653    List<ColumnFamilyDescriptor> families = Arrays.asList((htd.getColumnFamilies()));
654    for (int i = 0; i < writtenRowCount; i++) {
655      Put put = new Put(Bytes.toBytes(tableName + Integer.toString(i)));
656      put.addColumn(families.get(i % families.size()).getName(), Bytes.toBytes("q"),
657        Bytes.toBytes("val"));
658      region.put(put);
659    }
660
661    // Now assert edits made it in.
662    RegionScanner scanner = region.getScanner(new Scan());
663    assertEquals(writtenRowCount, getScannedCount(scanner));
664
665    // Let us flush the region
666    CustomStoreFlusher.throwExceptionWhenFlushing.set(true);
667    try {
668      region.flush(true);
669      fail("Injected exception hasn't been thrown");
670    } catch (IOException e) {
671      LOG.info("Expected simulated exception when flushing region, {}", e.getMessage());
672      // simulated to abort server
673      Mockito.doReturn(true).when(rsServices).isAborted();
674      region.setClosing(false); // region normally does not accept writes after
675      // DroppedSnapshotException. We mock around it for this test.
676    }
677    // writing more data
678    int moreRow = 10;
679    for (int i = writtenRowCount; i < writtenRowCount + moreRow; i++) {
680      Put put = new Put(Bytes.toBytes(tableName + Integer.toString(i)));
681      put.addColumn(families.get(i % families.size()).getName(), Bytes.toBytes("q"),
682        Bytes.toBytes("val"));
683      region.put(put);
684    }
685    writtenRowCount += moreRow;
686    // call flush again
687    CustomStoreFlusher.throwExceptionWhenFlushing.set(false);
688    try {
689      region.flush(true);
690    } catch (IOException t) {
691      LOG.info(
692        "Expected exception when flushing region because server is stopped," + t.getMessage());
693    }
694
695    region.close(true);
696    wal.shutdown();
697
698    // Let us try to split and recover
699    runWALSplit(this.conf);
700    WAL wal2 = createWAL(this.conf, hbaseRootDir, logName);
701    Mockito.doReturn(false).when(rsServices).isAborted();
702    HRegion region2 =
703      HRegion.openHRegion(this.hbaseRootDir, hri, htd, wal2, this.conf, rsServices, null);
704    scanner = region2.getScanner(new Scan());
705    assertEquals(writtenRowCount, getScannedCount(scanner));
706  }
707
708  private int getScannedCount(RegionScanner scanner) throws IOException {
709    int scannedCount = 0;
710    List<Cell> results = new ArrayList<>();
711    while (true) {
712      boolean existMore = scanner.next(results);
713      if (!results.isEmpty()) {
714        scannedCount++;
715      }
716      if (!existMore) {
717        break;
718      }
719      results.clear();
720    }
721    return scannedCount;
722  }
723
724  /**
725   * Create an HRegion with the result of a WAL split and test we only see the good edits=
726   */
727  @Test
728  public void testReplayEditsWrittenIntoWAL() throws Exception {
729    final TableName tableName = TableName.valueOf("testReplayEditsWrittenIntoWAL");
730    final MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl();
731    final RegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
732    final Path basedir = CommonFSUtils.getTableDir(hbaseRootDir, tableName);
733    deleteDir(basedir);
734
735    final TableDescriptor htd = createBasic3FamilyHTD(tableName);
736    HRegion region2 = HBaseTestingUtil.createRegionAndWAL(hri, hbaseRootDir, this.conf, htd);
737    HBaseTestingUtil.closeRegionAndWAL(region2);
738    final WAL wal = createWAL(this.conf, hbaseRootDir, logName);
739    final byte[] rowName = tableName.getName();
740    final byte[] regionName = hri.getEncodedNameAsBytes();
741
742    // Add 1k to each family.
743    final int countPerFamily = 1000;
744    Set<byte[]> familyNames = new HashSet<>();
745    NavigableMap<byte[], Integer> scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR);
746    for (byte[] fam : htd.getColumnFamilyNames()) {
747      scopes.put(fam, 0);
748    }
749    for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) {
750      addWALEdits(tableName, hri, rowName, hcd.getName(), countPerFamily, ee, wal, mvcc, scopes);
751      familyNames.add(hcd.getName());
752    }
753
754    // Add a cache flush, shouldn't have any effect
755    wal.startCacheFlush(regionName, familyNames);
756    wal.completeCacheFlush(regionName, HConstants.NO_SEQNUM);
757
758    // Add an edit to another family, should be skipped.
759    WALEdit edit = new WALEdit();
760    long now = ee.currentTime();
761    WALEditInternalHelper.addExtendedCell(edit,
762      new KeyValue(rowName, Bytes.toBytes("another family"), rowName, now, rowName));
763    wal.appendData(hri, new WALKeyImpl(hri.getEncodedNameAsBytes(), tableName, now, mvcc, scopes),
764      edit);
765
766    // Delete the c family to verify deletes make it over.
767    edit = new WALEdit();
768    now = ee.currentTime();
769    WALEditInternalHelper.addExtendedCell(edit,
770      new KeyValue(rowName, Bytes.toBytes("c"), null, now, KeyValue.Type.DeleteFamily));
771    wal.appendData(hri, new WALKeyImpl(hri.getEncodedNameAsBytes(), tableName, now, mvcc, scopes),
772      edit);
773
774    // Sync.
775    wal.sync();
776    // Make a new conf and a new fs for the splitter to run on so we can take
777    // over old wal.
778    final Configuration newConf = HBaseConfiguration.create(this.conf);
779    User user = HBaseTestingUtil.getDifferentUser(newConf, ".replay.wal.secondtime");
780    user.runAs(new PrivilegedExceptionAction<Void>() {
781      @Override
782      public Void run() throws Exception {
783        runWALSplit(newConf);
784        FileSystem newFS = FileSystem.get(newConf);
785        // 100k seems to make for about 4 flushes during HRegion#initialize.
786        newConf.setInt(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 1024 * 100);
787        // Make a new wal for new region.
788        WAL newWal = createWAL(newConf, hbaseRootDir, logName);
789        final AtomicInteger flushcount = new AtomicInteger(0);
790        try {
791          final HRegion region = new HRegion(basedir, newWal, newFS, newConf, hri, htd, null) {
792            @Override
793            protected FlushResultImpl internalFlushcache(final WAL wal, final long myseqid,
794              final Collection<HStore> storesToFlush, MonitoredTask status,
795              boolean writeFlushWalMarker, FlushLifeCycleTracker tracker) throws IOException {
796              LOG.info("InternalFlushCache Invoked");
797              FlushResultImpl fs = super.internalFlushcache(wal, myseqid, storesToFlush,
798                Mockito.mock(MonitoredTask.class), writeFlushWalMarker, tracker);
799              flushcount.incrementAndGet();
800              return fs;
801            }
802          };
803          // The seq id this region has opened up with
804          long seqid = region.initialize();
805
806          // The mvcc readpoint of from inserting data.
807          long writePoint = mvcc.getWritePoint();
808
809          // We flushed during init.
810          assertTrue("Flushcount=" + flushcount.get(), flushcount.get() > 0);
811          assertTrue((seqid - 1) == writePoint);
812
813          Get get = new Get(rowName);
814          Result result = region.get(get);
815          // Make sure we only see the good edits
816          assertEquals(countPerFamily * (htd.getColumnFamilies().length - 1), result.size());
817          region.close();
818        } finally {
819          newWal.close();
820        }
821        return null;
822      }
823    });
824  }
825
826  @Test
827  // the following test is for HBASE-6065
828  public void testSequentialEditLogSeqNum() throws IOException {
829    final TableName tableName = TableName.valueOf(currentTest.getMethodName());
830    final RegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
831    final Path basedir = CommonFSUtils.getWALTableDir(conf, tableName);
832    deleteDir(basedir);
833    final byte[] rowName = tableName.getName();
834    final int countPerFamily = 10;
835    final TableDescriptor htd = createBasic1FamilyHTD(tableName);
836
837    // Mock the WAL
838    MockWAL wal = createMockWAL();
839
840    HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseRootDir, hri, htd, wal);
841    for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) {
842      addRegionEdits(rowName, hcd.getName(), countPerFamily, this.ee, region, "x");
843    }
844
845    // Let us flush the region
846    // But this time completeflushcache is not yet done
847    region.flush(true);
848    for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) {
849      addRegionEdits(rowName, hcd.getName(), 5, this.ee, region, "x");
850    }
851    long lastestSeqNumber = region.getReadPoint(null);
852    // get the current seq no
853    wal.doCompleteCacheFlush = true;
854    // allow complete cache flush with the previous seq number got after first
855    // set of edits.
856    wal.completeCacheFlush(hri.getEncodedNameAsBytes(), HConstants.NO_SEQNUM);
857    wal.shutdown();
858    FileStatus[] listStatus = wal.getFiles();
859    assertNotNull(listStatus);
860    assertTrue(listStatus.length > 0);
861    WALSplitter.splitLogFile(hbaseRootDir, listStatus[0], this.fs, this.conf, null, null, null,
862      wals, null);
863    FileStatus[] listStatus1 =
864      this.fs.listStatus(new Path(CommonFSUtils.getWALTableDir(conf, tableName),
865        new Path(hri.getEncodedName(), "recovered.edits")), new PathFilter() {
866          @Override
867          public boolean accept(Path p) {
868            return !WALSplitUtil.isSequenceIdFile(p);
869          }
870        });
871    int editCount = 0;
872    for (FileStatus fileStatus : listStatus1) {
873      editCount = Integer.parseInt(fileStatus.getPath().getName());
874    }
875    // The sequence number should be same
876    assertEquals(
877      "The sequence number of the recoverd.edits and the current edit seq should be same",
878      lastestSeqNumber, editCount);
879  }
880
881  /**
882   * testcase for https://issues.apache.org/jira/browse/HBASE-15252
883   */
884  @Test
885  public void testDatalossWhenInputError() throws Exception {
886    final TableName tableName = TableName.valueOf("testDatalossWhenInputError");
887    final RegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
888    final Path basedir = CommonFSUtils.getWALTableDir(conf, tableName);
889    deleteDir(basedir);
890    final byte[] rowName = tableName.getName();
891    final int countPerFamily = 10;
892    final TableDescriptor htd = createBasic1FamilyHTD(tableName);
893    HRegion region1 = HBaseTestingUtil.createRegionAndWAL(hri, hbaseRootDir, this.conf, htd);
894    Path regionDir = region1.getWALRegionDir();
895    HBaseTestingUtil.closeRegionAndWAL(region1);
896
897    WAL wal = createWAL(this.conf, hbaseRootDir, logName);
898    HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseRootDir, hri, htd, wal);
899    for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) {
900      addRegionEdits(rowName, hcd.getName(), countPerFamily, this.ee, region, "x");
901    }
902    // Now assert edits made it in.
903    final Get g = new Get(rowName);
904    Result result = region.get(g);
905    assertEquals(countPerFamily * htd.getColumnFamilies().length, result.size());
906    // Now close the region (without flush), split the log, reopen the region and assert that
907    // replay of log has the correct effect.
908    region.close(true);
909    wal.shutdown();
910
911    runWALSplit(this.conf);
912
913    // here we let the DFSInputStream throw an IOException just after the WALHeader.
914    Path editFile = WALSplitUtil.getSplitEditFilesSorted(this.fs, regionDir).first();
915    final long headerLength;
916    try (WALStreamReader reader = WALFactory.createStreamReader(fs, editFile, conf)) {
917      headerLength = reader.getPosition();
918    }
919    FileSystem spyFs = spy(this.fs);
920    doAnswer(new Answer<FSDataInputStream>() {
921
922      @Override
923      public FSDataInputStream answer(InvocationOnMock invocation) throws Throwable {
924        FSDataInputStream stream = (FSDataInputStream) invocation.callRealMethod();
925        Field field = FilterInputStream.class.getDeclaredField("in");
926        field.setAccessible(true);
927        final DFSInputStream in = (DFSInputStream) field.get(stream);
928        DFSInputStream spyIn = spy(in);
929        doAnswer(new Answer<Integer>() {
930
931          private long pos;
932
933          @Override
934          public Integer answer(InvocationOnMock invocation) throws Throwable {
935            if (pos >= headerLength) {
936              throw new IOException("read over limit");
937            }
938            int b = (Integer) invocation.callRealMethod();
939            if (b > 0) {
940              pos += b;
941            }
942            return b;
943          }
944        }).when(spyIn).read(any(byte[].class), anyInt(), anyInt());
945        doAnswer(new Answer<Void>() {
946
947          @Override
948          public Void answer(InvocationOnMock invocation) throws Throwable {
949            invocation.callRealMethod();
950            in.close();
951            return null;
952          }
953        }).when(spyIn).close();
954        field.set(stream, spyIn);
955        return stream;
956      }
957    }).when(spyFs).open(eq(editFile));
958
959    WAL wal2 = createWAL(this.conf, hbaseRootDir, logName);
960    HRegion region2;
961    try {
962      // log replay should fail due to the IOException, otherwise we may lose data.
963      region2 = HRegion.openHRegion(conf, spyFs, hbaseRootDir, hri, htd, wal2);
964      assertEquals(result.size(), region2.get(g).size());
965    } catch (IOException e) {
966      assertEquals("read over limit", e.getMessage());
967    }
968    region2 = HRegion.openHRegion(conf, fs, hbaseRootDir, hri, htd, wal2);
969    assertEquals(result.size(), region2.get(g).size());
970  }
971
972  /**
973   * testcase for https://issues.apache.org/jira/browse/HBASE-14949.
974   */
975  private void testNameConflictWhenSplit(boolean largeFirst)
976    throws IOException, StreamLacksCapabilityException {
977    final TableName tableName = TableName.valueOf("testReplayEditsWrittenIntoWAL");
978    final MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl();
979    final RegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
980    final Path basedir = CommonFSUtils.getTableDir(hbaseRootDir, tableName);
981    deleteDir(basedir);
982
983    final TableDescriptor htd = createBasic1FamilyHTD(tableName);
984    NavigableMap<byte[], Integer> scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR);
985    for (byte[] fam : htd.getColumnFamilyNames()) {
986      scopes.put(fam, 0);
987    }
988    HRegion region = HBaseTestingUtil.createRegionAndWAL(hri, hbaseRootDir, this.conf, htd);
989    HBaseTestingUtil.closeRegionAndWAL(region);
990    final byte[] family = htd.getColumnFamilies()[0].getName();
991    final byte[] rowName = tableName.getName();
992    FSWALEntry entry1 = createFSWALEntry(htd, hri, 1L, rowName, family, ee, mvcc, 1, scopes);
993    FSWALEntry entry2 = createFSWALEntry(htd, hri, 2L, rowName, family, ee, mvcc, 2, scopes);
994
995    Path largeFile = new Path(logDir, "wal-1");
996    Path smallFile = new Path(logDir, "wal-2");
997    writerWALFile(largeFile, Arrays.asList(entry1, entry2));
998    writerWALFile(smallFile, Arrays.asList(entry2));
999    FileStatus first, second;
1000    if (largeFirst) {
1001      first = fs.getFileStatus(largeFile);
1002      second = fs.getFileStatus(smallFile);
1003    } else {
1004      first = fs.getFileStatus(smallFile);
1005      second = fs.getFileStatus(largeFile);
1006    }
1007    WALSplitter.splitLogFile(hbaseRootDir, first, fs, conf, null, null, null, wals, null);
1008    WALSplitter.splitLogFile(hbaseRootDir, second, fs, conf, null, null, null, wals, null);
1009    WAL wal = createWAL(this.conf, hbaseRootDir, logName);
1010    region = HRegion.openHRegion(conf, this.fs, hbaseRootDir, hri, htd, wal);
1011    assertTrue(region.getOpenSeqNum() > mvcc.getWritePoint());
1012    assertEquals(2, region.get(new Get(rowName)).size());
1013  }
1014
1015  @Test
1016  public void testNameConflictWhenSplit0() throws IOException, StreamLacksCapabilityException {
1017    testNameConflictWhenSplit(true);
1018  }
1019
1020  @Test
1021  public void testNameConflictWhenSplit1() throws IOException, StreamLacksCapabilityException {
1022    testNameConflictWhenSplit(false);
1023  }
1024
1025  static class MockWAL extends FSHLog {
1026    boolean doCompleteCacheFlush = false;
1027
1028    public MockWAL(FileSystem fs, Path rootDir, String logName, Configuration conf)
1029      throws IOException {
1030      super(fs, rootDir, logName, HConstants.HREGION_OLDLOGDIR_NAME, conf, null, true, null, null);
1031    }
1032
1033    @Override
1034    public void completeCacheFlush(byte[] encodedRegionName, long maxFlushedSeqId) {
1035      if (!doCompleteCacheFlush) {
1036        return;
1037      }
1038      super.completeCacheFlush(encodedRegionName, maxFlushedSeqId);
1039    }
1040  }
1041
1042  private TableDescriptor createBasic1FamilyHTD(final TableName tableName) {
1043    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
1044    builder.setColumnFamily(ColumnFamilyDescriptorBuilder.of(Bytes.toBytes("a")));
1045    return builder.build();
1046  }
1047
1048  private MockWAL createMockWAL() throws IOException {
1049    MockWAL wal = new MockWAL(fs, hbaseRootDir, logName, conf);
1050    wal.init();
1051    // Set down maximum recovery so we dfsclient doesn't linger retrying something
1052    // long gone.
1053    HBaseTestingUtil.setMaxRecoveryErrorCount(wal.getOutputStream(), 1);
1054    return wal;
1055  }
1056
1057  // Flusher used in this test. Keep count of how often we are called and
1058  // actually run the flush inside here.
1059  static class TestFlusher implements FlushRequester {
1060    private HRegion r;
1061
1062    @Override
1063    public boolean requestFlush(HRegion region, FlushLifeCycleTracker tracker) {
1064      try {
1065        r.flush(false);
1066        return true;
1067      } catch (IOException e) {
1068        throw new RuntimeException("Exception flushing", e);
1069      }
1070    }
1071
1072    @Override
1073    public boolean requestFlush(HRegion region, List<byte[]> families,
1074      FlushLifeCycleTracker tracker) {
1075      return true;
1076    }
1077
1078    @Override
1079    public boolean requestDelayedFlush(HRegion region, long when) {
1080      return true;
1081    }
1082
1083    @Override
1084    public void registerFlushRequestListener(FlushRequestListener listener) {
1085
1086    }
1087
1088    @Override
1089    public boolean unregisterFlushRequestListener(FlushRequestListener listener) {
1090      return false;
1091    }
1092
1093    @Override
1094    public void setGlobalMemStoreLimit(long globalMemStoreSize) {
1095
1096    }
1097  }
1098
1099  private WALKeyImpl createWALKey(final TableName tableName, final RegionInfo hri,
1100    final MultiVersionConcurrencyControl mvcc, NavigableMap<byte[], Integer> scopes) {
1101    return new WALKeyImpl(hri.getEncodedNameAsBytes(), tableName, 999, mvcc, scopes);
1102  }
1103
1104  private WALEdit createWALEdit(final byte[] rowName, final byte[] family, EnvironmentEdge ee,
1105    int index) {
1106    byte[] qualifierBytes = Bytes.toBytes(Integer.toString(index));
1107    byte[] columnBytes = Bytes.toBytes(Bytes.toString(family) + ":" + Integer.toString(index));
1108    WALEdit edit = new WALEdit();
1109    WALEditInternalHelper.addExtendedCell(edit,
1110      new KeyValue(rowName, family, qualifierBytes, ee.currentTime(), columnBytes));
1111    return edit;
1112  }
1113
1114  private FSWALEntry createFSWALEntry(TableDescriptor htd, RegionInfo hri, long sequence,
1115    byte[] rowName, byte[] family, EnvironmentEdge ee, MultiVersionConcurrencyControl mvcc,
1116    int index, NavigableMap<byte[], Integer> scopes) throws IOException {
1117    FSWALEntry entry = new FSWALEntry(sequence, createWALKey(htd.getTableName(), hri, mvcc, scopes),
1118      createWALEdit(rowName, family, ee, index), hri, true, null);
1119    entry.stampRegionSequenceId(mvcc.begin());
1120    return entry;
1121  }
1122
1123  private void addWALEdits(final TableName tableName, final RegionInfo hri, final byte[] rowName,
1124    final byte[] family, final int count, EnvironmentEdge ee, final WAL wal,
1125    final MultiVersionConcurrencyControl mvcc, NavigableMap<byte[], Integer> scopes)
1126    throws IOException {
1127    for (int j = 0; j < count; j++) {
1128      wal.appendData(hri, createWALKey(tableName, hri, mvcc, scopes),
1129        createWALEdit(rowName, family, ee, j));
1130    }
1131    wal.sync();
1132  }
1133
1134  public static List<Put> addRegionEdits(final byte[] rowName, final byte[] family, final int count,
1135    EnvironmentEdge ee, final Region r, final String qualifierPrefix) throws IOException {
1136    List<Put> puts = new ArrayList<>();
1137    for (int j = 0; j < count; j++) {
1138      byte[] qualifier = Bytes.toBytes(qualifierPrefix + Integer.toString(j));
1139      Put p = new Put(rowName);
1140      p.addColumn(family, qualifier, ee.currentTime(), rowName);
1141      r.put(p);
1142      puts.add(p);
1143    }
1144    return puts;
1145  }
1146
1147  /**
1148   * Creates an HRI around an HTD that has <code>tableName</code> and three column families named
1149   * 'a','b', and 'c'.
1150   * @param tableName Name of table to use when we create HTableDescriptor.
1151   */
1152  private RegionInfo createBasic3FamilyHRegionInfo(final TableName tableName) {
1153    return RegionInfoBuilder.newBuilder(tableName).build();
1154  }
1155
1156  /**
1157   * Run the split. Verify only single split file made.
1158   * @return The single split file made
1159   */
1160  private Path runWALSplit(final Configuration c) throws IOException {
1161    List<Path> splits =
1162      WALSplitter.split(hbaseRootDir, logDir, oldLogDir, FileSystem.get(c), c, wals);
1163    // Split should generate only 1 file since there's only 1 region
1164    assertEquals("splits=" + splits, 1, splits.size());
1165    // Make sure the file exists
1166    assertTrue(fs.exists(splits.get(0)));
1167    LOG.info("Split file=" + splits.get(0));
1168    return splits.get(0);
1169  }
1170
1171  private TableDescriptor createBasic3FamilyHTD(final TableName tableName) {
1172    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
1173    builder.setColumnFamily(ColumnFamilyDescriptorBuilder.of(Bytes.toBytes("a")));
1174    builder.setColumnFamily(ColumnFamilyDescriptorBuilder.of(Bytes.toBytes("b")));
1175    builder.setColumnFamily(ColumnFamilyDescriptorBuilder.of(Bytes.toBytes("c")));
1176    return builder.build();
1177  }
1178
1179  private void writerWALFile(Path file, List<FSWALEntry> entries)
1180    throws IOException, StreamLacksCapabilityException {
1181    fs.mkdirs(file.getParent());
1182    ProtobufLogWriter writer = new ProtobufLogWriter();
1183    writer.init(fs, file, conf, true, WALUtil.getWALBlockSize(conf, fs, file),
1184      StreamSlowMonitor.create(conf, "testMonitor"));
1185    for (FSWALEntry entry : entries) {
1186      writer.append(entry);
1187    }
1188    writer.sync(false);
1189    writer.close();
1190  }
1191
1192  protected abstract WAL createWAL(Configuration c, Path hbaseRootDir, String logName)
1193    throws IOException;
1194}