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