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;
019
020import static org.junit.Assert.assertTrue;
021
022import java.io.IOException;
023import java.util.NavigableMap;
024import java.util.TreeMap;
025import java.util.concurrent.CountDownLatch;
026import java.util.concurrent.TimeUnit;
027import org.apache.hadoop.conf.Configuration;
028import org.apache.hadoop.fs.FileSystem;
029import org.apache.hadoop.fs.Path;
030import org.apache.hadoop.hbase.CellScanner;
031import org.apache.hadoop.hbase.ChoreService;
032import org.apache.hadoop.hbase.CoordinatedStateManager;
033import org.apache.hadoop.hbase.HBaseClassTestRule;
034import org.apache.hadoop.hbase.HBaseTestingUtility;
035import org.apache.hadoop.hbase.HConstants;
036import org.apache.hadoop.hbase.Server;
037import org.apache.hadoop.hbase.ServerName;
038import org.apache.hadoop.hbase.TableName;
039import org.apache.hadoop.hbase.client.ClusterConnection;
040import org.apache.hadoop.hbase.client.Connection;
041import org.apache.hadoop.hbase.client.Durability;
042import org.apache.hadoop.hbase.client.Put;
043import org.apache.hadoop.hbase.regionserver.wal.DamagedWALException;
044import org.apache.hadoop.hbase.regionserver.wal.FSHLog;
045import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
046import org.apache.hadoop.hbase.testclassification.MediumTests;
047import org.apache.hadoop.hbase.testclassification.RegionServerTests;
048import org.apache.hadoop.hbase.util.Bytes;
049import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
050import org.apache.hadoop.hbase.util.Threads;
051import org.apache.hadoop.hbase.wal.WAL;
052import org.apache.hadoop.hbase.wal.WALEdit;
053import org.apache.hadoop.hbase.wal.WALKey;
054import org.apache.hadoop.hbase.wal.WALKeyImpl;
055import org.apache.hadoop.hbase.wal.WALProvider.Writer;
056import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
057import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
058import org.junit.After;
059import org.junit.Assert;
060import org.junit.Before;
061import org.junit.ClassRule;
062import org.junit.Rule;
063import org.junit.Test;
064import org.junit.experimental.categories.Category;
065import org.junit.rules.TestName;
066import org.mockito.Mockito;
067import org.slf4j.Logger;
068import org.slf4j.LoggerFactory;
069
070import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
071
072/**
073 * Testing for lock up of FSHLog.
074 */
075@Category({ RegionServerTests.class, MediumTests.class })
076public class TestWALLockup {
077
078  @ClassRule
079  public static final HBaseClassTestRule CLASS_RULE =
080      HBaseClassTestRule.forClass(TestWALLockup.class);
081
082  private static final Logger LOG = LoggerFactory.getLogger(TestWALLockup.class);
083
084  @Rule
085  public TestName name = new TestName();
086
087  private static final String COLUMN_FAMILY = "MyCF";
088  private static final byte [] COLUMN_FAMILY_BYTES = Bytes.toBytes(COLUMN_FAMILY);
089
090  HRegion region = null;
091  private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
092  private static Configuration CONF ;
093  private String dir;
094
095  // Test names
096  protected TableName tableName;
097
098  @Before
099  public void setup() throws IOException {
100    CONF = TEST_UTIL.getConfiguration();
101    // Disable block cache.
102    CONF.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0f);
103    dir = TEST_UTIL.getDataTestDir("TestHRegion").toString();
104    tableName = TableName.valueOf(name.getMethodName());
105  }
106
107  @After
108  public void tearDown() throws Exception {
109    EnvironmentEdgeManagerTestHelper.reset();
110    LOG.info("Cleaning test directory: " + TEST_UTIL.getDataTestDir());
111    TEST_UTIL.cleanupTestDir();
112  }
113
114  private String getName() {
115    return name.getMethodName();
116  }
117
118  // A WAL that we can have throw exceptions when a flag is set.
119  private static final class DodgyFSLog extends FSHLog {
120    // Set this when want the WAL to start throwing exceptions.
121    volatile boolean throwException = false;
122
123    // Latch to hold up processing until after another operation has had time to run.
124    CountDownLatch latch = new CountDownLatch(1);
125
126    public DodgyFSLog(FileSystem fs, Path root, String logDir, Configuration conf)
127        throws IOException {
128      super(fs, root, logDir, conf);
129    }
130
131    @Override
132    protected void afterCreatingZigZagLatch() {
133      // If throwException set, then append will throw an exception causing the WAL to be
134      // rolled. We'll come in here. Hold up processing until a sync can get in before
135      // the zigzag has time to complete its setup and get its own sync in. This is what causes
136      // the lock up we've seen in production.
137      if (throwException) {
138        try {
139          LOG.info("LATCHED");
140          // So, timing can have it that the test can run and the bad flush below happens
141          // before we get here. In this case, we'll be stuck waiting on this latch but there
142          // is nothing in the WAL pipeline to get us to the below beforeWaitOnSafePoint...
143          // because all WALs have rolled. In this case, just give up on test.
144          if (!this.latch.await(5, TimeUnit.SECONDS)) {
145            LOG.warn("GIVE UP! Failed waiting on latch...Test is ABORTED!");
146          }
147        } catch (InterruptedException e) {
148        }
149      }
150    }
151
152    @Override
153    protected void beforeWaitOnSafePoint() {
154      if (throwException) {
155        LOG.info("COUNTDOWN");
156        // Don't countdown latch until someone waiting on it otherwise, the above
157        // afterCreatingZigZagLatch will get to the latch and no one will ever free it and we'll
158        // be stuck; test won't go down
159        while (this.latch.getCount() <= 0)
160          Threads.sleep(1);
161        this.latch.countDown();
162      }
163    }
164
165    @Override
166    protected Writer createWriterInstance(Path path) throws IOException {
167      final Writer w = super.createWriterInstance(path);
168      return new Writer() {
169        @Override
170        public void close() throws IOException {
171          w.close();
172        }
173
174        @Override
175        public void sync(boolean forceSync) throws IOException {
176          if (throwException) {
177            throw new IOException("FAKE! Failed to replace a bad datanode...SYNC");
178          }
179          w.sync(forceSync);
180        }
181
182        @Override
183        public void append(Entry entry) throws IOException {
184          if (throwException) {
185            throw new IOException("FAKE! Failed to replace a bad datanode...APPEND");
186          }
187          w.append(entry);
188        }
189
190        @Override
191        public long getLength() {
192          return w.getLength();
193        }
194      };
195    }
196  }
197
198  /**
199   * Reproduce locking up that happens when we get an inopportune sync during setup for
200   * zigzaglatch wait. See HBASE-14317. If below is broken, we will see this test timeout because
201   * it is locked up.
202   * <p>First I need to set up some mocks for Server and RegionServerServices. I also need to
203   * set up a dodgy WAL that will throw an exception when we go to append to it.
204   */
205  @Test
206  public void testLockupWhenSyncInMiddleOfZigZagSetup() throws IOException {
207    // Mocked up server and regionserver services. Needed below.
208    RegionServerServices services = Mockito.mock(RegionServerServices.class);
209    Mockito.when(services.getConfiguration()).thenReturn(CONF);
210    Mockito.when(services.isStopped()).thenReturn(false);
211    Mockito.when(services.isAborted()).thenReturn(false);
212
213    // OK. Now I have my mocked up Server & RegionServerServices and dodgy WAL, go ahead with test.
214    FileSystem fs = FileSystem.get(CONF);
215    Path rootDir = new Path(dir + getName());
216    DodgyFSLog dodgyWAL = new DodgyFSLog(fs, rootDir, getName(), CONF);
217    dodgyWAL.init();
218    Path originalWAL = dodgyWAL.getCurrentFileName();
219    // I need a log roller running.
220    LogRoller logRoller = new LogRoller(services);
221    logRoller.addWAL(dodgyWAL);
222    // There is no 'stop' once a logRoller is running.. it just dies.
223    logRoller.start();
224    // Now get a region and start adding in edits.
225    final HRegion region = initHRegion(tableName, null, null, dodgyWAL);
226    byte [] bytes = Bytes.toBytes(getName());
227    NavigableMap<byte[], Integer> scopes = new TreeMap<>(
228        Bytes.BYTES_COMPARATOR);
229    scopes.put(COLUMN_FAMILY_BYTES, 0);
230    MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl();
231    try {
232      // First get something into memstore. Make a Put and then pull the Cell out of it. Will
233      // manage append and sync carefully in below to manufacture hang. We keep adding same
234      // edit. WAL subsystem doesn't care.
235      Put put = new Put(bytes);
236      put.addColumn(COLUMN_FAMILY_BYTES, Bytes.toBytes("1"), bytes);
237      WALKeyImpl key = new WALKeyImpl(region.getRegionInfo().getEncodedNameAsBytes(),
238        TableName.META_TABLE_NAME, System.currentTimeMillis(), mvcc, scopes);
239      WALEdit edit = new WALEdit();
240      CellScanner CellScanner = put.cellScanner();
241      assertTrue(CellScanner.advance());
242      edit.add(CellScanner.current());
243      // Put something in memstore and out in the WAL. Do a big number of appends so we push
244      // out other side of the ringbuffer. If small numbers, stuff doesn't make it to WAL
245      for (int i = 0; i < 1000; i++) {
246        region.put(put);
247      }
248      // Set it so we start throwing exceptions.
249      LOG.info("SET throwing of exception on append");
250      dodgyWAL.throwException = true;
251      // This append provokes a WAL roll request
252      dodgyWAL.appendData(region.getRegionInfo(), key, edit);
253      boolean exception = false;
254      try {
255        dodgyWAL.sync(false);
256      } catch (Exception e) {
257        exception = true;
258      }
259      assertTrue("Did not get sync exception", exception);
260
261      // Get a memstore flush going too so we have same hung profile as up in the issue over
262      // in HBASE-14317. Flush hangs trying to get sequenceid because the ringbuffer is held up
263      // by the zigzaglatch waiting on syncs to come home.
264      Thread t = new Thread ("Flusher") {
265        @Override
266        public void run() {
267          try {
268            if (region.getMemStoreDataSize() <= 0) {
269              throw new IOException("memstore size=" + region.getMemStoreDataSize());
270            }
271            region.flush(false);
272          } catch (IOException e) {
273            // Can fail trying to flush in middle of a roll. Not a failure. Will succeed later
274            // when roll completes.
275            LOG.info("In flush", e);
276          }
277          LOG.info("Exiting");
278        };
279      };
280      t.setDaemon(true);
281      t.start();
282      // Wait until
283      while (dodgyWAL.latch.getCount() > 0) {
284        Threads.sleep(1);
285      }
286      // Now assert I got a new WAL file put in place even though loads of errors above.
287      assertTrue(originalWAL != dodgyWAL.getCurrentFileName());
288      // Can I append to it?
289      dodgyWAL.throwException = false;
290      try {
291        region.put(put);
292      } catch (Exception e) {
293        LOG.info("In the put", e);
294      }
295    } finally {
296      // To stop logRoller, its server has to say it is stopped.
297      Mockito.when(services.isStopped()).thenReturn(true);
298      Closeables.close(logRoller, true);
299      try {
300        if (region != null) {
301          region.close();
302        }
303        if (dodgyWAL != null) {
304          dodgyWAL.close();
305        }
306      } catch (Exception e) {
307        LOG.info("On way out", e);
308      }
309    }
310  }
311
312  /**
313   *
314   * If below is broken, we will see this test timeout because RingBufferEventHandler was stuck in
315   * attainSafePoint. Everyone will wait for sync to finish forever. See HBASE-14317.
316   */
317  @Test
318  public void testRingBufferEventHandlerStuckWhenSyncFailed()
319    throws IOException, InterruptedException {
320
321    // A WAL that we can have throw exceptions and slow FSHLog.replaceWriter down
322    class DodgyFSLog extends FSHLog {
323
324      private volatile boolean zigZagCreated = false;
325
326      public DodgyFSLog(FileSystem fs, Path root, String logDir, Configuration conf)
327        throws IOException {
328        super(fs, root, logDir, conf);
329      }
330
331      @Override
332      protected void afterCreatingZigZagLatch() {
333        zigZagCreated = true;
334        // Sleep a while to wait for RingBufferEventHandler to get stuck first.
335        try {
336          Thread.sleep(3000);
337        } catch (InterruptedException ignore) {
338        }
339      }
340
341      @Override
342      protected long getSequenceOnRingBuffer() {
343        return super.getSequenceOnRingBuffer();
344      }
345
346      protected void publishSyncOnRingBufferAndBlock(long sequence) {
347        try {
348          super.blockOnSync(super.publishSyncOnRingBuffer(sequence, false));
349          Assert.fail("Expect an IOException here.");
350        } catch (IOException ignore) {
351          // Here, we will get an IOException.
352        }
353      }
354
355      @Override
356      protected Writer createWriterInstance(Path path) throws IOException {
357        final Writer w = super.createWriterInstance(path);
358        return new Writer() {
359          @Override
360          public void close() throws IOException {
361            w.close();
362          }
363
364          @Override
365          public void sync(boolean forceSync) throws IOException {
366            throw new IOException("FAKE! Failed to replace a bad datanode...SYNC");
367          }
368
369          @Override
370          public void append(Entry entry) throws IOException {
371            w.append(entry);
372          }
373
374          @Override
375          public long getLength() {
376            return w.getLength();
377          }
378        };
379      }
380    }
381
382    // Mocked up server and regionserver services. Needed below.
383    RegionServerServices services = Mockito.mock(RegionServerServices.class);
384    Mockito.when(services.getConfiguration()).thenReturn(CONF);
385    Mockito.when(services.isStopped()).thenReturn(false);
386    Mockito.when(services.isAborted()).thenReturn(false);
387
388    // OK. Now I have my mocked up Server & RegionServerServices and dodgy WAL, go ahead with test.
389    FileSystem fs = FileSystem.get(CONF);
390    Path rootDir = new Path(dir + getName());
391    final DodgyFSLog dodgyWAL = new DodgyFSLog(fs, rootDir, getName(), CONF);
392    dodgyWAL.init();
393    // I need a log roller running.
394    LogRoller logRoller = new LogRoller(services);
395    logRoller.addWAL(dodgyWAL);
396    // There is no 'stop' once a logRoller is running.. it just dies.
397    logRoller.start();
398
399    try {
400      final long seqForSync = dodgyWAL.getSequenceOnRingBuffer();
401
402      // This call provokes a WAL roll, and we will get a new RingBufferEventHandler.ZigZagLatch
403      // in LogRoller.
404      // After creating ZigZagLatch, RingBufferEventHandler would get stuck due to sync event,
405      // as long as HBASE-14317 hasn't be fixed.
406      LOG.info("Trigger log roll for creating a ZigZagLatch.");
407      logRoller.requestRollAll();
408
409      while (!dodgyWAL.zigZagCreated) {
410        Thread.sleep(10);
411      }
412
413      // Send a sync event for RingBufferEventHandler,
414      // and it gets blocked in RingBufferEventHandler.attainSafePoint
415      LOG.info("Send sync for RingBufferEventHandler");
416      Thread syncThread = new Thread() {
417        @Override
418        public void run() {
419          dodgyWAL.publishSyncOnRingBufferAndBlock(seqForSync);
420        }
421      };
422      // Sync in another thread to avoid reset SyncFuture again.
423      syncThread.start();
424      syncThread.join();
425
426      try {
427        LOG.info("Call sync for testing whether RingBufferEventHandler is hanging.");
428        dodgyWAL.sync(false); // Should not get a hang here, otherwise we will see timeout in this test.
429        Assert.fail("Expect an IOException here.");
430      } catch (IOException ignore) {
431      }
432
433    } finally {
434      // To stop logRoller, its server has to say it is stopped.
435      Mockito.when(services.isStopped()).thenReturn(true);
436      if (logRoller != null) {
437        logRoller.close();
438      }
439      if (dodgyWAL != null) {
440        dodgyWAL.close();
441      }
442    }
443  }
444
445
446  static class DummyServer implements Server {
447    private Configuration conf;
448    private String serverName;
449    private boolean isAborted = false;
450
451    public DummyServer(Configuration conf, String serverName) {
452      this.conf = conf;
453      this.serverName = serverName;
454    }
455
456    @Override
457    public Configuration getConfiguration() {
458      return conf;
459    }
460
461    @Override
462    public ZKWatcher getZooKeeper() {
463      return null;
464    }
465
466    @Override
467    public CoordinatedStateManager getCoordinatedStateManager() {
468      return null;
469    }
470
471    @Override
472    public ClusterConnection getConnection() {
473      return null;
474    }
475
476    @Override
477    public MetaTableLocator getMetaTableLocator() {
478      return null;
479    }
480
481    @Override
482    public ServerName getServerName() {
483      return ServerName.valueOf(this.serverName);
484    }
485
486    @Override
487    public void abort(String why, Throwable e) {
488      LOG.info("Aborting " + serverName);
489      this.isAborted = true;
490    }
491
492    @Override
493    public boolean isAborted() {
494      return this.isAborted;
495    }
496
497    @Override
498    public void stop(String why) {
499      this.isAborted = true;
500    }
501
502    @Override
503    public boolean isStopped() {
504      return this.isAborted;
505    }
506
507    @Override
508    public ChoreService getChoreService() {
509      return null;
510    }
511
512    @Override
513    public ClusterConnection getClusterConnection() {
514      return null;
515    }
516
517    @Override
518    public FileSystem getFileSystem() {
519      return null;
520    }
521
522    @Override
523    public boolean isStopping() {
524      return false;
525    }
526
527    @Override
528    public Connection createConnection(Configuration conf) throws IOException {
529      return null;
530    }
531  }
532
533  static class DummyWALActionsListener implements WALActionsListener {
534
535    @Override
536    public void visitLogEntryBeforeWrite(WALKey logKey, WALEdit logEdit)
537        throws IOException {
538      if (logKey.getTableName().getNameAsString().equalsIgnoreCase("sleep")) {
539        try {
540          Thread.sleep(1000);
541        } catch (InterruptedException e) {
542          e.printStackTrace();
543        }
544      }
545      if (logKey.getTableName().getNameAsString()
546          .equalsIgnoreCase("DamagedWALException")) {
547        throw new DamagedWALException("Failed appending");
548      }
549    }
550
551  }
552
553  /**
554   * @return A region on which you must call {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)}
555   *         when done.
556   */
557  private static HRegion initHRegion(TableName tableName, byte[] startKey, byte[] stopKey, WAL wal)
558      throws IOException {
559    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
560    return TEST_UTIL.createLocalHRegion(tableName, startKey, stopKey, false, Durability.SYNC_WAL,
561      wal, COLUMN_FAMILY_BYTES);
562  }
563}