HBASE-3366 WALObservers should be notified before the lock
HBASE-3367 Failed log split not retried HBASE-3370 ReplicationSource.openReader fails to locate HLogs when they aren't split yet HBASE-3371 Race in TestReplication can make it fail HBASE-3372 HRS shouldn't print a full stack for ServerNotRunningException git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1050542 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
f84ff410bd
commit
ef96f37844
|
@ -790,6 +790,11 @@ Release 0.90.0 - Unreleased
|
|||
location in .META. is possible
|
||||
HBASE-3368 Split message can come in before region opened message; results
|
||||
in 'Region has been PENDING_CLOSE for too long' cycle
|
||||
HBASE-3366 WALObservers should be notified before the lock
|
||||
HBASE-3367 Failed log split not retried
|
||||
HBASE-3370 ReplicationSource.openReader fails to locate HLogs when they
|
||||
aren't split yet
|
||||
HBASE-3371 Race in TestReplication can make it fail
|
||||
|
||||
|
||||
IMPROVEMENTS
|
||||
|
@ -1274,6 +1279,7 @@ Release 0.90.0 - Unreleased
|
|||
HBASE-3303 Lower hbase.regionserver.handler.count from 25 back to 10
|
||||
HBASE-2467 Concurrent flushers in HLog sync using HDFS-895
|
||||
HBASE-3349 Pass HBase configuration to HttpServer
|
||||
HBASE-3372 HRS shouldn't print a full stack for ServerNotRunningException
|
||||
|
||||
|
||||
NEW FEATURES
|
||||
|
|
|
@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.regionserver.HRegion;
|
|||
import org.apache.hadoop.hbase.regionserver.Store;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLog;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.OrphanHLogAfterSplitException;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
|
||||
|
@ -190,7 +191,12 @@ public class MasterFileSystem {
|
|||
Path logDir = new Path(this.rootdir, HLog.getHLogDirectoryName(serverName));
|
||||
try {
|
||||
HLogSplitter splitter = HLogSplitter.createLogSplitter(conf);
|
||||
splitter.splitLog(this.rootdir, logDir, oldLogDir, this.fs, conf);
|
||||
try {
|
||||
splitter.splitLog(this.rootdir, logDir, oldLogDir, this.fs, conf);
|
||||
} catch (OrphanHLogAfterSplitException e) {
|
||||
LOG.warn("Retrying splitting because of:", e);
|
||||
splitter.splitLog(this.rootdir, logDir, oldLogDir, this.fs, conf);
|
||||
}
|
||||
splitTime = splitter.getTime();
|
||||
splitLogSize = splitter.getSize();
|
||||
} catch (IOException e) {
|
||||
|
|
|
@ -103,6 +103,7 @@ import org.apache.hadoop.hbase.ipc.HMasterRegionInterface;
|
|||
import org.apache.hadoop.hbase.ipc.HRegionInterface;
|
||||
import org.apache.hadoop.hbase.ipc.Invocation;
|
||||
import org.apache.hadoop.hbase.ipc.RpcServer;
|
||||
import org.apache.hadoop.hbase.ipc.ServerNotRunningException;
|
||||
import org.apache.hadoop.hbase.regionserver.Leases.LeaseStillHeldException;
|
||||
import org.apache.hadoop.hbase.regionserver.handler.CloseMetaHandler;
|
||||
import org.apache.hadoop.hbase.regionserver.handler.CloseRegionHandler;
|
||||
|
@ -1419,7 +1420,13 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
|
|||
masterAddress.getInetSocketAddress(), this.conf, -1,
|
||||
this.rpcTimeout, this.rpcTimeout);
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Unable to connect to master. Retrying. Error was:", e);
|
||||
e = e instanceof RemoteException ?
|
||||
((RemoteException)e).unwrapRemoteException() : e;
|
||||
if (e instanceof ServerNotRunningException) {
|
||||
LOG.info("Master isn't available yet, retrying");
|
||||
} else {
|
||||
LOG.warn("Unable to connect to master. Retrying. Error was:", e);
|
||||
}
|
||||
sleeper.sleep();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -493,6 +493,13 @@ public class HLog implements Syncable {
|
|||
nextHdfsOut =
|
||||
((SequenceFileLogWriter)nextWriter).getDFSCOutputStream();
|
||||
}
|
||||
// Tell our listeners that a new log was created
|
||||
if (!this.listeners.isEmpty()) {
|
||||
for (WALObserver i : this.listeners) {
|
||||
i.logRolled(newPath);
|
||||
}
|
||||
}
|
||||
|
||||
synchronized (updateLock) {
|
||||
// Clean up current writer.
|
||||
Path oldFile = cleanupCurrentWriter(currentFilenum);
|
||||
|
@ -509,12 +516,6 @@ public class HLog implements Syncable {
|
|||
this.numEntries.set(0);
|
||||
this.logRollRequested = false;
|
||||
}
|
||||
// Tell our listeners that a new log was created
|
||||
if (!this.listeners.isEmpty()) {
|
||||
for (WALObserver i : this.listeners) {
|
||||
i.logRolled(newPath);
|
||||
}
|
||||
}
|
||||
// Can we delete any of the old log files?
|
||||
if (this.outputfiles.size() > 0) {
|
||||
if (this.lastSeqWritten.isEmpty()) {
|
||||
|
|
|
@ -287,7 +287,8 @@ public class HLogSplitter {
|
|||
}
|
||||
if (fs.listStatus(srcDir).length > processedLogs.size()
|
||||
+ corruptedLogs.size()) {
|
||||
throw new IOException("Discovered orphan hlog after split. Maybe "
|
||||
throw new OrphanHLogAfterSplitException(
|
||||
"Discovered orphan hlog after split. Maybe the "
|
||||
+ "HRegionServer was not dead when we started");
|
||||
}
|
||||
archiveLogs(corruptedLogs, processedLogs, oldLogDir, fs, conf);
|
||||
|
|
|
@ -0,0 +1,40 @@
|
|||
/**
|
||||
* Copyright 2010 The Apache Software Foundation
|
||||
*
|
||||
* 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 java.io.IOException;
|
||||
|
||||
public class OrphanHLogAfterSplitException extends IOException {
|
||||
|
||||
/**
|
||||
* Create this exception without a message
|
||||
*/
|
||||
public OrphanHLogAfterSplitException() {
|
||||
super();
|
||||
}
|
||||
|
||||
/**
|
||||
* Create this exception with a message
|
||||
* @param message why it failed
|
||||
*/
|
||||
public OrphanHLogAfterSplitException(String message) {
|
||||
super(message);
|
||||
}
|
||||
}
|
|
@ -438,17 +438,20 @@ public class ReplicationSource extends Thread
|
|||
// We didn't find the log in the archive directory, look if it still
|
||||
// exists in the dead RS folder (there could be a chain of failures
|
||||
// to look at)
|
||||
for (int i = this.deadRegionServers.length - 1; i > 0; i--) {
|
||||
LOG.info("NB dead servers : " + deadRegionServers.length);
|
||||
for (int i = this.deadRegionServers.length - 1; i >= 0; i--) {
|
||||
|
||||
Path deadRsDirectory =
|
||||
new Path(this.manager.getLogDir(), this.deadRegionServers[i]);
|
||||
new Path(manager.getLogDir().getParent(), this.deadRegionServers[i]);
|
||||
Path possibleLogLocation =
|
||||
new Path(deadRsDirectory, currentPath.getName());
|
||||
LOG.info("Possible location " + possibleLogLocation.toUri().toString());
|
||||
if (this.manager.getFs().exists(possibleLogLocation)) {
|
||||
// We found the right new location
|
||||
LOG.info("Log " + this.currentPath + " still exists at " +
|
||||
possibleLogLocation);
|
||||
// Breaking here will make us sleep since reader is null
|
||||
break;
|
||||
return true;
|
||||
}
|
||||
}
|
||||
// TODO What happens if the log was missing from every single location?
|
||||
|
|
|
@ -599,6 +599,7 @@ public class HBaseTestingUtility {
|
|||
Delete del = new Delete(res.getRow());
|
||||
table.delete(del);
|
||||
}
|
||||
resScan = table.getScanner(scan);
|
||||
return table;
|
||||
}
|
||||
|
||||
|
|
|
@ -157,7 +157,7 @@ public class TestHLogSplit {
|
|||
assertEquals(parentOfParent, HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
|
||||
}
|
||||
|
||||
@Test(expected = IOException.class)
|
||||
@Test(expected = OrphanHLogAfterSplitException.class)
|
||||
public void testSplitFailsIfNewHLogGetsCreatedAfterSplitStarted()
|
||||
throws IOException {
|
||||
AtomicBoolean stop = new AtomicBoolean(false);
|
||||
|
|
|
@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.client.Scan;
|
|||
import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
|
||||
import org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.JVMClusterUtil;
|
||||
import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.hadoop.mapreduce.Job;
|
||||
|
@ -126,7 +127,6 @@ public class TestReplication {
|
|||
utility2.startMiniCluster(2);
|
||||
|
||||
HTableDescriptor table = new HTableDescriptor(tableName);
|
||||
table.setDeferredLogFlush(false);
|
||||
HColumnDescriptor fam = new HColumnDescriptor(famName);
|
||||
fam.setScope(HConstants.REPLICATION_SCOPE_GLOBAL);
|
||||
table.addFamily(fam);
|
||||
|
@ -153,8 +153,15 @@ public class TestReplication {
|
|||
*/
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
|
||||
// Starting and stopping replication can make us miss new logs,
|
||||
// rolling like this makes sure the most recent one gets added to the queue
|
||||
for ( JVMClusterUtil.RegionServerThread r :
|
||||
utility1.getHBaseCluster().getRegionServerThreads()) {
|
||||
r.getRegionServer().getWAL().rollWriter();
|
||||
}
|
||||
utility1.truncateTable(tableName);
|
||||
// truncating the table will send on Delete per row to the slave cluster
|
||||
// truncating the table will send one Delete per row to the slave cluster
|
||||
// in an async fashion, which is why we cannot just call truncateTable on
|
||||
// utility2 since late writes could make it to the slave in some way.
|
||||
// Instead, we truncate the first table and wait for all the Deletes to
|
||||
|
@ -172,6 +179,7 @@ public class TestReplication {
|
|||
if (lastCount < res.length) {
|
||||
i--; // Don't increment timeout if we make progress
|
||||
}
|
||||
lastCount = res.length;
|
||||
LOG.info("Still got " + res.length + " rows");
|
||||
Thread.sleep(SLEEP_TIME);
|
||||
} else {
|
||||
|
|
Loading…
Reference in New Issue