From 7e536e96ab099570916164970f5bee359b5ece84 Mon Sep 17 00:00:00 2001 From: comnetwork Date: Fri, 22 Jul 2022 10:16:29 +0800 Subject: [PATCH 1/9] HBASE-27230 RegionServer should be aborted when WAL.sync throws TimeoutIOException --- .../hadoop/hbase/regionserver/HRegion.java | 40 ++- .../hadoop/hbase/regionserver/HStore.java | 3 +- .../hbase/regionserver/wal/AbstractFSWAL.java | 4 +- .../wal/WALSyncTimeoutIOException.java | 47 ++++ .../hadoop/hbase/wal/AsyncFSWALProvider.java | 4 +- .../java/org/apache/hadoop/hbase/wal/WAL.java | 5 + .../apache/hadoop/hbase/wal/WALFactory.java | 7 + .../wal/TestWALSyncTimeoutException.java | 259 ++++++++++++++++++ 8 files changed, 355 insertions(+), 14 deletions(-) create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALSyncTimeoutIOException.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALSyncTimeoutException.java diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java index 0857ace1f663..3a53fa0b11b4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -153,6 +153,7 @@ import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController; import org.apache.hadoop.hbase.regionserver.throttle.StoreHotnessProtector; import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController; +import org.apache.hadoop.hbase.regionserver.wal.WALSyncTimeoutIOException; import org.apache.hadoop.hbase.regionserver.wal.WALUtil; import org.apache.hadoop.hbase.replication.ReplicationUtils; import org.apache.hadoop.hbase.replication.regionserver.ReplicationObserver; @@ -1367,8 +1368,12 @@ public RegionInfo getRegionInfo() { return this.fs.getRegionInfo(); } - /** Returns Instance of {@link RegionServerServices} used by this HRegion. Can be null. */ - RegionServerServices getRegionServerServices() { + /** + * Returns Instance of {@link RegionServerServices} used by this HRegion. Can be null. + **/ + @RestrictedApi(explanation = "Should only be called in tests", link = "", + allowedOnPath = ".*/src/test/.*") + public RegionServerServices getRegionServerServices() { return this.rsServices; } @@ -2863,7 +2868,7 @@ private boolean writeCanNotFlushMarkerToWAL(WriteEntry flushOpSeqIdMVCCEntry, WA if (sink != null && !writeFlushWalMarker) { /** * Here for replication to secondary region replica could use {@link FlushAction#CANNOT_FLUSH} - * to recover writeFlushWalMarker is false, we create {@link WALEdit} for + * to recover when writeFlushWalMarker is false, we create {@link WALEdit} for * {@link FlushDescriptor} and attach the {@link RegionReplicationSink#add} to the * flushOpSeqIdMVCCEntry,see HBASE-26960 for more details. */ @@ -3694,7 +3699,7 @@ public void doPostOpCleanupForMiniBatch( * @param familyMap Map of Cells by family */ protected void applyFamilyMapToMemStore(Map> familyMap, - MemStoreSizing memstoreAccounting) throws IOException { + MemStoreSizing memstoreAccounting) { for (Map.Entry> e : familyMap.entrySet()) { byte[] family = e.getKey(); List cells = e.getValue(); @@ -5231,7 +5236,7 @@ public void setReadsEnabled(boolean readsEnabled) { * scenario but that do not make sense otherwise. */ private void applyToMemStore(HStore store, List cells, boolean delta, - MemStoreSizing memstoreAccounting) throws IOException { + MemStoreSizing memstoreAccounting) { // Any change in how we update Store/MemStore needs to also be done in other applyToMemStore!!!! boolean upsert = delta && store.getColumnFamilyDescriptor().getMaxVersions() == 1; if (upsert) { @@ -8037,16 +8042,35 @@ private WriteEntry doWALAppend(WALEdit walEdit, BatchOperation batchOp, try { long txid = this.wal.appendData(this.getRegionInfo(), walKey, walEdit); WriteEntry writeEntry = walKey.getWriteEntry(); - this.attachRegionReplicationInWALAppend(batchOp, miniBatchOp, walKey, walEdit, writeEntry); // Call sync on our edit. if (txid != 0) { sync(txid, batchOp.durability); } + /** + * If above {@link HRegion#sync} throws Exception, the RegionServer should be aborted and + * following {@link BatchOperation#writeMiniBatchOperationsToMemStore} will not be executed, + * so there is no need to replicate to secondary replica, for this reason here we attach the + * region replication action after the {@link HRegion#sync} is successful. + */ + this.attachRegionReplicationInWALAppend(batchOp, miniBatchOp, walKey, walEdit, writeEntry); return writeEntry; } catch (IOException ioe) { if (walKey.getWriteEntry() != null) { mvcc.complete(walKey.getWriteEntry()); } + + /** + * If {@link WAL#sync} get a timeout exception, the only correct way is to abort the region + * server, as the design of {@link WAL#sync}, is to succeed or die, there is no 'failure'. It + * is usually not a big deal is because we set a very large default value(5 minutes) for + * {@link AbstractFSWAL#WAL_SYNC_TIMEOUT_MS}, usually the WAL system will abort the region + * server if it can not finish the sync within 5 minutes. + */ + if (ioe instanceof WALSyncTimeoutIOException) { + if (rsServices != null) { + rsServices.abort("WAL sync timeout,forcing server shutdown", ioe); + } + } throw ioe; } } @@ -8057,7 +8081,7 @@ private WriteEntry doWALAppend(WALEdit walEdit, BatchOperation batchOp, */ private void attachRegionReplicationInWALAppend(BatchOperation batchOp, MiniBatchOperationInProgress miniBatchOp, WALKeyImpl walKey, WALEdit walEdit, - WriteEntry writeEntry) throws IOException { + WriteEntry writeEntry) { if (!regionReplicationSink.isPresent()) { return; } @@ -8086,7 +8110,7 @@ private void attachRegionReplicationInWALAppend(BatchOperation batchOp, * replica. */ private void doAttachReplicateRegionReplicaAction(WALKeyImpl walKey, WALEdit walEdit, - WriteEntry writeEntry) throws IOException { + WriteEntry writeEntry) { if (walEdit == null || walEdit.isEmpty()) { return; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java index 54e18d619822..300ff4f818c7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java @@ -1880,8 +1880,7 @@ public long getSmallestReadPoint() { * across all of them. * @param readpoint readpoint below which we can safely remove duplicate KVs */ - public void upsert(Iterable cells, long readpoint, MemStoreSizing memstoreSizing) - throws IOException { + public void upsert(Iterable cells, long readpoint, MemStoreSizing memstoreSizing) { this.storeEngine.readLock(); try { this.memstore.upsert(cells, readpoint, memstoreSizing); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java index c05f71011721..77708c39590d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java @@ -142,7 +142,7 @@ public abstract class AbstractFSWAL implements WAL { "hbase.regionserver.wal.slowsync.roll.interval.ms"; protected static final int DEFAULT_SLOW_SYNC_ROLL_INTERVAL_MS = 60 * 1000; // in ms, 1 minute - protected static final String WAL_SYNC_TIMEOUT_MS = "hbase.regionserver.wal.sync.timeout"; + public static final String WAL_SYNC_TIMEOUT_MS = "hbase.regionserver.wal.sync.timeout"; protected static final int DEFAULT_WAL_SYNC_TIMEOUT_MS = 5 * 60 * 1000; // in ms, 5min public static final String WAL_ROLL_MULTIPLIER = "hbase.regionserver.logroll.multiplier"; @@ -881,7 +881,7 @@ protected final void blockOnSync(SyncFuture syncFuture) throws IOException { } } } catch (TimeoutIOException tioe) { - throw tioe; + throw new WALSyncTimeoutIOException(tioe); } catch (InterruptedException ie) { LOG.warn("Interrupted", ie); throw convertInterruptedExceptionToIOException(ie); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALSyncTimeoutIOException.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALSyncTimeoutIOException.java new file mode 100644 index 000000000000..b12a66d5ddff --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALSyncTimeoutIOException.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.regionserver.wal; + +import org.apache.hadoop.hbase.HBaseIOException; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * Thrown when {@link WAL#sync} timeout. + */ +@SuppressWarnings("serial") +@InterfaceAudience.Private +public class WALSyncTimeoutIOException extends HBaseIOException { + + public WALSyncTimeoutIOException() { + super(); + } + + public WALSyncTimeoutIOException(String message, Throwable cause) { + super(message, cause); + } + + public WALSyncTimeoutIOException(String message) { + super(message); + } + + public WALSyncTimeoutIOException(Throwable cause) { + super(cause); + } + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java index 2fe0402ca4df..02e62a4de407 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java @@ -62,9 +62,9 @@ void init(FileSystem fs, Path path, Configuration c, boolean overwritable, long StreamSlowMonitor monitor) throws IOException, CommonFSUtils.StreamLacksCapabilityException; } - private EventLoopGroup eventLoopGroup; + protected EventLoopGroup eventLoopGroup; - private Class channelClass; + protected Class channelClass; @Override protected AsyncFSWAL createWAL() throws IOException { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java index 390a9251751a..21accbcf99f4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java @@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException; import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener; import org.apache.hadoop.hbase.regionserver.wal.WALCoprocessorHost; +import org.apache.hadoop.hbase.regionserver.wal.WALSyncTimeoutIOException; import org.apache.hadoop.hbase.replication.regionserver.WALFileLengthProvider; import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceStability; @@ -133,18 +134,21 @@ void updateStore(byte[] encodedRegionName, byte[] familyName, Long sequenceid, /** * Sync what we have in the WAL. + * @throws when timeout, it would throw {@link WALSyncTimeoutIOException}. */ void sync() throws IOException; /** * Sync the WAL if the txId was not already sync'd. * @param txid Transaction id to sync to. + * @throws when timeout, it would throw {@link WALSyncTimeoutIOException}. */ void sync(long txid) throws IOException; /** * @param forceSync Flag to force sync rather than flushing to the buffer. Example - Hadoop hflush * vs hsync. + * @throws when timeout, it would throw {@link WALSyncTimeoutIOException}. */ default void sync(boolean forceSync) throws IOException { sync(); @@ -154,6 +158,7 @@ default void sync(boolean forceSync) throws IOException { * @param txid Transaction id to sync to. * @param forceSync Flag to force sync rather than flushing to the buffer. Example - Hadoop hflush * vs hsync. + * @throws when timeout, it would throw {@link WALSyncTimeoutIOException}. */ default void sync(long txid, boolean forceSync) throws IOException { sync(txid); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java index 3a019ce84368..9136099defdc 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hbase.wal; +import com.google.errorprone.annotations.RestrictedApi; import java.io.IOException; import java.io.InterruptedIOException; import java.util.List; @@ -505,4 +506,10 @@ public final WALProvider getMetaWALProvider() { public ExcludeDatanodeManager getExcludeDatanodeManager() { return excludeDatanodeManager; } + + @RestrictedApi(explanation = "Should only be called in tests", link = "", + allowedOnPath = ".*/src/test/.*") + public String getFactoryId() { + return this.factoryId; + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALSyncTimeoutException.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALSyncTimeoutException.java new file mode 100644 index 000000000000..a956af84c214 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALSyncTimeoutException.java @@ -0,0 +1,259 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.regionserver.wal; + +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.io.IOException; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.Abortable; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.StartTestingClusterOption; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.RegionReplicaUtil; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.io.asyncfs.monitor.StreamSlowMonitor; +import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; +import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.regionserver.RegionServerServices; +import org.apache.hadoop.hbase.regionserver.regionreplication.RegionReplicationSink; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.CommonFSUtils; +import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil; +import org.apache.hadoop.hbase.wal.AsyncFSWALProvider; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALFactory; +import org.apache.hadoop.hbase.wal.WALKeyImpl; +import org.apache.hadoop.hbase.wal.WALProvider; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.mockito.Mockito; + +import org.apache.hbase.thirdparty.io.netty.channel.Channel; +import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup; + +@Category({ RegionServerTests.class, LargeTests.class }) +public class TestWALSyncTimeoutException { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestWALSyncTimeoutException.class); + + private static final byte[] FAMILY = Bytes.toBytes("family_test"); + + private static final byte[] QUAL = Bytes.toBytes("qualifier_test"); + + private static final HBaseTestingUtil HTU = new HBaseTestingUtil(); + private static final int NB_SERVERS = 2; + + private static TableName tableName = TableName.valueOf("TestWALSyncTimeoutException"); + private static volatile boolean testWALTimout = false; + private static final long timeoutMIlliseconds = 3000; + private static final String USER_THREAD_NAME = tableName.getNameAsString(); + + @BeforeClass + public static void setUp() throws Exception { + Configuration conf = HTU.getConfiguration(); + conf.setBoolean(ServerRegionReplicaUtil.REGION_REPLICA_REPLICATION_CONF_KEY, true); + conf.setClass(HConstants.REGION_IMPL, HRegionForTest.class, HRegion.class); + conf.setInt(RegionReplicationSink.RETRIES_NUMBER, 1); + conf.setLong(RegionReplicationSink.RPC_TIMEOUT_MS, 10 * 60 * 1000); + conf.setLong(RegionReplicationSink.OPERATION_TIMEOUT_MS, 20 * 60 * 1000); + conf.setLong(RegionReplicationSink.META_EDIT_RPC_TIMEOUT_MS, 10 * 60 * 1000); + conf.setLong(RegionReplicationSink.META_EDIT_OPERATION_TIMEOUT_MS, 20 * 60 * 1000); + conf.setBoolean(RegionReplicaUtil.REGION_REPLICA_WAIT_FOR_PRIMARY_FLUSH_CONF_KEY, false); + conf.setClass(WALFactory.WAL_PROVIDER, SlowAsyncFSWALProvider.class, WALProvider.class); + conf.setLong(AbstractFSWAL.WAL_SYNC_TIMEOUT_MS, timeoutMIlliseconds); + HTU.startMiniCluster(StartTestingClusterOption.builder().numRegionServers(NB_SERVERS).build()); + + } + + @AfterClass + public static void tearDown() throws Exception { + HTU.shutdownMiniCluster(); + } + + /** + * This test is for HBASE-27230. When {@link WAL#sync} timeout, it would throws + * {@link WALSyncTimeoutIOException},and when {@link HRegion#doWALAppend} catches this exception + * it aborts the RegionServer. + */ + @Test + public void testWALSyncWriteException() throws Exception { + final HRegionForTest[] regions = this.createTable(); + RegionReplicationSink regionReplicationSink = regions[0].getRegionReplicationSink().get(); + assertTrue(regionReplicationSink != null); + final AtomicInteger replicateCounter = new AtomicInteger(0); + setUpSpiedRegionReplicationSink(regionReplicationSink, regions[0], replicateCounter); + + String oldThreadName = Thread.currentThread().getName(); + Thread.currentThread().setName(USER_THREAD_NAME); + try { + byte[] rowKey1 = Bytes.toBytes(1); + byte[] value1 = Bytes.toBytes(3); + Thread.sleep(2000); + testWALTimout = true; + + /** + * The {@link WAL#sync} would timeout and throws {@link WALSyncTimeoutIOException},when + * {@link HRegion#doWALAppend} catches this exception it aborts the RegionServer. + */ + try { + regions[0].put(new Put(rowKey1).addColumn(FAMILY, QUAL, value1)); + fail(); + } catch (WALSyncTimeoutIOException e) { + assertTrue(e != null); + } + assertTrue(regions[0].getRegionServerServices().isAborted()); + assertTrue(replicateCounter.get() == 0); + } finally { + Thread.currentThread().setName(oldThreadName); + testWALTimout = false; + } + } + + private static RegionReplicationSink setUpSpiedRegionReplicationSink( + final RegionReplicationSink regionReplicationSink, final HRegionForTest primaryRegion, + final AtomicInteger counter) { + RegionReplicationSink spiedRegionReplicationSink = Mockito.spy(regionReplicationSink); + + Mockito.doAnswer((invocationOnMock) -> { + if (!testWALTimout || !USER_THREAD_NAME.equals(Thread.currentThread().getName())) { + invocationOnMock.callRealMethod(); + return null; + } + WALKeyImpl walKey = invocationOnMock.getArgument(0); + if (!walKey.getTableName().equals(tableName)) { + invocationOnMock.callRealMethod(); + return null; + } + counter.incrementAndGet(); + invocationOnMock.callRealMethod(); + return null; + }).when(spiedRegionReplicationSink).add(Mockito.any(), Mockito.any(), Mockito.any()); + primaryRegion.setRegionReplicationSink(spiedRegionReplicationSink); + return spiedRegionReplicationSink; + } + + private HRegionForTest[] createTable() throws Exception { + TableDescriptor tableDescriptor = + TableDescriptorBuilder.newBuilder(tableName).setRegionReplication(NB_SERVERS) + .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY)).build(); + HTU.getAdmin().createTable(tableDescriptor); + final HRegionForTest[] regions = new HRegionForTest[NB_SERVERS]; + for (int i = 0; i < NB_SERVERS; i++) { + HRegionServer rs = HTU.getMiniHBaseCluster().getRegionServer(i); + List onlineRegions = rs.getRegions(tableName); + for (HRegion region : onlineRegions) { + int replicaId = region.getRegionInfo().getReplicaId(); + assertTrue(regions[replicaId] == null); + regions[region.getRegionInfo().getReplicaId()] = (HRegionForTest) region; + } + } + for (HRegionForTest region : regions) { + assertNotNull(region); + } + return regions; + } + + public static final class HRegionForTest extends HRegion { + + public HRegionForTest(HRegionFileSystem fs, WAL wal, Configuration confParam, + TableDescriptor htd, RegionServerServices rsServices) { + super(fs, wal, confParam, htd, rsServices); + } + + @SuppressWarnings("deprecation") + public HRegionForTest(Path tableDir, WAL wal, FileSystem fs, Configuration confParam, + RegionInfo regionInfo, TableDescriptor htd, RegionServerServices rsServices) { + super(tableDir, wal, fs, confParam, regionInfo, htd, rsServices); + } + + public void setRegionReplicationSink(RegionReplicationSink regionReplicationSink) { + this.regionReplicationSink = Optional.of(regionReplicationSink); + } + + } + + public static class SlowAsyncFSWAL extends AsyncFSWAL { + + public SlowAsyncFSWAL(FileSystem fs, Abortable abortable, Path rootDir, String logDir, + String archiveDir, Configuration conf, List listeners, + boolean failIfWALExists, String prefix, String suffix, EventLoopGroup eventLoopGroup, + Class channelClass, StreamSlowMonitor monitor) + throws FailedLogCloseException, IOException { + super(fs, abortable, rootDir, logDir, archiveDir, conf, listeners, failIfWALExists, prefix, + suffix, eventLoopGroup, channelClass, monitor); + + } + + public SlowAsyncFSWAL(FileSystem fs, Path rootDir, String logDir, String archiveDir, + Configuration conf, List listeners, boolean failIfWALExists, + String prefix, String suffix, EventLoopGroup eventLoopGroup, + Class channelClass) throws FailedLogCloseException, IOException { + super(fs, rootDir, logDir, archiveDir, conf, listeners, failIfWALExists, prefix, suffix, + eventLoopGroup, channelClass); + + } + + @Override + protected void atHeadOfRingBufferEventHandlerAppend() { + if (testWALTimout) { + try { + Thread.sleep(timeoutMIlliseconds + 1000); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + super.atHeadOfRingBufferEventHandlerAppend(); + } + + } + + public static class SlowAsyncFSWALProvider extends AsyncFSWALProvider { + + @Override + protected AsyncFSWAL createWAL() throws IOException { + return new SlowAsyncFSWAL(CommonFSUtils.getWALFileSystem(conf), this.abortable, + CommonFSUtils.getWALRootDir(conf), getWALDirectoryName(factory.getFactoryId()), + getWALArchiveDirectoryName(conf, factory.getFactoryId()), conf, listeners, true, logPrefix, + META_WAL_PROVIDER_ID.equals(providerId) ? META_WAL_PROVIDER_ID : null, eventLoopGroup, + channelClass, factory.getExcludeDatanodeManager().getStreamSlowMonitor(providerId)); + } + + } +} From 968297c72e65194ccea1894b49dea5f9de89e0f9 Mon Sep 17 00:00:00 2001 From: comnetwork Date: Fri, 22 Jul 2022 14:09:34 +0800 Subject: [PATCH 2/9] fix error --- .../java/org/apache/hadoop/hbase/regionserver/HRegion.java | 4 +--- .../hbase/regionserver/wal/TestWALSyncTimeoutException.java | 6 +++++- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java index 3a53fa0b11b4..e36e3c311e5a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -1371,9 +1371,7 @@ public RegionInfo getRegionInfo() { /** * Returns Instance of {@link RegionServerServices} used by this HRegion. Can be null. **/ - @RestrictedApi(explanation = "Should only be called in tests", link = "", - allowedOnPath = ".*/src/test/.*") - public RegionServerServices getRegionServerServices() { + RegionServerServices getRegionServerServices() { return this.rsServices; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALSyncTimeoutException.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALSyncTimeoutException.java index a956af84c214..dfdbb7369544 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALSyncTimeoutException.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALSyncTimeoutException.java @@ -138,7 +138,7 @@ public void testWALSyncWriteException() throws Exception { } catch (WALSyncTimeoutIOException e) { assertTrue(e != null); } - assertTrue(regions[0].getRegionServerServices().isAborted()); + assertTrue(regions[0].getRSServices().isAborted()); assertTrue(replicateCounter.get() == 0); } finally { Thread.currentThread().setName(oldThreadName); @@ -207,6 +207,10 @@ public void setRegionReplicationSink(RegionReplicationSink regionReplicationSink this.regionReplicationSink = Optional.of(regionReplicationSink); } + public RegionServerServices getRSServices() { + return this.rsServices; + } + } public static class SlowAsyncFSWAL extends AsyncFSWAL { From 83c56f32fc4e51ddd4e1ccf4f2c2f4a4f92a0507 Mon Sep 17 00:00:00 2001 From: comnetwork Date: Fri, 22 Jul 2022 14:13:15 +0800 Subject: [PATCH 3/9] fix error --- .../main/java/org/apache/hadoop/hbase/regionserver/HRegion.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java index e36e3c311e5a..4fce06587415 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -1370,7 +1370,7 @@ public RegionInfo getRegionInfo() { /** * Returns Instance of {@link RegionServerServices} used by this HRegion. Can be null. - **/ + */ RegionServerServices getRegionServerServices() { return this.rsServices; } From b6e5ab9b6ec647ee29e261c7631dc5eaaeef941e Mon Sep 17 00:00:00 2001 From: comnetwork Date: Sat, 23 Jul 2022 10:33:12 +0800 Subject: [PATCH 4/9] fix serialId --- .../org/apache/hadoop/hbase/regionserver/HRegion.java | 8 +------- .../hbase/regionserver/wal/WALSyncTimeoutIOException.java | 3 ++- 2 files changed, 3 insertions(+), 8 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java index 4fce06587415..951758d6b424 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -8040,17 +8040,11 @@ private WriteEntry doWALAppend(WALEdit walEdit, BatchOperation batchOp, try { long txid = this.wal.appendData(this.getRegionInfo(), walKey, walEdit); WriteEntry writeEntry = walKey.getWriteEntry(); + this.attachRegionReplicationInWALAppend(batchOp, miniBatchOp, walKey, walEdit, writeEntry); // Call sync on our edit. if (txid != 0) { sync(txid, batchOp.durability); } - /** - * If above {@link HRegion#sync} throws Exception, the RegionServer should be aborted and - * following {@link BatchOperation#writeMiniBatchOperationsToMemStore} will not be executed, - * so there is no need to replicate to secondary replica, for this reason here we attach the - * region replication action after the {@link HRegion#sync} is successful. - */ - this.attachRegionReplicationInWALAppend(batchOp, miniBatchOp, walKey, walEdit, writeEntry); return writeEntry; } catch (IOException ioe) { if (walKey.getWriteEntry() != null) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALSyncTimeoutIOException.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALSyncTimeoutIOException.java index b12a66d5ddff..8cef9c9dfd3d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALSyncTimeoutIOException.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALSyncTimeoutIOException.java @@ -24,10 +24,11 @@ /** * Thrown when {@link WAL#sync} timeout. */ -@SuppressWarnings("serial") @InterfaceAudience.Private public class WALSyncTimeoutIOException extends HBaseIOException { + private static final long serialVersionUID = 5067699288291906985L; + public WALSyncTimeoutIOException() { super(); } From 499ff33ab2e3432438956836abf96ef092f2e3d7 Mon Sep 17 00:00:00 2001 From: comnetwork Date: Sat, 23 Jul 2022 10:47:10 +0800 Subject: [PATCH 5/9] fix serialId --- .../wal/TestWALSyncTimeoutException.java | 69 +++---------------- 1 file changed, 8 insertions(+), 61 deletions(-) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALSyncTimeoutException.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALSyncTimeoutException.java index dfdbb7369544..514b5e6dbda0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALSyncTimeoutException.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALSyncTimeoutException.java @@ -17,14 +17,11 @@ */ package org.apache.hadoop.hbase.regionserver.wal; -import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import java.io.IOException; import java.util.List; -import java.util.Optional; -import java.util.concurrent.atomic.AtomicInteger; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -37,7 +34,6 @@ import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.RegionInfo; -import org.apache.hadoop.hbase.client.RegionReplicaUtil; import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.io.asyncfs.monitor.StreamSlowMonitor; @@ -50,18 +46,15 @@ import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.CommonFSUtils; -import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil; import org.apache.hadoop.hbase.wal.AsyncFSWALProvider; import org.apache.hadoop.hbase.wal.WAL; import org.apache.hadoop.hbase.wal.WALFactory; -import org.apache.hadoop.hbase.wal.WALKeyImpl; import org.apache.hadoop.hbase.wal.WALProvider; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import org.mockito.Mockito; import org.apache.hbase.thirdparty.io.netty.channel.Channel; import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup; @@ -78,7 +71,6 @@ public class TestWALSyncTimeoutException { private static final byte[] QUAL = Bytes.toBytes("qualifier_test"); private static final HBaseTestingUtil HTU = new HBaseTestingUtil(); - private static final int NB_SERVERS = 2; private static TableName tableName = TableName.valueOf("TestWALSyncTimeoutException"); private static volatile boolean testWALTimout = false; @@ -88,17 +80,15 @@ public class TestWALSyncTimeoutException { @BeforeClass public static void setUp() throws Exception { Configuration conf = HTU.getConfiguration(); - conf.setBoolean(ServerRegionReplicaUtil.REGION_REPLICA_REPLICATION_CONF_KEY, true); conf.setClass(HConstants.REGION_IMPL, HRegionForTest.class, HRegion.class); conf.setInt(RegionReplicationSink.RETRIES_NUMBER, 1); conf.setLong(RegionReplicationSink.RPC_TIMEOUT_MS, 10 * 60 * 1000); conf.setLong(RegionReplicationSink.OPERATION_TIMEOUT_MS, 20 * 60 * 1000); conf.setLong(RegionReplicationSink.META_EDIT_RPC_TIMEOUT_MS, 10 * 60 * 1000); conf.setLong(RegionReplicationSink.META_EDIT_OPERATION_TIMEOUT_MS, 20 * 60 * 1000); - conf.setBoolean(RegionReplicaUtil.REGION_REPLICA_WAIT_FOR_PRIMARY_FLUSH_CONF_KEY, false); conf.setClass(WALFactory.WAL_PROVIDER, SlowAsyncFSWALProvider.class, WALProvider.class); conf.setLong(AbstractFSWAL.WAL_SYNC_TIMEOUT_MS, timeoutMIlliseconds); - HTU.startMiniCluster(StartTestingClusterOption.builder().numRegionServers(NB_SERVERS).build()); + HTU.startMiniCluster(StartTestingClusterOption.builder().numRegionServers(1).build()); } @@ -114,11 +104,7 @@ public static void tearDown() throws Exception { */ @Test public void testWALSyncWriteException() throws Exception { - final HRegionForTest[] regions = this.createTable(); - RegionReplicationSink regionReplicationSink = regions[0].getRegionReplicationSink().get(); - assertTrue(regionReplicationSink != null); - final AtomicInteger replicateCounter = new AtomicInteger(0); - setUpSpiedRegionReplicationSink(regionReplicationSink, regions[0], replicateCounter); + final HRegionForTest region = this.createTable(); String oldThreadName = Thread.currentThread().getName(); Thread.currentThread().setName(USER_THREAD_NAME); @@ -133,61 +119,25 @@ public void testWALSyncWriteException() throws Exception { * {@link HRegion#doWALAppend} catches this exception it aborts the RegionServer. */ try { - regions[0].put(new Put(rowKey1).addColumn(FAMILY, QUAL, value1)); + region.put(new Put(rowKey1).addColumn(FAMILY, QUAL, value1)); fail(); } catch (WALSyncTimeoutIOException e) { assertTrue(e != null); } - assertTrue(regions[0].getRSServices().isAborted()); - assertTrue(replicateCounter.get() == 0); + assertTrue(region.getRSServices().isAborted()); } finally { Thread.currentThread().setName(oldThreadName); testWALTimout = false; } } - private static RegionReplicationSink setUpSpiedRegionReplicationSink( - final RegionReplicationSink regionReplicationSink, final HRegionForTest primaryRegion, - final AtomicInteger counter) { - RegionReplicationSink spiedRegionReplicationSink = Mockito.spy(regionReplicationSink); - - Mockito.doAnswer((invocationOnMock) -> { - if (!testWALTimout || !USER_THREAD_NAME.equals(Thread.currentThread().getName())) { - invocationOnMock.callRealMethod(); - return null; - } - WALKeyImpl walKey = invocationOnMock.getArgument(0); - if (!walKey.getTableName().equals(tableName)) { - invocationOnMock.callRealMethod(); - return null; - } - counter.incrementAndGet(); - invocationOnMock.callRealMethod(); - return null; - }).when(spiedRegionReplicationSink).add(Mockito.any(), Mockito.any(), Mockito.any()); - primaryRegion.setRegionReplicationSink(spiedRegionReplicationSink); - return spiedRegionReplicationSink; - } - - private HRegionForTest[] createTable() throws Exception { + private HRegionForTest createTable() throws Exception { TableDescriptor tableDescriptor = - TableDescriptorBuilder.newBuilder(tableName).setRegionReplication(NB_SERVERS) + TableDescriptorBuilder.newBuilder(tableName) .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY)).build(); HTU.getAdmin().createTable(tableDescriptor); - final HRegionForTest[] regions = new HRegionForTest[NB_SERVERS]; - for (int i = 0; i < NB_SERVERS; i++) { - HRegionServer rs = HTU.getMiniHBaseCluster().getRegionServer(i); - List onlineRegions = rs.getRegions(tableName); - for (HRegion region : onlineRegions) { - int replicaId = region.getRegionInfo().getReplicaId(); - assertTrue(regions[replicaId] == null); - regions[region.getRegionInfo().getReplicaId()] = (HRegionForTest) region; - } - } - for (HRegionForTest region : regions) { - assertNotNull(region); - } - return regions; + HRegionServer rs = HTU.getMiniHBaseCluster().getRegionServer(0); + return (HRegionForTest) rs.getRegions(tableName).get(0); } public static final class HRegionForTest extends HRegion { @@ -203,9 +153,6 @@ public HRegionForTest(Path tableDir, WAL wal, FileSystem fs, Configuration confP super(tableDir, wal, fs, confParam, regionInfo, htd, rsServices); } - public void setRegionReplicationSink(RegionReplicationSink regionReplicationSink) { - this.regionReplicationSink = Optional.of(regionReplicationSink); - } public RegionServerServices getRSServices() { return this.rsServices; From 01b114c7cf0ce7d27f634267b97f9a6a5b0b0047 Mon Sep 17 00:00:00 2001 From: comnetwork Date: Sat, 23 Jul 2022 10:55:23 +0800 Subject: [PATCH 6/9] fix serialId --- .../hbase/regionserver/wal/TestWALSyncTimeoutException.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALSyncTimeoutException.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALSyncTimeoutException.java index 514b5e6dbda0..939c56a54472 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALSyncTimeoutException.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALSyncTimeoutException.java @@ -132,9 +132,8 @@ public void testWALSyncWriteException() throws Exception { } private HRegionForTest createTable() throws Exception { - TableDescriptor tableDescriptor = - TableDescriptorBuilder.newBuilder(tableName) - .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY)).build(); + TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName) + .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY)).build(); HTU.getAdmin().createTable(tableDescriptor); HRegionServer rs = HTU.getMiniHBaseCluster().getRegionServer(0); return (HRegionForTest) rs.getRegions(tableName).get(0); @@ -153,7 +152,6 @@ public HRegionForTest(Path tableDir, WAL wal, FileSystem fs, Configuration confP super(tableDir, wal, fs, confParam, regionInfo, htd, rsServices); } - public RegionServerServices getRSServices() { return this.rsServices; } From 1b8eee8f14714ef8efa6dfe87de95db0f65d16f6 Mon Sep 17 00:00:00 2001 From: comnetwork Date: Sat, 23 Jul 2022 22:12:40 +0800 Subject: [PATCH 7/9] add comments --- .../org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java index 02e62a4de407..884197bcd011 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java @@ -62,8 +62,14 @@ void init(FileSystem fs, Path path, Configuration c, boolean overwritable, long StreamSlowMonitor monitor) throws IOException, CommonFSUtils.StreamLacksCapabilityException; } + /** + * Protected visible for used in tests. + */ protected EventLoopGroup eventLoopGroup; + /** + * Protected visible for used in tests. + */ protected Class channelClass; @Override From bd7fc27808044a9dd6885a4c00f623de938b5ed3 Mon Sep 17 00:00:00 2001 From: comnetwork Date: Sat, 23 Jul 2022 22:15:21 +0800 Subject: [PATCH 8/9] add comments --- .../java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java index 884197bcd011..b171a401a2e3 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java @@ -63,12 +63,12 @@ void init(FileSystem fs, Path path, Configuration c, boolean overwritable, long } /** - * Protected visible for used in tests. + * Protected visibility for used in tests. */ protected EventLoopGroup eventLoopGroup; /** - * Protected visible for used in tests. + * Protected visibility for used in tests. */ protected Class channelClass; From e8ed08ca3352a9e574e50253d996c0ff1b2ed04a Mon Sep 17 00:00:00 2001 From: comnetwork Date: Sun, 24 Jul 2022 10:21:44 +0800 Subject: [PATCH 9/9] add comments --- .../org/apache/hadoop/hbase/regionserver/wal/WALUtil.java | 6 ++++++ .../org/apache/hadoop/hbase/master/TestWarmupRegion.java | 6 ------ 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java index 243e5eb983e3..a6c4bda6ee0b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java @@ -176,6 +176,12 @@ private static WALKeyImpl doFullMarkerAppendTransaction(final WAL wal, if (walKey.getWriteEntry() != null) { mvcc.complete(walKey.getWriteEntry()); } + /** + * Here we do not abort the RegionServer for {@link WALSyncTimeoutIOException} as + * {@link HRegion#doWALAppend} does,because WAL Marker just records the internal state and + * seems it is no need to always abort the RegionServer when {@link WAL#sync} timeout,it is + * the internal state transition that determines whether RegionServer is aborted or not. + */ throw ioe; } return walKey; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestWarmupRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestWarmupRegion.java index 4adfd33a6186..420c0a6cac60 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestWarmupRegion.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestWarmupRegion.java @@ -21,10 +21,8 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; -import org.apache.hadoop.hbase.SingleProcessHBaseCluster; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.client.CompactionState; @@ -54,7 +52,6 @@ * named for the method and does its stuff against that. */ @Category({ MasterTests.class, LargeTests.class }) -@SuppressWarnings("deprecation") public class TestWarmupRegion { @ClassRule @@ -66,12 +63,10 @@ public class TestWarmupRegion { protected final static HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); private static byte[] ROW = Bytes.toBytes("testRow"); private static byte[] FAMILY = Bytes.toBytes("testFamily"); - private static byte[] QUALIFIER = Bytes.toBytes("testQualifier"); private static byte[] VALUE = Bytes.toBytes("testValue"); private static byte[] COLUMN = Bytes.toBytes("column"); private static int numRows = 10000; protected static int SLAVES = 3; - private static SingleProcessHBaseCluster myCluster; private static Table table; /** @@ -79,7 +74,6 @@ public class TestWarmupRegion { */ @BeforeClass public static void setUpBeforeClass() throws Exception { - Configuration conf = TEST_UTIL.getConfiguration(); TEST_UTIL.startMiniCluster(SLAVES); }