Skip to content

Commit

Permalink
HBASE-23340 hmaster /hbase/replication/rs session expired (hbase repl… (
Browse files Browse the repository at this point in the history
#2739)

Signed-off-by: Duo Zhang <zhangduo@apache.org>
  • Loading branch information
cuibo01 committed Dec 12, 2020
1 parent f098461 commit d50816f
Show file tree
Hide file tree
Showing 5 changed files with 47 additions and 20 deletions.
Expand Up @@ -1336,17 +1336,17 @@ private void startServiceThreads() throws IOException {

// Create cleaner thread pool
cleanerPool = new DirScanPool(conf);
Map<String, Object> params = new HashMap<>();
params.put(MASTER, this);
// Start log cleaner thread
int cleanerInterval =
conf.getInt(HBASE_MASTER_CLEANER_INTERVAL, DEFAULT_HBASE_MASTER_CLEANER_INTERVAL);
this.logCleaner = new LogCleaner(cleanerInterval, this, conf,
getMasterWalManager().getFileSystem(), getMasterWalManager().getOldLogDir(), cleanerPool);
getMasterWalManager().getFileSystem(), getMasterWalManager().getOldLogDir(), cleanerPool, params);
getChoreService().scheduleChore(logCleaner);

// start the hfile archive cleaner thread
Path archiveDir = HFileArchiveUtil.getArchivePath(conf);
Map<String, Object> params = new HashMap<>();
params.put(MASTER, this);
this.hfileCleaner = new HFileCleaner(cleanerInterval, this, conf,
getMasterFileSystem().getFileSystem(), archiveDir, cleanerPool, params);
getChoreService().scheduleChore(hfileCleaner);
Expand Down
Expand Up @@ -22,6 +22,7 @@
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.TimeUnit;
Expand Down Expand Up @@ -72,9 +73,9 @@ public class LogCleaner extends CleanerChore<BaseLogCleanerDelegate>
* @param pool the thread pool used to scan directories
*/
public LogCleaner(final int period, final Stoppable stopper, Configuration conf, FileSystem fs,
Path oldLogDir, DirScanPool pool) {
Path oldLogDir, DirScanPool pool, Map<String, Object> params) {
super("LogsCleaner", period, stopper, conf, fs, oldLogDir, HBASE_MASTER_LOGCLEANER_PLUGINS,
pool);
pool, params);
this.pendingDelete = new LinkedBlockingQueue<>();
int size = conf.getInt(OLD_WALS_CLEANER_THREAD_SIZE, DEFAULT_OLD_WALS_CLEANER_THREAD_SIZE);
this.oldWALsCleaner = createOldWalsCleaner(size);
Expand Down
Expand Up @@ -19,16 +19,19 @@

import java.io.IOException;
import java.util.Collections;
import java.util.Map;
import java.util.Set;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.cleaner.BaseLogCleanerDelegate;
import org.apache.hadoop.hbase.replication.ReplicationException;
import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hbase.thirdparty.org.apache.commons.collections4.MapUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand All @@ -43,7 +46,8 @@
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
public class ReplicationLogCleaner extends BaseLogCleanerDelegate {
private static final Logger LOG = LoggerFactory.getLogger(ReplicationLogCleaner.class);
private ZKWatcher zkw;
private ZKWatcher zkw = null;
private boolean shareZK = false;
private ReplicationQueueStorage queueStorage;
private boolean stopped = false;
private Set<String> wals;
Expand Down Expand Up @@ -92,12 +96,20 @@ public boolean apply(FileStatus file) {
}

@Override
public void setConf(Configuration config) {
// Make my own Configuration. Then I'll have my own connection to zk that
// I can close myself when comes time.
Configuration conf = new Configuration(config);
public void init(Map<String, Object> params) {
super.init(params);
try {
setConf(conf, new ZKWatcher(conf, "replicationLogCleaner", null));
if (MapUtils.isNotEmpty(params)) {
Object master = params.get(HMaster.MASTER);
if (master != null && master instanceof HMaster) {
zkw = ((HMaster) master).getZooKeeper();
shareZK = true;
}
}
if (zkw == null) {
zkw = new ZKWatcher(getConf(), "replicationLogCleaner", null);
}
this.queueStorage = ReplicationStorageFactory.getReplicationQueueStorage(zkw, getConf());
} catch (IOException e) {
LOG.error("Error while configuring " + this.getClass().getName(), e);
}
Expand Down Expand Up @@ -126,7 +138,7 @@ public void setConf(Configuration conf, ZKWatcher zk,
public void stop(String why) {
if (this.stopped) return;
this.stopped = true;
if (this.zkw != null) {
if (!shareZK && this.zkw != null) {
LOG.info("Stopping " + this.zkw);
this.zkw.close();
}
Expand Down
Expand Up @@ -205,7 +205,7 @@ public void testLogCleaning() throws Exception {
// 10 procedure WALs
assertEquals(10, fs.listStatus(OLD_PROCEDURE_WALS_DIR).length);

LogCleaner cleaner = new LogCleaner(1000, server, conf, fs, OLD_WALS_DIR, POOL);
LogCleaner cleaner = new LogCleaner(1000, server, conf, fs, OLD_WALS_DIR, POOL, null);
cleaner.chore();

// In oldWALs we end up with the current WAL, a newer WAL, the 3 old WALs which
Expand All @@ -226,7 +226,7 @@ public void testLogCleaning() throws Exception {
}

@Test
public void testZooKeeperAbortDuringGetListOfReplicators() throws Exception {
public void testZooKeeperRecoveryDuringGetListOfReplicators() throws Exception {
ReplicationLogCleaner cleaner = new ReplicationLogCleaner();

List<FileStatus> dummyFiles = Arrays.asList(
Expand All @@ -239,7 +239,7 @@ public void testZooKeeperAbortDuringGetListOfReplicators() throws Exception {
final AtomicBoolean getListOfReplicatorsFailed = new AtomicBoolean(false);

try {
faultyZK.init();
faultyZK.init(false);
ReplicationQueueStorage queueStorage = spy(ReplicationStorageFactory
.getReplicationQueueStorage(faultyZK, conf));
doAnswer(new Answer<Object>() {
Expand All @@ -263,6 +263,18 @@ public Object answer(InvocationOnMock invocation) throws Throwable {
assertTrue(getListOfReplicatorsFailed.get());
assertFalse(toDelete.iterator().hasNext());
assertFalse(cleaner.isStopped());

//zk recovery.
faultyZK.init(true);
cleaner.preClean();
Iterable<FileStatus> filesToDelete = cleaner.getDeletableFiles(dummyFiles);
Iterator<FileStatus> iter = filesToDelete.iterator();
assertTrue(iter.hasNext());
assertEquals(new Path("log1"), iter.next().getPath());
assertTrue(iter.hasNext());
assertEquals(new Path("log2"), iter.next().getPath());
assertFalse(iter.hasNext());

} finally {
faultyZK.close();
}
Expand Down Expand Up @@ -306,7 +318,7 @@ public void testOnConfigurationChange() throws Exception {
Server server = new DummyServer();

FileSystem fs = TEST_UTIL.getDFSCluster().getFileSystem();
LogCleaner cleaner = new LogCleaner(3000, server, conf, fs, OLD_WALS_DIR, POOL);
LogCleaner cleaner = new LogCleaner(3000, server, conf, fs, OLD_WALS_DIR, POOL, null);
int size = cleaner.getSizeOfCleaners();
assertEquals(LogCleaner.DEFAULT_OLD_WALS_CLEANER_THREAD_TIMEOUT_MSEC,
cleaner.getCleanerThreadTimeoutMsec());
Expand Down Expand Up @@ -426,10 +438,12 @@ public FaultyZooKeeperWatcher(Configuration conf, String identifier, Abortable a
super(conf, identifier, abortable);
}

public void init() throws Exception {
public void init(boolean autoRecovery) throws Exception {
this.zk = spy(super.getRecoverableZooKeeper());
doThrow(new KeeperException.ConnectionLossException())
.when(zk).getChildren("/hbase/replication/rs", null);
if (!autoRecovery) {
doThrow(new KeeperException.ConnectionLossException())
.when(zk).getChildren("/hbase/replication/rs", null);
}
}

@Override
Expand Down
Expand Up @@ -72,7 +72,7 @@ public void stop(String why) {
public boolean isStopped() {
return stopped;
}
}, conf, fs, globalWALArchiveDir, cleanerPool);
}, conf, fs, globalWALArchiveDir, cleanerPool, null);
choreService.scheduleChore(logCleaner);
}

Expand Down

0 comments on commit d50816f

Please sign in to comment.