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.assertFalse; 022import static org.junit.Assert.assertNotNull; 023import static org.junit.Assert.assertTrue; 024 025import java.io.EOFException; 026import java.io.IOException; 027import java.io.InterruptedIOException; 028import java.util.ArrayList; 029import java.util.HashSet; 030import java.util.List; 031import java.util.Set; 032import java.util.concurrent.atomic.AtomicBoolean; 033import org.apache.hadoop.conf.Configuration; 034import org.apache.hadoop.fs.Path; 035import org.apache.hadoop.hbase.Cell; 036import org.apache.hadoop.hbase.HBaseClassTestRule; 037import org.apache.hadoop.hbase.HConstants; 038import org.apache.hadoop.hbase.TableName; 039import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; 040import org.apache.hadoop.hbase.client.Put; 041import org.apache.hadoop.hbase.client.RegionInfo; 042import org.apache.hadoop.hbase.client.Result; 043import org.apache.hadoop.hbase.client.ResultScanner; 044import org.apache.hadoop.hbase.client.Scan; 045import org.apache.hadoop.hbase.client.Table; 046import org.apache.hadoop.hbase.client.TableDescriptor; 047import org.apache.hadoop.hbase.client.TableDescriptorBuilder; 048import org.apache.hadoop.hbase.fs.HFileSystem; 049import org.apache.hadoop.hbase.regionserver.HRegion; 050import org.apache.hadoop.hbase.testclassification.LargeTests; 051import org.apache.hadoop.hbase.testclassification.VerySlowRegionServerTests; 052import org.apache.hadoop.hbase.util.Bytes; 053import org.apache.hadoop.hbase.util.CommonFSUtils; 054import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; 055import org.apache.hadoop.hbase.util.JVMClusterUtil; 056import org.apache.hadoop.hbase.util.RecoverLeaseFSUtils; 057import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; 058import org.apache.hadoop.hbase.wal.FSHLogProvider; 059import org.apache.hadoop.hbase.wal.WAL; 060import org.apache.hadoop.hbase.wal.WALFactory; 061import org.apache.hadoop.hbase.wal.WALStreamReader; 062import org.apache.hadoop.hdfs.protocol.DatanodeInfo; 063import org.apache.hadoop.hdfs.server.datanode.DataNode; 064import org.junit.BeforeClass; 065import org.junit.ClassRule; 066import org.junit.Test; 067import org.junit.experimental.categories.Category; 068import org.slf4j.Logger; 069import org.slf4j.LoggerFactory; 070 071@Category({ VerySlowRegionServerTests.class, LargeTests.class }) 072public class TestLogRolling extends AbstractTestLogRolling { 073 074 @ClassRule 075 public static final HBaseClassTestRule CLASS_RULE = 076 HBaseClassTestRule.forClass(TestLogRolling.class); 077 078 private static final Logger LOG = LoggerFactory.getLogger(TestLogRolling.class); 079 080 @BeforeClass 081 public static void setUpBeforeClass() throws Exception { 082 // TODO: testLogRollOnDatanodeDeath fails if short circuit reads are on under the hadoop2 083 // profile. See HBASE-9337 for related issues. 084 System.setProperty("hbase.tests.use.shortcircuit.reads", "false"); 085 086 /**** configuration for testLogRollOnDatanodeDeath ****/ 087 // lower the namenode & datanode heartbeat so the namenode 088 // quickly detects datanode failures 089 Configuration conf = TEST_UTIL.getConfiguration(); 090 conf.setInt("dfs.namenode.heartbeat.recheck-interval", 5000); 091 conf.setInt("dfs.heartbeat.interval", 1); 092 // the namenode might still try to choose the recently-dead datanode 093 // for a pipeline, so try to a new pipeline multiple times 094 conf.setInt("dfs.client.block.write.retries", 30); 095 conf.setInt("hbase.regionserver.hlog.tolerable.lowreplication", 2); 096 conf.setInt("hbase.regionserver.hlog.lowreplication.rolllimit", 3); 097 conf.set(WALFactory.WAL_PROVIDER, "filesystem"); 098 AbstractTestLogRolling.setUpBeforeClass(); 099 } 100 101 public static class SlowSyncLogWriter extends ProtobufLogWriter { 102 @Override 103 public void sync(boolean forceSync) throws IOException { 104 try { 105 Thread.sleep(syncLatencyMillis); 106 } catch (InterruptedException e) { 107 InterruptedIOException ex = new InterruptedIOException(); 108 ex.initCause(e); 109 throw ex; 110 } 111 super.sync(forceSync); 112 } 113 } 114 115 @Override 116 protected void setSlowLogWriter(Configuration conf) { 117 conf.set(FSHLogProvider.WRITER_IMPL, SlowSyncLogWriter.class.getName()); 118 } 119 120 @Override 121 protected void setDefaultLogWriter(Configuration conf) { 122 conf.set(FSHLogProvider.WRITER_IMPL, ProtobufLogWriter.class.getName()); 123 } 124 125 void batchWriteAndWait(Table table, final FSHLog log, int start, boolean expect, int timeout) 126 throws IOException { 127 for (int i = 0; i < 10; i++) { 128 Put put = new Put(Bytes.toBytes("row" + String.format("%1$04d", (start + i)))); 129 put.addColumn(HConstants.CATALOG_FAMILY, null, value); 130 table.put(put); 131 } 132 Put tmpPut = new Put(Bytes.toBytes("tmprow")); 133 tmpPut.addColumn(HConstants.CATALOG_FAMILY, null, value); 134 long startTime = EnvironmentEdgeManager.currentTime(); 135 long remaining = timeout; 136 while (remaining > 0) { 137 if (log.isLowReplicationRollEnabled() == expect) { 138 break; 139 } else { 140 // Trigger calling FSHlog#checkLowReplication() 141 table.put(tmpPut); 142 try { 143 Thread.sleep(200); 144 } catch (InterruptedException e) { 145 // continue 146 } 147 remaining = timeout - (EnvironmentEdgeManager.currentTime() - startTime); 148 } 149 } 150 } 151 152 @Test 153 public void testSlowSyncLogRolling() throws Exception { 154 // Create the test table 155 TableDescriptor desc = TableDescriptorBuilder.newBuilder(TableName.valueOf(getName())) 156 .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build(); 157 admin.createTable(desc); 158 try (Table table = TEST_UTIL.getConnection().getTable(desc.getTableName())) { 159 server = TEST_UTIL.getRSForFirstRegionInTable(desc.getTableName()); 160 RegionInfo region = server.getRegions(desc.getTableName()).get(0).getRegionInfo(); 161 final AbstractFSWAL<?> log = getWALAndRegisterSlowSyncHook(region); 162 163 // Set default log writer, no additional latency to any sync on the hlog. 164 checkSlowSync(log, table, -1, 10, false); 165 166 // Adds 200 ms of latency to any sync on the hlog. This should be more than sufficient to 167 // trigger slow sync warnings. 168 // Write some data. 169 // We need to write at least 5 times, but double it. We should only request 170 // a SLOW_SYNC roll once in the current interval. 171 checkSlowSync(log, table, 200, 10, true); 172 173 // Adds 5000 ms of latency to any sync on the hlog. This will trip the other threshold. 174 // Write some data. Should only take one sync. 175 checkSlowSync(log, table, 5000, 1, true); 176 177 // Set default log writer, no additional latency to any sync on the hlog. 178 checkSlowSync(log, table, -1, 10, false); 179 } 180 } 181 182 /** 183 * Tests that logs are rolled upon detecting datanode death Requires an HDFS jar with HDFS-826 & 184 * syncFs() support (HDFS-200) 185 */ 186 @Test 187 public void testLogRollOnDatanodeDeath() throws Exception { 188 TEST_UTIL.ensureSomeRegionServersAvailable(2); 189 assertTrue("This test requires WAL file replication set to 2.", 190 fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()) == 2); 191 LOG.info("Replication=" + fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS())); 192 193 this.server = cluster.getRegionServer(0); 194 195 // Create the test table and open it 196 TableDescriptor desc = TableDescriptorBuilder.newBuilder(TableName.valueOf(getName())) 197 .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build(); 198 199 admin.createTable(desc); 200 Table table = TEST_UTIL.getConnection().getTable(desc.getTableName()); 201 202 server = TEST_UTIL.getRSForFirstRegionInTable(desc.getTableName()); 203 RegionInfo region = server.getRegions(desc.getTableName()).get(0).getRegionInfo(); 204 final FSHLog log = (FSHLog) server.getWAL(region); 205 final AtomicBoolean lowReplicationHookCalled = new AtomicBoolean(false); 206 207 log.registerWALActionsListener(new WALActionsListener() { 208 @Override 209 public void logRollRequested(WALActionsListener.RollRequestReason reason) { 210 switch (reason) { 211 case LOW_REPLICATION: 212 lowReplicationHookCalled.lazySet(true); 213 break; 214 default: 215 break; 216 } 217 } 218 }); 219 220 // add up the datanode count, to ensure proper replication when we kill 1 221 // This function is synchronous; when it returns, the dfs cluster is active 222 // We start 3 servers and then stop 2 to avoid a directory naming conflict 223 // when we stop/start a namenode later, as mentioned in HBASE-5163 224 List<DataNode> existingNodes = dfsCluster.getDataNodes(); 225 int numDataNodes = 3; 226 dfsCluster.startDataNodes(TEST_UTIL.getConfiguration(), numDataNodes, true, null, null); 227 List<DataNode> allNodes = dfsCluster.getDataNodes(); 228 for (int i = allNodes.size() - 1; i >= 0; i--) { 229 if (existingNodes.contains(allNodes.get(i))) { 230 dfsCluster.stopDataNode(i); 231 } 232 } 233 234 assertTrue( 235 "DataNodes " + dfsCluster.getDataNodes().size() + " default replication " 236 + fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()), 237 dfsCluster.getDataNodes().size() 238 >= fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()) + 1); 239 240 writeData(table, 2); 241 242 long curTime = EnvironmentEdgeManager.currentTime(); 243 LOG.info("log.getCurrentFileName(): " + log.getCurrentFileName()); 244 long oldFilenum = AbstractFSWALProvider.extractFileNumFromWAL(log); 245 assertTrue("Log should have a timestamp older than now", 246 curTime > oldFilenum && oldFilenum != -1); 247 248 assertTrue("The log shouldn't have rolled yet", 249 oldFilenum == AbstractFSWALProvider.extractFileNumFromWAL(log)); 250 final DatanodeInfo[] pipeline = log.getPipeline(); 251 assertTrue(pipeline.length == fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS())); 252 253 // kill a datanode in the pipeline to force a log roll on the next sync() 254 // This function is synchronous, when it returns the node is killed. 255 assertTrue(dfsCluster.stopDataNode(pipeline[0].getName()) != null); 256 257 // this write should succeed, but trigger a log roll 258 writeData(table, 2); 259 long newFilenum = AbstractFSWALProvider.extractFileNumFromWAL(log); 260 261 assertTrue("Missing datanode should've triggered a log roll", 262 newFilenum > oldFilenum && newFilenum > curTime); 263 264 assertTrue("The log rolling hook should have been called with the low replication flag", 265 lowReplicationHookCalled.get()); 266 267 // write some more log data (this should use a new hdfs_out) 268 writeData(table, 3); 269 assertTrue("The log should not roll again.", 270 AbstractFSWALProvider.extractFileNumFromWAL(log) == newFilenum); 271 // kill another datanode in the pipeline, so the replicas will be lower than 272 // the configured value 2. 273 assertTrue(dfsCluster.stopDataNode(pipeline[1].getName()) != null); 274 275 batchWriteAndWait(table, log, 3, false, 14000); 276 int replication = log.getLogReplication(); 277 assertTrue("LowReplication Roller should've been disabled, current replication=" + replication, 278 !log.isLowReplicationRollEnabled()); 279 280 dfsCluster.startDataNodes(TEST_UTIL.getConfiguration(), 1, true, null, null); 281 282 // Force roll writer. The new log file will have the default replications, 283 // and the LowReplication Roller will be enabled. 284 log.rollWriter(true); 285 batchWriteAndWait(table, log, 13, true, 10000); 286 replication = log.getLogReplication(); 287 assertTrue("New log file should have the default replication instead of " + replication, 288 replication == fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS())); 289 assertTrue("LowReplication Roller should've been enabled", log.isLowReplicationRollEnabled()); 290 } 291 292 /** 293 * Test that WAL is rolled when all data nodes in the pipeline have been restarted. 294 */ 295 @Test 296 public void testLogRollOnPipelineRestart() throws Exception { 297 LOG.info("Starting testLogRollOnPipelineRestart"); 298 assertTrue("This test requires WAL file replication.", 299 fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()) > 1); 300 LOG.info("Replication=" + fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS())); 301 // When the hbase:meta table can be opened, the region servers are running 302 Table t = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME); 303 try { 304 this.server = cluster.getRegionServer(0); 305 306 // Create the test table and open it 307 TableDescriptor desc = TableDescriptorBuilder.newBuilder(TableName.valueOf(getName())) 308 .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build(); 309 310 admin.createTable(desc); 311 Table table = TEST_UTIL.getConnection().getTable(desc.getTableName()); 312 313 server = TEST_UTIL.getRSForFirstRegionInTable(desc.getTableName()); 314 RegionInfo region = server.getRegions(desc.getTableName()).get(0).getRegionInfo(); 315 final WAL log = server.getWAL(region); 316 final List<Path> paths = new ArrayList<>(1); 317 final List<Integer> preLogRolledCalled = new ArrayList<>(); 318 319 paths.add(AbstractFSWALProvider.getCurrentFileName(log)); 320 log.registerWALActionsListener(new WALActionsListener() { 321 322 @Override 323 public void preLogRoll(Path oldFile, Path newFile) { 324 LOG.debug("preLogRoll: oldFile=" + oldFile + " newFile=" + newFile); 325 preLogRolledCalled.add(new Integer(1)); 326 } 327 328 @Override 329 public void postLogRoll(Path oldFile, Path newFile) { 330 paths.add(newFile); 331 } 332 }); 333 334 writeData(table, 1002); 335 336 long curTime = EnvironmentEdgeManager.currentTime(); 337 LOG.info("log.getCurrentFileName()): " + AbstractFSWALProvider.getCurrentFileName(log)); 338 long oldFilenum = AbstractFSWALProvider.extractFileNumFromWAL(log); 339 assertTrue("Log should have a timestamp older than now", 340 curTime > oldFilenum && oldFilenum != -1); 341 342 assertTrue("The log shouldn't have rolled yet", 343 oldFilenum == AbstractFSWALProvider.extractFileNumFromWAL(log)); 344 345 // roll all datanodes in the pipeline 346 dfsCluster.restartDataNodes(); 347 Thread.sleep(1000); 348 dfsCluster.waitActive(); 349 LOG.info("Data Nodes restarted"); 350 validateData(table, 1002); 351 352 // this write should succeed, but trigger a log roll 353 writeData(table, 1003); 354 long newFilenum = AbstractFSWALProvider.extractFileNumFromWAL(log); 355 356 assertTrue("Missing datanode should've triggered a log roll", 357 newFilenum > oldFilenum && newFilenum > curTime); 358 validateData(table, 1003); 359 360 writeData(table, 1004); 361 362 // roll all datanode again 363 dfsCluster.restartDataNodes(); 364 Thread.sleep(1000); 365 dfsCluster.waitActive(); 366 LOG.info("Data Nodes restarted"); 367 validateData(table, 1004); 368 369 // this write should succeed, but trigger a log roll 370 writeData(table, 1005); 371 372 // force a log roll to read back and verify previously written logs 373 log.rollWriter(true); 374 assertTrue("preLogRolledCalled has size of " + preLogRolledCalled.size(), 375 preLogRolledCalled.size() >= 1); 376 377 // read back the data written 378 Set<String> loggedRows = new HashSet<>(); 379 for (Path p : paths) { 380 LOG.debug("recovering lease for " + p); 381 RecoverLeaseFSUtils.recoverFileLease(((HFileSystem) fs).getBackingFs(), p, 382 TEST_UTIL.getConfiguration(), null); 383 384 LOG.debug("Reading WAL " + CommonFSUtils.getPath(p)); 385 try (WALStreamReader reader = 386 WALFactory.createStreamReader(fs, p, TEST_UTIL.getConfiguration())) { 387 WAL.Entry entry; 388 while ((entry = reader.next()) != null) { 389 LOG.debug("#" + entry.getKey().getSequenceId() + ": " + entry.getEdit().getCells()); 390 for (Cell cell : entry.getEdit().getCells()) { 391 loggedRows.add( 392 Bytes.toStringBinary(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength())); 393 } 394 } 395 } catch (EOFException e) { 396 LOG.debug("EOF reading file " + CommonFSUtils.getPath(p)); 397 } 398 } 399 400 // verify the written rows are there 401 assertTrue(loggedRows.contains("row1002")); 402 assertTrue(loggedRows.contains("row1003")); 403 assertTrue(loggedRows.contains("row1004")); 404 assertTrue(loggedRows.contains("row1005")); 405 406 // flush all regions 407 for (HRegion r : server.getOnlineRegionsLocalContext()) { 408 try { 409 r.flush(true); 410 } catch (Exception e) { 411 // This try/catch was added by HBASE-14317. It is needed 412 // because this issue tightened up the semantic such that 413 // a failed append could not be followed by a successful 414 // sync. What is coming out here is a failed sync, a sync 415 // that used to 'pass'. 416 LOG.info(e.toString(), e); 417 } 418 } 419 420 ResultScanner scanner = table.getScanner(new Scan()); 421 try { 422 for (int i = 2; i <= 5; i++) { 423 Result r = scanner.next(); 424 assertNotNull(r); 425 assertFalse(r.isEmpty()); 426 assertEquals("row100" + i, Bytes.toString(r.getRow())); 427 } 428 } finally { 429 scanner.close(); 430 } 431 432 // verify that no region servers aborted 433 for (JVMClusterUtil.RegionServerThread rsThread : TEST_UTIL.getHBaseCluster() 434 .getRegionServerThreads()) { 435 assertFalse(rsThread.getRegionServer().isAborted()); 436 } 437 } finally { 438 if (t != null) t.close(); 439 } 440 } 441 442}