HBASE-27230 RegionServer should be aborted when WAL.sync throws Timeo… (#4641)
Signed-off-by: Duo Zhang <zhangduo@apache.org>
This commit is contained in:
parent
31fc97edee
commit
0ae42ddaee
|
@ -153,6 +153,7 @@ import org.apache.hadoop.hbase.regionserver.throttle.CompactionThroughputControl
|
||||||
import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
|
import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
|
||||||
import org.apache.hadoop.hbase.regionserver.throttle.StoreHotnessProtector;
|
import org.apache.hadoop.hbase.regionserver.throttle.StoreHotnessProtector;
|
||||||
import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
|
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.regionserver.wal.WALUtil;
|
||||||
import org.apache.hadoop.hbase.replication.ReplicationUtils;
|
import org.apache.hadoop.hbase.replication.ReplicationUtils;
|
||||||
import org.apache.hadoop.hbase.replication.regionserver.ReplicationObserver;
|
import org.apache.hadoop.hbase.replication.regionserver.ReplicationObserver;
|
||||||
|
@ -1367,7 +1368,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
||||||
return this.fs.getRegionInfo();
|
return this.fs.getRegionInfo();
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Returns Instance of {@link RegionServerServices} used by this HRegion. Can be null. */
|
/**
|
||||||
|
* Returns Instance of {@link RegionServerServices} used by this HRegion. Can be null.
|
||||||
|
*/
|
||||||
RegionServerServices getRegionServerServices() {
|
RegionServerServices getRegionServerServices() {
|
||||||
return this.rsServices;
|
return this.rsServices;
|
||||||
}
|
}
|
||||||
|
@ -2863,7 +2866,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
||||||
if (sink != null && !writeFlushWalMarker) {
|
if (sink != null && !writeFlushWalMarker) {
|
||||||
/**
|
/**
|
||||||
* Here for replication to secondary region replica could use {@link FlushAction#CANNOT_FLUSH}
|
* 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
|
* {@link FlushDescriptor} and attach the {@link RegionReplicationSink#add} to the
|
||||||
* flushOpSeqIdMVCCEntry,see HBASE-26960 for more details.
|
* flushOpSeqIdMVCCEntry,see HBASE-26960 for more details.
|
||||||
*/
|
*/
|
||||||
|
@ -3694,7 +3697,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
||||||
* @param familyMap Map of Cells by family
|
* @param familyMap Map of Cells by family
|
||||||
*/
|
*/
|
||||||
protected void applyFamilyMapToMemStore(Map<byte[], List<Cell>> familyMap,
|
protected void applyFamilyMapToMemStore(Map<byte[], List<Cell>> familyMap,
|
||||||
MemStoreSizing memstoreAccounting) throws IOException {
|
MemStoreSizing memstoreAccounting) {
|
||||||
for (Map.Entry<byte[], List<Cell>> e : familyMap.entrySet()) {
|
for (Map.Entry<byte[], List<Cell>> e : familyMap.entrySet()) {
|
||||||
byte[] family = e.getKey();
|
byte[] family = e.getKey();
|
||||||
List<Cell> cells = e.getValue();
|
List<Cell> cells = e.getValue();
|
||||||
|
@ -5231,7 +5234,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
||||||
* scenario but that do not make sense otherwise.
|
* scenario but that do not make sense otherwise.
|
||||||
*/
|
*/
|
||||||
private void applyToMemStore(HStore store, List<Cell> cells, boolean delta,
|
private void applyToMemStore(HStore store, List<Cell> 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!!!!
|
// Any change in how we update Store/MemStore needs to also be done in other applyToMemStore!!!!
|
||||||
boolean upsert = delta && store.getColumnFamilyDescriptor().getMaxVersions() == 1;
|
boolean upsert = delta && store.getColumnFamilyDescriptor().getMaxVersions() == 1;
|
||||||
if (upsert) {
|
if (upsert) {
|
||||||
|
@ -8047,6 +8050,19 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
||||||
if (walKey.getWriteEntry() != null) {
|
if (walKey.getWriteEntry() != null) {
|
||||||
mvcc.complete(walKey.getWriteEntry());
|
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;
|
throw ioe;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -8057,7 +8073,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
||||||
*/
|
*/
|
||||||
private void attachRegionReplicationInWALAppend(BatchOperation<?> batchOp,
|
private void attachRegionReplicationInWALAppend(BatchOperation<?> batchOp,
|
||||||
MiniBatchOperationInProgress<Mutation> miniBatchOp, WALKeyImpl walKey, WALEdit walEdit,
|
MiniBatchOperationInProgress<Mutation> miniBatchOp, WALKeyImpl walKey, WALEdit walEdit,
|
||||||
WriteEntry writeEntry) throws IOException {
|
WriteEntry writeEntry) {
|
||||||
if (!regionReplicationSink.isPresent()) {
|
if (!regionReplicationSink.isPresent()) {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
@ -8086,7 +8102,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
||||||
* replica.
|
* replica.
|
||||||
*/
|
*/
|
||||||
private void doAttachReplicateRegionReplicaAction(WALKeyImpl walKey, WALEdit walEdit,
|
private void doAttachReplicateRegionReplicaAction(WALKeyImpl walKey, WALEdit walEdit,
|
||||||
WriteEntry writeEntry) throws IOException {
|
WriteEntry writeEntry) {
|
||||||
if (walEdit == null || walEdit.isEmpty()) {
|
if (walEdit == null || walEdit.isEmpty()) {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
|
@ -1880,8 +1880,7 @@ public class HStore
|
||||||
* across all of them.
|
* across all of them.
|
||||||
* @param readpoint readpoint below which we can safely remove duplicate KVs
|
* @param readpoint readpoint below which we can safely remove duplicate KVs
|
||||||
*/
|
*/
|
||||||
public void upsert(Iterable<Cell> cells, long readpoint, MemStoreSizing memstoreSizing)
|
public void upsert(Iterable<Cell> cells, long readpoint, MemStoreSizing memstoreSizing) {
|
||||||
throws IOException {
|
|
||||||
this.storeEngine.readLock();
|
this.storeEngine.readLock();
|
||||||
try {
|
try {
|
||||||
this.memstore.upsert(cells, readpoint, memstoreSizing);
|
this.memstore.upsert(cells, readpoint, memstoreSizing);
|
||||||
|
|
|
@ -142,7 +142,7 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
|
||||||
"hbase.regionserver.wal.slowsync.roll.interval.ms";
|
"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 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
|
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";
|
public static final String WAL_ROLL_MULTIPLIER = "hbase.regionserver.logroll.multiplier";
|
||||||
|
@ -881,7 +881,7 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
} catch (TimeoutIOException tioe) {
|
} catch (TimeoutIOException tioe) {
|
||||||
throw tioe;
|
throw new WALSyncTimeoutIOException(tioe);
|
||||||
} catch (InterruptedException ie) {
|
} catch (InterruptedException ie) {
|
||||||
LOG.warn("Interrupted", ie);
|
LOG.warn("Interrupted", ie);
|
||||||
throw convertInterruptedExceptionToIOException(ie);
|
throw convertInterruptedExceptionToIOException(ie);
|
||||||
|
|
|
@ -0,0 +1,48 @@
|
||||||
|
/*
|
||||||
|
* 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.
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
public class WALSyncTimeoutIOException extends HBaseIOException {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 5067699288291906985L;
|
||||||
|
|
||||||
|
public WALSyncTimeoutIOException() {
|
||||||
|
super();
|
||||||
|
}
|
||||||
|
|
||||||
|
public WALSyncTimeoutIOException(String message, Throwable cause) {
|
||||||
|
super(message, cause);
|
||||||
|
}
|
||||||
|
|
||||||
|
public WALSyncTimeoutIOException(String message) {
|
||||||
|
super(message);
|
||||||
|
}
|
||||||
|
|
||||||
|
public WALSyncTimeoutIOException(Throwable cause) {
|
||||||
|
super(cause);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -176,6 +176,12 @@ public class WALUtil {
|
||||||
if (walKey.getWriteEntry() != null) {
|
if (walKey.getWriteEntry() != null) {
|
||||||
mvcc.complete(walKey.getWriteEntry());
|
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;
|
throw ioe;
|
||||||
}
|
}
|
||||||
return walKey;
|
return walKey;
|
||||||
|
|
|
@ -62,9 +62,15 @@ public class AsyncFSWALProvider extends AbstractFSWALProvider<AsyncFSWAL> {
|
||||||
StreamSlowMonitor monitor) throws IOException, CommonFSUtils.StreamLacksCapabilityException;
|
StreamSlowMonitor monitor) throws IOException, CommonFSUtils.StreamLacksCapabilityException;
|
||||||
}
|
}
|
||||||
|
|
||||||
private EventLoopGroup eventLoopGroup;
|
/**
|
||||||
|
* Protected visibility for used in tests.
|
||||||
|
*/
|
||||||
|
protected EventLoopGroup eventLoopGroup;
|
||||||
|
|
||||||
private Class<? extends Channel> channelClass;
|
/**
|
||||||
|
* Protected visibility for used in tests.
|
||||||
|
*/
|
||||||
|
protected Class<? extends Channel> channelClass;
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected AsyncFSWAL createWAL() throws IOException {
|
protected AsyncFSWAL createWAL() throws IOException {
|
||||||
|
|
|
@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.client.RegionInfo;
|
||||||
import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
|
import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
|
||||||
import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
|
import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
|
||||||
import org.apache.hadoop.hbase.regionserver.wal.WALCoprocessorHost;
|
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.hadoop.hbase.replication.regionserver.WALFileLengthProvider;
|
||||||
import org.apache.yetus.audience.InterfaceAudience;
|
import org.apache.yetus.audience.InterfaceAudience;
|
||||||
import org.apache.yetus.audience.InterfaceStability;
|
import org.apache.yetus.audience.InterfaceStability;
|
||||||
|
@ -133,18 +134,21 @@ public interface WAL extends Closeable, WALFileLengthProvider {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Sync what we have in the WAL.
|
* Sync what we have in the WAL.
|
||||||
|
* @throws when timeout, it would throw {@link WALSyncTimeoutIOException}.
|
||||||
*/
|
*/
|
||||||
void sync() throws IOException;
|
void sync() throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Sync the WAL if the txId was not already sync'd.
|
* Sync the WAL if the txId was not already sync'd.
|
||||||
* @param txid Transaction id to sync to.
|
* @param txid Transaction id to sync to.
|
||||||
|
* @throws when timeout, it would throw {@link WALSyncTimeoutIOException}.
|
||||||
*/
|
*/
|
||||||
void sync(long txid) throws IOException;
|
void sync(long txid) throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @param forceSync Flag to force sync rather than flushing to the buffer. Example - Hadoop hflush
|
* @param forceSync Flag to force sync rather than flushing to the buffer. Example - Hadoop hflush
|
||||||
* vs hsync.
|
* vs hsync.
|
||||||
|
* @throws when timeout, it would throw {@link WALSyncTimeoutIOException}.
|
||||||
*/
|
*/
|
||||||
default void sync(boolean forceSync) throws IOException {
|
default void sync(boolean forceSync) throws IOException {
|
||||||
sync();
|
sync();
|
||||||
|
@ -154,6 +158,7 @@ public interface WAL extends Closeable, WALFileLengthProvider {
|
||||||
* @param txid Transaction id to sync to.
|
* @param txid Transaction id to sync to.
|
||||||
* @param forceSync Flag to force sync rather than flushing to the buffer. Example - Hadoop hflush
|
* @param forceSync Flag to force sync rather than flushing to the buffer. Example - Hadoop hflush
|
||||||
* vs hsync.
|
* vs hsync.
|
||||||
|
* @throws when timeout, it would throw {@link WALSyncTimeoutIOException}.
|
||||||
*/
|
*/
|
||||||
default void sync(long txid, boolean forceSync) throws IOException {
|
default void sync(long txid, boolean forceSync) throws IOException {
|
||||||
sync(txid);
|
sync(txid);
|
||||||
|
|
|
@ -17,6 +17,7 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.hbase.wal;
|
package org.apache.hadoop.hbase.wal;
|
||||||
|
|
||||||
|
import com.google.errorprone.annotations.RestrictedApi;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.InterruptedIOException;
|
import java.io.InterruptedIOException;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
@ -505,4 +506,10 @@ public class WALFactory {
|
||||||
public ExcludeDatanodeManager getExcludeDatanodeManager() {
|
public ExcludeDatanodeManager getExcludeDatanodeManager() {
|
||||||
return excludeDatanodeManager;
|
return excludeDatanodeManager;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@RestrictedApi(explanation = "Should only be called in tests", link = "",
|
||||||
|
allowedOnPath = ".*/src/test/.*")
|
||||||
|
public String getFactoryId() {
|
||||||
|
return this.factoryId;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -21,10 +21,8 @@ import static org.apache.hadoop.hbase.regionserver.HRegion.warmupHRegion;
|
||||||
import static org.junit.Assert.assertTrue;
|
import static org.junit.Assert.assertTrue;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
|
||||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||||
import org.apache.hadoop.hbase.HBaseTestingUtil;
|
import org.apache.hadoop.hbase.HBaseTestingUtil;
|
||||||
import org.apache.hadoop.hbase.SingleProcessHBaseCluster;
|
|
||||||
import org.apache.hadoop.hbase.TableName;
|
import org.apache.hadoop.hbase.TableName;
|
||||||
import org.apache.hadoop.hbase.Waiter;
|
import org.apache.hadoop.hbase.Waiter;
|
||||||
import org.apache.hadoop.hbase.client.CompactionState;
|
import org.apache.hadoop.hbase.client.CompactionState;
|
||||||
|
@ -54,7 +52,6 @@ import org.slf4j.LoggerFactory;
|
||||||
* named for the method and does its stuff against that.
|
* named for the method and does its stuff against that.
|
||||||
*/
|
*/
|
||||||
@Category({ MasterTests.class, LargeTests.class })
|
@Category({ MasterTests.class, LargeTests.class })
|
||||||
@SuppressWarnings("deprecation")
|
|
||||||
public class TestWarmupRegion {
|
public class TestWarmupRegion {
|
||||||
|
|
||||||
@ClassRule
|
@ClassRule
|
||||||
|
@ -66,12 +63,10 @@ public class TestWarmupRegion {
|
||||||
protected final static HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil();
|
protected final static HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil();
|
||||||
private static byte[] ROW = Bytes.toBytes("testRow");
|
private static byte[] ROW = Bytes.toBytes("testRow");
|
||||||
private static byte[] FAMILY = Bytes.toBytes("testFamily");
|
private static byte[] FAMILY = Bytes.toBytes("testFamily");
|
||||||
private static byte[] QUALIFIER = Bytes.toBytes("testQualifier");
|
|
||||||
private static byte[] VALUE = Bytes.toBytes("testValue");
|
private static byte[] VALUE = Bytes.toBytes("testValue");
|
||||||
private static byte[] COLUMN = Bytes.toBytes("column");
|
private static byte[] COLUMN = Bytes.toBytes("column");
|
||||||
private static int numRows = 10000;
|
private static int numRows = 10000;
|
||||||
protected static int SLAVES = 3;
|
protected static int SLAVES = 3;
|
||||||
private static SingleProcessHBaseCluster myCluster;
|
|
||||||
private static Table table;
|
private static Table table;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -79,7 +74,6 @@ public class TestWarmupRegion {
|
||||||
*/
|
*/
|
||||||
@BeforeClass
|
@BeforeClass
|
||||||
public static void setUpBeforeClass() throws Exception {
|
public static void setUpBeforeClass() throws Exception {
|
||||||
Configuration conf = TEST_UTIL.getConfiguration();
|
|
||||||
TEST_UTIL.startMiniCluster(SLAVES);
|
TEST_UTIL.startMiniCluster(SLAVES);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -0,0 +1,208 @@
|
||||||
|
/*
|
||||||
|
* 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.assertTrue;
|
||||||
|
import static org.junit.Assert.fail;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.List;
|
||||||
|
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.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.wal.AsyncFSWALProvider;
|
||||||
|
import org.apache.hadoop.hbase.wal.WAL;
|
||||||
|
import org.apache.hadoop.hbase.wal.WALFactory;
|
||||||
|
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.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 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.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.setClass(WALFactory.WAL_PROVIDER, SlowAsyncFSWALProvider.class, WALProvider.class);
|
||||||
|
conf.setLong(AbstractFSWAL.WAL_SYNC_TIMEOUT_MS, timeoutMIlliseconds);
|
||||||
|
HTU.startMiniCluster(StartTestingClusterOption.builder().numRegionServers(1).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 region = this.createTable();
|
||||||
|
|
||||||
|
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 {
|
||||||
|
region.put(new Put(rowKey1).addColumn(FAMILY, QUAL, value1));
|
||||||
|
fail();
|
||||||
|
} catch (WALSyncTimeoutIOException e) {
|
||||||
|
assertTrue(e != null);
|
||||||
|
}
|
||||||
|
assertTrue(region.getRSServices().isAborted());
|
||||||
|
} finally {
|
||||||
|
Thread.currentThread().setName(oldThreadName);
|
||||||
|
testWALTimout = false;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private HRegionForTest createTable() throws Exception {
|
||||||
|
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);
|
||||||
|
}
|
||||||
|
|
||||||
|
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 RegionServerServices getRSServices() {
|
||||||
|
return this.rsServices;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class SlowAsyncFSWAL extends AsyncFSWAL {
|
||||||
|
|
||||||
|
public SlowAsyncFSWAL(FileSystem fs, Abortable abortable, Path rootDir, String logDir,
|
||||||
|
String archiveDir, Configuration conf, List<WALActionsListener> listeners,
|
||||||
|
boolean failIfWALExists, String prefix, String suffix, EventLoopGroup eventLoopGroup,
|
||||||
|
Class<? extends Channel> 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<WALActionsListener> listeners, boolean failIfWALExists,
|
||||||
|
String prefix, String suffix, EventLoopGroup eventLoopGroup,
|
||||||
|
Class<? extends Channel> 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));
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
Loading…
Reference in New Issue