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}