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.hamcrest.CoreMatchers.containsString;
021import static org.junit.Assert.assertEquals;
022import static org.junit.Assert.assertNotEquals;
023import static org.junit.Assert.assertNotNull;
024import static org.junit.Assert.assertNull;
025import static org.junit.Assert.assertThat;
026import static org.junit.Assert.assertTrue;
027import static org.junit.Assert.fail;
028import static org.mockito.Mockito.mock;
029import static org.mockito.Mockito.when;
030
031import java.io.IOException;
032import java.util.ArrayList;
033import java.util.Collection;
034import java.util.Collections;
035import java.util.Comparator;
036import java.util.HashSet;
037import java.util.List;
038import java.util.Map;
039import java.util.NavigableMap;
040import java.util.Set;
041import java.util.TreeMap;
042import java.util.UUID;
043import java.util.concurrent.ConcurrentSkipListMap;
044import java.util.concurrent.CountDownLatch;
045import java.util.concurrent.ExecutorService;
046import java.util.concurrent.Executors;
047import java.util.concurrent.atomic.AtomicBoolean;
048import java.util.stream.Collectors;
049import org.apache.hadoop.conf.Configuration;
050import org.apache.hadoop.fs.FileStatus;
051import org.apache.hadoop.fs.FileSystem;
052import org.apache.hadoop.fs.Path;
053import org.apache.hadoop.hbase.CellScanner;
054import org.apache.hadoop.hbase.Coprocessor;
055import org.apache.hadoop.hbase.HBaseConfiguration;
056import org.apache.hadoop.hbase.HBaseTestingUtility;
057import org.apache.hadoop.hbase.HConstants;
058import org.apache.hadoop.hbase.KeyValue;
059import org.apache.hadoop.hbase.ServerName;
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.Durability;
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.TableDescriptor;
070import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
071import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
072import org.apache.hadoop.hbase.coprocessor.SampleRegionWALCoprocessor;
073import org.apache.hadoop.hbase.regionserver.ChunkCreator;
074import org.apache.hadoop.hbase.regionserver.FlushPolicy;
075import org.apache.hadoop.hbase.regionserver.FlushPolicyFactory;
076import org.apache.hadoop.hbase.regionserver.HRegion;
077import org.apache.hadoop.hbase.regionserver.HStore;
078import org.apache.hadoop.hbase.regionserver.MemStoreLAB;
079import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
080import org.apache.hadoop.hbase.regionserver.RegionServerServices;
081import org.apache.hadoop.hbase.regionserver.SequenceId;
082import org.apache.hadoop.hbase.util.Bytes;
083import org.apache.hadoop.hbase.util.CommonFSUtils;
084import org.apache.hadoop.hbase.util.EnvironmentEdge;
085import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
086import org.apache.hadoop.hbase.util.Threads;
087import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
088import org.apache.hadoop.hbase.wal.WAL;
089import org.apache.hadoop.hbase.wal.WALEdit;
090import org.apache.hadoop.hbase.wal.WALKey;
091import org.apache.hadoop.hbase.wal.WALKeyImpl;
092import org.junit.AfterClass;
093import org.junit.Before;
094import org.junit.BeforeClass;
095import org.junit.Rule;
096import org.junit.Test;
097import org.junit.rules.TestName;
098import org.slf4j.Logger;
099import org.slf4j.LoggerFactory;
100
101public abstract class AbstractTestFSWAL {
102
103  protected static final Logger LOG = LoggerFactory.getLogger(AbstractTestFSWAL.class);
104
105  protected static Configuration CONF;
106  protected static FileSystem FS;
107  protected static Path DIR;
108  protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
109
110  @Rule
111  public final TestName currentTest = new TestName();
112
113  @Before
114  public void setUp() throws Exception {
115    FileStatus[] entries = FS.listStatus(new Path("/"));
116    for (FileStatus dir : entries) {
117      FS.delete(dir.getPath(), true);
118    }
119    final Path hbaseDir = TEST_UTIL.createRootDir();
120    final Path hbaseWALDir = TEST_UTIL.createWALRootDir();
121    DIR = new Path(hbaseWALDir, currentTest.getMethodName());
122    assertNotEquals(hbaseDir, hbaseWALDir);
123  }
124
125  @BeforeClass
126  public static void setUpBeforeClass() throws Exception {
127    // Make block sizes small.
128    TEST_UTIL.getConfiguration().setInt("dfs.blocksize", 1024 * 1024);
129    // quicker heartbeat interval for faster DN death notification
130    TEST_UTIL.getConfiguration().setInt("dfs.namenode.heartbeat.recheck-interval", 5000);
131    TEST_UTIL.getConfiguration().setInt("dfs.heartbeat.interval", 1);
132    TEST_UTIL.getConfiguration().setInt("dfs.client.socket-timeout", 5000);
133
134    // faster failover with cluster.shutdown();fs.close() idiom
135    TEST_UTIL.getConfiguration().setInt("hbase.ipc.client.connect.max.retries", 1);
136    TEST_UTIL.getConfiguration().setInt("dfs.client.block.recovery.retries", 1);
137    TEST_UTIL.getConfiguration().setInt("hbase.ipc.client.connection.maxidletime", 500);
138    TEST_UTIL.getConfiguration().set(CoprocessorHost.WAL_COPROCESSOR_CONF_KEY,
139      SampleRegionWALCoprocessor.class.getName());
140    TEST_UTIL.startMiniDFSCluster(3);
141
142    CONF = TEST_UTIL.getConfiguration();
143    FS = TEST_UTIL.getDFSCluster().getFileSystem();
144  }
145
146  @AfterClass
147  public static void tearDownAfterClass() throws Exception {
148    TEST_UTIL.shutdownMiniCluster();
149  }
150
151  protected abstract AbstractFSWAL<?> newWAL(FileSystem fs, Path rootDir, String WALDir,
152      String archiveDir, Configuration conf, List<WALActionsListener> listeners,
153      boolean failIfWALExists, String prefix, String suffix) throws IOException;
154
155  protected abstract AbstractFSWAL<?> newSlowWAL(FileSystem fs, Path rootDir, String WALDir,
156      String archiveDir, Configuration conf, List<WALActionsListener> listeners,
157      boolean failIfWALExists, String prefix, String suffix, Runnable action) throws IOException;
158
159  /**
160   * A loaded WAL coprocessor won't break existing WAL test cases.
161   */
162  @Test
163  public void testWALCoprocessorLoaded() throws Exception {
164    // test to see whether the coprocessor is loaded or not.
165    AbstractFSWAL<?> wal = null;
166    try {
167      wal = newWAL(FS, CommonFSUtils.getWALRootDir(CONF), DIR.toString(),
168          HConstants.HREGION_OLDLOGDIR_NAME, CONF, null, true, null, null);
169      WALCoprocessorHost host = wal.getCoprocessorHost();
170      Coprocessor c = host.findCoprocessor(SampleRegionWALCoprocessor.class);
171      assertNotNull(c);
172    } finally {
173      if (wal != null) {
174        wal.close();
175      }
176    }
177  }
178
179  protected void addEdits(WAL log, RegionInfo hri, TableDescriptor htd, int times,
180      MultiVersionConcurrencyControl mvcc, NavigableMap<byte[], Integer> scopes, String cf)
181      throws IOException {
182    final byte[] row = Bytes.toBytes(cf);
183    for (int i = 0; i < times; i++) {
184      long timestamp = System.currentTimeMillis();
185      WALEdit cols = new WALEdit();
186      cols.add(new KeyValue(row, row, row, timestamp, row));
187      WALKeyImpl key = new WALKeyImpl(hri.getEncodedNameAsBytes(), htd.getTableName(),
188          SequenceId.NO_SEQUENCE_ID, timestamp, WALKey.EMPTY_UUIDS, HConstants.NO_NONCE,
189          HConstants.NO_NONCE, mvcc, scopes);
190      log.appendData(hri, key, cols);
191    }
192    log.sync();
193  }
194
195  /**
196   * helper method to simulate region flush for a WAL.
197   */
198  protected void flushRegion(WAL wal, byte[] regionEncodedName, Set<byte[]> flushedFamilyNames) {
199    wal.startCacheFlush(regionEncodedName, flushedFamilyNames);
200    wal.completeCacheFlush(regionEncodedName, HConstants.NO_SEQNUM);
201  }
202
203  /**
204   * tests the log comparator. Ensure that we are not mixing meta logs with non-meta logs (throws
205   * exception if we do). Comparison is based on the timestamp present in the wal name.
206   * @throws Exception
207   */
208  @Test
209  public void testWALComparator() throws Exception {
210    AbstractFSWAL<?> wal1 = null;
211    AbstractFSWAL<?> walMeta = null;
212    try {
213      wal1 = newWAL(FS, CommonFSUtils.getWALRootDir(CONF), DIR.toString(),
214          HConstants.HREGION_OLDLOGDIR_NAME, CONF, null, true, null, null);
215      LOG.debug("Log obtained is: " + wal1);
216      Comparator<Path> comp = wal1.LOG_NAME_COMPARATOR;
217      Path p1 = wal1.computeFilename(11);
218      Path p2 = wal1.computeFilename(12);
219      // comparing with itself returns 0
220      assertTrue(comp.compare(p1, p1) == 0);
221      // comparing with different filenum.
222      assertTrue(comp.compare(p1, p2) < 0);
223      walMeta = newWAL(FS, CommonFSUtils.getWALRootDir(CONF), DIR.toString(),
224          HConstants.HREGION_OLDLOGDIR_NAME, CONF, null, true, null,
225          AbstractFSWALProvider.META_WAL_PROVIDER_ID);
226      Comparator<Path> compMeta = walMeta.LOG_NAME_COMPARATOR;
227
228      Path p1WithMeta = walMeta.computeFilename(11);
229      Path p2WithMeta = walMeta.computeFilename(12);
230      assertTrue(compMeta.compare(p1WithMeta, p1WithMeta) == 0);
231      assertTrue(compMeta.compare(p1WithMeta, p2WithMeta) < 0);
232      // mixing meta and non-meta logs gives error
233      boolean ex = false;
234      try {
235        comp.compare(p1WithMeta, p2);
236      } catch (IllegalArgumentException e) {
237        ex = true;
238      }
239      assertTrue("Comparator doesn't complain while checking meta log files", ex);
240      boolean exMeta = false;
241      try {
242        compMeta.compare(p1WithMeta, p2);
243      } catch (IllegalArgumentException e) {
244        exMeta = true;
245      }
246      assertTrue("Meta comparator doesn't complain while checking log files", exMeta);
247    } finally {
248      if (wal1 != null) {
249        wal1.close();
250      }
251      if (walMeta != null) {
252        walMeta.close();
253      }
254    }
255  }
256
257  /**
258   * On rolling a wal after reaching the threshold, {@link WAL#rollWriter()} returns the list of
259   * regions which should be flushed in order to archive the oldest wal file.
260   * <p>
261   * This method tests this behavior by inserting edits and rolling the wal enough times to reach
262   * the max number of logs threshold. It checks whether we get the "right regions and stores" for
263   * flush on rolling the wal.
264   * @throws Exception
265   */
266  @Test
267  public void testFindMemStoresEligibleForFlush() throws Exception {
268    LOG.debug("testFindMemStoresEligibleForFlush");
269    Configuration conf1 = HBaseConfiguration.create(CONF);
270    conf1.setInt("hbase.regionserver.maxlogs", 1);
271    AbstractFSWAL<?> wal = newWAL(FS, CommonFSUtils.getWALRootDir(conf1), DIR.toString(),
272      HConstants.HREGION_OLDLOGDIR_NAME, conf1, null, true, null, null);
273    String cf1 = "cf1";
274    String cf2 = "cf2";
275    String cf3 = "cf3";
276    TableDescriptor t1 = TableDescriptorBuilder.newBuilder(TableName.valueOf("t1"))
277      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(cf1)).build();
278    TableDescriptor t2 = TableDescriptorBuilder.newBuilder(TableName.valueOf("t2"))
279      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(cf1)).build();
280    RegionInfo hri1 = RegionInfoBuilder.newBuilder(t1.getTableName()).build();
281    RegionInfo hri2 = RegionInfoBuilder.newBuilder(t2.getTableName()).build();
282
283    List<ColumnFamilyDescriptor> cfs = new ArrayList();
284    cfs.add(ColumnFamilyDescriptorBuilder.of(cf1));
285    cfs.add(ColumnFamilyDescriptorBuilder.of(cf2));
286    TableDescriptor t3 = TableDescriptorBuilder.newBuilder(TableName.valueOf("t3"))
287      .setColumnFamilies(cfs).build();
288    RegionInfo hri3 = RegionInfoBuilder.newBuilder(t3.getTableName()).build();
289
290    // add edits and roll the wal
291    MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl();
292    NavigableMap<byte[], Integer> scopes1 = new TreeMap<>(Bytes.BYTES_COMPARATOR);
293    for (byte[] fam : t1.getColumnFamilyNames()) {
294      scopes1.put(fam, 0);
295    }
296    NavigableMap<byte[], Integer> scopes2 = new TreeMap<>(Bytes.BYTES_COMPARATOR);
297    for (byte[] fam : t2.getColumnFamilyNames()) {
298      scopes2.put(fam, 0);
299    }
300    NavigableMap<byte[], Integer> scopes3 = new TreeMap<>(Bytes.BYTES_COMPARATOR);
301    for (byte[] fam : t3.getColumnFamilyNames()) {
302      scopes3.put(fam, 0);
303    }
304    try {
305      addEdits(wal, hri1, t1, 2, mvcc, scopes1, cf1);
306      wal.rollWriter();
307      // add some more edits and roll the wal. This would reach the log number threshold
308      addEdits(wal, hri1, t1, 2, mvcc, scopes1, cf1);
309      wal.rollWriter();
310      // with above rollWriter call, the max logs limit is reached.
311      assertTrue(wal.getNumRolledLogFiles() == 2);
312
313      // get the regions to flush; since there is only one region in the oldest wal, it should
314      // return only one region.
315      Map<byte[], List<byte[]>> regionsToFlush = wal.findRegionsToForceFlush();
316      assertEquals(1, regionsToFlush.size());
317      assertEquals(hri1.getEncodedNameAsBytes(), (byte[])regionsToFlush.keySet().toArray()[0]);
318      // insert edits in second region
319      addEdits(wal, hri2, t2, 2, mvcc, scopes2, cf1);
320      // get the regions to flush, it should still read region1.
321      regionsToFlush = wal.findRegionsToForceFlush();
322      assertEquals(1, regionsToFlush.size());
323      assertEquals(hri1.getEncodedNameAsBytes(), (byte[])regionsToFlush.keySet().toArray()[0]);
324      // flush region 1, and roll the wal file. Only last wal which has entries for region1 should
325      // remain.
326      flushRegion(wal, hri1.getEncodedNameAsBytes(), t1.getColumnFamilyNames());
327      wal.rollWriter();
328      // only one wal should remain now (that is for the second region).
329      assertEquals(1, wal.getNumRolledLogFiles());
330      // flush the second region
331      flushRegion(wal, hri2.getEncodedNameAsBytes(), t2.getColumnFamilyNames());
332      wal.rollWriter(true);
333      // no wal should remain now.
334      assertEquals(0, wal.getNumRolledLogFiles());
335      // add edits both to region 1 and region 2, and roll.
336      addEdits(wal, hri1, t1, 2, mvcc, scopes1, cf1);
337      addEdits(wal, hri2, t2, 2, mvcc, scopes2, cf1);
338      wal.rollWriter();
339      // add edits and roll the writer, to reach the max logs limit.
340      assertEquals(1, wal.getNumRolledLogFiles());
341      addEdits(wal, hri1, t1, 2, mvcc, scopes1, cf1);
342      wal.rollWriter();
343      // it should return two regions to flush, as the oldest wal file has entries
344      // for both regions.
345      regionsToFlush = wal.findRegionsToForceFlush();
346      assertEquals(2, regionsToFlush.size());
347      // flush both regions
348      flushRegion(wal, hri1.getEncodedNameAsBytes(), t1.getColumnFamilyNames());
349      flushRegion(wal, hri2.getEncodedNameAsBytes(), t2.getColumnFamilyNames());
350      wal.rollWriter(true);
351      assertEquals(0, wal.getNumRolledLogFiles());
352      // Add an edit to region1, and roll the wal.
353      addEdits(wal, hri1, t1, 2, mvcc, scopes1, cf1);
354      // tests partial flush: roll on a partial flush, and ensure that wal is not archived.
355      wal.startCacheFlush(hri1.getEncodedNameAsBytes(), t1.getColumnFamilyNames());
356      wal.rollWriter();
357      wal.completeCacheFlush(hri1.getEncodedNameAsBytes(), HConstants.NO_SEQNUM);
358      assertEquals(1, wal.getNumRolledLogFiles());
359
360      // clear test data
361      flushRegion(wal, hri1.getEncodedNameAsBytes(), t1.getColumnFamilyNames());
362      wal.rollWriter(true);
363      // add edits for three familes
364      addEdits(wal, hri3, t3, 2, mvcc, scopes3, cf1);
365      addEdits(wal, hri3, t3, 2, mvcc, scopes3, cf2);
366      addEdits(wal, hri3, t3, 2, mvcc, scopes3, cf3);
367      wal.rollWriter();
368      addEdits(wal, hri3, t3, 2, mvcc, scopes3, cf1);
369      wal.rollWriter();
370      assertEquals(2, wal.getNumRolledLogFiles());
371      // flush one family before archive oldest wal
372      Set<byte[]> flushedFamilyNames = new HashSet<>();
373      flushedFamilyNames.add(Bytes.toBytes(cf1));
374      flushRegion(wal, hri3.getEncodedNameAsBytes(), flushedFamilyNames);
375      regionsToFlush = wal.findRegionsToForceFlush();
376      // then only two family need to be flushed when archive oldest wal
377      assertEquals(1, regionsToFlush.size());
378      assertEquals(hri3.getEncodedNameAsBytes(), (byte[])regionsToFlush.keySet().toArray()[0]);
379      assertEquals(2, regionsToFlush.get(hri3.getEncodedNameAsBytes()).size());
380    } finally {
381      if (wal != null) {
382        wal.close();
383      }
384    }
385  }
386
387  @Test(expected = IOException.class)
388  public void testFailedToCreateWALIfParentRenamed() throws IOException,
389      CommonFSUtils.StreamLacksCapabilityException {
390    final String name = "testFailedToCreateWALIfParentRenamed";
391    AbstractFSWAL<?> wal = newWAL(FS, CommonFSUtils.getWALRootDir(CONF), name,
392      HConstants.HREGION_OLDLOGDIR_NAME, CONF, null, true, null, null);
393    long filenum = System.currentTimeMillis();
394    Path path = wal.computeFilename(filenum);
395    wal.createWriterInstance(path);
396    Path parent = path.getParent();
397    path = wal.computeFilename(filenum + 1);
398    Path newPath = new Path(parent.getParent(), parent.getName() + "-splitting");
399    FS.rename(parent, newPath);
400    wal.createWriterInstance(path);
401    fail("It should fail to create the new WAL");
402  }
403
404  /**
405   * Test flush for sure has a sequence id that is beyond the last edit appended. We do this by
406   * slowing appends in the background ring buffer thread while in foreground we call flush. The
407   * addition of the sync over HRegion in flush should fix an issue where flush was returning before
408   * all of its appends had made it out to the WAL (HBASE-11109).
409   * @throws IOException
410   * @see <a href="https://issues.apache.org/jira/browse/HBASE-11109">HBASE-11109</a>
411   */
412  @Test
413  public void testFlushSequenceIdIsGreaterThanAllEditsInHFile() throws IOException {
414    String testName = currentTest.getMethodName();
415    final TableName tableName = TableName.valueOf(testName);
416    final RegionInfo hri = RegionInfoBuilder.newBuilder(tableName).build();
417    final byte[] rowName = tableName.getName();
418    final TableDescriptor htd = TableDescriptorBuilder.newBuilder(tableName)
419      .setColumnFamily(ColumnFamilyDescriptorBuilder.of("f")).build();
420    HRegion r = HBaseTestingUtility.createRegionAndWAL(hri, TEST_UTIL.getDefaultRootDirPath(),
421      TEST_UTIL.getConfiguration(), htd);
422    HBaseTestingUtility.closeRegionAndWAL(r);
423    final int countPerFamily = 10;
424    final AtomicBoolean goslow = new AtomicBoolean(false);
425    NavigableMap<byte[], Integer> scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR);
426    for (byte[] fam : htd.getColumnFamilyNames()) {
427      scopes.put(fam, 0);
428    }
429    // subclass and doctor a method.
430    AbstractFSWAL<?> wal = newSlowWAL(FS, CommonFSUtils.getWALRootDir(CONF), DIR.toString(),
431        testName, CONF, null, true, null, null, new Runnable() {
432
433          @Override
434          public void run() {
435            if (goslow.get()) {
436              Threads.sleep(100);
437              LOG.debug("Sleeping before appending 100ms");
438            }
439          }
440        });
441    HRegion region = HRegion.openHRegion(TEST_UTIL.getConfiguration(),
442      TEST_UTIL.getTestFileSystem(), TEST_UTIL.getDefaultRootDirPath(), hri, htd, wal);
443    EnvironmentEdge ee = EnvironmentEdgeManager.getDelegate();
444    try {
445      List<Put> puts = null;
446      for (byte[] fam : htd.getColumnFamilyNames()) {
447        puts =
448            TestWALReplay.addRegionEdits(rowName, fam, countPerFamily, ee, region, "x");
449      }
450
451      // Now assert edits made it in.
452      final Get g = new Get(rowName);
453      Result result = region.get(g);
454      assertEquals(countPerFamily * htd.getColumnFamilyNames().size(), result.size());
455
456      // Construct a WALEdit and add it a few times to the WAL.
457      WALEdit edits = new WALEdit();
458      for (Put p : puts) {
459        CellScanner cs = p.cellScanner();
460        while (cs.advance()) {
461          edits.add(cs.current());
462        }
463      }
464      // Add any old cluster id.
465      List<UUID> clusterIds = new ArrayList<>(1);
466      clusterIds.add(TEST_UTIL.getRandomUUID());
467      // Now make appends run slow.
468      goslow.set(true);
469      for (int i = 0; i < countPerFamily; i++) {
470        final RegionInfo info = region.getRegionInfo();
471        final WALKeyImpl logkey = new WALKeyImpl(info.getEncodedNameAsBytes(), tableName,
472            System.currentTimeMillis(), clusterIds, -1, -1, region.getMVCC(), scopes);
473        wal.append(info, logkey, edits, true);
474        region.getMVCC().completeAndWait(logkey.getWriteEntry());
475      }
476      region.flush(true);
477      // FlushResult.flushSequenceId is not visible here so go get the current sequence id.
478      long currentSequenceId = region.getReadPoint(null);
479      // Now release the appends
480      goslow.set(false);
481      assertTrue(currentSequenceId >= region.getReadPoint(null));
482    } finally {
483      region.close(true);
484      wal.close();
485    }
486  }
487
488  @Test
489  public void testSyncNoAppend() throws IOException {
490    String testName = currentTest.getMethodName();
491    AbstractFSWAL<?> wal = newWAL(FS, CommonFSUtils.getWALRootDir(CONF), DIR.toString(), testName,
492        CONF, null, true, null, null);
493    wal.init();
494    try {
495      wal.sync();
496    } finally {
497      wal.close();
498    }
499  }
500
501  @Test
502  public void testWriteEntryCanBeNull() throws IOException {
503    String testName = currentTest.getMethodName();
504    AbstractFSWAL<?> wal = newWAL(FS, CommonFSUtils.getWALRootDir(CONF), DIR.toString(), testName,
505      CONF, null, true, null, null);
506    wal.close();
507    TableDescriptor td = TableDescriptorBuilder.newBuilder(TableName.valueOf("table"))
508      .setColumnFamily(ColumnFamilyDescriptorBuilder.of("row")).build();
509    RegionInfo ri = RegionInfoBuilder.newBuilder(td.getTableName()).build();
510    MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl();
511    NavigableMap<byte[], Integer> scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR);
512    for (byte[] fam : td.getColumnFamilyNames()) {
513      scopes.put(fam, 0);
514    }
515    long timestamp = System.currentTimeMillis();
516    byte[] row = Bytes.toBytes("row");
517    WALEdit cols = new WALEdit();
518    cols.add(new KeyValue(row, row, row, timestamp, row));
519    WALKeyImpl key =
520        new WALKeyImpl(ri.getEncodedNameAsBytes(), td.getTableName(), SequenceId.NO_SEQUENCE_ID,
521          timestamp, WALKey.EMPTY_UUIDS, HConstants.NO_NONCE, HConstants.NO_NONCE, mvcc, scopes);
522    try {
523      wal.append(ri, key, cols, true);
524      fail("Should fail since the wal has already been closed");
525    } catch (IOException e) {
526      // expected
527      assertThat(e.getMessage(), containsString("log is closed"));
528      // the WriteEntry should be null since we fail before setting it.
529      assertNull(key.getWriteEntry());
530    }
531  }
532
533  private AbstractFSWAL<?> createHoldingWAL(String testName, AtomicBoolean startHoldingForAppend,
534    CountDownLatch holdAppend) throws IOException {
535    AbstractFSWAL<?> wal = newWAL(FS, CommonFSUtils.getRootDir(CONF), testName,
536      HConstants.HREGION_OLDLOGDIR_NAME, CONF, null, true, null, null);
537    wal.init();
538    wal.registerWALActionsListener(new WALActionsListener() {
539      @Override
540      public void visitLogEntryBeforeWrite(WALKey logKey, WALEdit logEdit) throws IOException {
541        if (startHoldingForAppend.get()) {
542          try {
543            holdAppend.await();
544          } catch (InterruptedException e) {
545            LOG.error(e.toString(), e);
546          }
547        }
548      }
549    });
550    return wal;
551  }
552
553  private HRegion createHoldingHRegion(Configuration conf, TableDescriptor htd, WAL wal)
554    throws IOException {
555    RegionInfo hri = RegionInfoBuilder.newBuilder(htd.getTableName()).build();
556    ChunkCreator.initialize(MemStoreLAB.CHUNK_SIZE_DEFAULT, false, 0, 0,
557      0, null, MemStoreLAB.INDEX_CHUNK_SIZE_PERCENTAGE_DEFAULT);
558    TEST_UTIL.createLocalHRegion(hri, CONF, htd, wal).close();
559    RegionServerServices rsServices = mock(RegionServerServices.class);
560    when(rsServices.getServerName()).thenReturn(ServerName.valueOf("localhost:12345", 123456));
561    when(rsServices.getConfiguration()).thenReturn(conf);
562    return HRegion.openHRegion(TEST_UTIL.getDataTestDir(), hri, htd, wal, conf, rsServices, null);
563  }
564
565  private void doPutWithAsyncWAL(ExecutorService exec, HRegion region, Put put,
566    Runnable flushOrCloseRegion, AtomicBoolean startHoldingForAppend,
567    CountDownLatch flushOrCloseFinished, CountDownLatch holdAppend)
568    throws InterruptedException, IOException {
569    // do a regular write first because of memstore size calculation.
570    region.put(put);
571
572    startHoldingForAppend.set(true);
573    region.put(new Put(put).setDurability(Durability.ASYNC_WAL));
574
575    // give the put a chance to start
576    Threads.sleep(3000);
577
578    exec.submit(flushOrCloseRegion);
579
580    // give the flush a chance to start. Flush should have got the region lock, and
581    // should have been waiting on the mvcc complete after this.
582    Threads.sleep(3000);
583
584    // let the append to WAL go through now that the flush already started
585    holdAppend.countDown();
586    flushOrCloseFinished.await();
587  }
588
589  // Testcase for HBASE-23181
590  @Test
591  public void testUnflushedSeqIdTrackingWithAsyncWal() throws IOException, InterruptedException {
592    String testName = currentTest.getMethodName();
593    byte[] b = Bytes.toBytes("b");
594    TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf("table"))
595      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(b)).build();
596
597    AtomicBoolean startHoldingForAppend = new AtomicBoolean(false);
598    CountDownLatch holdAppend = new CountDownLatch(1);
599    CountDownLatch closeFinished = new CountDownLatch(1);
600    ExecutorService exec = Executors.newFixedThreadPool(1);
601    AbstractFSWAL<?> wal = createHoldingWAL(testName, startHoldingForAppend, holdAppend);
602    // open a new region which uses this WAL
603    HRegion region = createHoldingHRegion(TEST_UTIL.getConfiguration(), htd, wal);
604    try {
605      doPutWithAsyncWAL(exec, region, new Put(b).addColumn(b, b, b), () -> {
606        try {
607          Map<?, ?> closeResult = region.close();
608          LOG.info("Close result:" + closeResult);
609          closeFinished.countDown();
610        } catch (IOException e) {
611          LOG.error(e.toString(), e);
612        }
613      }, startHoldingForAppend, closeFinished, holdAppend);
614
615      // now check the region's unflushed seqIds.
616      long seqId = wal.getEarliestMemStoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes());
617      assertEquals("Found seqId for the region which is already closed", HConstants.NO_SEQNUM,
618        seqId);
619    } finally {
620      exec.shutdownNow();
621      region.close();
622      wal.close();
623    }
624  }
625
626  private static final Set<byte[]> STORES_TO_FLUSH =
627    Collections.newSetFromMap(new ConcurrentSkipListMap<byte[], Boolean>(Bytes.BYTES_COMPARATOR));
628
629  // Testcase for HBASE-23157
630  @Test
631  public void testMaxFlushedSequenceIdGoBackwards() throws IOException, InterruptedException {
632    String testName = currentTest.getMethodName();
633    byte[] a = Bytes.toBytes("a");
634    byte[] b = Bytes.toBytes("b");
635    TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf("table"))
636      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(a))
637      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(b)).build();
638
639    AtomicBoolean startHoldingForAppend = new AtomicBoolean(false);
640    CountDownLatch holdAppend = new CountDownLatch(1);
641    CountDownLatch flushFinished = new CountDownLatch(1);
642    ExecutorService exec = Executors.newFixedThreadPool(2);
643    Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
644    conf.setClass(FlushPolicyFactory.HBASE_FLUSH_POLICY_KEY, FlushSpecificStoresPolicy.class,
645      FlushPolicy.class);
646    AbstractFSWAL<?> wal = createHoldingWAL(testName, startHoldingForAppend, holdAppend);
647    // open a new region which uses this WAL
648    HRegion region = createHoldingHRegion(conf, htd, wal);
649    try {
650      Put put = new Put(a).addColumn(a, a, a).addColumn(b, b, b);
651      doPutWithAsyncWAL(exec, region, put, () -> {
652        try {
653          HRegion.FlushResult flushResult = region.flush(true);
654          LOG.info("Flush result:" + flushResult.getResult());
655          LOG.info("Flush succeeded:" + flushResult.isFlushSucceeded());
656          flushFinished.countDown();
657        } catch (IOException e) {
658          LOG.error(e.toString(), e);
659        }
660      }, startHoldingForAppend, flushFinished, holdAppend);
661
662      // get the max flushed sequence id after the first flush
663      long maxFlushedSeqId1 = region.getMaxFlushedSeqId();
664
665      region.put(put);
666      // this time we only flush family a
667      STORES_TO_FLUSH.add(a);
668      region.flush(false);
669
670      // get the max flushed sequence id after the second flush
671      long maxFlushedSeqId2 = region.getMaxFlushedSeqId();
672      // make sure that the maxFlushedSequenceId does not go backwards
673      assertTrue(
674        "maxFlushedSeqId1(" + maxFlushedSeqId1 +
675          ") is not greater than or equal to maxFlushedSeqId2(" + maxFlushedSeqId2 + ")",
676        maxFlushedSeqId1 <= maxFlushedSeqId2);
677    } finally {
678      exec.shutdownNow();
679      region.close();
680      wal.close();
681    }
682  }
683
684  public static final class FlushSpecificStoresPolicy extends FlushPolicy {
685
686    @Override
687    public Collection<HStore> selectStoresToFlush() {
688      if (STORES_TO_FLUSH.isEmpty()) {
689        return region.getStores();
690      } else {
691        return STORES_TO_FLUSH.stream().map(region::getStore).collect(Collectors.toList());
692      }
693    }
694  }
695}