HBASE-10378 Refactor write-ahead-log implementation -- ADDEDNUM
This commit is contained in:
parent
f5e05eb836
commit
b94e6738d9
|
@ -83,7 +83,7 @@ if [ $# = 0 ]; then
|
|||
echo "Some commands take arguments. Pass no args or -h for usage."
|
||||
echo " shell Run the HBase shell"
|
||||
echo " hbck Run the hbase 'fsck' tool"
|
||||
echo " hlog Write-ahead-log analyzer"
|
||||
echo " wal Write-ahead-log analyzer"
|
||||
echo " hfile Store file analyzer"
|
||||
echo " zkcli Run the ZooKeeper shell"
|
||||
echo " upgrade Upgrade hbase"
|
||||
|
@ -288,8 +288,9 @@ if [ "$COMMAND" = "shell" ] ; then
|
|||
CLASS="org.jruby.Main -X+O ${JRUBY_OPTS} ${HBASE_HOME}/bin/hirb.rb"
|
||||
elif [ "$COMMAND" = "hbck" ] ; then
|
||||
CLASS='org.apache.hadoop.hbase.util.HBaseFsck'
|
||||
elif [ "$COMMAND" = "hlog" ] ; then
|
||||
CLASS='org.apache.hadoop.hbase.regionserver.wal.HLogPrettyPrinter'
|
||||
# TODO remove old 'hlog' version
|
||||
elif [ "$COMMAND" = "hlog" -o "$COMMAND" = "wal" ] ; then
|
||||
CLASS='org.apache.hadoop.hbase.wal.WALPrettyPrinter'
|
||||
elif [ "$COMMAND" = "hfile" ] ; then
|
||||
CLASS='org.apache.hadoop.hbase.io.hfile.HFilePrettyPrinter'
|
||||
elif [ "$COMMAND" = "zkcli" ] ; then
|
||||
|
|
|
@ -200,7 +200,7 @@ goto :MakeCmdArgsLoop
|
|||
set hbase-command-arguments=%_hbasearguments%
|
||||
|
||||
@rem figure out which class to run
|
||||
set corecommands=shell master regionserver thrift thrift2 rest avro hlog hbck hfile zookeeper zkcli upgrade mapredcp
|
||||
set corecommands=shell master regionserver thrift thrift2 rest avro hlog wal hbck hfile zookeeper zkcli upgrade mapredcp
|
||||
for %%i in ( %corecommands% ) do (
|
||||
if "%hbase-command%"=="%%i" set corecommand=true
|
||||
)
|
||||
|
@ -364,8 +364,13 @@ goto :eof
|
|||
set CLASS=org.apache.hadoop.hbase.util.HBaseFsck
|
||||
goto :eof
|
||||
|
||||
@rem TODO remove older 'hlog' command
|
||||
:hlog
|
||||
set CLASS=org.apache.hadoop.hbase.regionserver.wal.HLogPrettyPrinter
|
||||
set CLASS=org.apache.hadoop.hbase.wal.WALPrettyPrinter
|
||||
goto :eof
|
||||
|
||||
:wal
|
||||
set CLASS=org.apache.hadoop.hbase.wal.WALPrettyPrinter
|
||||
goto :eof
|
||||
|
||||
:hfile
|
||||
|
@ -405,7 +410,7 @@ goto :eof
|
|||
echo Some commands take arguments. Pass no args or -h for usage."
|
||||
echo shell Run the HBase shell
|
||||
echo hbck Run the hbase 'fsck' tool
|
||||
echo hlog Write-ahead-log analyzer
|
||||
echo wal Write-ahead-log analyzer
|
||||
echo hfile Store file analyzer
|
||||
echo zkcli Run the ZooKeeper shell
|
||||
echo upgrade Upgrade hbase
|
||||
|
|
|
@ -22,7 +22,7 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
|
|||
|
||||
/**
|
||||
* Thrown during flush if the possibility snapshot content was not properly
|
||||
* persisted into store files. Response should include replay of hlog content.
|
||||
* persisted into store files. Response should include replay of wal content.
|
||||
*/
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Stable
|
||||
|
|
|
@ -890,18 +890,17 @@ public interface Admin extends Abortable, Closeable {
|
|||
HTableDescriptor[] getTableDescriptors(List<String> names) throws IOException;
|
||||
|
||||
/**
|
||||
* Roll the log writer. That is, start writing log messages to a new file.
|
||||
* Roll the log writer. I.e. for filesystem based write ahead logs, start writing to a new file.
|
||||
*
|
||||
* @param serverName The servername of the regionserver. A server name is made of host, port and
|
||||
* startcode. This is mandatory. Here is an example:
|
||||
* <code> host187.example.com,60020,1289493121758</code>
|
||||
* @return If lots of logs, flush the returned regions so next time through we can clean logs.
|
||||
* Returns null if nothing to flush. Names are actual region names as returned by {@link
|
||||
* HRegionInfo#getEncodedName()}
|
||||
* Note that the actual rolling of the log writer is asynchronous and may not be complete when
|
||||
* this method returns. As a side effect of this call, the named region server may schedule
|
||||
* store flushes at the request of the wal.
|
||||
*
|
||||
* @param serverName The servername of the regionserver.
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
* @throws org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException
|
||||
*/
|
||||
byte[][] rollHLogWriter(String serverName) throws IOException, FailedLogCloseException;
|
||||
void rollWALWriter(ServerName serverName) throws IOException, FailedLogCloseException;
|
||||
|
||||
/**
|
||||
* Helper delegage to getClusterStatus().getMasterCoprocessors().
|
||||
|
|
|
@ -64,7 +64,7 @@ import com.google.common.collect.Lists;
|
|||
* Adding a new peer results in creating new outbound connections from every
|
||||
* region server to a subset of region servers on the slave cluster. Each
|
||||
* new stream of replication will start replicating from the beginning of the
|
||||
* current HLog, meaning that edits from that past will be replicated.
|
||||
* current WAL, meaning that edits from that past will be replicated.
|
||||
* </p>
|
||||
* <p>
|
||||
* Removing a peer is a destructive and irreversible operation that stops
|
||||
|
|
|
@ -15,7 +15,7 @@ import java.io.IOException;
|
|||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.client.ClusterConnection;
|
||||
|
|
|
@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRespons
|
|||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetOnlineRegionResponse;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoResponse;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionResponse;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ServerInfo;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest;
|
||||
|
@ -183,21 +182,6 @@ public final class ResponseConverter {
|
|||
// End utilities for Client
|
||||
// Start utilities for Admin
|
||||
|
||||
/**
|
||||
* Get the list of regions to flush from a RollLogWriterResponse
|
||||
*
|
||||
* @param proto the RollLogWriterResponse
|
||||
* @return the the list of regions to flush
|
||||
*/
|
||||
public static byte[][] getRegions(final RollWALWriterResponse proto) {
|
||||
if (proto == null || proto.getRegionToFlushCount() == 0) return null;
|
||||
List<byte[]> regions = new ArrayList<byte[]>();
|
||||
for (ByteString region: proto.getRegionToFlushList()) {
|
||||
regions.add(region.toByteArray());
|
||||
}
|
||||
return (byte[][])regions.toArray();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the list of region info from a GetOnlineRegionResponse
|
||||
*
|
||||
|
|
|
@ -26,7 +26,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
|||
|
||||
/**
|
||||
* This provides an interface for maintaining a region server's replication queues. These queues
|
||||
* keep track of the HLogs that still need to be replicated to remote clusters.
|
||||
* keep track of the WALs that still need to be replicated to remote clusters.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public interface ReplicationQueues {
|
||||
|
@ -45,31 +45,31 @@ public interface ReplicationQueues {
|
|||
void removeQueue(String queueId);
|
||||
|
||||
/**
|
||||
* Add a new HLog file to the given queue. If the queue does not exist it is created.
|
||||
* Add a new WAL file to the given queue. If the queue does not exist it is created.
|
||||
* @param queueId a String that identifies the queue.
|
||||
* @param filename name of the HLog
|
||||
* @param filename name of the WAL
|
||||
*/
|
||||
void addLog(String queueId, String filename) throws ReplicationException;
|
||||
|
||||
/**
|
||||
* Remove an HLog file from the given queue.
|
||||
* Remove an WAL file from the given queue.
|
||||
* @param queueId a String that identifies the queue.
|
||||
* @param filename name of the HLog
|
||||
* @param filename name of the WAL
|
||||
*/
|
||||
void removeLog(String queueId, String filename);
|
||||
|
||||
/**
|
||||
* Set the current position for a specific HLog in a given queue.
|
||||
* Set the current position for a specific WAL in a given queue.
|
||||
* @param queueId a String that identifies the queue
|
||||
* @param filename name of the HLog
|
||||
* @param filename name of the WAL
|
||||
* @param position the current position in the file
|
||||
*/
|
||||
void setLogPosition(String queueId, String filename, long position);
|
||||
|
||||
/**
|
||||
* Get the current position for a specific HLog in a given queue.
|
||||
* Get the current position for a specific WAL in a given queue.
|
||||
* @param queueId a String that identifies the queue
|
||||
* @param filename name of the HLog
|
||||
* @param filename name of the WAL
|
||||
* @return the current position in the file
|
||||
*/
|
||||
long getLogPosition(String queueId, String filename) throws ReplicationException;
|
||||
|
@ -80,9 +80,9 @@ public interface ReplicationQueues {
|
|||
void removeAllQueues();
|
||||
|
||||
/**
|
||||
* Get a list of all HLogs in the given queue.
|
||||
* Get a list of all WALs in the given queue.
|
||||
* @param queueId a String that identifies the queue
|
||||
* @return a list of HLogs, null if this region server is dead and has no outstanding queues
|
||||
* @return a list of WALs, null if this region server is dead and has no outstanding queues
|
||||
*/
|
||||
List<String> getLogsInQueue(String queueId);
|
||||
|
||||
|
@ -95,7 +95,7 @@ public interface ReplicationQueues {
|
|||
/**
|
||||
* Take ownership for the set of queues belonging to a dead region server.
|
||||
* @param regionserver the id of the dead region server
|
||||
* @return A SortedMap of the queues that have been claimed, including a SortedSet of HLogs in
|
||||
* @return A SortedMap of the queues that have been claimed, including a SortedSet of WALs in
|
||||
* each queue. Returns an empty map if no queues were failed-over.
|
||||
*/
|
||||
SortedMap<String, SortedSet<String>> claimQueues(String regionserver);
|
||||
|
|
|
@ -23,7 +23,7 @@ import java.util.List;
|
|||
|
||||
/**
|
||||
* This provides an interface for clients of replication to view replication queues. These queues
|
||||
* keep track of the HLogs that still need to be replicated to remote clusters.
|
||||
* keep track of the WALs that still need to be replicated to remote clusters.
|
||||
*/
|
||||
public interface ReplicationQueuesClient {
|
||||
|
||||
|
@ -40,10 +40,10 @@ public interface ReplicationQueuesClient {
|
|||
List<String> getListOfReplicators();
|
||||
|
||||
/**
|
||||
* Get a list of all HLogs in the given queue on the given region server.
|
||||
* Get a list of all WALs in the given queue on the given region server.
|
||||
* @param serverName the server name of the region server that owns the queue
|
||||
* @param queueId a String that identifies the queue
|
||||
* @return a list of HLogs, null if this region server is dead and has no outstanding queues
|
||||
* @return a list of WALs, null if this region server is dead and has no outstanding queues
|
||||
*/
|
||||
List<String> getLogsInQueue(String serverName, String queueId);
|
||||
|
||||
|
|
|
@ -51,7 +51,7 @@ public class ReplicationQueuesClientZKImpl extends ReplicationStateZKBase implem
|
|||
try {
|
||||
result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode);
|
||||
} catch (KeeperException e) {
|
||||
this.abortable.abort("Failed to get list of hlogs for queueId=" + queueId
|
||||
this.abortable.abort("Failed to get list of wals for queueId=" + queueId
|
||||
+ " and serverName=" + serverName, e);
|
||||
}
|
||||
return result;
|
||||
|
|
|
@ -42,21 +42,21 @@ import org.apache.zookeeper.KeeperException;
|
|||
/**
|
||||
* This class provides an implementation of the ReplicationQueues interface using Zookeeper. The
|
||||
* base znode that this class works at is the myQueuesZnode. The myQueuesZnode contains a list of
|
||||
* all outstanding HLog files on this region server that need to be replicated. The myQueuesZnode is
|
||||
* all outstanding WAL files on this region server that need to be replicated. The myQueuesZnode is
|
||||
* the regionserver name (a concatenation of the region server’s hostname, client port and start
|
||||
* code). For example:
|
||||
*
|
||||
* /hbase/replication/rs/hostname.example.org,6020,1234
|
||||
*
|
||||
* Within this znode, the region server maintains a set of HLog replication queues. These queues are
|
||||
* Within this znode, the region server maintains a set of WAL replication queues. These queues are
|
||||
* represented by child znodes named using there give queue id. For example:
|
||||
*
|
||||
* /hbase/replication/rs/hostname.example.org,6020,1234/1
|
||||
* /hbase/replication/rs/hostname.example.org,6020,1234/2
|
||||
*
|
||||
* Each queue has one child znode for every HLog that still needs to be replicated. The value of
|
||||
* these HLog child znodes is the latest position that has been replicated. This position is updated
|
||||
* every time a HLog entry is replicated. For example:
|
||||
* Each queue has one child znode for every WAL that still needs to be replicated. The value of
|
||||
* these WAL child znodes is the latest position that has been replicated. This position is updated
|
||||
* every time a WAL entry is replicated. For example:
|
||||
*
|
||||
* /hbase/replication/rs/hostname.example.org,6020,1234/1/23522342.23422 [VALUE: 254]
|
||||
*/
|
||||
|
@ -113,7 +113,7 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
|
|||
znode = ZKUtil.joinZNode(znode, filename);
|
||||
ZKUtil.deleteNode(this.zookeeper, znode);
|
||||
} catch (KeeperException e) {
|
||||
this.abortable.abort("Failed to remove hlog from queue (queueId=" + queueId + ", filename="
|
||||
this.abortable.abort("Failed to remove wal from queue (queueId=" + queueId + ", filename="
|
||||
+ filename + ")", e);
|
||||
}
|
||||
}
|
||||
|
@ -126,7 +126,7 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
|
|||
// Why serialize String of Long and not Long as bytes?
|
||||
ZKUtil.setData(this.zookeeper, znode, ZKUtil.positionToByteArray(position));
|
||||
} catch (KeeperException e) {
|
||||
this.abortable.abort("Failed to write replication hlog position (filename=" + filename
|
||||
this.abortable.abort("Failed to write replication wal position (filename=" + filename
|
||||
+ ", position=" + position + ")", e);
|
||||
}
|
||||
}
|
||||
|
@ -146,12 +146,12 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
|
|||
return 0;
|
||||
}
|
||||
try {
|
||||
return ZKUtil.parseHLogPositionFrom(bytes);
|
||||
return ZKUtil.parseWALPositionFrom(bytes);
|
||||
} catch (DeserializationException de) {
|
||||
LOG.warn("Failed to parse HLogPosition for queueId=" + queueId + " and hlog=" + filename
|
||||
LOG.warn("Failed to parse WALPosition for queueId=" + queueId + " and wal=" + filename
|
||||
+ "znode content, continuing.");
|
||||
}
|
||||
// if we can not parse the position, start at the beginning of the hlog file
|
||||
// if we can not parse the position, start at the beginning of the wal file
|
||||
// again
|
||||
return 0;
|
||||
}
|
||||
|
@ -166,10 +166,10 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
|
|||
SortedMap<String, SortedSet<String>> newQueues = new TreeMap<String, SortedSet<String>>();
|
||||
// check whether there is multi support. If yes, use it.
|
||||
if (conf.getBoolean(HConstants.ZOOKEEPER_USEMULTI, true)) {
|
||||
LOG.info("Atomically moving " + regionserverZnode + "'s hlogs to my queue");
|
||||
LOG.info("Atomically moving " + regionserverZnode + "'s wals to my queue");
|
||||
newQueues = copyQueuesFromRSUsingMulti(regionserverZnode);
|
||||
} else {
|
||||
LOG.info("Moving " + regionserverZnode + "'s hlogs to my queue");
|
||||
LOG.info("Moving " + regionserverZnode + "'s wals to my queue");
|
||||
if (!lockOtherRS(regionserverZnode)) {
|
||||
return newQueues;
|
||||
}
|
||||
|
@ -200,7 +200,7 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
|
|||
try {
|
||||
result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode);
|
||||
} catch (KeeperException e) {
|
||||
this.abortable.abort("Failed to get list of hlogs for queueId=" + queueId, e);
|
||||
this.abortable.abort("Failed to get list of wals for queueId=" + queueId, e);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
@ -283,10 +283,10 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
|
|||
}
|
||||
|
||||
/**
|
||||
* It "atomically" copies all the hlogs queues from another region server and returns them all
|
||||
* It "atomically" copies all the wals queues from another region server and returns them all
|
||||
* sorted per peer cluster (appended with the dead server's znode).
|
||||
* @param znode pertaining to the region server to copy the queues from
|
||||
* @return HLog queues sorted per peer cluster
|
||||
* @return WAL queues sorted per peer cluster
|
||||
*/
|
||||
private SortedMap<String, SortedSet<String>> copyQueuesFromRSUsingMulti(String znode) {
|
||||
SortedMap<String, SortedSet<String>> queues = new TreeMap<String, SortedSet<String>>();
|
||||
|
@ -308,8 +308,8 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
|
|||
String newPeerZnode = ZKUtil.joinZNode(this.myQueuesZnode, newPeerId);
|
||||
// check the logs queue for the old peer cluster
|
||||
String oldClusterZnode = ZKUtil.joinZNode(deadRSZnodePath, peerId);
|
||||
List<String> hlogs = ZKUtil.listChildrenNoWatch(this.zookeeper, oldClusterZnode);
|
||||
if (hlogs == null || hlogs.size() == 0) {
|
||||
List<String> wals = ZKUtil.listChildrenNoWatch(this.zookeeper, oldClusterZnode);
|
||||
if (wals == null || wals.size() == 0) {
|
||||
listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldClusterZnode));
|
||||
continue; // empty log queue.
|
||||
}
|
||||
|
@ -319,15 +319,15 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
|
|||
ZKUtilOp op = ZKUtilOp.createAndFailSilent(newPeerZnode, HConstants.EMPTY_BYTE_ARRAY);
|
||||
listOfOps.add(op);
|
||||
// get the offset of the logs and set it to new znodes
|
||||
for (String hlog : hlogs) {
|
||||
String oldHlogZnode = ZKUtil.joinZNode(oldClusterZnode, hlog);
|
||||
byte[] logOffset = ZKUtil.getData(this.zookeeper, oldHlogZnode);
|
||||
LOG.debug("Creating " + hlog + " with data " + Bytes.toString(logOffset));
|
||||
String newLogZnode = ZKUtil.joinZNode(newPeerZnode, hlog);
|
||||
for (String wal : wals) {
|
||||
String oldWalZnode = ZKUtil.joinZNode(oldClusterZnode, wal);
|
||||
byte[] logOffset = ZKUtil.getData(this.zookeeper, oldWalZnode);
|
||||
LOG.debug("Creating " + wal + " with data " + Bytes.toString(logOffset));
|
||||
String newLogZnode = ZKUtil.joinZNode(newPeerZnode, wal);
|
||||
listOfOps.add(ZKUtilOp.createAndFailSilent(newLogZnode, logOffset));
|
||||
// add ops for deleting
|
||||
listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldHlogZnode));
|
||||
logQueue.add(hlog);
|
||||
listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldWalZnode));
|
||||
logQueue.add(wal);
|
||||
}
|
||||
// add delete op for peer
|
||||
listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldClusterZnode));
|
||||
|
@ -350,10 +350,10 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
|
|||
}
|
||||
|
||||
/**
|
||||
* This methods copies all the hlogs queues from another region server and returns them all sorted
|
||||
* This methods copies all the wals queues from another region server and returns them all sorted
|
||||
* per peer cluster (appended with the dead server's znode)
|
||||
* @param znode server names to copy
|
||||
* @return all hlogs for all peers of that cluster, null if an error occurred
|
||||
* @return all wals for all peers of that cluster, null if an error occurred
|
||||
*/
|
||||
private SortedMap<String, SortedSet<String>> copyQueuesFromRS(String znode) {
|
||||
// TODO this method isn't atomic enough, we could start copying and then
|
||||
|
@ -381,31 +381,31 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
|
|||
String newCluster = cluster + "-" + znode;
|
||||
String newClusterZnode = ZKUtil.joinZNode(this.myQueuesZnode, newCluster);
|
||||
String clusterPath = ZKUtil.joinZNode(nodePath, cluster);
|
||||
List<String> hlogs = ZKUtil.listChildrenNoWatch(this.zookeeper, clusterPath);
|
||||
List<String> wals = ZKUtil.listChildrenNoWatch(this.zookeeper, clusterPath);
|
||||
// That region server didn't have anything to replicate for this cluster
|
||||
if (hlogs == null || hlogs.size() == 0) {
|
||||
if (wals == null || wals.size() == 0) {
|
||||
continue;
|
||||
}
|
||||
ZKUtil.createNodeIfNotExistsAndWatch(this.zookeeper, newClusterZnode,
|
||||
HConstants.EMPTY_BYTE_ARRAY);
|
||||
SortedSet<String> logQueue = new TreeSet<String>();
|
||||
queues.put(newCluster, logQueue);
|
||||
for (String hlog : hlogs) {
|
||||
String z = ZKUtil.joinZNode(clusterPath, hlog);
|
||||
for (String wal : wals) {
|
||||
String z = ZKUtil.joinZNode(clusterPath, wal);
|
||||
byte[] positionBytes = ZKUtil.getData(this.zookeeper, z);
|
||||
long position = 0;
|
||||
try {
|
||||
position = ZKUtil.parseHLogPositionFrom(positionBytes);
|
||||
position = ZKUtil.parseWALPositionFrom(positionBytes);
|
||||
} catch (DeserializationException e) {
|
||||
LOG.warn("Failed parse of hlog position from the following znode: " + z
|
||||
LOG.warn("Failed parse of wal position from the following znode: " + z
|
||||
+ ", Exception: " + e);
|
||||
}
|
||||
LOG.debug("Creating " + hlog + " with data " + position);
|
||||
String child = ZKUtil.joinZNode(newClusterZnode, hlog);
|
||||
LOG.debug("Creating " + wal + " with data " + position);
|
||||
String child = ZKUtil.joinZNode(newClusterZnode, wal);
|
||||
// Position doesn't actually change, we are just deserializing it for
|
||||
// logging, so just use the already serialized version
|
||||
ZKUtil.createAndWatch(this.zookeeper, child, positionBytes);
|
||||
logQueue.add(hlog);
|
||||
logQueue.add(wal);
|
||||
}
|
||||
}
|
||||
} catch (KeeperException e) {
|
||||
|
|
|
@ -1670,7 +1670,7 @@ public class ZKUtil {
|
|||
if (data != null && data.length > 0) { // log position
|
||||
long position = 0;
|
||||
try {
|
||||
position = ZKUtil.parseHLogPositionFrom(ZKUtil.getData(zkw, znodeToProcess));
|
||||
position = ZKUtil.parseWALPositionFrom(ZKUtil.getData(zkw, znodeToProcess));
|
||||
sb.append(position);
|
||||
} catch (DeserializationException ignored) {
|
||||
} catch (InterruptedException e) {
|
||||
|
@ -1884,7 +1884,7 @@ public class ZKUtil {
|
|||
/**
|
||||
* @param position
|
||||
* @return Serialized protobuf of <code>position</code> with pb magic prefix prepended suitable
|
||||
* for use as content of an hlog position in a replication queue.
|
||||
* for use as content of an wal position in a replication queue.
|
||||
*/
|
||||
public static byte[] positionToByteArray(final long position) {
|
||||
byte[] bytes = ZooKeeperProtos.ReplicationHLogPosition.newBuilder().setPosition(position)
|
||||
|
@ -1893,13 +1893,13 @@ public class ZKUtil {
|
|||
}
|
||||
|
||||
/**
|
||||
* @param bytes - Content of a HLog position znode.
|
||||
* @return long - The current HLog position.
|
||||
* @param bytes - Content of a WAL position znode.
|
||||
* @return long - The current WAL position.
|
||||
* @throws DeserializationException
|
||||
*/
|
||||
public static long parseHLogPositionFrom(final byte[] bytes) throws DeserializationException {
|
||||
public static long parseWALPositionFrom(final byte[] bytes) throws DeserializationException {
|
||||
if (bytes == null) {
|
||||
throw new DeserializationException("Unable to parse null HLog position.");
|
||||
throw new DeserializationException("Unable to parse null WAL position.");
|
||||
}
|
||||
if (ProtobufUtil.isPBMagicPrefix(bytes)) {
|
||||
int pblen = ProtobufUtil.lengthOfPBMagic();
|
||||
|
|
|
@ -899,7 +899,7 @@ public final class HConstants {
|
|||
/** File permission umask to use when creating hbase data files */
|
||||
public static final String DATA_FILE_UMASK_KEY = "hbase.data.umask";
|
||||
|
||||
/** Configuration name of HLog Compression */
|
||||
/** Configuration name of WAL Compression */
|
||||
public static final String ENABLE_WAL_COMPRESSION =
|
||||
"hbase.regionserver.wal.enablecompression";
|
||||
|
||||
|
@ -1031,7 +1031,7 @@ public final class HConstants {
|
|||
/** Configuration key for the name of the master WAL encryption key for the cluster, a string */
|
||||
public static final String CRYPTO_WAL_KEY_NAME_CONF_KEY = "hbase.crypto.wal.key.name";
|
||||
|
||||
/** Configuration key for enabling HLog encryption, a boolean */
|
||||
/** Configuration key for enabling WAL encryption, a boolean */
|
||||
public static final String ENABLE_WAL_ENCRYPTION = "hbase.regionserver.wal.encryption";
|
||||
|
||||
/** Configuration key for setting RPC codec class name */
|
||||
|
|
|
@ -1135,7 +1135,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
|
|||
|
||||
/**
|
||||
* Produces a string map for this key/value pair. Useful for programmatic use
|
||||
* and manipulation of the data stored in an HLogKey, for example, printing
|
||||
* and manipulation of the data stored in an WALKey, for example, printing
|
||||
* as JSON. Values are left out due to their tendency to be large. If needed,
|
||||
* they can be added manually.
|
||||
*
|
||||
|
|
|
@ -112,8 +112,8 @@ possible configurations would overwhelm and obscure the important.
|
|||
<name>hbase.master.logcleaner.plugins</name>
|
||||
<value>org.apache.hadoop.hbase.master.cleaner.TimeToLiveLogCleaner</value>
|
||||
<description>A comma-separated list of BaseLogCleanerDelegate invoked by
|
||||
the LogsCleaner service. These WAL/HLog cleaners are called in order,
|
||||
so put the HLog cleaner that prunes the most HLog files in front. To
|
||||
the LogsCleaner service. These WAL cleaners are called in order,
|
||||
so put the cleaner that prunes the most files in front. To
|
||||
implement your own BaseLogCleanerDelegate, just put it in HBase's classpath
|
||||
and add the fully qualified class name here. Always add the above
|
||||
default log cleaners in the list.</description>
|
||||
|
@ -121,7 +121,7 @@ possible configurations would overwhelm and obscure the important.
|
|||
<property>
|
||||
<name>hbase.master.logcleaner.ttl</name>
|
||||
<value>600000</value>
|
||||
<description>Maximum time a HLog can stay in the .oldlogdir directory,
|
||||
<description>Maximum time a WAL can stay in the .oldlogdir directory,
|
||||
after which it will be cleaned by a Master thread.</description>
|
||||
</property>
|
||||
<property>
|
||||
|
@ -265,12 +265,12 @@ possible configurations would overwhelm and obscure the important.
|
|||
<property>
|
||||
<name>hbase.regionserver.hlog.reader.impl</name>
|
||||
<value>org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader</value>
|
||||
<description>The HLog file reader implementation.</description>
|
||||
<description>The WAL file reader implementation.</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>hbase.regionserver.hlog.writer.impl</name>
|
||||
<value>org.apache.hadoop.hbase.regionserver.wal.ProtobufLogWriter</value>
|
||||
<description>The HLog file writer implementation.</description>
|
||||
<description>The WAL file writer implementation.</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>hbase.master.distributed.log.replay</name>
|
||||
|
|
|
@ -48,9 +48,9 @@ public interface MetricsMasterFileSystemSource extends BaseSource {
|
|||
String SPLIT_SIZE_NAME = "hlogSplitSize";
|
||||
|
||||
String META_SPLIT_TIME_DESC = "Time it takes to finish splitMetaLog()";
|
||||
String META_SPLIT_SIZE_DESC = "Size of hbase:meta HLog files being split";
|
||||
String SPLIT_TIME_DESC = "Time it takes to finish HLog.splitLog()";
|
||||
String SPLIT_SIZE_DESC = "Size of HLog files being split";
|
||||
String META_SPLIT_SIZE_DESC = "Size of hbase:meta WAL files being split";
|
||||
String SPLIT_TIME_DESC = "Time it takes to finish WAL.splitLog()";
|
||||
String SPLIT_SIZE_DESC = "Size of WAL files being split";
|
||||
|
||||
|
||||
void updateMetaWALSplitTime(long time);
|
||||
|
|
|
@ -129,10 +129,10 @@ public interface MetricsRegionServerSource extends BaseSource {
|
|||
String REGION_COUNT_DESC = "Number of regions";
|
||||
String STORE_COUNT = "storeCount";
|
||||
String STORE_COUNT_DESC = "Number of Stores";
|
||||
String HLOGFILE_COUNT = "hlogFileCount";
|
||||
String HLOGFILE_COUNT_DESC = "Number of HLog Files";
|
||||
String HLOGFILE_SIZE = "hlogFileSize";
|
||||
String HLOGFILE_SIZE_DESC = "Size of all HLog Files";
|
||||
String WALFILE_COUNT = "hlogFileCount";
|
||||
String WALFILE_COUNT_DESC = "Number of WAL Files";
|
||||
String WALFILE_SIZE = "hlogFileSize";
|
||||
String WALFILE_SIZE_DESC = "Size of all WAL Files";
|
||||
String STOREFILE_COUNT = "storeFileCount";
|
||||
String STOREFILE_COUNT_DESC = "Number of Store Files";
|
||||
String MEMSTORE_SIZE = "memStoreSize";
|
||||
|
|
|
@ -68,14 +68,14 @@ public interface MetricsRegionServerWrapper {
|
|||
long getNumStores();
|
||||
|
||||
/**
|
||||
* Get the number of HLog files of this region server.
|
||||
* Get the number of WAL files of this region server.
|
||||
*/
|
||||
public long getNumHLogFiles();
|
||||
public long getNumWALFiles();
|
||||
|
||||
/**
|
||||
* Get the size of HLog files of this region server.
|
||||
* Get the size of WAL files of this region server.
|
||||
*/
|
||||
public long getHLogFileSize();
|
||||
public long getWALFileSize();
|
||||
|
||||
/**
|
||||
* Get the number of store files hosted on this region server.
|
||||
|
|
|
@ -39,7 +39,7 @@ public interface MetricsEditsReplaySource extends BaseSource {
|
|||
/**
|
||||
* Description
|
||||
*/
|
||||
String METRICS_DESCRIPTION = "Metrics about HBase RegionServer HLog Edits Replay";
|
||||
String METRICS_DESCRIPTION = "Metrics about HBase RegionServer WAL Edits Replay";
|
||||
|
||||
/**
|
||||
* The name of the metrics context that metrics will be under in jmx
|
||||
|
|
|
@ -21,7 +21,7 @@ package org.apache.hadoop.hbase.regionserver.wal;
|
|||
import org.apache.hadoop.hbase.metrics.BaseSource;
|
||||
|
||||
/**
|
||||
* Interface of the source that will export metrics about the region server's HLog.
|
||||
* Interface of the source that will export metrics about the region server's WAL.
|
||||
*/
|
||||
public interface MetricsWALSource extends BaseSource {
|
||||
|
||||
|
@ -39,7 +39,7 @@ public interface MetricsWALSource extends BaseSource {
|
|||
/**
|
||||
* Description
|
||||
*/
|
||||
String METRICS_DESCRIPTION = "Metrics about HBase RegionServer HLog";
|
||||
String METRICS_DESCRIPTION = "Metrics about HBase RegionServer WAL";
|
||||
|
||||
/**
|
||||
* The name of the metrics context that metrics will be under in jmx
|
||||
|
@ -52,11 +52,11 @@ public interface MetricsWALSource extends BaseSource {
|
|||
String APPEND_COUNT = "appendCount";
|
||||
String APPEND_COUNT_DESC = "Number of appends to the write ahead log.";
|
||||
String APPEND_SIZE = "appendSize";
|
||||
String APPEND_SIZE_DESC = "Size (in bytes) of the data appended to the HLog.";
|
||||
String APPEND_SIZE_DESC = "Size (in bytes) of the data appended to the WAL.";
|
||||
String SLOW_APPEND_COUNT = "slowAppendCount";
|
||||
String SLOW_APPEND_COUNT_DESC = "Number of appends that were slow.";
|
||||
String SYNC_TIME = "syncTime";
|
||||
String SYNC_TIME_DESC = "The time it took to sync the HLog to HDFS.";
|
||||
String SYNC_TIME_DESC = "The time it took to sync the WAL to HDFS.";
|
||||
|
||||
/**
|
||||
* Add the append size.
|
||||
|
@ -69,7 +69,7 @@ public interface MetricsWALSource extends BaseSource {
|
|||
void incrementAppendTime(long time);
|
||||
|
||||
/**
|
||||
* Increment the count of hlog appends
|
||||
* Increment the count of wal appends
|
||||
*/
|
||||
void incrementAppendCount();
|
||||
|
||||
|
@ -79,7 +79,7 @@ public interface MetricsWALSource extends BaseSource {
|
|||
void incrementSlowAppendCount();
|
||||
|
||||
/**
|
||||
* Add the time it took to sync the hlog.
|
||||
* Add the time it took to sync the wal.
|
||||
*/
|
||||
void incrementSyncTime(long time);
|
||||
|
||||
|
|
|
@ -1,32 +0,0 @@
|
|||
/**
|
||||
* 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.CompatibilitySingletonFactory;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestMetricsHLogSource {
|
||||
|
||||
@Test(expected=RuntimeException.class)
|
||||
public void testGetInstanceNoHadoopCompat() throws Exception {
|
||||
//This should throw an exception because there is no compat lib on the class path.
|
||||
CompatibilitySingletonFactory.getInstance(MetricsWALSource.class);
|
||||
|
||||
}
|
||||
}
|
|
@ -168,8 +168,8 @@ public class MetricsRegionServerSourceImpl
|
|||
if (rsWrap != null) {
|
||||
mrb.addGauge(Interns.info(REGION_COUNT, REGION_COUNT_DESC), rsWrap.getNumOnlineRegions())
|
||||
.addGauge(Interns.info(STORE_COUNT, STORE_COUNT_DESC), rsWrap.getNumStores())
|
||||
.addGauge(Interns.info(HLOGFILE_COUNT, HLOGFILE_COUNT_DESC), rsWrap.getNumHLogFiles())
|
||||
.addGauge(Interns.info(HLOGFILE_SIZE, HLOGFILE_SIZE_DESC), rsWrap.getHLogFileSize())
|
||||
.addGauge(Interns.info(WALFILE_COUNT, WALFILE_COUNT_DESC), rsWrap.getNumWALFiles())
|
||||
.addGauge(Interns.info(WALFILE_SIZE, WALFILE_SIZE_DESC), rsWrap.getWALFileSize())
|
||||
.addGauge(Interns.info(STOREFILE_COUNT, STOREFILE_COUNT_DESC), rsWrap.getNumStoreFiles())
|
||||
.addGauge(Interns.info(MEMSTORE_SIZE, MEMSTORE_SIZE_DESC), rsWrap.getMemstoreSize())
|
||||
.addGauge(Interns.info(STOREFILE_SIZE, STOREFILE_SIZE_DESC), rsWrap.getStoreFileSize())
|
||||
|
|
|
@ -25,9 +25,10 @@ import org.apache.hadoop.metrics2.lib.MutableCounterLong;
|
|||
|
||||
|
||||
/**
|
||||
* Class that transitions metrics from HLog's MetricsWAL into the metrics subsystem.
|
||||
* Class that transitions metrics from MetricsWAL into the metrics subsystem.
|
||||
*
|
||||
* Implements BaseSource through BaseSourceImpl, following the pattern.
|
||||
* @see org.apache.hadoop.hbase.regionserver.wal.MetricsWAL
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class MetricsWALSourceImpl extends BaseSourceImpl implements MetricsWALSource {
|
||||
|
|
|
@ -26,7 +26,8 @@ import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting;
|
|||
import org.apache.hadoop.hbase.io.hfile.HFile;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileReaderV3;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileWriterV3;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLog;
|
||||
import org.apache.hadoop.hbase.wal.WAL.Reader;
|
||||
import org.apache.hadoop.hbase.wal.WALProvider.Writer;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.SecureProtobufLogReader;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.SecureProtobufLogWriter;
|
||||
import org.apache.hadoop.hbase.testclassification.IntegrationTests;
|
||||
|
@ -61,9 +62,9 @@ public class IntegrationTestIngestWithEncryption extends IntegrationTestIngest {
|
|||
conf.set(HConstants.CRYPTO_KEYPROVIDER_CONF_KEY, KeyProviderForTesting.class.getName());
|
||||
conf.set(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY, "hbase");
|
||||
conf.setClass("hbase.regionserver.hlog.reader.impl", SecureProtobufLogReader.class,
|
||||
HLog.Reader.class);
|
||||
Reader.class);
|
||||
conf.setClass("hbase.regionserver.hlog.writer.impl", SecureProtobufLogWriter.class,
|
||||
HLog.Writer.class);
|
||||
Writer.class);
|
||||
conf.setBoolean(HConstants.ENABLE_WAL_ENCRYPTION, true);
|
||||
}
|
||||
super.setUpCluster();
|
||||
|
|
|
@ -82,7 +82,7 @@ import com.google.common.base.Objects;
|
|||
* <ol>
|
||||
* <li>
|
||||
* Load Test Tool.<br/>
|
||||
* This runs so that all RegionServers will have some load and HLogs will be full.
|
||||
* This runs so that all RegionServers will have some load and WALs will be full.
|
||||
* </li>
|
||||
* <li>
|
||||
* Scan thread.<br/>
|
||||
|
@ -151,7 +151,7 @@ public class IntegrationTestMTTR {
|
|||
private static Action restartMasterAction;
|
||||
|
||||
/**
|
||||
* The load test tool used to create load and make sure that HLogs aren't empty.
|
||||
* The load test tool used to create load and make sure that WALs aren't empty.
|
||||
*/
|
||||
private static LoadTestTool loadTool;
|
||||
|
||||
|
|
|
@ -16956,6 +16956,12 @@ public final class AdminProtos {
|
|||
}
|
||||
/**
|
||||
* Protobuf type {@code RollWALWriterResponse}
|
||||
*
|
||||
* <pre>
|
||||
*
|
||||
* Roll request responses no longer include regions to flush
|
||||
* this list will always be empty when talking to a 1.0 server
|
||||
* </pre>
|
||||
*/
|
||||
public static final class RollWALWriterResponse extends
|
||||
com.google.protobuf.GeneratedMessage
|
||||
|
@ -17241,6 +17247,12 @@ public final class AdminProtos {
|
|||
}
|
||||
/**
|
||||
* Protobuf type {@code RollWALWriterResponse}
|
||||
*
|
||||
* <pre>
|
||||
*
|
||||
* Roll request responses no longer include regions to flush
|
||||
* this list will always be empty when talking to a 1.0 server
|
||||
* </pre>
|
||||
*/
|
||||
public static final class Builder extends
|
||||
com.google.protobuf.GeneratedMessage.Builder<Builder>
|
||||
|
|
|
@ -4496,7 +4496,7 @@ public final class RegionServerStatusProtos {
|
|||
* <code>required uint64 last_flushed_sequence_id = 1;</code>
|
||||
*
|
||||
* <pre>
|
||||
** the last HLog sequence id flushed from MemStore to HFile for the region
|
||||
* the last WAL sequence id flushed from MemStore to HFile for the region
|
||||
* </pre>
|
||||
*/
|
||||
boolean hasLastFlushedSequenceId();
|
||||
|
@ -4504,7 +4504,7 @@ public final class RegionServerStatusProtos {
|
|||
* <code>required uint64 last_flushed_sequence_id = 1;</code>
|
||||
*
|
||||
* <pre>
|
||||
** the last HLog sequence id flushed from MemStore to HFile for the region
|
||||
* the last WAL sequence id flushed from MemStore to HFile for the region
|
||||
* </pre>
|
||||
*/
|
||||
long getLastFlushedSequenceId();
|
||||
|
@ -4612,7 +4612,7 @@ public final class RegionServerStatusProtos {
|
|||
* <code>required uint64 last_flushed_sequence_id = 1;</code>
|
||||
*
|
||||
* <pre>
|
||||
** the last HLog sequence id flushed from MemStore to HFile for the region
|
||||
* the last WAL sequence id flushed from MemStore to HFile for the region
|
||||
* </pre>
|
||||
*/
|
||||
public boolean hasLastFlushedSequenceId() {
|
||||
|
@ -4622,7 +4622,7 @@ public final class RegionServerStatusProtos {
|
|||
* <code>required uint64 last_flushed_sequence_id = 1;</code>
|
||||
*
|
||||
* <pre>
|
||||
** the last HLog sequence id flushed from MemStore to HFile for the region
|
||||
* the last WAL sequence id flushed from MemStore to HFile for the region
|
||||
* </pre>
|
||||
*/
|
||||
public long getLastFlushedSequenceId() {
|
||||
|
@ -4908,7 +4908,7 @@ public final class RegionServerStatusProtos {
|
|||
* <code>required uint64 last_flushed_sequence_id = 1;</code>
|
||||
*
|
||||
* <pre>
|
||||
** the last HLog sequence id flushed from MemStore to HFile for the region
|
||||
* the last WAL sequence id flushed from MemStore to HFile for the region
|
||||
* </pre>
|
||||
*/
|
||||
public boolean hasLastFlushedSequenceId() {
|
||||
|
@ -4918,7 +4918,7 @@ public final class RegionServerStatusProtos {
|
|||
* <code>required uint64 last_flushed_sequence_id = 1;</code>
|
||||
*
|
||||
* <pre>
|
||||
** the last HLog sequence id flushed from MemStore to HFile for the region
|
||||
* the last WAL sequence id flushed from MemStore to HFile for the region
|
||||
* </pre>
|
||||
*/
|
||||
public long getLastFlushedSequenceId() {
|
||||
|
@ -4928,7 +4928,7 @@ public final class RegionServerStatusProtos {
|
|||
* <code>required uint64 last_flushed_sequence_id = 1;</code>
|
||||
*
|
||||
* <pre>
|
||||
** the last HLog sequence id flushed from MemStore to HFile for the region
|
||||
* the last WAL sequence id flushed from MemStore to HFile for the region
|
||||
* </pre>
|
||||
*/
|
||||
public Builder setLastFlushedSequenceId(long value) {
|
||||
|
@ -4941,7 +4941,7 @@ public final class RegionServerStatusProtos {
|
|||
* <code>required uint64 last_flushed_sequence_id = 1;</code>
|
||||
*
|
||||
* <pre>
|
||||
** the last HLog sequence id flushed from MemStore to HFile for the region
|
||||
* the last WAL sequence id flushed from MemStore to HFile for the region
|
||||
* </pre>
|
||||
*/
|
||||
public Builder clearLastFlushedSequenceId() {
|
||||
|
|
|
@ -1242,7 +1242,9 @@ public final class WALProtos {
|
|||
* Protobuf type {@code WALKey}
|
||||
*
|
||||
* <pre>
|
||||
* Protocol buffer version of HLogKey; see HLogKey comment, not really a key but WALEdit header for some KVs
|
||||
*
|
||||
* Protocol buffer version of WALKey; see WALKey comment, not really a key but WALEdit header
|
||||
* for some KVs
|
||||
* </pre>
|
||||
*/
|
||||
public static final class WALKey extends
|
||||
|
@ -2033,7 +2035,9 @@ public final class WALProtos {
|
|||
* Protobuf type {@code WALKey}
|
||||
*
|
||||
* <pre>
|
||||
* Protocol buffer version of HLogKey; see HLogKey comment, not really a key but WALEdit header for some KVs
|
||||
*
|
||||
* Protocol buffer version of WALKey; see WALKey comment, not really a key but WALEdit header
|
||||
* for some KVs
|
||||
* </pre>
|
||||
*/
|
||||
public static final class Builder extends
|
||||
|
@ -10021,8 +10025,10 @@ public final class WALProtos {
|
|||
*
|
||||
* <pre>
|
||||
**
|
||||
* A trailer that is appended to the end of a properly closed HLog WAL file.
|
||||
* A trailer that is appended to the end of a properly closed WAL file.
|
||||
* If missing, this is either a legacy or a corrupted WAL file.
|
||||
* N.B. This trailer currently doesn't contain any information and we
|
||||
* purposefully don't expose it in the WAL APIs. It's for future growth.
|
||||
* </pre>
|
||||
*/
|
||||
public static final class WALTrailer extends
|
||||
|
@ -10246,8 +10252,10 @@ public final class WALProtos {
|
|||
*
|
||||
* <pre>
|
||||
**
|
||||
* A trailer that is appended to the end of a properly closed HLog WAL file.
|
||||
* A trailer that is appended to the end of a properly closed WAL file.
|
||||
* If missing, this is either a legacy or a corrupted WAL file.
|
||||
* N.B. This trailer currently doesn't contain any information and we
|
||||
* purposefully don't expose it in the WAL APIs. It's for future growth.
|
||||
* </pre>
|
||||
*/
|
||||
public static final class Builder extends
|
||||
|
|
|
@ -6020,7 +6020,7 @@ public final class ZooKeeperProtos {
|
|||
*
|
||||
* <pre>
|
||||
**
|
||||
* Used by replication. Holds the current position in an HLog file.
|
||||
* Used by replication. Holds the current position in an WAL file.
|
||||
* </pre>
|
||||
*/
|
||||
public static final class ReplicationHLogPosition extends
|
||||
|
@ -6288,7 +6288,7 @@ public final class ZooKeeperProtos {
|
|||
*
|
||||
* <pre>
|
||||
**
|
||||
* Used by replication. Holds the current position in an HLog file.
|
||||
* Used by replication. Holds the current position in an WAL file.
|
||||
* </pre>
|
||||
*/
|
||||
public static final class Builder extends
|
||||
|
|
|
@ -204,6 +204,10 @@ message ReplicateWALEntryResponse {
|
|||
message RollWALWriterRequest {
|
||||
}
|
||||
|
||||
/*
|
||||
* Roll request responses no longer include regions to flush
|
||||
* this list will always be empty when talking to a 1.0 server
|
||||
*/
|
||||
message RollWALWriterResponse {
|
||||
// A list of encoded name of regions to flush
|
||||
repeated bytes region_to_flush = 1;
|
||||
|
|
|
@ -74,7 +74,7 @@ message GetLastFlushedSequenceIdRequest {
|
|||
}
|
||||
|
||||
message GetLastFlushedSequenceIdResponse {
|
||||
/** the last HLog sequence id flushed from MemStore to HFile for the region */
|
||||
/* the last WAL sequence id flushed from MemStore to HFile for the region */
|
||||
required uint64 last_flushed_sequence_id = 1;
|
||||
}
|
||||
|
||||
|
|
|
@ -31,7 +31,10 @@ message WALHeader {
|
|||
optional string cell_codec_cls_name = 5;
|
||||
}
|
||||
|
||||
// Protocol buffer version of HLogKey; see HLogKey comment, not really a key but WALEdit header for some KVs
|
||||
/*
|
||||
* Protocol buffer version of WALKey; see WALKey comment, not really a key but WALEdit header
|
||||
* for some KVs
|
||||
*/
|
||||
message WALKey {
|
||||
required bytes encoded_region_name = 1;
|
||||
required bytes table_name = 2;
|
||||
|
@ -144,8 +147,10 @@ message RegionEventDescriptor {
|
|||
}
|
||||
|
||||
/**
|
||||
* A trailer that is appended to the end of a properly closed HLog WAL file.
|
||||
* A trailer that is appended to the end of a properly closed WAL file.
|
||||
* If missing, this is either a legacy or a corrupted WAL file.
|
||||
* N.B. This trailer currently doesn't contain any information and we
|
||||
* purposefully don't expose it in the WAL APIs. It's for future growth.
|
||||
*/
|
||||
message WALTrailer {
|
||||
}
|
||||
|
|
|
@ -128,7 +128,7 @@ message ReplicationState {
|
|||
}
|
||||
|
||||
/**
|
||||
* Used by replication. Holds the current position in an HLog file.
|
||||
* Used by replication. Holds the current position in an WAL file.
|
||||
*/
|
||||
message ReplicationHLogPosition {
|
||||
required int64 position = 1;
|
||||
|
|
|
@ -40,7 +40,7 @@ java.lang.management.ManagementFactory;
|
|||
<li class="active"><a href="#tab_baseStats" data-toggle="tab">Base Stats</a></li>
|
||||
<li class=""><a href="#tab_memoryStats" data-toggle="tab">Memory</a></li>
|
||||
<li class=""><a href="#tab_requestStats" data-toggle="tab">Requests</a></li>
|
||||
<li class=""><a href="#tab_hlogStats" data-toggle="tab">hlogs</a></li>
|
||||
<li class=""><a href="#tab_hlogStats" data-toggle="tab">wals</a></li>
|
||||
<li class=""><a href="#tab_storeStats" data-toggle="tab">Storefiles</a></li>
|
||||
<li class=""><a href="#tab_queueStats" data-toggle="tab">Queues</a></li>
|
||||
</ul>
|
||||
|
@ -55,7 +55,7 @@ java.lang.management.ManagementFactory;
|
|||
<& requestStats; mWrap = mWrap &>
|
||||
</div>
|
||||
<div class="tab-pane" id="tab_hlogStats">
|
||||
<& hlogStats; mWrap = mWrap &>
|
||||
<& walStats; mWrap = mWrap &>
|
||||
</div>
|
||||
<div class="tab-pane" id="tab_storeStats">
|
||||
<& storeStats; mWrap = mWrap &>
|
||||
|
@ -76,7 +76,7 @@ java.lang.management.ManagementFactory;
|
|||
<th>Requests Per Second</th>
|
||||
<th>Num. Regions</th>
|
||||
<th>Block locality</th>
|
||||
<th>Slow HLog Append Count</th>
|
||||
<th>Slow WAL Append Count</th>
|
||||
</tr>
|
||||
<tr>
|
||||
<td><% String.format("%.0f", mWrap.getRequestsPerSecond()) %></td>
|
||||
|
@ -119,20 +119,20 @@ MetricsRegionServerWrapper mWrap;
|
|||
</table>
|
||||
</%def>
|
||||
|
||||
<%def hlogStats>
|
||||
<%def walStats>
|
||||
<%args>
|
||||
MetricsRegionServerWrapper mWrap;
|
||||
</%args>
|
||||
<table class="table table-striped">
|
||||
<tr>
|
||||
<tr>
|
||||
<th>Num. HLog Files</th>
|
||||
<th>Size. HLog Files (bytes)</th>
|
||||
<th>Num. WAL Files</th>
|
||||
<th>Size. WAL Files (bytes)</th>
|
||||
</tr>
|
||||
</tr>
|
||||
<tr>
|
||||
<td><% mWrap.getNumHLogFiles() %></td>
|
||||
<td><% mWrap.getHLogFileSize() %></td>
|
||||
<td><% mWrap.getNumWALFiles() %></td>
|
||||
<td><% mWrap.getWALFileSize() %></td>
|
||||
</tr>
|
||||
</table>
|
||||
</%def>
|
||||
|
|
|
@ -32,7 +32,7 @@ public class SplitLogCounters {
|
|||
public final static AtomicLong tot_mgr_log_split_batch_start = new AtomicLong(0);
|
||||
public final static AtomicLong tot_mgr_log_split_batch_success = new AtomicLong(0);
|
||||
public final static AtomicLong tot_mgr_log_split_batch_err = new AtomicLong(0);
|
||||
public final static AtomicLong tot_mgr_new_unexpected_hlogs = new AtomicLong(0);
|
||||
public final static AtomicLong tot_mgr_new_unexpected_wals = new AtomicLong(0);
|
||||
public final static AtomicLong tot_mgr_log_split_start = new AtomicLong(0);
|
||||
public final static AtomicLong tot_mgr_log_split_success = new AtomicLong(0);
|
||||
public final static AtomicLong tot_mgr_log_split_err = new AtomicLong(0);
|
||||
|
|
|
@ -24,7 +24,6 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
|||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
|
|
|
@ -29,12 +29,12 @@ import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionStoreSeq
|
|||
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
|
||||
import org.apache.hadoop.hbase.regionserver.SplitLogWorker;
|
||||
import org.apache.hadoop.hbase.regionserver.SplitLogWorker.TaskExecutor;
|
||||
import org.apache.hadoop.hbase.regionserver.handler.HLogSplitterHandler;
|
||||
import org.apache.hadoop.hbase.regionserver.handler.WALSplitterHandler;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
/**
|
||||
* Coordinated operations for {@link SplitLogWorker} and {@link HLogSplitterHandler} Important
|
||||
* Coordinated operations for {@link SplitLogWorker} and {@link WALSplitterHandler} Important
|
||||
* methods for SplitLogWorker: <BR>
|
||||
* {@link #isReady()} called from {@link SplitLogWorker#run()} to check whether the coordination is
|
||||
* ready to supply the tasks <BR>
|
||||
|
@ -44,7 +44,7 @@ import com.google.common.annotations.VisibleForTesting;
|
|||
* for external changes in coordination (if required) <BR>
|
||||
* {@link #endTask(SplitLogTask, AtomicLong, SplitTaskDetails)} notify coordination engine that
|
||||
* <p>
|
||||
* Important methods for HLogSplitterHandler: <BR>
|
||||
* Important methods for WALSplitterHandler: <BR>
|
||||
* splitting task has completed.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
|
@ -112,7 +112,7 @@ public interface SplitLogWorkerCoordination {
|
|||
*/
|
||||
void removeListener();
|
||||
|
||||
/* HLogSplitterHandler part */
|
||||
/* WALSplitterHandler part */
|
||||
|
||||
/**
|
||||
* Notify coordination engine that splitting task has completed.
|
||||
|
|
|
@ -54,8 +54,8 @@ import org.apache.hadoop.hbase.master.SplitLogManager.Task;
|
|||
import org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
|
||||
import org.apache.hadoop.hbase.regionserver.SplitLogWorker;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
|
||||
import org.apache.hadoop.hbase.wal.DefaultWALProvider;
|
||||
import org.apache.hadoop.hbase.wal.WALSplitter;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
||||
|
@ -115,7 +115,7 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements
|
|||
@Override
|
||||
public Status finish(ServerName workerName, String logfile) {
|
||||
try {
|
||||
HLogSplitter.finishSplitLogFile(logfile, manager.getServer().getConfiguration());
|
||||
WALSplitter.finishSplitLogFile(logfile, manager.getServer().getConfiguration());
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Could not finish splitting of log file " + logfile, e);
|
||||
return Status.ERR;
|
||||
|
@ -716,7 +716,7 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements
|
|||
}
|
||||
// decode the file name
|
||||
t = ZKSplitLog.getFileName(t);
|
||||
ServerName serverName = HLogUtil.getServerNameFromHLogDirectoryName(new Path(t));
|
||||
ServerName serverName = DefaultWALProvider.getServerNameFromWALDirectoryName(new Path(t));
|
||||
if (serverName != null) {
|
||||
knownFailedServers.add(serverName.getServerName());
|
||||
} else {
|
||||
|
|
|
@ -46,8 +46,8 @@ import org.apache.hadoop.hbase.regionserver.HRegion;
|
|||
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
|
||||
import org.apache.hadoop.hbase.regionserver.SplitLogWorker;
|
||||
import org.apache.hadoop.hbase.regionserver.SplitLogWorker.TaskExecutor;
|
||||
import org.apache.hadoop.hbase.regionserver.handler.HLogSplitterHandler;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
|
||||
import org.apache.hadoop.hbase.regionserver.handler.WALSplitterHandler;
|
||||
import org.apache.hadoop.hbase.wal.DefaultWALProvider;
|
||||
import org.apache.hadoop.hbase.util.CancelableProgressable;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
|
||||
|
@ -318,8 +318,8 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements
|
|||
splitTaskDetails.setTaskNode(curTask);
|
||||
splitTaskDetails.setCurTaskZKVersion(zkVersion);
|
||||
|
||||
HLogSplitterHandler hsh =
|
||||
new HLogSplitterHandler(server, this, splitTaskDetails, reporter,
|
||||
WALSplitterHandler hsh =
|
||||
new WALSplitterHandler(server, this, splitTaskDetails, reporter,
|
||||
this.tasksInProgress, splitTaskExecutor, mode);
|
||||
server.getExecutorService().submit(hsh);
|
||||
}
|
||||
|
@ -418,7 +418,7 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements
|
|||
// pick meta wal firstly
|
||||
int offset = (int) (Math.random() * paths.size());
|
||||
for (int i = 0; i < paths.size(); i++) {
|
||||
if (HLogUtil.isMetaFile(paths.get(i))) {
|
||||
if (DefaultWALProvider.isMetaFile(paths.get(i))) {
|
||||
offset = i;
|
||||
break;
|
||||
}
|
||||
|
@ -581,7 +581,7 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements
|
|||
}
|
||||
|
||||
/*
|
||||
* Next part is related to HLogSplitterHandler
|
||||
* Next part is related to WALSplitterHandler
|
||||
*/
|
||||
/**
|
||||
* endTask() can fail and the only way to recover out of it is for the {@link SplitLogManager} to
|
||||
|
|
|
@ -54,6 +54,7 @@ import org.apache.hadoop.hbase.regionserver.StoreFile;
|
|||
import org.apache.hadoop.hbase.regionserver.StoreFile.Reader;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
|
||||
import org.apache.hadoop.hbase.wal.WALKey;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
|
||||
|
@ -444,14 +445,32 @@ public abstract class BaseRegionObserver implements RegionObserver {
|
|||
final InternalScanner s) throws IOException {
|
||||
}
|
||||
|
||||
/**
|
||||
* Implementers should override this version of the method and leave the deprecated one as-is.
|
||||
*/
|
||||
@Override
|
||||
public void preWALRestore(ObserverContext<? extends RegionCoprocessorEnvironment> env,
|
||||
HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void preWALRestore(ObserverContext<RegionCoprocessorEnvironment> env, HRegionInfo info,
|
||||
HLogKey logKey, WALEdit logEdit) throws IOException {
|
||||
preWALRestore(env, info, (WALKey)logKey, logEdit);
|
||||
}
|
||||
|
||||
/**
|
||||
* Implementers should override this version of the method and leave the deprecated one as-is.
|
||||
*/
|
||||
@Override
|
||||
public void postWALRestore(ObserverContext<? extends RegionCoprocessorEnvironment> env,
|
||||
HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postWALRestore(ObserverContext<RegionCoprocessorEnvironment> env,
|
||||
HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException {
|
||||
postWALRestore(env, info, (WALKey)logKey, logEdit);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -26,6 +26,7 @@ import org.apache.hadoop.hbase.CoprocessorEnvironment;
|
|||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
|
||||
import org.apache.hadoop.hbase.wal.WALKey;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
|
||||
/**
|
||||
|
@ -42,13 +43,31 @@ public class BaseWALObserver implements WALObserver {
|
|||
@Override
|
||||
public void stop(CoprocessorEnvironment e) throws IOException { }
|
||||
|
||||
/**
|
||||
* Implementers should override this method and leave the deprecated version as-is.
|
||||
*/
|
||||
@Override
|
||||
public boolean preWALWrite(ObserverContext<WALCoprocessorEnvironment> ctx, HRegionInfo info,
|
||||
HLogKey logKey, WALEdit logEdit) throws IOException {
|
||||
public boolean preWALWrite(ObserverContext<? extends WALCoprocessorEnvironment> ctx,
|
||||
HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean preWALWrite(ObserverContext<WALCoprocessorEnvironment> ctx, HRegionInfo info,
|
||||
HLogKey logKey, WALEdit logEdit) throws IOException {
|
||||
return preWALWrite(ctx, info, (WALKey)logKey, logEdit);
|
||||
}
|
||||
|
||||
/**
|
||||
* Implementers should override this method and leave the deprecated version as-is.
|
||||
*/
|
||||
@Override
|
||||
public void postWALWrite(ObserverContext<? extends WALCoprocessorEnvironment> ctx,
|
||||
HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException { }
|
||||
|
||||
@Override
|
||||
public void postWALWrite(ObserverContext<WALCoprocessorEnvironment> ctx, HRegionInfo info,
|
||||
HLogKey logKey, WALEdit logEdit) throws IOException { }
|
||||
HLogKey logKey, WALEdit logEdit) throws IOException {
|
||||
postWALWrite(ctx, info, (WALKey)logKey, logEdit);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -29,6 +29,7 @@ import java.util.Set;
|
|||
import java.util.SortedSet;
|
||||
import java.util.TreeSet;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.ConcurrentSkipListSet;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
|
@ -557,4 +558,77 @@ public abstract class CoprocessorHost<E extends CoprocessorEnvironment> {
|
|||
"coprocessor set.", e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Used to gracefully handle fallback to deprecated methods when we
|
||||
* evolve coprocessor APIs.
|
||||
*
|
||||
* When a particular Coprocessor API is updated to change methods, hosts can support fallback
|
||||
* to the deprecated API by using this method to determine if an instance implements the new API.
|
||||
* In the event that said support is partial, then in the face of a runtime issue that prevents
|
||||
* proper operation {@link #legacyWarning(Class, String)} should be used to let operators know.
|
||||
*
|
||||
* For examples of this in action, see the implementation of
|
||||
* <ul>
|
||||
* <li>{@link org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost}
|
||||
* <li>{@link org.apache.hadoop.hbase.regionserver.wal.WALCoprocessorHost}
|
||||
* </ul>
|
||||
*
|
||||
* @param clazz Coprocessor you wish to evaluate
|
||||
* @param methodName the name of the non-deprecated method version
|
||||
* @param parameterTypes the Class of the non-deprecated method's arguments in the order they are
|
||||
* declared.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
protected static boolean useLegacyMethod(final Class<? extends Coprocessor> clazz,
|
||||
final String methodName, final Class<?>... parameterTypes) {
|
||||
boolean useLegacy;
|
||||
// Use reflection to see if they implement the non-deprecated version
|
||||
try {
|
||||
clazz.getDeclaredMethod(methodName, parameterTypes);
|
||||
LOG.debug("Found an implementation of '" + methodName + "' that uses updated method " +
|
||||
"signature. Skipping legacy support for invocations in '" + clazz +"'.");
|
||||
useLegacy = false;
|
||||
} catch (NoSuchMethodException exception) {
|
||||
useLegacy = true;
|
||||
} catch (SecurityException exception) {
|
||||
LOG.warn("The Security Manager denied our attempt to detect if the coprocessor '" + clazz +
|
||||
"' requires legacy support; assuming it does. If you get later errors about legacy " +
|
||||
"coprocessor use, consider updating your security policy to allow access to the package" +
|
||||
" and declared members of your implementation.");
|
||||
LOG.debug("Details of Security Manager rejection.", exception);
|
||||
useLegacy = true;
|
||||
}
|
||||
return useLegacy;
|
||||
}
|
||||
|
||||
/**
|
||||
* Used to limit legacy handling to once per Coprocessor class per classloader.
|
||||
*/
|
||||
private static final Set<Class<? extends Coprocessor>> legacyWarning =
|
||||
new ConcurrentSkipListSet<Class<? extends Coprocessor>>(
|
||||
new Comparator<Class<? extends Coprocessor>>() {
|
||||
@Override
|
||||
public int compare(Class<? extends Coprocessor> c1, Class<? extends Coprocessor> c2) {
|
||||
if (c1.equals(c2)) {
|
||||
return 0;
|
||||
}
|
||||
return c1.getName().compareTo(c2.getName());
|
||||
}
|
||||
});
|
||||
|
||||
/**
|
||||
* limits the amount of logging to once per coprocessor class.
|
||||
* Used in concert with {@link #useLegacyMethod(Class, String, Class[])} when a runtime issue
|
||||
* prevents properly supporting the legacy version of a coprocessor API.
|
||||
* Since coprocessors can be in tight loops this serves to limit the amount of log spam we create.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
protected void legacyWarning(final Class<? extends Coprocessor> clazz, final String message) {
|
||||
if(legacyWarning.add(clazz)) {
|
||||
LOG.error("You have a legacy coprocessor loaded and there are events we can't map to the " +
|
||||
" deprecated API. Your coprocessor will not see these events. Please update '" + clazz +
|
||||
"'. Details of the problem: " + message);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -56,6 +56,7 @@ import org.apache.hadoop.hbase.regionserver.StoreFile;
|
|||
import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
|
||||
import org.apache.hadoop.hbase.wal.WALKey;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
|
||||
|
@ -67,6 +68,9 @@ import com.google.common.collect.ImmutableList;
|
|||
*/
|
||||
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
|
||||
@InterfaceStability.Evolving
|
||||
// TODO as method signatures need to break, update to
|
||||
// ObserverContext<? extends RegionCoprocessorEnvironment>
|
||||
// so we can use additional environment state that isn't exposed to coprocessors.
|
||||
public interface RegionObserver extends Coprocessor {
|
||||
|
||||
/** Mutation type for postMutationBeforeWAL hook */
|
||||
|
@ -1109,26 +1113,62 @@ public interface RegionObserver extends Coprocessor {
|
|||
/**
|
||||
* Called before a {@link org.apache.hadoop.hbase.regionserver.wal.WALEdit}
|
||||
* replayed for this region.
|
||||
*
|
||||
* @param ctx
|
||||
* @param info
|
||||
* @param logKey
|
||||
* @param logEdit
|
||||
* @throws IOException
|
||||
*/
|
||||
void preWALRestore(final ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
|
||||
HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException;
|
||||
|
||||
/**
|
||||
* Called before a {@link org.apache.hadoop.hbase.regionserver.wal.WALEdit}
|
||||
* replayed for this region.
|
||||
*
|
||||
* This method is left in place to maintain binary compatibility with older
|
||||
* {@link RegionObserver}s. If an implementation directly overrides
|
||||
* {@link #preWALRestore(ObserverContext, HRegionInfo, WALKey, WALEdit)} then this version
|
||||
* won't be called at all, barring problems with the Security Manager. To work correctly
|
||||
* in the presence of a strict Security Manager, or in the case of an implementation that
|
||||
* relies on a parent class to implement preWALRestore, you should implement this method
|
||||
* as a call to the non-deprecated version.
|
||||
*
|
||||
* Users of this method will see all edits that can be treated as HLogKey. If there are
|
||||
* edits that can't be treated as HLogKey they won't be offered to coprocessors that rely
|
||||
* on this method. If a coprocessor gets skipped because of this mechanism, a log message
|
||||
* at ERROR will be generated per coprocessor on the logger for {@link CoprocessorHost} once per
|
||||
* classloader.
|
||||
*
|
||||
* @deprecated use {@link #preWALRestore(ObserverContext, HRegionInfo, WALKey, WALEdit)}
|
||||
*/
|
||||
@Deprecated
|
||||
void preWALRestore(final ObserverContext<RegionCoprocessorEnvironment> ctx,
|
||||
HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException;
|
||||
|
||||
/**
|
||||
* Called after a {@link org.apache.hadoop.hbase.regionserver.wal.WALEdit}
|
||||
* replayed for this region.
|
||||
*
|
||||
* @param ctx
|
||||
* @param info
|
||||
* @param logKey
|
||||
* @param logEdit
|
||||
* @throws IOException
|
||||
*/
|
||||
void postWALRestore(final ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
|
||||
HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException;
|
||||
|
||||
/**
|
||||
* Called after a {@link org.apache.hadoop.hbase.regionserver.wal.WALEdit}
|
||||
* replayed for this region.
|
||||
*
|
||||
* This method is left in place to maintain binary compatibility with older
|
||||
* {@link RegionObserver}s. If an implementation directly overrides
|
||||
* {@link #postWALRestore(ObserverContext, HRegionInfo, WALKey, WALEdit)} then this version
|
||||
* won't be called at all, barring problems with the Security Manager. To work correctly
|
||||
* in the presence of a strict Security Manager, or in the case of an implementation that
|
||||
* relies on a parent class to implement preWALRestore, you should implement this method
|
||||
* as a call to the non-deprecated version.
|
||||
*
|
||||
* Users of this method will see all edits that can be treated as HLogKey. If there are
|
||||
* edits that can't be treated as HLogKey they won't be offered to coprocessors that rely
|
||||
* on this method. If a coprocessor gets skipped because of this mechanism, a log message
|
||||
* at ERROR will be generated per coprocessor on the logger for {@link CoprocessorHost} once per
|
||||
* classloader.
|
||||
*
|
||||
* @deprecated use {@link #postWALRestore(ObserverContext, HRegionInfo, WALKey, WALEdit)}
|
||||
*/
|
||||
@Deprecated
|
||||
void postWALRestore(final ObserverContext<RegionCoprocessorEnvironment> ctx,
|
||||
HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException;
|
||||
|
||||
|
|
|
@ -23,11 +23,11 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
|||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.CoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLog;
|
||||
import org.apache.hadoop.hbase.wal.WAL;
|
||||
|
||||
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
|
||||
@InterfaceStability.Evolving
|
||||
public interface WALCoprocessorEnvironment extends CoprocessorEnvironment {
|
||||
/** @return reference to the region server services */
|
||||
HLog getWAL();
|
||||
/** @return reference to the region server's WAL */
|
||||
WAL getWAL();
|
||||
}
|
||||
|
|
|
@ -25,6 +25,7 @@ import org.apache.hadoop.hbase.Coprocessor;
|
|||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
|
||||
import org.apache.hadoop.hbase.wal.WALKey;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -40,7 +41,7 @@ import java.io.IOException;
|
|||
* hooks for adding logic for WALEdits in the region context during reconstruction,
|
||||
*
|
||||
* Defines coprocessor hooks for interacting with operations on the
|
||||
* {@link org.apache.hadoop.hbase.regionserver.wal.HLog}.
|
||||
* {@link org.apache.hadoop.hbase.wal.WAL}.
|
||||
*/
|
||||
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
|
||||
@InterfaceStability.Evolving
|
||||
|
@ -50,27 +51,65 @@ public interface WALObserver extends Coprocessor {
|
|||
* Called before a {@link org.apache.hadoop.hbase.regionserver.wal.WALEdit}
|
||||
* is writen to WAL.
|
||||
*
|
||||
* @param ctx
|
||||
* @param info
|
||||
* @param logKey
|
||||
* @param logEdit
|
||||
* @return true if default behavior should be bypassed, false otherwise
|
||||
* @throws IOException
|
||||
*/
|
||||
// TODO: return value is not used
|
||||
boolean preWALWrite(ObserverContext<? extends WALCoprocessorEnvironment> ctx,
|
||||
HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException;
|
||||
|
||||
/**
|
||||
* Called before a {@link org.apache.hadoop.hbase.regionserver.wal.WALEdit}
|
||||
* is writen to WAL.
|
||||
*
|
||||
* This method is left in place to maintain binary compatibility with older
|
||||
* {@link WALObserver}s. If an implementation directly overrides
|
||||
* {@link #preWALWrite(ObserverContext, HRegionInfo, WALKey, WALEdit)} then this version
|
||||
* won't be called at all, barring problems with the Security Manager. To work correctly
|
||||
* in the presence of a strict Security Manager, or in the case of an implementation that
|
||||
* relies on a parent class to implement preWALWrite, you should implement this method
|
||||
* as a call to the non-deprecated version.
|
||||
*
|
||||
* Users of this method will see all edits that can be treated as HLogKey. If there are
|
||||
* edits that can't be treated as HLogKey they won't be offered to coprocessors that rely
|
||||
* on this method. If a coprocessor gets skipped because of this mechanism, a log message
|
||||
* at ERROR will be generated per coprocessor on the logger for {@link CoprocessorHost} once per
|
||||
* classloader.
|
||||
*
|
||||
* @return true if default behavior should be bypassed, false otherwise
|
||||
* @deprecated use {@link #preWALWrite(ObserverContext, HRegionInfo, WALKey, WALEdit)}
|
||||
*/
|
||||
@Deprecated
|
||||
boolean preWALWrite(ObserverContext<WALCoprocessorEnvironment> ctx,
|
||||
HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException;
|
||||
|
||||
/**
|
||||
* Called after a {@link org.apache.hadoop.hbase.regionserver.wal.WALEdit}
|
||||
* is writen to WAL.
|
||||
*
|
||||
* @param ctx
|
||||
* @param info
|
||||
* @param logKey
|
||||
* @param logEdit
|
||||
* @throws IOException
|
||||
*/
|
||||
void postWALWrite(ObserverContext<? extends WALCoprocessorEnvironment> ctx,
|
||||
HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException;
|
||||
|
||||
/**
|
||||
* Called after a {@link org.apache.hadoop.hbase.regionserver.wal.WALEdit}
|
||||
* is writen to WAL.
|
||||
*
|
||||
* This method is left in place to maintain binary compatibility with older
|
||||
* {@link WALObserver}s. If an implementation directly overrides
|
||||
* {@link #postWALWrite(ObserverContext, HRegionInfo, WALKey, WALEdit)} then this version
|
||||
* won't be called at all, barring problems with the Security Manager. To work correctly
|
||||
* in the presence of a strict Security Manager, or in the case of an implementation that
|
||||
* relies on a parent class to implement preWALWrite, you should implement this method
|
||||
* as a call to the non-deprecated version.
|
||||
*
|
||||
* Users of this method will see all edits that can be treated as HLogKey. If there are
|
||||
* edits that can't be treated as HLogKey they won't be offered to coprocessors that rely
|
||||
* on this method. If a coprocessor gets skipped because of this mechanism, a log message
|
||||
* at ERROR will be generated per coprocessor on the logger for {@link CoprocessorHost} once per
|
||||
* classloader.
|
||||
*
|
||||
* @deprecated use {@link #postWALWrite(ObserverContext, HRegionInfo, WALKey, WALEdit)}
|
||||
*/
|
||||
@Deprecated
|
||||
void postWALWrite(ObserverContext<WALCoprocessorEnvironment> ctx,
|
||||
HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException;
|
||||
}
|
||||
|
|
|
@ -40,7 +40,7 @@ import org.apache.hadoop.fs.FilterFileSystem;
|
|||
import org.apache.hadoop.fs.LocalFileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
|
||||
import org.apache.hadoop.hbase.wal.DefaultWALProvider;
|
||||
import org.apache.hadoop.hdfs.DFSClient;
|
||||
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
||||
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
|
||||
|
@ -54,8 +54,8 @@ import org.apache.hadoop.util.ReflectionUtils;
|
|||
/**
|
||||
* An encapsulation for the FileSystem object that hbase uses to access
|
||||
* data. This class allows the flexibility of using
|
||||
* separate filesystem objects for reading and writing hfiles and hlogs.
|
||||
* In future, if we want to make hlogs be in a different filesystem,
|
||||
* separate filesystem objects for reading and writing hfiles and wals.
|
||||
* In future, if we want to make wals be in a different filesystem,
|
||||
* this is the place to make it happen.
|
||||
*/
|
||||
public class HFileSystem extends FilterFileSystem {
|
||||
|
@ -322,7 +322,7 @@ public class HFileSystem extends FilterFileSystem {
|
|||
}
|
||||
|
||||
/**
|
||||
* We're putting at lowest priority the hlog files blocks that are on the same datanode
|
||||
* We're putting at lowest priority the wal files blocks that are on the same datanode
|
||||
* as the original regionserver which created these files. This because we fear that the
|
||||
* datanode is actually dead, so if we use it it will timeout.
|
||||
*/
|
||||
|
@ -330,17 +330,17 @@ public class HFileSystem extends FilterFileSystem {
|
|||
public void reorderBlocks(Configuration conf, LocatedBlocks lbs, String src)
|
||||
throws IOException {
|
||||
|
||||
ServerName sn = HLogUtil.getServerNameFromHLogDirectoryName(conf, src);
|
||||
ServerName sn = DefaultWALProvider.getServerNameFromWALDirectoryName(conf, src);
|
||||
if (sn == null) {
|
||||
// It's not an HLOG
|
||||
// It's not an WAL
|
||||
return;
|
||||
}
|
||||
|
||||
// Ok, so it's an HLog
|
||||
// Ok, so it's an WAL
|
||||
String hostName = sn.getHostname();
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace(src +
|
||||
" is an HLog file, so reordering blocks, last hostname will be:" + hostName);
|
||||
" is an WAL file, so reordering blocks, last hostname will be:" + hostName);
|
||||
}
|
||||
|
||||
// Just check for all blocks
|
||||
|
|
|
@ -1,69 +0,0 @@
|
|||
/**
|
||||
* 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.io;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
|
||||
/**
|
||||
* HLogLink describes a link to a WAL.
|
||||
*
|
||||
* An hlog can be in /hbase/.logs/<server>/<hlog>
|
||||
* or it can be in /hbase/.oldlogs/<hlog>
|
||||
*
|
||||
* The link checks first in the original path,
|
||||
* if it is not present it fallbacks to the archived path.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class HLogLink extends FileLink {
|
||||
/**
|
||||
* @param conf {@link Configuration} from which to extract specific archive locations
|
||||
* @param serverName Region Server owner of the log
|
||||
* @param logName WAL file name
|
||||
* @throws IOException on unexpected error.
|
||||
*/
|
||||
public HLogLink(final Configuration conf,
|
||||
final String serverName, final String logName) throws IOException {
|
||||
this(FSUtils.getRootDir(conf), serverName, logName);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param rootDir Path to the root directory where hbase files are stored
|
||||
* @param serverName Region Server owner of the log
|
||||
* @param logName WAL file name
|
||||
*/
|
||||
public HLogLink(final Path rootDir, final String serverName, final String logName) {
|
||||
final Path oldLogDir = new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME);
|
||||
final Path logDir = new Path(new Path(rootDir, HConstants.HREGION_LOGDIR_NAME), serverName);
|
||||
setLocations(new Path(logDir, logName), new Path(oldLogDir, logName));
|
||||
}
|
||||
|
||||
/**
|
||||
* @param originPath Path to the wal in the log directory
|
||||
* @param archivePath Path to the wal in the archived log directory
|
||||
*/
|
||||
public HLogLink(final Path originPath, final Path archivePath) {
|
||||
setLocations(originPath, archivePath);
|
||||
}
|
||||
}
|
|
@ -186,12 +186,12 @@ public class HFileOutputFormat2
|
|||
rollWriters();
|
||||
}
|
||||
|
||||
// create a new HLog writer, if necessary
|
||||
// create a new WAL writer, if necessary
|
||||
if (wl == null || wl.writer == null) {
|
||||
wl = getNewWriter(family, conf);
|
||||
}
|
||||
|
||||
// we now have the proper HLog writer. full steam ahead
|
||||
// we now have the proper WAL writer. full steam ahead
|
||||
kv.updateLatestStamp(this.now);
|
||||
wl.writer.append(kv);
|
||||
wl.written += length;
|
||||
|
|
|
@ -17,26 +17,15 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.mapreduce;
|
||||
|
||||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.EOFException;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLog;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogFactory;
|
||||
import org.apache.hadoop.hbase.wal.WAL;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
import org.apache.hadoop.mapreduce.InputFormat;
|
||||
import org.apache.hadoop.mapreduce.InputSplit;
|
||||
import org.apache.hadoop.mapreduce.JobContext;
|
||||
|
@ -44,227 +33,51 @@ import org.apache.hadoop.mapreduce.RecordReader;
|
|||
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
||||
|
||||
/**
|
||||
* Simple {@link InputFormat} for {@link HLog} files.
|
||||
* Simple {@link InputFormat} for {@link WAL} files.
|
||||
* @deprecated use {@link WALInputFormat}
|
||||
*/
|
||||
@Deprecated
|
||||
@InterfaceAudience.Public
|
||||
public class HLogInputFormat extends InputFormat<HLogKey, WALEdit> {
|
||||
private static final Log LOG = LogFactory.getLog(HLogInputFormat.class);
|
||||
|
||||
public static final String START_TIME_KEY = "hlog.start.time";
|
||||
public static final String END_TIME_KEY = "hlog.end.time";
|
||||
|
||||
/**
|
||||
* {@link InputSplit} for {@link HLog} files. Each split represent
|
||||
* exactly one log file.
|
||||
*/
|
||||
static class HLogSplit extends InputSplit implements Writable {
|
||||
private String logFileName;
|
||||
private long fileSize;
|
||||
private long startTime;
|
||||
private long endTime;
|
||||
|
||||
/** for serialization */
|
||||
public HLogSplit() {}
|
||||
|
||||
/**
|
||||
* Represent an HLogSplit, i.e. a single HLog file.
|
||||
* Start- and EndTime are managed by the split, so that HLog files can be
|
||||
* filtered before WALEdits are passed to the mapper(s).
|
||||
* @param logFileName
|
||||
* @param fileSize
|
||||
* @param startTime
|
||||
* @param endTime
|
||||
*/
|
||||
public HLogSplit(String logFileName, long fileSize, long startTime, long endTime) {
|
||||
this.logFileName = logFileName;
|
||||
this.fileSize = fileSize;
|
||||
this.startTime = startTime;
|
||||
this.endTime = endTime;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLength() throws IOException, InterruptedException {
|
||||
return fileSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String[] getLocations() throws IOException, InterruptedException {
|
||||
// TODO: Find the data node with the most blocks for this HLog?
|
||||
return new String[] {};
|
||||
}
|
||||
|
||||
public String getLogFileName() {
|
||||
return logFileName;
|
||||
}
|
||||
|
||||
public long getStartTime() {
|
||||
return startTime;
|
||||
}
|
||||
|
||||
public long getEndTime() {
|
||||
return endTime;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFields(DataInput in) throws IOException {
|
||||
logFileName = in.readUTF();
|
||||
fileSize = in.readLong();
|
||||
startTime = in.readLong();
|
||||
endTime = in.readLong();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(DataOutput out) throws IOException {
|
||||
out.writeUTF(logFileName);
|
||||
out.writeLong(fileSize);
|
||||
out.writeLong(startTime);
|
||||
out.writeLong(endTime);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return logFileName + " (" + startTime + ":" + endTime + ") length:" + fileSize;
|
||||
}
|
||||
}
|
||||
// Delegate to WALInputFormat for implementation.
|
||||
private final WALInputFormat delegate = new WALInputFormat();
|
||||
|
||||
/**
|
||||
* {@link RecordReader} for an {@link HLog} file.
|
||||
* {@link RecordReader} that pulls out the legacy HLogKey format directly.
|
||||
*/
|
||||
static class HLogRecordReader extends RecordReader<HLogKey, WALEdit> {
|
||||
private HLog.Reader reader = null;
|
||||
private HLog.Entry currentEntry = new HLog.Entry();
|
||||
private long startTime;
|
||||
private long endTime;
|
||||
|
||||
@Override
|
||||
public void initialize(InputSplit split, TaskAttemptContext context)
|
||||
throws IOException, InterruptedException {
|
||||
HLogSplit hsplit = (HLogSplit)split;
|
||||
Path logFile = new Path(hsplit.getLogFileName());
|
||||
Configuration conf = context.getConfiguration();
|
||||
LOG.info("Opening reader for "+split);
|
||||
try {
|
||||
this.reader = HLogFactory.createReader(logFile.getFileSystem(conf),
|
||||
logFile, conf);
|
||||
} catch (EOFException x) {
|
||||
LOG.info("Ignoring corrupted HLog file: " + logFile
|
||||
+ " (This is normal when a RegionServer crashed.)");
|
||||
}
|
||||
this.startTime = hsplit.getStartTime();
|
||||
this.endTime = hsplit.getEndTime();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean nextKeyValue() throws IOException, InterruptedException {
|
||||
if (reader == null) return false;
|
||||
|
||||
HLog.Entry temp;
|
||||
long i = -1;
|
||||
do {
|
||||
// skip older entries
|
||||
try {
|
||||
temp = reader.next(currentEntry);
|
||||
i++;
|
||||
} catch (EOFException x) {
|
||||
LOG.info("Corrupted entry detected. Ignoring the rest of the file."
|
||||
+ " (This is normal when a RegionServer crashed.)");
|
||||
return false;
|
||||
}
|
||||
}
|
||||
while(temp != null && temp.getKey().getWriteTime() < startTime);
|
||||
|
||||
if (temp == null) {
|
||||
if (i > 0) LOG.info("Skipped " + i + " entries.");
|
||||
LOG.info("Reached end of file.");
|
||||
return false;
|
||||
} else if (i > 0) {
|
||||
LOG.info("Skipped " + i + " entries, until ts: " + temp.getKey().getWriteTime() + ".");
|
||||
}
|
||||
boolean res = temp.getKey().getWriteTime() <= endTime;
|
||||
if (!res) {
|
||||
LOG.info("Reached ts: " + temp.getKey().getWriteTime() + " ignoring the rest of the file.");
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
static class HLogKeyRecordReader extends WALInputFormat.WALRecordReader<HLogKey> {
|
||||
@Override
|
||||
public HLogKey getCurrentKey() throws IOException, InterruptedException {
|
||||
return currentEntry.getKey();
|
||||
}
|
||||
|
||||
@Override
|
||||
public WALEdit getCurrentValue() throws IOException, InterruptedException {
|
||||
return currentEntry.getEdit();
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getProgress() throws IOException, InterruptedException {
|
||||
// N/A depends on total number of entries, which is unknown
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
LOG.info("Closing reader");
|
||||
if (reader != null) this.reader.close();
|
||||
if (!(currentEntry.getKey() instanceof HLogKey)) {
|
||||
final IllegalStateException exception = new IllegalStateException(
|
||||
"HLogInputFormat only works when given entries that have HLogKey for keys. This" +
|
||||
" one had '" + currentEntry.getKey().getClass() + "'");
|
||||
LOG.error("The deprecated HLogInputFormat has to work with the deprecated HLogKey class, " +
|
||||
" but HBase internals read the wal entry using some other class." +
|
||||
" This is a bug; please file an issue or email the developer mailing list. It is " +
|
||||
"likely that you would not have this problem if you updated to use WALInputFormat. " +
|
||||
"You will need the following exception details when seeking help from the HBase " +
|
||||
"community.",
|
||||
exception);
|
||||
throw exception;
|
||||
}
|
||||
return (HLogKey)currentEntry.getKey();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<InputSplit> getSplits(JobContext context) throws IOException,
|
||||
InterruptedException {
|
||||
Configuration conf = context.getConfiguration();
|
||||
Path inputDir = new Path(conf.get("mapreduce.input.fileinputformat.inputdir"));
|
||||
|
||||
long startTime = conf.getLong(START_TIME_KEY, Long.MIN_VALUE);
|
||||
long endTime = conf.getLong(END_TIME_KEY, Long.MAX_VALUE);
|
||||
|
||||
FileSystem fs = inputDir.getFileSystem(conf);
|
||||
List<FileStatus> files = getFiles(fs, inputDir, startTime, endTime);
|
||||
|
||||
List<InputSplit> splits = new ArrayList<InputSplit>(files.size());
|
||||
for (FileStatus file : files) {
|
||||
splits.add(new HLogSplit(file.getPath().toString(), file.getLen(), startTime, endTime));
|
||||
}
|
||||
return splits;
|
||||
}
|
||||
|
||||
private List<FileStatus> getFiles(FileSystem fs, Path dir, long startTime, long endTime)
|
||||
throws IOException {
|
||||
List<FileStatus> result = new ArrayList<FileStatus>();
|
||||
LOG.debug("Scanning " + dir.toString() + " for HLog files");
|
||||
|
||||
FileStatus[] files = fs.listStatus(dir);
|
||||
if (files == null) return Collections.emptyList();
|
||||
for (FileStatus file : files) {
|
||||
if (file.isDirectory()) {
|
||||
// recurse into sub directories
|
||||
result.addAll(getFiles(fs, file.getPath(), startTime, endTime));
|
||||
} else {
|
||||
String name = file.getPath().toString();
|
||||
int idx = name.lastIndexOf('.');
|
||||
if (idx > 0) {
|
||||
try {
|
||||
long fileStartTime = Long.parseLong(name.substring(idx+1));
|
||||
if (fileStartTime <= endTime) {
|
||||
LOG.info("Found: " + name);
|
||||
result.add(file);
|
||||
}
|
||||
} catch (NumberFormatException x) {
|
||||
idx = 0;
|
||||
}
|
||||
}
|
||||
if (idx == 0) {
|
||||
LOG.warn("File " + name + " does not appear to be an HLog file. Skipping...");
|
||||
}
|
||||
}
|
||||
}
|
||||
return result;
|
||||
return delegate.getSplits(context, START_TIME_KEY, END_TIME_KEY);
|
||||
}
|
||||
|
||||
@Override
|
||||
public RecordReader<HLogKey, WALEdit> createRecordReader(InputSplit split,
|
||||
TaskAttemptContext context) throws IOException, InterruptedException {
|
||||
return new HLogRecordReader();
|
||||
return new HLogKeyRecordReader();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -51,7 +51,7 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
|||
* </p>
|
||||
*
|
||||
* <p>
|
||||
* Write-ahead logging (HLog) for Puts can be disabled by setting
|
||||
* Write-ahead logging (WAL) for Puts can be disabled by setting
|
||||
* {@link #WAL_PROPERTY} to {@link #WAL_OFF}. Default value is {@link #WAL_ON}.
|
||||
* Note that disabling write-ahead logging is only appropriate for jobs where
|
||||
* loss of data due to region server failure can be tolerated (for example,
|
||||
|
@ -61,7 +61,7 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
|||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Stable
|
||||
public class MultiTableOutputFormat extends OutputFormat<ImmutableBytesWritable, Mutation> {
|
||||
/** Set this to {@link #WAL_OFF} to turn off write-ahead logging (HLog) */
|
||||
/** Set this to {@link #WAL_OFF} to turn off write-ahead logging (WAL) */
|
||||
public static final String WAL_PROPERTY = "hbase.mapreduce.multitableoutputformat.wal";
|
||||
/** Property value to use write-ahead logging */
|
||||
public static final boolean WAL_ON = true;
|
||||
|
|
|
@ -49,7 +49,7 @@ import com.google.common.annotations.VisibleForTesting;
|
|||
/**
|
||||
* TableSnapshotInputFormat allows a MapReduce job to run over a table snapshot. The job
|
||||
* bypasses HBase servers, and directly accesses the underlying files (hfile, recovered edits,
|
||||
* hlogs, etc) directly to provide maximum performance. The snapshot is not required to be
|
||||
* wals, etc) directly to provide maximum performance. The snapshot is not required to be
|
||||
* restored to the live cluster or cloned. This also allows to run the mapreduce job from an
|
||||
* online or offline hbase cluster. The snapshot files can be exported by using the
|
||||
* {@link ExportSnapshot} tool, to a pure-hdfs cluster, and this InputFormat can be used to
|
||||
|
|
|
@ -23,6 +23,8 @@ import java.text.SimpleDateFormat;
|
|||
import java.util.Map;
|
||||
import java.util.TreeMap;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
@ -39,7 +41,7 @@ import org.apache.hadoop.hbase.client.HTable;
|
|||
import org.apache.hadoop.hbase.client.Mutation;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
|
||||
import org.apache.hadoop.hbase.wal.WALKey;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.mapreduce.Job;
|
||||
|
@ -63,11 +65,21 @@ import org.apache.hadoop.util.ToolRunner;
|
|||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Stable
|
||||
public class WALPlayer extends Configured implements Tool {
|
||||
final static Log LOG = LogFactory.getLog(WALPlayer.class);
|
||||
final static String NAME = "WALPlayer";
|
||||
final static String BULK_OUTPUT_CONF_KEY = "hlog.bulk.output";
|
||||
final static String HLOG_INPUT_KEY = "hlog.input.dir";
|
||||
final static String TABLES_KEY = "hlog.input.tables";
|
||||
final static String TABLE_MAP_KEY = "hlog.input.tablesmap";
|
||||
final static String BULK_OUTPUT_CONF_KEY = "wal.bulk.output";
|
||||
final static String TABLES_KEY = "wal.input.tables";
|
||||
final static String TABLE_MAP_KEY = "wal.input.tablesmap";
|
||||
|
||||
// This relies on Hadoop Configuration to handle warning about deprecated configs and
|
||||
// to set the correct non-deprecated configs when an old one shows up.
|
||||
static {
|
||||
Configuration.addDeprecation("hlog.bulk.output", BULK_OUTPUT_CONF_KEY);
|
||||
Configuration.addDeprecation("hlog.input.tables", TABLES_KEY);
|
||||
Configuration.addDeprecation("hlog.input.tablesmap", TABLE_MAP_KEY);
|
||||
Configuration.addDeprecation(HLogInputFormat.START_TIME_KEY, WALInputFormat.START_TIME_KEY);
|
||||
Configuration.addDeprecation(HLogInputFormat.END_TIME_KEY, WALInputFormat.END_TIME_KEY);
|
||||
}
|
||||
|
||||
private final static String JOB_NAME_CONF_KEY = "mapreduce.job.name";
|
||||
|
||||
|
@ -75,12 +87,12 @@ public class WALPlayer extends Configured implements Tool {
|
|||
* A mapper that just writes out KeyValues.
|
||||
* This one can be used together with {@link KeyValueSortReducer}
|
||||
*/
|
||||
static class HLogKeyValueMapper
|
||||
extends Mapper<HLogKey, WALEdit, ImmutableBytesWritable, KeyValue> {
|
||||
static class WALKeyValueMapper
|
||||
extends Mapper<WALKey, WALEdit, ImmutableBytesWritable, KeyValue> {
|
||||
private byte[] table;
|
||||
|
||||
@Override
|
||||
public void map(HLogKey key, WALEdit value,
|
||||
public void map(WALKey key, WALEdit value,
|
||||
Context context)
|
||||
throws IOException {
|
||||
try {
|
||||
|
@ -102,7 +114,7 @@ public class WALPlayer extends Configured implements Tool {
|
|||
// only a single table is supported when HFiles are generated with HFileOutputFormat
|
||||
String tables[] = context.getConfiguration().getStrings(TABLES_KEY);
|
||||
if (tables == null || tables.length != 1) {
|
||||
// this can only happen when HLogMapper is used directly by a class other than WALPlayer
|
||||
// this can only happen when WALMapper is used directly by a class other than WALPlayer
|
||||
throw new IOException("Exactly one table must be specified for bulk HFile case.");
|
||||
}
|
||||
table = Bytes.toBytes(tables[0]);
|
||||
|
@ -113,13 +125,13 @@ public class WALPlayer extends Configured implements Tool {
|
|||
* A mapper that writes out {@link Mutation} to be directly applied to
|
||||
* a running HBase instance.
|
||||
*/
|
||||
static class HLogMapper
|
||||
extends Mapper<HLogKey, WALEdit, ImmutableBytesWritable, Mutation> {
|
||||
static class WALMapper
|
||||
extends Mapper<WALKey, WALEdit, ImmutableBytesWritable, Mutation> {
|
||||
private Map<TableName, TableName> tables =
|
||||
new TreeMap<TableName, TableName>();
|
||||
|
||||
@Override
|
||||
public void map(HLogKey key, WALEdit value,
|
||||
public void map(WALKey key, WALEdit value,
|
||||
Context context)
|
||||
throws IOException {
|
||||
try {
|
||||
|
@ -132,7 +144,7 @@ public class WALPlayer extends Configured implements Tool {
|
|||
Delete del = null;
|
||||
Cell lastCell = null;
|
||||
for (Cell cell : value.getCells()) {
|
||||
// filtering HLog meta entries
|
||||
// filtering WAL meta entries
|
||||
if (WALEdit.isMetaEditFamily(cell.getFamily())) continue;
|
||||
|
||||
// A WALEdit may contain multiple operations (HBASE-3584) and/or
|
||||
|
@ -172,7 +184,7 @@ public class WALPlayer extends Configured implements Tool {
|
|||
String[] tableMap = context.getConfiguration().getStrings(TABLE_MAP_KEY);
|
||||
String[] tablesToUse = context.getConfiguration().getStrings(TABLES_KEY);
|
||||
if (tablesToUse == null || tableMap == null || tablesToUse.length != tableMap.length) {
|
||||
// this can only happen when HLogMapper is used directly by a class other than WALPlayer
|
||||
// this can only happen when WALMapper is used directly by a class other than WALPlayer
|
||||
throw new IOException("No tables or incorrect table mapping specified.");
|
||||
}
|
||||
int i = 0;
|
||||
|
@ -192,7 +204,7 @@ public class WALPlayer extends Configured implements Tool {
|
|||
|
||||
void setupTime(Configuration conf, String option) throws IOException {
|
||||
String val = conf.get(option);
|
||||
if (val == null) return;
|
||||
if (null == val) return;
|
||||
long ms;
|
||||
try {
|
||||
// first try to parse in user friendly form
|
||||
|
@ -239,7 +251,7 @@ public class WALPlayer extends Configured implements Tool {
|
|||
Job job = Job.getInstance(conf, conf.get(JOB_NAME_CONF_KEY, NAME + "_" + inputDir));
|
||||
job.setJarByClass(WALPlayer.class);
|
||||
FileInputFormat.setInputPaths(job, inputDir);
|
||||
job.setInputFormatClass(HLogInputFormat.class);
|
||||
job.setInputFormatClass(WALInputFormat.class);
|
||||
job.setMapOutputKeyClass(ImmutableBytesWritable.class);
|
||||
String hfileOutPath = conf.get(BULK_OUTPUT_CONF_KEY);
|
||||
if (hfileOutPath != null) {
|
||||
|
@ -248,7 +260,7 @@ public class WALPlayer extends Configured implements Tool {
|
|||
throw new IOException("Exactly one table must be specified for the bulk export option");
|
||||
}
|
||||
HTable table = new HTable(conf, TableName.valueOf(tables[0]));
|
||||
job.setMapperClass(HLogKeyValueMapper.class);
|
||||
job.setMapperClass(WALKeyValueMapper.class);
|
||||
job.setReducerClass(KeyValueSortReducer.class);
|
||||
Path outputDir = new Path(hfileOutPath);
|
||||
FileOutputFormat.setOutputPath(job, outputDir);
|
||||
|
@ -258,7 +270,7 @@ public class WALPlayer extends Configured implements Tool {
|
|||
com.google.common.base.Preconditions.class);
|
||||
} else {
|
||||
// output to live cluster
|
||||
job.setMapperClass(HLogMapper.class);
|
||||
job.setMapperClass(WALMapper.class);
|
||||
job.setOutputFormatClass(MultiTableOutputFormat.class);
|
||||
TableMapReduceUtil.addDependencyJars(job);
|
||||
TableMapReduceUtil.initCredentials(job);
|
||||
|
@ -288,8 +300,8 @@ public class WALPlayer extends Configured implements Tool {
|
|||
System.err.println(" -D" + BULK_OUTPUT_CONF_KEY + "=/path/for/output");
|
||||
System.err.println(" (Only one table can be specified, and no mapping is allowed!)");
|
||||
System.err.println("Other options: (specify time range to WAL edit to consider)");
|
||||
System.err.println(" -D" + HLogInputFormat.START_TIME_KEY + "=[date|ms]");
|
||||
System.err.println(" -D" + HLogInputFormat.END_TIME_KEY + "=[date|ms]");
|
||||
System.err.println(" -D" + WALInputFormat.START_TIME_KEY + "=[date|ms]");
|
||||
System.err.println(" -D" + WALInputFormat.END_TIME_KEY + "=[date|ms]");
|
||||
System.err.println(" -D " + JOB_NAME_CONF_KEY
|
||||
+ "=jobName - use the specified mapreduce job name for the wal player");
|
||||
System.err.println("For performance also consider the following options:\n"
|
||||
|
|
|
@ -76,8 +76,7 @@ import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.Regio
|
|||
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLog;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
|
||||
import org.apache.hadoop.hbase.wal.DefaultWALProvider;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.KeyLocker;
|
||||
|
@ -463,17 +462,20 @@ public class AssignmentManager {
|
|||
}
|
||||
if (!failover) {
|
||||
// If we get here, we have a full cluster restart. It is a failover only
|
||||
// if there are some HLogs are not split yet. For meta HLogs, they should have
|
||||
// if there are some WALs are not split yet. For meta WALs, they should have
|
||||
// been split already, if any. We can walk through those queued dead servers,
|
||||
// if they don't have any HLogs, this restart should be considered as a clean one
|
||||
// if they don't have any WALs, this restart should be considered as a clean one
|
||||
Set<ServerName> queuedDeadServers = serverManager.getRequeuedDeadServers().keySet();
|
||||
if (!queuedDeadServers.isEmpty()) {
|
||||
Configuration conf = server.getConfiguration();
|
||||
Path rootdir = FSUtils.getRootDir(conf);
|
||||
FileSystem fs = rootdir.getFileSystem(conf);
|
||||
for (ServerName serverName: queuedDeadServers) {
|
||||
Path logDir = new Path(rootdir, HLogUtil.getHLogDirectoryName(serverName.toString()));
|
||||
Path splitDir = logDir.suffix(HLog.SPLITTING_EXT);
|
||||
// In the case of a clean exit, the shutdown handler would have presplit any WALs and
|
||||
// removed empty directories.
|
||||
Path logDir = new Path(rootdir,
|
||||
DefaultWALProvider.getWALDirectoryName(serverName.toString()));
|
||||
Path splitDir = logDir.suffix(DefaultWALProvider.SPLITTING_EXT);
|
||||
if (fs.exists(logDir) || fs.exists(splitDir)) {
|
||||
LOG.debug("Found queued dead server " + serverName);
|
||||
failover = true;
|
||||
|
|
|
@ -51,8 +51,8 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
|||
import org.apache.hadoop.hbase.fs.HFileSystem;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLog;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
|
||||
import org.apache.hadoop.hbase.wal.DefaultWALProvider;
|
||||
import org.apache.hadoop.hbase.wal.WALSplitter;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.FSTableDescriptors;
|
||||
|
@ -94,14 +94,14 @@ public class MasterFileSystem {
|
|||
final static PathFilter META_FILTER = new PathFilter() {
|
||||
@Override
|
||||
public boolean accept(Path p) {
|
||||
return HLogUtil.isMetaFile(p);
|
||||
return DefaultWALProvider.isMetaFile(p);
|
||||
}
|
||||
};
|
||||
|
||||
final static PathFilter NON_META_FILTER = new PathFilter() {
|
||||
@Override
|
||||
public boolean accept(Path p) {
|
||||
return !HLogUtil.isMetaFile(p);
|
||||
return !DefaultWALProvider.isMetaFile(p);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -216,7 +216,7 @@ public class MasterFileSystem {
|
|||
*/
|
||||
Set<ServerName> getFailedServersFromLogFolders() {
|
||||
boolean retrySplitting = !conf.getBoolean("hbase.hlog.split.skip.errors",
|
||||
HLog.SPLIT_SKIP_ERRORS_DEFAULT);
|
||||
WALSplitter.SPLIT_SKIP_ERRORS_DEFAULT);
|
||||
|
||||
Set<ServerName> serverNames = new HashSet<ServerName>();
|
||||
Path logsDirPath = new Path(this.rootdir, HConstants.HREGION_LOGDIR_NAME);
|
||||
|
@ -239,13 +239,13 @@ public class MasterFileSystem {
|
|||
return serverNames;
|
||||
}
|
||||
for (FileStatus status : logFolders) {
|
||||
String sn = status.getPath().getName();
|
||||
// truncate splitting suffix if present (for ServerName parsing)
|
||||
if (sn.endsWith(HLog.SPLITTING_EXT)) {
|
||||
sn = sn.substring(0, sn.length() - HLog.SPLITTING_EXT.length());
|
||||
}
|
||||
ServerName serverName = ServerName.parseServerName(sn);
|
||||
if (!onlineServers.contains(serverName)) {
|
||||
final ServerName serverName = DefaultWALProvider.getServerNameFromWALDirectoryName(
|
||||
status.getPath());
|
||||
if (null == serverName) {
|
||||
LOG.warn("Log folder " + status.getPath() + " doesn't look like its name includes a " +
|
||||
"region server name; leaving in place. If you see later errors about missing " +
|
||||
"write ahead logs they may be saved in this location.");
|
||||
} else if (!onlineServers.contains(serverName)) {
|
||||
LOG.info("Log folder " + status.getPath() + " doesn't belong "
|
||||
+ "to a known region server, splitting");
|
||||
serverNames.add(serverName);
|
||||
|
@ -283,7 +283,7 @@ public class MasterFileSystem {
|
|||
}
|
||||
|
||||
/**
|
||||
* Specialized method to handle the splitting for meta HLog
|
||||
* Specialized method to handle the splitting for meta WAL
|
||||
* @param serverName
|
||||
* @throws IOException
|
||||
*/
|
||||
|
@ -294,7 +294,7 @@ public class MasterFileSystem {
|
|||
}
|
||||
|
||||
/**
|
||||
* Specialized method to handle the splitting for meta HLog
|
||||
* Specialized method to handle the splitting for meta WAL
|
||||
* @param serverNames
|
||||
* @throws IOException
|
||||
*/
|
||||
|
@ -302,6 +302,9 @@ public class MasterFileSystem {
|
|||
splitLog(serverNames, META_FILTER);
|
||||
}
|
||||
|
||||
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="UL_UNRELEASED_LOCK", justification=
|
||||
"We only release this lock when we set it. Updates to code that uses it should verify use " +
|
||||
"of the guard boolean.")
|
||||
private List<Path> getLogDirs(final Set<ServerName> serverNames) throws IOException {
|
||||
List<Path> logDirs = new ArrayList<Path>();
|
||||
boolean needReleaseLock = false;
|
||||
|
@ -312,9 +315,10 @@ public class MasterFileSystem {
|
|||
}
|
||||
try {
|
||||
for (ServerName serverName : serverNames) {
|
||||
Path logDir = new Path(this.rootdir, HLogUtil.getHLogDirectoryName(serverName.toString()));
|
||||
Path splitDir = logDir.suffix(HLog.SPLITTING_EXT);
|
||||
// Rename the directory so a rogue RS doesn't create more HLogs
|
||||
Path logDir = new Path(this.rootdir,
|
||||
DefaultWALProvider.getWALDirectoryName(serverName.toString()));
|
||||
Path splitDir = logDir.suffix(DefaultWALProvider.SPLITTING_EXT);
|
||||
// Rename the directory so a rogue RS doesn't create more WALs
|
||||
if (fs.exists(logDir)) {
|
||||
if (!this.fs.rename(logDir, splitDir)) {
|
||||
throw new IOException("Failed fs.rename for log split: " + logDir);
|
||||
|
@ -367,9 +371,10 @@ public class MasterFileSystem {
|
|||
}
|
||||
|
||||
/**
|
||||
* This method is the base split method that splits HLog files matching a filter. Callers should
|
||||
* pass the appropriate filter for meta and non-meta HLogs.
|
||||
* @param serverNames
|
||||
* This method is the base split method that splits WAL files matching a filter. Callers should
|
||||
* pass the appropriate filter for meta and non-meta WALs.
|
||||
* @param serverNames logs belonging to these servers will be split; this will rename the log
|
||||
* directory out from under a soft-failed server
|
||||
* @param filter
|
||||
* @throws IOException
|
||||
*/
|
||||
|
|
|
@ -31,7 +31,7 @@ public class MetricsMasterFileSystem {
|
|||
/**
|
||||
* Record a single instance of a split
|
||||
* @param time time that the split took
|
||||
* @param size length of original HLogs that were split
|
||||
* @param size length of original WALs that were split
|
||||
*/
|
||||
public synchronized void addSplit(long time, long size) {
|
||||
source.updateSplitTime(time);
|
||||
|
@ -41,7 +41,7 @@ public class MetricsMasterFileSystem {
|
|||
/**
|
||||
* Record a single instance of a split
|
||||
* @param time time that the split took
|
||||
* @param size length of original HLogs that were split
|
||||
* @param size length of original WALs that were split
|
||||
*/
|
||||
public synchronized void addMetaWALSplit(long time, long size) {
|
||||
source.updateMetaWALSplitTime(time);
|
||||
|
|
|
@ -448,7 +448,7 @@ public class RegionStates {
|
|||
}
|
||||
|
||||
/**
|
||||
* A dead server's hlogs have been split so that all the regions
|
||||
* A dead server's wals have been split so that all the regions
|
||||
* used to be open on it can be safely assigned now. Mark them assignable.
|
||||
*/
|
||||
public synchronized void logSplit(final ServerName serverName) {
|
||||
|
@ -688,7 +688,7 @@ public class RegionStates {
|
|||
|
||||
/**
|
||||
* Checking if a region was assigned to a server which is not online now.
|
||||
* If so, we should hold re-assign this region till SSH has split its hlogs.
|
||||
* If so, we should hold re-assign this region till SSH has split its wals.
|
||||
* Once logs are split, the last assignment of this region will be reset,
|
||||
* which means a null last assignment server is ok for re-assigning.
|
||||
*
|
||||
|
|
|
@ -591,7 +591,7 @@ public class ServerManager {
|
|||
this.processDeadServer(serverName, false);
|
||||
}
|
||||
|
||||
public synchronized void processDeadServer(final ServerName serverName, boolean shouldSplitHlog) {
|
||||
public synchronized void processDeadServer(final ServerName serverName, boolean shouldSplitWal) {
|
||||
// When assignment manager is cleaning up the zookeeper nodes and rebuilding the
|
||||
// in-memory region states, region servers could be down. Meta table can and
|
||||
// should be re-assigned, log splitting can be done too. However, it is better to
|
||||
|
@ -601,14 +601,14 @@ public class ServerManager {
|
|||
// the handler threads and meta table could not be re-assigned in case
|
||||
// the corresponding server is down. So we queue them up here instead.
|
||||
if (!services.getAssignmentManager().isFailoverCleanupDone()) {
|
||||
requeuedDeadServers.put(serverName, shouldSplitHlog);
|
||||
requeuedDeadServers.put(serverName, shouldSplitWal);
|
||||
return;
|
||||
}
|
||||
|
||||
this.deadservers.add(serverName);
|
||||
this.services.getExecutorService().submit(
|
||||
new ServerShutdownHandler(this.master, this.services, this.deadservers, serverName,
|
||||
shouldSplitHlog));
|
||||
shouldSplitWal));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -947,7 +947,7 @@ public class ServerManager {
|
|||
|
||||
/**
|
||||
* During startup, if we figure it is not a failover, i.e. there is
|
||||
* no more HLog files to split, we won't try to recover these dead servers.
|
||||
* no more WAL files to split, we won't try to recover these dead servers.
|
||||
* So we just remove them from the queue. Use caution in calling this.
|
||||
*/
|
||||
void removeRequeuedDeadServers() {
|
||||
|
|
|
@ -59,7 +59,7 @@ import org.apache.hadoop.hbase.monitoring.MonitoredTask;
|
|||
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
|
||||
import org.apache.hadoop.hbase.regionserver.SplitLogWorker;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
|
||||
import org.apache.hadoop.hbase.wal.DefaultWALProvider;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
|
@ -102,8 +102,7 @@ public class SplitLogManager {
|
|||
private Server server;
|
||||
|
||||
private final Stoppable stopper;
|
||||
private FileSystem fs;
|
||||
private Configuration conf;
|
||||
private final Configuration conf;
|
||||
|
||||
public static final int DEFAULT_UNASSIGNED_TIMEOUT = (3 * 60 * 1000); // 3 min
|
||||
|
||||
|
@ -161,16 +160,34 @@ public class SplitLogManager {
|
|||
}
|
||||
|
||||
private FileStatus[] getFileList(List<Path> logDirs, PathFilter filter) throws IOException {
|
||||
return getFileList(conf, logDirs, filter);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get a list of paths that need to be split given a set of server-specific directories and
|
||||
* optinally a filter.
|
||||
*
|
||||
* See {@link DefaultWALProvider#getServerNameFromWALDirectoryName} for more info on directory
|
||||
* layout.
|
||||
*
|
||||
* Should be package-private, but is needed by
|
||||
* {@link org.apache.hadoop.hbase.wal.WALSplitter#split(Path, Path, Path, FileSystem,
|
||||
* Configuration, WALFactory)} for tests.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public static FileStatus[] getFileList(final Configuration conf, final List<Path> logDirs,
|
||||
final PathFilter filter)
|
||||
throws IOException {
|
||||
List<FileStatus> fileStatus = new ArrayList<FileStatus>();
|
||||
for (Path hLogDir : logDirs) {
|
||||
this.fs = hLogDir.getFileSystem(conf);
|
||||
if (!fs.exists(hLogDir)) {
|
||||
LOG.warn(hLogDir + " doesn't exist. Nothing to do!");
|
||||
for (Path logDir : logDirs) {
|
||||
final FileSystem fs = logDir.getFileSystem(conf);
|
||||
if (!fs.exists(logDir)) {
|
||||
LOG.warn(logDir + " doesn't exist. Nothing to do!");
|
||||
continue;
|
||||
}
|
||||
FileStatus[] logfiles = FSUtils.listStatus(fs, hLogDir, filter);
|
||||
FileStatus[] logfiles = FSUtils.listStatus(fs, logDir, filter);
|
||||
if (logfiles == null || logfiles.length == 0) {
|
||||
LOG.info(hLogDir + " is empty dir, no logs to split");
|
||||
LOG.info(logDir + " is empty dir, no logs to split");
|
||||
} else {
|
||||
Collections.addAll(fileStatus, logfiles);
|
||||
}
|
||||
|
@ -180,7 +197,7 @@ public class SplitLogManager {
|
|||
}
|
||||
|
||||
/**
|
||||
* @param logDir one region sever hlog dir path in .logs
|
||||
* @param logDir one region sever wal dir path in .logs
|
||||
* @throws IOException if there was an error while splitting any log file
|
||||
* @return cumulative size of the logfiles split
|
||||
* @throws IOException
|
||||
|
@ -206,7 +223,7 @@ public class SplitLogManager {
|
|||
Set<ServerName> serverNames = new HashSet<ServerName>();
|
||||
for (Path logDir : logDirs) {
|
||||
try {
|
||||
ServerName serverName = HLogUtil.getServerNameFromHLogDirectoryName(logDir);
|
||||
ServerName serverName = DefaultWALProvider.getServerNameFromWALDirectoryName(logDir);
|
||||
if (serverName != null) {
|
||||
serverNames.add(serverName);
|
||||
}
|
||||
|
@ -273,6 +290,7 @@ public class SplitLogManager {
|
|||
}
|
||||
for (Path logDir : logDirs) {
|
||||
status.setStatus("Cleaning up log directory...");
|
||||
final FileSystem fs = logDir.getFileSystem(conf);
|
||||
try {
|
||||
if (fs.exists(logDir) && !fs.delete(logDir, false)) {
|
||||
LOG.warn("Unable to delete log src dir. Ignoring. " + logDir);
|
||||
|
|
|
@ -26,10 +26,10 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Stoppable;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
|
||||
import org.apache.hadoop.hbase.wal.DefaultWALProvider;
|
||||
|
||||
/**
|
||||
* This Chore, every time it runs, will attempt to delete the HLogs in the old logs folder. The HLog
|
||||
* This Chore, every time it runs, will attempt to delete the WALs in the old logs folder. The WAL
|
||||
* is only deleted if none of the cleaner delegates says otherwise.
|
||||
* @see BaseLogCleanerDelegate
|
||||
*/
|
||||
|
@ -51,6 +51,6 @@ public class LogCleaner extends CleanerChore<BaseLogCleanerDelegate> {
|
|||
|
||||
@Override
|
||||
protected boolean validate(Path file) {
|
||||
return HLogUtil.validateHLogFilename(file.getName());
|
||||
return DefaultWALProvider.validateWALFilename(file.getName());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -26,7 +26,7 @@ import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
|||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
|
||||
/**
|
||||
* Log cleaner that uses the timestamp of the hlog to determine if it should
|
||||
* Log cleaner that uses the timestamp of the wal to determine if it should
|
||||
* be deleted. By default they are allowed to live for 10 minutes.
|
||||
*/
|
||||
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
|
||||
|
|
|
@ -67,7 +67,7 @@ public class MetaServerShutdownHandler extends ServerShutdownHandler {
|
|||
boolean distributedLogReplay =
|
||||
(this.services.getMasterFileSystem().getLogRecoveryMode() == RecoveryMode.LOG_REPLAY);
|
||||
try {
|
||||
if (this.shouldSplitHlog) {
|
||||
if (this.shouldSplitWal) {
|
||||
LOG.info("Splitting hbase:meta logs for " + serverName);
|
||||
if (distributedLogReplay) {
|
||||
Set<HRegionInfo> regions = new HashSet<HRegionInfo>();
|
||||
|
@ -95,7 +95,7 @@ public class MetaServerShutdownHandler extends ServerShutdownHandler {
|
|||
}
|
||||
|
||||
try {
|
||||
if (this.shouldSplitHlog && distributedLogReplay) {
|
||||
if (this.shouldSplitWal && distributedLogReplay) {
|
||||
if (!am.waitOnRegionToClearRegionsInTransition(HRegionInfo.FIRST_META_REGIONINFO,
|
||||
regionAssignmentWaitTimeout)) {
|
||||
// Wait here is to avoid log replay hits current dead server and incur a RPC timeout
|
||||
|
|
|
@ -56,19 +56,19 @@ public class ServerShutdownHandler extends EventHandler {
|
|||
protected final ServerName serverName;
|
||||
protected final MasterServices services;
|
||||
protected final DeadServer deadServers;
|
||||
protected final boolean shouldSplitHlog; // whether to split HLog or not
|
||||
protected final boolean shouldSplitWal; // whether to split WAL or not
|
||||
protected final int regionAssignmentWaitTimeout;
|
||||
|
||||
public ServerShutdownHandler(final Server server, final MasterServices services,
|
||||
final DeadServer deadServers, final ServerName serverName,
|
||||
final boolean shouldSplitHlog) {
|
||||
final boolean shouldSplitWal) {
|
||||
this(server, services, deadServers, serverName, EventType.M_SERVER_SHUTDOWN,
|
||||
shouldSplitHlog);
|
||||
shouldSplitWal);
|
||||
}
|
||||
|
||||
ServerShutdownHandler(final Server server, final MasterServices services,
|
||||
final DeadServer deadServers, final ServerName serverName, EventType type,
|
||||
final boolean shouldSplitHlog) {
|
||||
final boolean shouldSplitWal) {
|
||||
super(server, type);
|
||||
this.serverName = serverName;
|
||||
this.server = server;
|
||||
|
@ -77,7 +77,7 @@ public class ServerShutdownHandler extends EventHandler {
|
|||
if (!this.deadServers.isDeadServer(this.serverName)) {
|
||||
LOG.warn(this.serverName + " is NOT in deadservers; it should be!");
|
||||
}
|
||||
this.shouldSplitHlog = shouldSplitHlog;
|
||||
this.shouldSplitWal = shouldSplitWal;
|
||||
this.regionAssignmentWaitTimeout = server.getConfiguration().getInt(
|
||||
HConstants.LOG_REPLAY_WAIT_REGION_TIMEOUT, 15000);
|
||||
}
|
||||
|
@ -133,7 +133,7 @@ public class ServerShutdownHandler extends EventHandler {
|
|||
AssignmentManager am = services.getAssignmentManager();
|
||||
ServerManager serverManager = services.getServerManager();
|
||||
if (isCarryingMeta() /* hbase:meta */ || !am.isFailoverCleanupDone()) {
|
||||
serverManager.processDeadServer(serverName, this.shouldSplitHlog);
|
||||
serverManager.processDeadServer(serverName, this.shouldSplitWal);
|
||||
return;
|
||||
}
|
||||
|
||||
|
@ -180,7 +180,7 @@ public class ServerShutdownHandler extends EventHandler {
|
|||
(this.services.getMasterFileSystem().getLogRecoveryMode() == RecoveryMode.LOG_REPLAY);
|
||||
|
||||
try {
|
||||
if (this.shouldSplitHlog) {
|
||||
if (this.shouldSplitWal) {
|
||||
if (distributedLogReplay) {
|
||||
LOG.info("Mark regions in recovery for crashed server " + serverName +
|
||||
" before assignment; regions=" + hris);
|
||||
|
@ -275,13 +275,13 @@ public class ServerShutdownHandler extends EventHandler {
|
|||
throw (InterruptedIOException)new InterruptedIOException().initCause(ie);
|
||||
} catch (IOException ioe) {
|
||||
LOG.info("Caught " + ioe + " during region assignment, will retry");
|
||||
// Only do HLog splitting if shouldSplitHlog and in DLR mode
|
||||
// Only do wal splitting if shouldSplitWal and in DLR mode
|
||||
serverManager.processDeadServer(serverName,
|
||||
this.shouldSplitHlog && distributedLogReplay);
|
||||
this.shouldSplitWal && distributedLogReplay);
|
||||
return;
|
||||
}
|
||||
|
||||
if (this.shouldSplitHlog && distributedLogReplay) {
|
||||
if (this.shouldSplitWal && distributedLogReplay) {
|
||||
// wait for region assignment completes
|
||||
for (HRegionInfo hri : toAssignRegions) {
|
||||
try {
|
||||
|
|
|
@ -46,11 +46,11 @@ public class SnapshotLogCleaner extends BaseLogCleanerDelegate {
|
|||
* Conf key for the frequency to attempt to refresh the cache of hfiles currently used in
|
||||
* snapshots (ms)
|
||||
*/
|
||||
static final String HLOG_CACHE_REFRESH_PERIOD_CONF_KEY =
|
||||
static final String WAL_CACHE_REFRESH_PERIOD_CONF_KEY =
|
||||
"hbase.master.hlogcleaner.plugins.snapshot.period";
|
||||
|
||||
/** Refresh cache, by default, every 5 minutes */
|
||||
private static final long DEFAULT_HLOG_CACHE_REFRESH_PERIOD = 300000;
|
||||
private static final long DEFAULT_WAL_CACHE_REFRESH_PERIOD = 300000;
|
||||
|
||||
private SnapshotFileCache cache;
|
||||
|
||||
|
@ -77,14 +77,14 @@ public class SnapshotLogCleaner extends BaseLogCleanerDelegate {
|
|||
super.setConf(conf);
|
||||
try {
|
||||
long cacheRefreshPeriod = conf.getLong(
|
||||
HLOG_CACHE_REFRESH_PERIOD_CONF_KEY, DEFAULT_HLOG_CACHE_REFRESH_PERIOD);
|
||||
WAL_CACHE_REFRESH_PERIOD_CONF_KEY, DEFAULT_WAL_CACHE_REFRESH_PERIOD);
|
||||
final FileSystem fs = FSUtils.getCurrentFileSystem(conf);
|
||||
Path rootDir = FSUtils.getRootDir(conf);
|
||||
cache = new SnapshotFileCache(fs, rootDir, cacheRefreshPeriod, cacheRefreshPeriod,
|
||||
"snapshot-log-cleaner-cache-refresher", new SnapshotFileCache.SnapshotFileInspector() {
|
||||
public Collection<String> filesUnderSnapshot(final Path snapshotDir)
|
||||
throws IOException {
|
||||
return SnapshotReferenceUtil.getHLogNames(fs, snapshotDir);
|
||||
return SnapshotReferenceUtil.getWALNames(fs, snapshotDir);
|
||||
}
|
||||
});
|
||||
} catch (IOException e) {
|
||||
|
|
|
@ -50,9 +50,8 @@ import org.apache.hadoop.hbase.client.TableState;
|
|||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLog;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogFactory;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
|
||||
import org.apache.hadoop.hbase.wal.WAL;
|
||||
import org.apache.hadoop.hbase.wal.WALFactory;
|
||||
import org.apache.hadoop.hbase.security.access.AccessControlLists;
|
||||
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -382,13 +381,11 @@ public class NamespaceUpgrade implements Tool {
|
|||
|
||||
|
||||
ServerName fakeServer = ServerName.valueOf("nsupgrade", 96, 123);
|
||||
String metaLogName = HLogUtil.getHLogDirectoryName(fakeServer.toString());
|
||||
HLog metaHLog = HLogFactory.createMetaHLog(fs, rootDir,
|
||||
metaLogName, conf, null,
|
||||
fakeServer.toString());
|
||||
final WALFactory walFactory = new WALFactory(conf, null, fakeServer.toString());
|
||||
WAL metawal = walFactory.getMetaWAL(HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes());
|
||||
FSTableDescriptors fst = new FSTableDescriptors(conf);
|
||||
HRegion meta = HRegion.openHRegion(rootDir, HRegionInfo.FIRST_META_REGIONINFO,
|
||||
fst.get(TableName.META_TABLE_NAME), metaHLog, conf);
|
||||
fst.get(TableName.META_TABLE_NAME), metawal, conf);
|
||||
HRegion region = null;
|
||||
try {
|
||||
for(Path regionDir : FSUtils.getRegionDirs(fs, oldTablePath)) {
|
||||
|
@ -405,7 +402,7 @@ public class NamespaceUpgrade implements Tool {
|
|||
new HRegion(
|
||||
HRegionFileSystem.openRegionFromFileSystem(conf, fs, oldTablePath,
|
||||
oldRegionInfo, false),
|
||||
metaHLog,
|
||||
metawal,
|
||||
conf,
|
||||
oldDesc,
|
||||
null);
|
||||
|
@ -442,7 +439,7 @@ public class NamespaceUpgrade implements Tool {
|
|||
meta.flushcache();
|
||||
meta.waitForFlushesAndCompactions();
|
||||
meta.close();
|
||||
metaHLog.closeAndDelete();
|
||||
metawal.close();
|
||||
if(region != null) {
|
||||
region.close();
|
||||
}
|
||||
|
|
|
@ -39,8 +39,8 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
|
|||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLog;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
|
||||
import org.apache.hadoop.hbase.wal.WAL.Entry;
|
||||
import org.apache.hadoop.hbase.wal.WALKey;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.util.ByteStringer;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
|
@ -50,14 +50,14 @@ import com.google.protobuf.ServiceException;
|
|||
@InterfaceAudience.Private
|
||||
public class ReplicationProtbufUtil {
|
||||
/**
|
||||
* A helper to replicate a list of HLog entries using admin protocol.
|
||||
* A helper to replicate a list of WAL entries using admin protocol.
|
||||
*
|
||||
* @param admin
|
||||
* @param entries
|
||||
* @throws java.io.IOException
|
||||
*/
|
||||
public static void replicateWALEntry(final AdminService.BlockingInterface admin,
|
||||
final HLog.Entry[] entries) throws IOException {
|
||||
final Entry[] entries) throws IOException {
|
||||
Pair<AdminProtos.ReplicateWALEntryRequest, CellScanner> p =
|
||||
buildReplicateWALEntryRequest(entries, null);
|
||||
PayloadCarryingRpcController controller = new PayloadCarryingRpcController(p.getSecond());
|
||||
|
@ -69,27 +69,27 @@ public class ReplicationProtbufUtil {
|
|||
}
|
||||
|
||||
/**
|
||||
* Create a new ReplicateWALEntryRequest from a list of HLog entries
|
||||
* Create a new ReplicateWALEntryRequest from a list of WAL entries
|
||||
*
|
||||
* @param entries the HLog entries to be replicated
|
||||
* @param entries the WAL entries to be replicated
|
||||
* @return a pair of ReplicateWALEntryRequest and a CellScanner over all the WALEdit values
|
||||
* found.
|
||||
*/
|
||||
public static Pair<AdminProtos.ReplicateWALEntryRequest, CellScanner>
|
||||
buildReplicateWALEntryRequest(final HLog.Entry[] entries) {
|
||||
buildReplicateWALEntryRequest(final Entry[] entries) {
|
||||
return buildReplicateWALEntryRequest(entries, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a new ReplicateWALEntryRequest from a list of HLog entries
|
||||
* Create a new ReplicateWALEntryRequest from a list of WAL entries
|
||||
*
|
||||
* @param entries the HLog entries to be replicated
|
||||
* @param entries the WAL entries to be replicated
|
||||
* @param encodedRegionName alternative region name to use if not null
|
||||
* @return a pair of ReplicateWALEntryRequest and a CellScanner over all the WALEdit values
|
||||
* found.
|
||||
*/
|
||||
public static Pair<AdminProtos.ReplicateWALEntryRequest, CellScanner>
|
||||
buildReplicateWALEntryRequest(final HLog.Entry[] entries, byte[] encodedRegionName) {
|
||||
buildReplicateWALEntryRequest(final Entry[] entries, byte[] encodedRegionName) {
|
||||
// Accumulate all the Cells seen in here.
|
||||
List<List<? extends Cell>> allCells = new ArrayList<List<? extends Cell>>(entries.length);
|
||||
int size = 0;
|
||||
|
@ -98,11 +98,11 @@ public class ReplicationProtbufUtil {
|
|||
AdminProtos.ReplicateWALEntryRequest.Builder builder =
|
||||
AdminProtos.ReplicateWALEntryRequest.newBuilder();
|
||||
HBaseProtos.UUID.Builder uuidBuilder = HBaseProtos.UUID.newBuilder();
|
||||
for (HLog.Entry entry: entries) {
|
||||
for (Entry entry: entries) {
|
||||
entryBuilder.clear();
|
||||
// TODO: this duplicates a lot in HLogKey#getBuilder
|
||||
// TODO: this duplicates a lot in WALKey#getBuilder
|
||||
WALProtos.WALKey.Builder keyBuilder = entryBuilder.getKeyBuilder();
|
||||
HLogKey key = entry.getKey();
|
||||
WALKey key = entry.getKey();
|
||||
keyBuilder.setEncodedRegionName(
|
||||
ByteStringer.wrap(encodedRegionName == null
|
||||
? key.getEncodedRegionName()
|
||||
|
|
|
@ -44,6 +44,7 @@ import java.util.concurrent.ConcurrentHashMap;
|
|||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.ConcurrentSkipListMap;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||
import java.net.InetAddress;
|
||||
|
||||
|
@ -124,9 +125,10 @@ import org.apache.hadoop.hbase.quotas.RegionServerQuotaManager;
|
|||
import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
|
||||
import org.apache.hadoop.hbase.regionserver.handler.CloseMetaHandler;
|
||||
import org.apache.hadoop.hbase.regionserver.handler.CloseRegionHandler;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLog;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogFactory;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
|
||||
import org.apache.hadoop.hbase.wal.DefaultWALProvider;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.MetricsWAL;
|
||||
import org.apache.hadoop.hbase.wal.WAL;
|
||||
import org.apache.hadoop.hbase.wal.WALFactory;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
|
||||
import org.apache.hadoop.hbase.security.UserProvider;
|
||||
import org.apache.hadoop.hbase.trace.SpanReceiverHost;
|
||||
|
@ -330,15 +332,13 @@ public class HRegionServer extends HasThread implements
|
|||
*/
|
||||
Chore periodicFlusher;
|
||||
|
||||
// HLog and HLog roller. log is protected rather than private to avoid
|
||||
// eclipse warning when accessed by inner classes
|
||||
protected volatile HLog hlog;
|
||||
// The meta updates are written to a different hlog. If this
|
||||
// regionserver holds meta regions, then this field will be non-null.
|
||||
protected volatile HLog hlogForMeta;
|
||||
protected volatile WALFactory walFactory;
|
||||
|
||||
LogRoller hlogRoller;
|
||||
LogRoller metaHLogRoller;
|
||||
// WAL roller. log is protected rather than private to avoid
|
||||
// eclipse warning when accessed by inner classes
|
||||
final LogRoller walRoller;
|
||||
// Lazily initialized if this RegionServer hosts a meta table.
|
||||
final AtomicReference<LogRoller> metawalRoller = new AtomicReference<LogRoller>();
|
||||
|
||||
// flag set after we're done setting up server threads
|
||||
final AtomicBoolean online = new AtomicBoolean(false);
|
||||
|
@ -546,6 +546,7 @@ public class HRegionServer extends HasThread implements
|
|||
|
||||
rpcServices.start();
|
||||
putUpWebUI();
|
||||
this.walRoller = new LogRoller(this, this);
|
||||
}
|
||||
|
||||
protected void login(UserProvider user, String host) throws IOException {
|
||||
|
@ -974,7 +975,7 @@ public class HRegionServer extends HasThread implements
|
|||
|
||||
//fsOk flag may be changed when closing regions throws exception.
|
||||
if (this.fsOk) {
|
||||
closeWAL(!abortRequested);
|
||||
shutdownWAL(!abortRequested);
|
||||
}
|
||||
|
||||
// Make sure the proxy is down.
|
||||
|
@ -1076,7 +1077,8 @@ public class HRegionServer extends HasThread implements
|
|||
}
|
||||
}
|
||||
|
||||
ClusterStatusProtos.ServerLoad buildServerLoad(long reportStartTime, long reportEndTime) {
|
||||
ClusterStatusProtos.ServerLoad buildServerLoad(long reportStartTime, long reportEndTime)
|
||||
throws IOException {
|
||||
// We're getting the MetricsRegionServerWrapper here because the wrapper computes requests
|
||||
// per second, and other metrics As long as metrics are part of ServerLoad it's best to use
|
||||
// the wrapper to compute those numbers in one place.
|
||||
|
@ -1095,7 +1097,7 @@ public class HRegionServer extends HasThread implements
|
|||
serverLoad.setTotalNumberOfRequests((int) regionServerWrapper.getTotalRequestCount());
|
||||
serverLoad.setUsedHeapMB((int)(memory.getUsed() / 1024 / 1024));
|
||||
serverLoad.setMaxHeapMB((int) (memory.getMax() / 1024 / 1024));
|
||||
Set<String> coprocessors = this.hlog.getCoprocessorHost().getCoprocessors();
|
||||
Set<String> coprocessors = getWAL(null).getCoprocessorHost().getCoprocessors();
|
||||
for (String coprocessor : coprocessors) {
|
||||
serverLoad.addCoprocessors(
|
||||
Coprocessor.newBuilder().setName(coprocessor).build());
|
||||
|
@ -1104,6 +1106,10 @@ public class HRegionServer extends HasThread implements
|
|||
RegionSpecifier.Builder regionSpecifier = RegionSpecifier.newBuilder();
|
||||
for (HRegion region : regions) {
|
||||
serverLoad.addRegionLoads(createRegionLoad(region, regionLoadBldr, regionSpecifier));
|
||||
for (String coprocessor :
|
||||
getWAL(region.getRegionInfo()).getCoprocessorHost().getCoprocessors()) {
|
||||
serverLoad.addCoprocessors(Coprocessor.newBuilder().setName(coprocessor).build());
|
||||
}
|
||||
}
|
||||
serverLoad.setReportStartTime(reportStartTime);
|
||||
serverLoad.setReportEndTime(reportEndTime);
|
||||
|
@ -1192,35 +1198,24 @@ public class HRegionServer extends HasThread implements
|
|||
return interrupted;
|
||||
}
|
||||
|
||||
private void closeWAL(final boolean delete) {
|
||||
if (this.hlogForMeta != null) {
|
||||
// All hlogs (meta and non-meta) are in the same directory. Don't call
|
||||
// closeAndDelete here since that would delete all hlogs not just the
|
||||
// meta ones. We will just 'close' the hlog for meta here, and leave
|
||||
// the directory cleanup to the follow-on closeAndDelete call.
|
||||
private void shutdownWAL(final boolean close) {
|
||||
if (this.walFactory != null) {
|
||||
try {
|
||||
this.hlogForMeta.close();
|
||||
} catch (Throwable e) {
|
||||
e = e instanceof RemoteException ? ((RemoteException) e).unwrapRemoteException() : e;
|
||||
LOG.error("Metalog close and delete failed", e);
|
||||
}
|
||||
}
|
||||
if (this.hlog != null) {
|
||||
try {
|
||||
if (delete) {
|
||||
hlog.closeAndDelete();
|
||||
if (close) {
|
||||
walFactory.close();
|
||||
} else {
|
||||
hlog.close();
|
||||
walFactory.shutdown();
|
||||
}
|
||||
} catch (Throwable e) {
|
||||
e = e instanceof RemoteException ? ((RemoteException) e).unwrapRemoteException() : e;
|
||||
LOG.error("Close and delete failed", e);
|
||||
LOG.error("Shutdown / close of WAL failed: " + e);
|
||||
LOG.debug("Shutdown / close exception details:", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
* Run init. Sets up hlog and starts up all server threads.
|
||||
* Run init. Sets up wal and starts up all server threads.
|
||||
*
|
||||
* @param c Extra configuration.
|
||||
*/
|
||||
|
@ -1258,7 +1253,7 @@ public class HRegionServer extends HasThread implements
|
|||
ZNodeClearer.writeMyEphemeralNodeOnDisk(getMyEphemeralNodePath());
|
||||
|
||||
this.cacheConfig = new CacheConfig(conf);
|
||||
this.hlog = setupWALAndReplication();
|
||||
this.walFactory = setupWALAndReplication();
|
||||
// Init in here rather than in constructor after thread name has been set
|
||||
this.metricsRegionServer = new MetricsRegionServer(new MetricsRegionServerWrapperImpl(this));
|
||||
|
||||
|
@ -1502,10 +1497,10 @@ public class HRegionServer extends HasThread implements
|
|||
* @return A WAL instance.
|
||||
* @throws IOException
|
||||
*/
|
||||
private HLog setupWALAndReplication() throws IOException {
|
||||
private WALFactory setupWALAndReplication() throws IOException {
|
||||
// TODO Replication make assumptions here based on the default filesystem impl
|
||||
final Path oldLogDir = new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME);
|
||||
final String logName
|
||||
= HLogUtil.getHLogDirectoryName(this.serverName.toString());
|
||||
final String logName = DefaultWALProvider.getWALDirectoryName(this.serverName.toString());
|
||||
|
||||
Path logdir = new Path(rootDir, logName);
|
||||
if (LOG.isDebugEnabled()) LOG.debug("logdir=" + logdir);
|
||||
|
@ -1518,66 +1513,44 @@ public class HRegionServer extends HasThread implements
|
|||
// log directories.
|
||||
createNewReplicationInstance(conf, this, this.fs, logdir, oldLogDir);
|
||||
|
||||
return instantiateHLog(rootDir, logName);
|
||||
}
|
||||
|
||||
private HLog getMetaWAL() throws IOException {
|
||||
if (this.hlogForMeta != null) return this.hlogForMeta;
|
||||
final String logName = HLogUtil.getHLogDirectoryName(this.serverName.toString());
|
||||
Path logdir = new Path(rootDir, logName);
|
||||
if (LOG.isDebugEnabled()) LOG.debug("logdir=" + logdir);
|
||||
this.hlogForMeta = HLogFactory.createMetaHLog(this.fs.getBackingFs(), rootDir, logName,
|
||||
this.conf, getMetaWALActionListeners(), this.serverName.toString());
|
||||
return this.hlogForMeta;
|
||||
}
|
||||
|
||||
/**
|
||||
* Called by {@link #setupWALAndReplication()} creating WAL instance.
|
||||
* @param rootdir
|
||||
* @param logName
|
||||
* @return WAL instance.
|
||||
* @throws IOException
|
||||
*/
|
||||
protected HLog instantiateHLog(Path rootdir, String logName) throws IOException {
|
||||
return HLogFactory.createHLog(this.fs.getBackingFs(), rootdir, logName, this.conf,
|
||||
getWALActionListeners(), this.serverName.toString());
|
||||
}
|
||||
|
||||
/**
|
||||
* Called by {@link #instantiateHLog(Path, String)} setting up WAL instance.
|
||||
* Add any {@link WALActionsListener}s you want inserted before WAL startup.
|
||||
* @return List of WALActionsListener that will be passed in to
|
||||
* {@link org.apache.hadoop.hbase.regionserver.wal.FSHLog} on construction.
|
||||
*/
|
||||
protected List<WALActionsListener> getWALActionListeners() {
|
||||
List<WALActionsListener> listeners = new ArrayList<WALActionsListener>();
|
||||
// Log roller.
|
||||
this.hlogRoller = new LogRoller(this, this);
|
||||
listeners.add(this.hlogRoller);
|
||||
// listeners the wal factory will add to wals it creates.
|
||||
final List<WALActionsListener> listeners = new ArrayList<WALActionsListener>();
|
||||
listeners.add(new MetricsWAL());
|
||||
if (this.replicationSourceHandler != null &&
|
||||
this.replicationSourceHandler.getWALActionsListener() != null) {
|
||||
// Replication handler is an implementation of WALActionsListener.
|
||||
listeners.add(this.replicationSourceHandler.getWALActionsListener());
|
||||
}
|
||||
return listeners;
|
||||
|
||||
return new WALFactory(conf, listeners, serverName.toString());
|
||||
}
|
||||
|
||||
protected List<WALActionsListener> getMetaWALActionListeners() {
|
||||
List<WALActionsListener> listeners = new ArrayList<WALActionsListener>();
|
||||
/**
|
||||
* We initialize the roller for the wal that handles meta lazily
|
||||
* since we don't know if this regionserver will handle it. All calls to
|
||||
* this method return a reference to the that same roller. As newly referenced
|
||||
* meta regions are brought online, they will be offered to the roller for maintenance.
|
||||
* As a part of that registration process, the roller will add itself as a
|
||||
* listener on the wal.
|
||||
*/
|
||||
protected LogRoller ensureMetaWALRoller() {
|
||||
// Using a tmp log roller to ensure metaLogRoller is alive once it is not
|
||||
// null
|
||||
MetaLogRoller tmpLogRoller = new MetaLogRoller(this, this);
|
||||
String n = Thread.currentThread().getName();
|
||||
Threads.setDaemonThreadRunning(tmpLogRoller.getThread(),
|
||||
n + "-MetaLogRoller", uncaughtExceptionHandler);
|
||||
this.metaHLogRoller = tmpLogRoller;
|
||||
tmpLogRoller = null;
|
||||
listeners.add(this.metaHLogRoller);
|
||||
return listeners;
|
||||
}
|
||||
|
||||
protected LogRoller getLogRoller() {
|
||||
return hlogRoller;
|
||||
LogRoller roller = metawalRoller.get();
|
||||
if (null == roller) {
|
||||
LogRoller tmpLogRoller = new LogRoller(this, this);
|
||||
String n = Thread.currentThread().getName();
|
||||
Threads.setDaemonThreadRunning(tmpLogRoller.getThread(),
|
||||
n + "-MetaLogRoller", uncaughtExceptionHandler);
|
||||
if (metawalRoller.compareAndSet(null, tmpLogRoller)) {
|
||||
roller = tmpLogRoller;
|
||||
} else {
|
||||
// Another thread won starting the roller
|
||||
Threads.shutdown(tmpLogRoller.getThread());
|
||||
roller = metawalRoller.get();
|
||||
}
|
||||
}
|
||||
return roller;
|
||||
}
|
||||
|
||||
public MetricsRegionServer getRegionServerMetrics() {
|
||||
|
@ -1620,7 +1593,7 @@ public class HRegionServer extends HasThread implements
|
|||
this.service.startExecutorService(ExecutorType.RS_LOG_REPLAY_OPS, conf.getInt(
|
||||
"hbase.regionserver.wal.max.splitters", SplitLogWorkerCoordination.DEFAULT_MAX_SPLITTERS));
|
||||
|
||||
Threads.setDaemonThreadRunning(this.hlogRoller.getThread(), getName() + ".logRoller",
|
||||
Threads.setDaemonThreadRunning(this.walRoller.getThread(), getName() + ".logRoller",
|
||||
uncaughtExceptionHandler);
|
||||
this.cacheFlusher.start(uncaughtExceptionHandler);
|
||||
Threads.setDaemonThreadRunning(this.compactionChecker.getThread(), getName() +
|
||||
|
@ -1667,7 +1640,7 @@ public class HRegionServer extends HasThread implements
|
|||
sinkConf.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY,
|
||||
conf.getInt("hbase.log.replay.rpc.timeout", 30000)); // default 30 seconds
|
||||
sinkConf.setInt("hbase.client.serverside.retries.multiplier", 1);
|
||||
this.splitLogWorker = new SplitLogWorker(this, sinkConf, this, this);
|
||||
this.splitLogWorker = new SplitLogWorker(this, sinkConf, this, this, walFactory);
|
||||
splitLogWorker.start();
|
||||
}
|
||||
|
||||
|
@ -1730,38 +1703,37 @@ public class HRegionServer extends HasThread implements
|
|||
}
|
||||
// Verify that all threads are alive
|
||||
if (!(leases.isAlive()
|
||||
&& cacheFlusher.isAlive() && hlogRoller.isAlive()
|
||||
&& cacheFlusher.isAlive() && walRoller.isAlive()
|
||||
&& this.compactionChecker.isAlive()
|
||||
&& this.periodicFlusher.isAlive())) {
|
||||
stop("One or more threads are no longer alive -- stop");
|
||||
return false;
|
||||
}
|
||||
if (metaHLogRoller != null && !metaHLogRoller.isAlive()) {
|
||||
stop("Meta HLog roller thread is no longer alive -- stop");
|
||||
final LogRoller metawalRoller = this.metawalRoller.get();
|
||||
if (metawalRoller != null && !metawalRoller.isAlive()) {
|
||||
stop("Meta WAL roller thread is no longer alive -- stop");
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
public HLog getWAL() {
|
||||
try {
|
||||
return getWAL(null);
|
||||
} catch (IOException e) {
|
||||
LOG.warn("getWAL threw exception " + e);
|
||||
return null;
|
||||
}
|
||||
}
|
||||
private static final byte[] UNSPECIFIED_REGION = new byte[]{};
|
||||
|
||||
@Override
|
||||
public HLog getWAL(HRegionInfo regionInfo) throws IOException {
|
||||
//TODO: at some point this should delegate to the HLogFactory
|
||||
//currently, we don't care about the region as much as we care about the
|
||||
//table.. (hence checking the tablename below)
|
||||
public WAL getWAL(HRegionInfo regionInfo) throws IOException {
|
||||
WAL wal;
|
||||
LogRoller roller = walRoller;
|
||||
//_ROOT_ and hbase:meta regions have separate WAL.
|
||||
if (regionInfo != null && regionInfo.isMetaTable()) {
|
||||
return getMetaWAL();
|
||||
roller = ensureMetaWALRoller();
|
||||
wal = walFactory.getMetaWAL(regionInfo.getEncodedNameAsBytes());
|
||||
} else if (regionInfo == null) {
|
||||
wal = walFactory.getWAL(UNSPECIFIED_REGION);
|
||||
} else {
|
||||
wal = walFactory.getWAL(regionInfo.getEncodedNameAsBytes());
|
||||
}
|
||||
return this.hlog;
|
||||
roller.addWAL(wal);
|
||||
return wal;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -2006,11 +1978,12 @@ public class HRegionServer extends HasThread implements
|
|||
if (this.spanReceiverHost != null) {
|
||||
this.spanReceiverHost.closeReceivers();
|
||||
}
|
||||
if (this.hlogRoller != null) {
|
||||
Threads.shutdown(this.hlogRoller.getThread());
|
||||
if (this.walRoller != null) {
|
||||
Threads.shutdown(this.walRoller.getThread());
|
||||
}
|
||||
if (this.metaHLogRoller != null) {
|
||||
Threads.shutdown(this.metaHLogRoller.getThread());
|
||||
final LogRoller metawalRoller = this.metawalRoller.get();
|
||||
if (metawalRoller != null) {
|
||||
Threads.shutdown(metawalRoller.getThread());
|
||||
}
|
||||
if (this.compactSplitThread != null) {
|
||||
this.compactSplitThread.join();
|
||||
|
@ -2518,7 +2491,7 @@ public class HRegionServer extends HasThread implements
|
|||
* @see org.apache.hadoop.hbase.regionserver.HRegionServerCommandLine
|
||||
*/
|
||||
public static void main(String[] args) throws Exception {
|
||||
VersionInfo.logVersion();
|
||||
VersionInfo.logVersion();
|
||||
Configuration conf = HBaseConfiguration.create();
|
||||
@SuppressWarnings("unchecked")
|
||||
Class<? extends HRegionServer> regionServerClass = (Class<? extends HRegionServer>) conf
|
||||
|
@ -2569,11 +2542,24 @@ public class HRegionServer extends HasThread implements
|
|||
|
||||
// used by org/apache/hbase/tmpl/regionserver/RSStatusTmpl.jamon (HBASE-4070).
|
||||
public String[] getRegionServerCoprocessors() {
|
||||
TreeSet<String> coprocessors = new TreeSet<String>(
|
||||
this.hlog.getCoprocessorHost().getCoprocessors());
|
||||
TreeSet<String> coprocessors = new TreeSet<String>();
|
||||
try {
|
||||
coprocessors.addAll(getWAL(null).getCoprocessorHost().getCoprocessors());
|
||||
} catch (IOException exception) {
|
||||
LOG.warn("Exception attempting to fetch wal coprocessor information for the common wal; " +
|
||||
"skipping.");
|
||||
LOG.debug("Exception details for failure to fetch wal coprocessor information.", exception);
|
||||
}
|
||||
Collection<HRegion> regions = getOnlineRegionsLocalContext();
|
||||
for (HRegion region: regions) {
|
||||
coprocessors.addAll(region.getCoprocessorHost().getCoprocessors());
|
||||
try {
|
||||
coprocessors.addAll(getWAL(region.getRegionInfo()).getCoprocessorHost().getCoprocessors());
|
||||
} catch (IOException exception) {
|
||||
LOG.warn("Exception attempting to fetch wal coprocessor information for region " + region +
|
||||
"; skipping.");
|
||||
LOG.debug("Exception details for failure to fetch wal coprocessor information.", exception);
|
||||
}
|
||||
}
|
||||
return coprocessors.toArray(new String[coprocessors.size()]);
|
||||
}
|
||||
|
@ -2696,16 +2682,22 @@ public class HRegionServer extends HasThread implements
|
|||
HRegion toReturn = this.onlineRegions.remove(r.getRegionInfo().getEncodedName());
|
||||
|
||||
if (destination != null) {
|
||||
HLog wal = getWAL();
|
||||
long closeSeqNum = wal.getEarliestMemstoreSeqNum(r.getRegionInfo().getEncodedNameAsBytes());
|
||||
if (closeSeqNum == HConstants.NO_SEQNUM) {
|
||||
// No edits in WAL for this region; get the sequence number when the region was opened.
|
||||
closeSeqNum = r.getOpenSeqNum();
|
||||
try {
|
||||
WAL wal = getWAL(r.getRegionInfo());
|
||||
long closeSeqNum = wal.getEarliestMemstoreSeqNum(r.getRegionInfo().getEncodedNameAsBytes());
|
||||
if (closeSeqNum == HConstants.NO_SEQNUM) {
|
||||
closeSeqNum = 0;
|
||||
// No edits in WAL for this region; get the sequence number when the region was opened.
|
||||
closeSeqNum = r.getOpenSeqNum();
|
||||
if (closeSeqNum == HConstants.NO_SEQNUM) {
|
||||
closeSeqNum = 0;
|
||||
}
|
||||
}
|
||||
addToMovedRegions(r.getRegionInfo().getEncodedName(), destination, closeSeqNum);
|
||||
} catch (IOException exception) {
|
||||
LOG.error("Could not retrieve WAL information for region " + r.getRegionInfo() +
|
||||
"; not adding to moved regions.");
|
||||
LOG.debug("Exception details for failure to get wal", exception);
|
||||
}
|
||||
addToMovedRegions(r.getRegionInfo().getEncodedName(), destination, closeSeqNum);
|
||||
}
|
||||
this.regionFavoredNodesMap.remove(r.getRegionInfo().getEncodedName());
|
||||
return toReturn != null;
|
||||
|
|
|
@ -78,7 +78,7 @@ import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
|
|||
import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.OffPeakHours;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
|
||||
import org.apache.hadoop.hbase.security.EncryptionUtil;
|
||||
import org.apache.hadoop.hbase.security.User;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -1216,7 +1216,7 @@ public class HStore implements Store {
|
|||
*/
|
||||
private void writeCompactionWalRecord(Collection<StoreFile> filesCompacted,
|
||||
Collection<StoreFile> newFiles) throws IOException {
|
||||
if (region.getLog() == null) return;
|
||||
if (region.getWAL() == null) return;
|
||||
List<Path> inputPaths = new ArrayList<Path>(filesCompacted.size());
|
||||
for (StoreFile f : filesCompacted) {
|
||||
inputPaths.add(f.getPath());
|
||||
|
@ -1228,7 +1228,7 @@ public class HStore implements Store {
|
|||
HRegionInfo info = this.region.getRegionInfo();
|
||||
CompactionDescriptor compactionDescriptor = ProtobufUtil.toCompactionDescriptor(info,
|
||||
family.getName(), inputPaths, outputPaths, fs.getStoreDir(getFamily().getNameAsString()));
|
||||
HLogUtil.writeCompactionMarker(region.getLog(), this.region.getTableDesc(),
|
||||
WALUtil.writeCompactionMarker(region.getWAL(), this.region.getTableDesc(),
|
||||
this.region.getRegionInfo(), compactionDescriptor, this.region.getSequenceId());
|
||||
}
|
||||
|
||||
|
|
|
@ -19,38 +19,39 @@
|
|||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLog;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
|
||||
import org.apache.hadoop.hbase.wal.WAL;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.HasThread;
|
||||
import org.apache.hadoop.ipc.RemoteException;
|
||||
|
||||
/**
|
||||
* Runs periodically to determine if the HLog should be rolled.
|
||||
* Runs periodically to determine if the WAL should be rolled.
|
||||
*
|
||||
* NOTE: This class extends Thread rather than Chore because the sleep time
|
||||
* can be interrupted when there is something to do, rather than the Chore
|
||||
* sleep time which is invariant.
|
||||
*
|
||||
* TODO: change to a pool of threads
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
class LogRoller extends HasThread implements WALActionsListener {
|
||||
class LogRoller extends HasThread {
|
||||
static final Log LOG = LogFactory.getLog(LogRoller.class);
|
||||
private final ReentrantLock rollLock = new ReentrantLock();
|
||||
private final AtomicBoolean rollLog = new AtomicBoolean(false);
|
||||
private final ConcurrentHashMap<WAL, Boolean> walNeedsRoll =
|
||||
new ConcurrentHashMap<WAL, Boolean>();
|
||||
private final Server server;
|
||||
protected final RegionServerServices services;
|
||||
private volatile long lastrolltime = System.currentTimeMillis();
|
||||
|
@ -58,6 +59,32 @@ class LogRoller extends HasThread implements WALActionsListener {
|
|||
private final long rollperiod;
|
||||
private final int threadWakeFrequency;
|
||||
|
||||
public void addWAL(final WAL wal) {
|
||||
if (null == walNeedsRoll.putIfAbsent(wal, Boolean.FALSE)) {
|
||||
wal.registerWALActionsListener(new WALActionsListener.Base() {
|
||||
@Override
|
||||
public void logRollRequested() {
|
||||
walNeedsRoll.put(wal, Boolean.TRUE);
|
||||
// TODO logs will contend with each other here, replace with e.g. DelayedQueue
|
||||
synchronized(rollLog) {
|
||||
rollLog.set(true);
|
||||
rollLog.notifyAll();
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
public void requestRollAll() {
|
||||
for (WAL wal : walNeedsRoll.keySet()) {
|
||||
walNeedsRoll.put(wal, Boolean.TRUE);
|
||||
}
|
||||
synchronized(rollLog) {
|
||||
rollLog.set(true);
|
||||
rollLog.notifyAll();
|
||||
}
|
||||
}
|
||||
|
||||
/** @param server */
|
||||
public LogRoller(final Server server, final RegionServerServices services) {
|
||||
super();
|
||||
|
@ -88,19 +115,24 @@ class LogRoller extends HasThread implements WALActionsListener {
|
|||
}
|
||||
// Time for periodic roll
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Hlog roll period " + this.rollperiod + "ms elapsed");
|
||||
LOG.debug("Wal roll period " + this.rollperiod + "ms elapsed");
|
||||
}
|
||||
} else if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("HLog roll requested");
|
||||
LOG.debug("WAL roll requested");
|
||||
}
|
||||
rollLock.lock(); // FindBugs UL_UNRELEASED_LOCK_EXCEPTION_PATH
|
||||
try {
|
||||
this.lastrolltime = now;
|
||||
// Force the roll if the logroll.period is elapsed or if a roll was requested.
|
||||
// The returned value is an array of actual region names.
|
||||
byte [][] regionsToFlush = getWAL().rollWriter(periodic || rollLog.get());
|
||||
if (regionsToFlush != null) {
|
||||
for (byte [] r: regionsToFlush) scheduleFlush(r);
|
||||
for (Entry<WAL, Boolean> entry : walNeedsRoll.entrySet()) {
|
||||
final WAL wal = entry.getKey();
|
||||
// Force the roll if the logroll.period is elapsed or if a roll was requested.
|
||||
// The returned value is an array of actual region names.
|
||||
final byte [][] regionsToFlush = wal.rollWriter(periodic ||
|
||||
entry.getValue().booleanValue());
|
||||
walNeedsRoll.put(wal, Boolean.FALSE);
|
||||
if (regionsToFlush != null) {
|
||||
for (byte [] r: regionsToFlush) scheduleFlush(r);
|
||||
}
|
||||
}
|
||||
} catch (FailedLogCloseException e) {
|
||||
server.abort("Failed log close in log roller", e);
|
||||
|
@ -145,51 +177,4 @@ class LogRoller extends HasThread implements WALActionsListener {
|
|||
}
|
||||
}
|
||||
|
||||
public void logRollRequested() {
|
||||
synchronized (rollLog) {
|
||||
rollLog.set(true);
|
||||
rollLog.notifyAll();
|
||||
}
|
||||
}
|
||||
|
||||
protected HLog getWAL() throws IOException {
|
||||
return this.services.getWAL(null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void preLogRoll(Path oldPath, Path newPath) throws IOException {
|
||||
// Not interested
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postLogRoll(Path oldPath, Path newPath) throws IOException {
|
||||
// Not interested
|
||||
}
|
||||
|
||||
@Override
|
||||
public void preLogArchive(Path oldPath, Path newPath) throws IOException {
|
||||
// Not interested
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postLogArchive(Path oldPath, Path newPath) throws IOException {
|
||||
// Not interested
|
||||
}
|
||||
|
||||
@Override
|
||||
public void visitLogEntryBeforeWrite(HRegionInfo info, HLogKey logKey,
|
||||
WALEdit logEdit) {
|
||||
// Not interested.
|
||||
}
|
||||
|
||||
@Override
|
||||
public void visitLogEntryBeforeWrite(HTableDescriptor htd, HLogKey logKey,
|
||||
WALEdit logEdit) {
|
||||
//Not interested
|
||||
}
|
||||
|
||||
@Override
|
||||
public void logCloseRequested() {
|
||||
// not interested
|
||||
}
|
||||
}
|
||||
|
|
|
@ -460,11 +460,11 @@ class MemStoreFlusher implements FlushRequester {
|
|||
}
|
||||
} catch (DroppedSnapshotException ex) {
|
||||
// Cache flush can fail in a few places. If it fails in a critical
|
||||
// section, we get a DroppedSnapshotException and a replay of hlog
|
||||
// section, we get a DroppedSnapshotException and a replay of wal
|
||||
// is required. Currently the only way to do this is a restart of
|
||||
// the server. Abort because hdfs is probably bad (HBASE-644 is a case
|
||||
// where hdfs was bad but passed the hdfs check).
|
||||
server.abort("Replay of HLog required. Forcing server shutdown", ex);
|
||||
server.abort("Replay of WAL required. Forcing server shutdown", ex);
|
||||
return false;
|
||||
} catch (IOException ex) {
|
||||
ex = ex instanceof RemoteException ? ((RemoteException) ex).unwrapRemoteException() : ex;
|
||||
|
|
|
@ -1,38 +0,0 @@
|
|||
/**
|
||||
* 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;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLog;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
class MetaLogRoller extends LogRoller {
|
||||
public MetaLogRoller(Server server, RegionServerServices services) {
|
||||
super(server, services);
|
||||
}
|
||||
@Override
|
||||
protected HLog getWAL() throws IOException {
|
||||
//The argument to getWAL below could either be HRegionInfo.FIRST_META_REGIONINFO or
|
||||
//HRegionInfo.ROOT_REGIONINFO. Both these share the same WAL.
|
||||
return services.getWAL(HRegionInfo.FIRST_META_REGIONINFO);
|
||||
}
|
||||
}
|
|
@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.ServerName;
|
|||
import org.apache.hadoop.hbase.io.hfile.BlockCache;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheStats;
|
||||
import org.apache.hadoop.hbase.wal.DefaultWALProvider;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
|
@ -53,8 +54,8 @@ class MetricsRegionServerWrapperImpl
|
|||
private BlockCache blockCache;
|
||||
|
||||
private volatile long numStores = 0;
|
||||
private volatile long numHLogFiles = 0;
|
||||
private volatile long hlogFileSize = 0;
|
||||
private volatile long numWALFiles = 0;
|
||||
private volatile long walFileSize = 0;
|
||||
private volatile long numStoreFiles = 0;
|
||||
private volatile long memstoreSize = 0;
|
||||
private volatile long storeFileSize = 0;
|
||||
|
@ -286,13 +287,13 @@ class MetricsRegionServerWrapperImpl
|
|||
}
|
||||
|
||||
@Override
|
||||
public long getNumHLogFiles() {
|
||||
return numHLogFiles;
|
||||
public long getNumWALFiles() {
|
||||
return numWALFiles;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getHLogFileSize() {
|
||||
return hlogFileSize;
|
||||
public long getWALFileSize() {
|
||||
return walFileSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -490,21 +491,11 @@ class MetricsRegionServerWrapperImpl
|
|||
}
|
||||
lastRan = currentTime;
|
||||
|
||||
numWALFiles = DefaultWALProvider.getNumLogFiles(regionServer.walFactory);
|
||||
walFileSize = DefaultWALProvider.getLogFileSize(regionServer.walFactory);
|
||||
|
||||
//Copy over computed values so that no thread sees half computed values.
|
||||
numStores = tempNumStores;
|
||||
long tempNumHLogFiles = regionServer.hlog.getNumLogFiles();
|
||||
// meta logs
|
||||
if (regionServer.hlogForMeta != null) {
|
||||
tempNumHLogFiles += regionServer.hlogForMeta.getNumLogFiles();
|
||||
}
|
||||
numHLogFiles = tempNumHLogFiles;
|
||||
|
||||
long tempHlogFileSize = regionServer.hlog.getLogFileSize();
|
||||
if (regionServer.hlogForMeta != null) {
|
||||
tempHlogFileSize += regionServer.hlogForMeta.getLogFileSize();
|
||||
}
|
||||
hlogFileSize = tempHlogFileSize;
|
||||
|
||||
numStoreFiles = tempNumStoreFiles;
|
||||
memstoreSize = tempMemstoreSize;
|
||||
storeFileSize = tempStoreFileSize;
|
||||
|
|
|
@ -149,9 +149,9 @@ import org.apache.hadoop.hbase.regionserver.HRegion.Operation;
|
|||
import org.apache.hadoop.hbase.regionserver.Leases.LeaseStillHeldException;
|
||||
import org.apache.hadoop.hbase.regionserver.handler.OpenMetaHandler;
|
||||
import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLog;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter;
|
||||
import org.apache.hadoop.hbase.wal.WAL;
|
||||
import org.apache.hadoop.hbase.wal.WALKey;
|
||||
import org.apache.hadoop.hbase.wal.WALSplitter;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.util.ByteStringer;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -693,13 +693,13 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
|||
* @throws IOException
|
||||
*/
|
||||
private OperationStatus [] doReplayBatchOp(final HRegion region,
|
||||
final List<HLogSplitter.MutationReplay> mutations, long replaySeqId) throws IOException {
|
||||
final List<WALSplitter.MutationReplay> mutations, long replaySeqId) throws IOException {
|
||||
|
||||
long before = EnvironmentEdgeManager.currentTime();
|
||||
boolean batchContainsPuts = false, batchContainsDelete = false;
|
||||
try {
|
||||
for (Iterator<HLogSplitter.MutationReplay> it = mutations.iterator(); it.hasNext();) {
|
||||
HLogSplitter.MutationReplay m = it.next();
|
||||
for (Iterator<WALSplitter.MutationReplay> it = mutations.iterator(); it.hasNext();) {
|
||||
WALSplitter.MutationReplay m = it.next();
|
||||
|
||||
if (m.type == MutationType.PUT) {
|
||||
batchContainsPuts = true;
|
||||
|
@ -724,7 +724,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
|||
regionServer.cacheFlusher.reclaimMemStoreMemory();
|
||||
}
|
||||
return region.batchReplay(mutations.toArray(
|
||||
new HLogSplitter.MutationReplay[mutations.size()]), replaySeqId);
|
||||
new WALSplitter.MutationReplay[mutations.size()]), replaySeqId);
|
||||
} finally {
|
||||
if (regionServer.metricsRegionServer != null) {
|
||||
long after = EnvironmentEdgeManager.currentTime();
|
||||
|
@ -1097,10 +1097,10 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
|||
return builder.build();
|
||||
} catch (DroppedSnapshotException ex) {
|
||||
// Cache flush can fail in a few places. If it fails in a critical
|
||||
// section, we get a DroppedSnapshotException and a replay of hlog
|
||||
// section, we get a DroppedSnapshotException and a replay of wal
|
||||
// is required. Currently the only way to do this is a restart of
|
||||
// the server.
|
||||
regionServer.abort("Replay of HLog required. Forcing server shutdown", ex);
|
||||
regionServer.abort("Replay of WAL required. Forcing server shutdown", ex);
|
||||
throw new ServiceException(ex);
|
||||
} catch (IOException ie) {
|
||||
throw new ServiceException(ie);
|
||||
|
@ -1431,7 +1431,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
|||
ServerRegionReplicaUtil.isDefaultReplica(region.getRegionInfo())
|
||||
? region.getCoprocessorHost()
|
||||
: null; // do not invoke coprocessors if this is a secondary region replica
|
||||
List<Pair<HLogKey, WALEdit>> walEntries = new ArrayList<Pair<HLogKey, WALEdit>>();
|
||||
List<Pair<WALKey, WALEdit>> walEntries = new ArrayList<Pair<WALKey, WALEdit>>();
|
||||
// when tag is enabled, we need tag replay edits with log sequence number
|
||||
boolean needAddReplayTag = (HFile.getFormatVersion(regionServer.conf) >= 3);
|
||||
|
||||
|
@ -1451,9 +1451,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
|||
long nonce = entry.getKey().hasNonce() ? entry.getKey().getNonce() : HConstants.NO_NONCE;
|
||||
regionServer.nonceManager.reportOperationFromWal(nonceGroup, nonce, entry.getKey().getWriteTime());
|
||||
}
|
||||
Pair<HLogKey, WALEdit> walEntry = (coprocessorHost == null) ? null :
|
||||
new Pair<HLogKey, WALEdit>();
|
||||
List<HLogSplitter.MutationReplay> edits = HLogSplitter.getMutationsFromWALEntry(entry,
|
||||
Pair<WALKey, WALEdit> walEntry = (coprocessorHost == null) ? null :
|
||||
new Pair<WALKey, WALEdit>();
|
||||
List<WALSplitter.MutationReplay> edits = WALSplitter.getMutationsFromWALEntry(entry,
|
||||
cells, walEntry, needAddReplayTag, durability);
|
||||
if (coprocessorHost != null) {
|
||||
// Start coprocessor replay here. The coprocessor is for each WALEdit instead of a
|
||||
|
@ -1482,7 +1482,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
|||
region.syncWal();
|
||||
|
||||
if (coprocessorHost != null) {
|
||||
for (Pair<HLogKey, WALEdit> wal : walEntries) {
|
||||
for (Pair<WALKey, WALEdit> wal : walEntries) {
|
||||
coprocessorHost.postWALRestore(region.getRegionInfo(), wal.getFirst(),
|
||||
wal.getSecond());
|
||||
}
|
||||
|
@ -1535,14 +1535,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
|||
checkOpen();
|
||||
requestCount.increment();
|
||||
regionServer.getRegionServerCoprocessorHost().preRollWALWriterRequest();
|
||||
HLog wal = regionServer.getWAL();
|
||||
byte[][] regionsToFlush = wal.rollWriter(true);
|
||||
regionServer.walRoller.requestRollAll();
|
||||
regionServer.getRegionServerCoprocessorHost().postRollWALWriterRequest();
|
||||
RollWALWriterResponse.Builder builder = RollWALWriterResponse.newBuilder();
|
||||
if (regionsToFlush != null) {
|
||||
for (byte[] region: regionsToFlush) {
|
||||
builder.addRegionToFlush(ByteStringer.wrap(region));
|
||||
}
|
||||
}
|
||||
return builder.build();
|
||||
} catch (IOException ie) {
|
||||
throw new ServiceException(ie);
|
||||
|
|
|
@ -77,6 +77,7 @@ import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
|||
import org.apache.hadoop.hbase.regionserver.HRegion.Operation;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
|
||||
import org.apache.hadoop.hbase.wal.WALKey;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
|
@ -107,6 +108,8 @@ public class RegionCoprocessorHost
|
|||
private static final int LATENCY_BUFFER_SIZE = 100;
|
||||
private final BlockingQueue<Long> coprocessorTimeNanos = new ArrayBlockingQueue<Long>(
|
||||
LATENCY_BUFFER_SIZE);
|
||||
private final boolean useLegacyPre;
|
||||
private final boolean useLegacyPost;
|
||||
|
||||
/**
|
||||
* Constructor
|
||||
|
@ -120,6 +123,14 @@ public class RegionCoprocessorHost
|
|||
this.region = region;
|
||||
this.rsServices = services;
|
||||
this.sharedData = sharedData;
|
||||
// Pick which version of the WAL related events we'll call.
|
||||
// This way we avoid calling the new version on older RegionObservers so
|
||||
// we can maintain binary compatibility.
|
||||
// See notes in javadoc for RegionObserver
|
||||
useLegacyPre = useLegacyMethod(impl.getClass(), "preWALRestore", ObserverContext.class,
|
||||
HRegionInfo.class, WALKey.class, WALEdit.class);
|
||||
useLegacyPost = useLegacyMethod(impl.getClass(), "postWALRestore", ObserverContext.class,
|
||||
HRegionInfo.class, WALKey.class, WALEdit.class);
|
||||
}
|
||||
|
||||
/** @return the region */
|
||||
|
@ -1307,34 +1318,75 @@ public class RegionCoprocessorHost
|
|||
* @return true if default behavior should be bypassed, false otherwise
|
||||
* @throws IOException
|
||||
*/
|
||||
public boolean preWALRestore(final HRegionInfo info, final HLogKey logKey,
|
||||
public boolean preWALRestore(final HRegionInfo info, final WALKey logKey,
|
||||
final WALEdit logEdit) throws IOException {
|
||||
return execOperation(coprocessors.isEmpty() ? null : new RegionOperation() {
|
||||
@Override
|
||||
public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
|
||||
throws IOException {
|
||||
oserver.preWALRestore(ctx, info, logKey, logEdit);
|
||||
// Once we don't need to support the legacy call, replace RegionOperation with a version
|
||||
// that's ObserverContext<RegionEnvironment> and avoid this cast.
|
||||
final RegionEnvironment env = (RegionEnvironment)ctx.getEnvironment();
|
||||
if (env.useLegacyPre) {
|
||||
if (logKey instanceof HLogKey) {
|
||||
oserver.preWALRestore(ctx, info, (HLogKey)logKey, logEdit);
|
||||
} else {
|
||||
legacyWarning(oserver.getClass(), "There are wal keys present that are not HLogKey.");
|
||||
}
|
||||
} else {
|
||||
oserver.preWALRestore(ctx, info, logKey, logEdit);
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* @return true if default behavior should be bypassed, false otherwise
|
||||
* @deprecated use {@link #preWALRestore(HRegionInfo, WALKey, WALEdit)}
|
||||
*/
|
||||
@Deprecated
|
||||
public boolean preWALRestore(final HRegionInfo info, final HLogKey logKey,
|
||||
final WALEdit logEdit) throws IOException {
|
||||
return preWALRestore(info, (WALKey)logKey, logEdit);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param info
|
||||
* @param logKey
|
||||
* @param logEdit
|
||||
* @throws IOException
|
||||
*/
|
||||
public void postWALRestore(final HRegionInfo info, final HLogKey logKey, final WALEdit logEdit)
|
||||
public void postWALRestore(final HRegionInfo info, final WALKey logKey, final WALEdit logEdit)
|
||||
throws IOException {
|
||||
execOperation(coprocessors.isEmpty() ? null : new RegionOperation() {
|
||||
@Override
|
||||
public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
|
||||
throws IOException {
|
||||
oserver.postWALRestore(ctx, info, logKey, logEdit);
|
||||
// Once we don't need to support the legacy call, replace RegionOperation with a version
|
||||
// that's ObserverContext<RegionEnvironment> and avoid this cast.
|
||||
final RegionEnvironment env = (RegionEnvironment)ctx.getEnvironment();
|
||||
if (env.useLegacyPost) {
|
||||
if (logKey instanceof HLogKey) {
|
||||
oserver.postWALRestore(ctx, info, (HLogKey)logKey, logEdit);
|
||||
} else {
|
||||
legacyWarning(oserver.getClass(), "There are wal keys present that are not HLogKey.");
|
||||
}
|
||||
} else {
|
||||
oserver.postWALRestore(ctx, info, logKey, logEdit);
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* @deprecated use {@link #postWALRestore(HRegionInfo, WALKey, WALEdit)}
|
||||
*/
|
||||
@Deprecated
|
||||
public void postWALRestore(final HRegionInfo info, final HLogKey logKey, final WALEdit logEdit)
|
||||
throws IOException {
|
||||
postWALRestore(info, (WALKey)logKey, logEdit);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param familyPaths pairs of { CF, file path } submitted for bulk load
|
||||
* @return true if the default operation should be bypassed
|
||||
|
|
|
@ -34,7 +34,7 @@ public class RegionServerAccounting {
|
|||
|
||||
private final AtomicLong atomicGlobalMemstoreSize = new AtomicLong(0);
|
||||
|
||||
// Store the edits size during replaying HLog. Use this to roll back the
|
||||
// Store the edits size during replaying WAL. Use this to roll back the
|
||||
// global memstore size once a region opening failed.
|
||||
private final ConcurrentMap<byte[], AtomicLong> replayEditsPerRegion =
|
||||
new ConcurrentSkipListMap<byte[], AtomicLong>(Bytes.BYTES_COMPARATOR);
|
||||
|
|
|
@ -33,8 +33,8 @@ import org.apache.hadoop.hbase.executor.ExecutorService;
|
|||
import org.apache.hadoop.hbase.ipc.RpcServerInterface;
|
||||
import org.apache.hadoop.hbase.master.TableLockManager;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLog;
|
||||
import org.apache.hadoop.hbase.quotas.RegionServerQuotaManager;
|
||||
import org.apache.hadoop.hbase.wal.WAL;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
/**
|
||||
|
@ -48,9 +48,9 @@ public interface RegionServerServices
|
|||
*/
|
||||
boolean isStopping();
|
||||
|
||||
/** @return the HLog for a particular region. Pass null for getting the
|
||||
/** @return the WAL for a particular region. Pass null for getting the
|
||||
* default (common) WAL */
|
||||
HLog getWAL(HRegionInfo regionInfo) throws IOException;
|
||||
WAL getWAL(HRegionInfo regionInfo) throws IOException;
|
||||
|
||||
/**
|
||||
* @return Implementation of {@link CompactionRequestor} or null.
|
||||
|
|
|
@ -36,7 +36,8 @@ import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
|
|||
import org.apache.hadoop.hbase.coordination.SplitLogWorkerCoordination;
|
||||
import org.apache.hadoop.hbase.master.SplitLogManager;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter;
|
||||
import org.apache.hadoop.hbase.wal.WALFactory;
|
||||
import org.apache.hadoop.hbase.wal.WALSplitter;
|
||||
import org.apache.hadoop.hbase.util.CancelableProgressable;
|
||||
import org.apache.hadoop.hbase.util.ExceptionUtil;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
|
@ -70,6 +71,7 @@ public class SplitLogWorker implements Runnable {
|
|||
private SplitLogWorkerCoordination coordination;
|
||||
private Configuration conf;
|
||||
private RegionServerServices server;
|
||||
|
||||
public SplitLogWorker(Server hserver, Configuration conf, RegionServerServices server,
|
||||
TaskExecutor splitTaskExecutor) {
|
||||
this.server = server;
|
||||
|
@ -82,7 +84,8 @@ public class SplitLogWorker implements Runnable {
|
|||
}
|
||||
|
||||
public SplitLogWorker(final Server hserver, final Configuration conf,
|
||||
final RegionServerServices server, final LastSequenceId sequenceIdChecker) {
|
||||
final RegionServerServices server, final LastSequenceId sequenceIdChecker,
|
||||
final WALFactory factory) {
|
||||
this(server, conf, server, new TaskExecutor() {
|
||||
@Override
|
||||
public Status exec(String filename, RecoveryMode mode, CancelableProgressable p) {
|
||||
|
@ -99,8 +102,8 @@ public class SplitLogWorker implements Runnable {
|
|||
// interrupted or has encountered a transient error and when it has
|
||||
// encountered a bad non-retry-able persistent error.
|
||||
try {
|
||||
if (!HLogSplitter.splitLogFile(rootdir, fs.getFileStatus(new Path(rootdir, filename)),
|
||||
fs, conf, p, sequenceIdChecker, server.getCoordinatedStateManager(), mode)) {
|
||||
if (!WALSplitter.splitLogFile(rootdir, fs.getFileStatus(new Path(rootdir, filename)),
|
||||
fs, conf, p, sequenceIdChecker, server.getCoordinatedStateManager(), mode, factory)) {
|
||||
return Status.PREEMPTED;
|
||||
}
|
||||
} catch (InterruptedIOException iioe) {
|
||||
|
@ -153,6 +156,7 @@ public class SplitLogWorker implements Runnable {
|
|||
LOG.info("SplitLogWorker " + server.getServerName() + " exiting");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* If the worker is doing a task i.e. splitting a log file then stop the task.
|
||||
* It doesn't exit the worker thread.
|
||||
|
|
|
@ -1,106 +0,0 @@
|
|||
/**
|
||||
*
|
||||
* 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.handler;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.SplitLogCounters;
|
||||
import org.apache.hadoop.hbase.SplitLogTask;
|
||||
import org.apache.hadoop.hbase.coordination.SplitLogWorkerCoordination;
|
||||
import org.apache.hadoop.hbase.executor.EventHandler;
|
||||
import org.apache.hadoop.hbase.executor.EventType;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
|
||||
import org.apache.hadoop.hbase.regionserver.SplitLogWorker.TaskExecutor;
|
||||
import org.apache.hadoop.hbase.regionserver.SplitLogWorker.TaskExecutor.Status;
|
||||
import org.apache.hadoop.hbase.util.CancelableProgressable;
|
||||
|
||||
/**
|
||||
* Handles log splitting a wal
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class HLogSplitterHandler extends EventHandler {
|
||||
private static final Log LOG = LogFactory.getLog(HLogSplitterHandler.class);
|
||||
private final ServerName serverName;
|
||||
private final CancelableProgressable reporter;
|
||||
private final AtomicInteger inProgressTasks;
|
||||
private final TaskExecutor splitTaskExecutor;
|
||||
private final RecoveryMode mode;
|
||||
private final SplitLogWorkerCoordination.SplitTaskDetails splitTaskDetails;
|
||||
private final SplitLogWorkerCoordination coordination;
|
||||
|
||||
|
||||
public HLogSplitterHandler(final Server server, SplitLogWorkerCoordination coordination,
|
||||
SplitLogWorkerCoordination.SplitTaskDetails splitDetails, CancelableProgressable reporter,
|
||||
AtomicInteger inProgressTasks, TaskExecutor splitTaskExecutor, RecoveryMode mode) {
|
||||
super(server, EventType.RS_LOG_REPLAY);
|
||||
this.splitTaskDetails = splitDetails;
|
||||
this.coordination = coordination;
|
||||
this.reporter = reporter;
|
||||
this.inProgressTasks = inProgressTasks;
|
||||
this.inProgressTasks.incrementAndGet();
|
||||
this.serverName = server.getServerName();
|
||||
this.splitTaskExecutor = splitTaskExecutor;
|
||||
this.mode = mode;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void process() throws IOException {
|
||||
long startTime = System.currentTimeMillis();
|
||||
try {
|
||||
Status status = this.splitTaskExecutor.exec(splitTaskDetails.getWALFile(), mode, reporter);
|
||||
switch (status) {
|
||||
case DONE:
|
||||
coordination.endTask(new SplitLogTask.Done(this.serverName,this.mode),
|
||||
SplitLogCounters.tot_wkr_task_done, splitTaskDetails);
|
||||
break;
|
||||
case PREEMPTED:
|
||||
SplitLogCounters.tot_wkr_preempt_task.incrementAndGet();
|
||||
LOG.warn("task execution prempted " + splitTaskDetails.getWALFile());
|
||||
break;
|
||||
case ERR:
|
||||
if (server != null && !server.isStopped()) {
|
||||
coordination.endTask(new SplitLogTask.Err(this.serverName, this.mode),
|
||||
SplitLogCounters.tot_wkr_task_err, splitTaskDetails);
|
||||
break;
|
||||
}
|
||||
// if the RS is exiting then there is probably a tons of stuff
|
||||
// that can go wrong. Resign instead of signaling error.
|
||||
//$FALL-THROUGH$
|
||||
case RESIGNED:
|
||||
if (server != null && server.isStopped()) {
|
||||
LOG.info("task execution interrupted because worker is exiting "
|
||||
+ splitTaskDetails.toString());
|
||||
}
|
||||
coordination.endTask(new SplitLogTask.Resigned(this.serverName, this.mode),
|
||||
SplitLogCounters.tot_wkr_task_resigned, splitTaskDetails);
|
||||
break;
|
||||
}
|
||||
} finally {
|
||||
LOG.info("worker " + serverName + " done with task " + splitTaskDetails.toString() + " in "
|
||||
+ (System.currentTimeMillis() - startTime) + "ms");
|
||||
this.inProgressTasks.decrementAndGet();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -26,17 +26,18 @@ import org.apache.hadoop.hbase.io.TagCompressionContext;
|
|||
import org.apache.hadoop.hbase.io.util.Dictionary;
|
||||
|
||||
/**
|
||||
* Context that holds the various dictionaries for compression in HLog.
|
||||
* Context that holds the various dictionaries for compression in WAL.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
class CompressionContext {
|
||||
public class CompressionContext {
|
||||
|
||||
static final String ENABLE_WAL_TAGS_COMPRESSION =
|
||||
"hbase.regionserver.wal.tags.enablecompression";
|
||||
|
||||
final Dictionary regionDict;
|
||||
final Dictionary tableDict;
|
||||
final Dictionary familyDict;
|
||||
// visible only for WALKey, until we move everything into o.a.h.h.wal
|
||||
public final Dictionary regionDict;
|
||||
public final Dictionary tableDict;
|
||||
public final Dictionary familyDict;
|
||||
final Dictionary qualifierDict;
|
||||
final Dictionary rowDict;
|
||||
// Context used for compressing tags
|
||||
|
|
|
@ -33,9 +33,13 @@ import org.apache.hadoop.io.WritableUtils;
|
|||
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
import org.apache.hadoop.hbase.wal.WAL;
|
||||
import org.apache.hadoop.hbase.wal.WALFactory;
|
||||
import org.apache.hadoop.hbase.wal.WALProvider;
|
||||
|
||||
/**
|
||||
* A set of static functions for running our custom WAL compression/decompression.
|
||||
* Also contains a command line tool to compress and uncompress HLogs.
|
||||
* Also contains a command line tool to compress and uncompress WALs.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class Compressor {
|
||||
|
@ -56,8 +60,8 @@ public class Compressor {
|
|||
|
||||
private static void printHelp() {
|
||||
System.err.println("usage: Compressor <input> <output>");
|
||||
System.err.println("If <input> HLog is compressed, <output> will be decompressed.");
|
||||
System.err.println("If <input> HLog is uncompressed, <output> will be compressed.");
|
||||
System.err.println("If <input> WAL is compressed, <output> will be decompressed.");
|
||||
System.err.println("If <input> WAL is uncompressed, <output> will be compressed.");
|
||||
return;
|
||||
}
|
||||
|
||||
|
@ -68,8 +72,8 @@ public class Compressor {
|
|||
FileSystem inFS = input.getFileSystem(conf);
|
||||
FileSystem outFS = output.getFileSystem(conf);
|
||||
|
||||
HLog.Reader in = HLogFactory.createReader(inFS, input, conf, null, false);
|
||||
HLog.Writer out = null;
|
||||
WAL.Reader in = WALFactory.createReaderIgnoreCustomClass(inFS, input, conf);
|
||||
WALProvider.Writer out = null;
|
||||
|
||||
try {
|
||||
if (!(in instanceof ReaderBase)) {
|
||||
|
@ -78,9 +82,9 @@ public class Compressor {
|
|||
}
|
||||
boolean compress = ((ReaderBase)in).hasCompression();
|
||||
conf.setBoolean(HConstants.ENABLE_WAL_COMPRESSION, !compress);
|
||||
out = HLogFactory.createWALWriter(outFS, output, conf);
|
||||
out = WALFactory.createWALWriter(outFS, output, conf);
|
||||
|
||||
HLog.Entry e = null;
|
||||
WAL.Entry e = null;
|
||||
while ((e = in.next()) != null) out.append(e);
|
||||
} finally {
|
||||
in.close();
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -27,16 +27,19 @@ import org.apache.hadoop.hbase.CellUtil;
|
|||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
|
||||
import org.apache.hadoop.hbase.wal.WAL.Entry;
|
||||
import org.apache.hadoop.hbase.wal.WALKey;
|
||||
|
||||
/**
|
||||
* A WAL Entry for {@link FSHLog} implementation. Immutable.
|
||||
* A subclass of {@link HLog.Entry} that carries extra info across the ring buffer such as
|
||||
* A subclass of {@link Entry} that carries extra info across the ring buffer such as
|
||||
* region sequence id (we want to use this later, just before we write the WAL to ensure region
|
||||
* edits maintain order). The extra info added here is not 'serialized' as part of the WALEdit
|
||||
* hence marked 'transient' to underline this fact. It also adds mechanism so we can wait on
|
||||
* the assign of the region sequence id. See {@link #stampRegionSequenceId()}.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
class FSWALEntry extends HLog.Entry {
|
||||
class FSWALEntry extends Entry {
|
||||
// The below data members are denoted 'transient' just to highlight these are not persisted;
|
||||
// they are only in memory and held here while passing over the ring buffer.
|
||||
private final transient long sequence;
|
||||
|
@ -46,7 +49,7 @@ class FSWALEntry extends HLog.Entry {
|
|||
private final transient HRegionInfo hri;
|
||||
private final transient List<Cell> memstoreCells;
|
||||
|
||||
FSWALEntry(final long sequence, final HLogKey key, final WALEdit edit,
|
||||
FSWALEntry(final long sequence, final WALKey key, final WALEdit edit,
|
||||
final AtomicLong referenceToRegionSequenceId, final boolean inMemstore,
|
||||
final HTableDescriptor htd, final HRegionInfo hri, List<Cell> memstoreCells) {
|
||||
super(key, edit);
|
||||
|
@ -98,7 +101,7 @@ class FSWALEntry extends HLog.Entry {
|
|||
CellUtil.setSequenceId(cell, regionSequenceId);
|
||||
}
|
||||
}
|
||||
HLogKey key = getKey();
|
||||
WALKey key = getKey();
|
||||
key.setLogSeqNum(regionSequenceId);
|
||||
return regionSequenceId;
|
||||
}
|
||||
|
|
|
@ -1,445 +0,0 @@
|
|||
/**
|
||||
*
|
||||
* 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.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALTrailer;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
/**
|
||||
* HLog records all the edits to HStore. It is the hbase write-ahead-log (WAL).
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
// TODO: Rename interface to WAL
|
||||
public interface HLog {
|
||||
Log LOG = LogFactory.getLog(HLog.class);
|
||||
public static final long NO_SEQUENCE_ID = -1;
|
||||
|
||||
/** File Extension used while splitting an HLog into regions (HBASE-2312) */
|
||||
// TODO: this seems like an implementation detail that does not belong here.
|
||||
String SPLITTING_EXT = "-splitting";
|
||||
boolean SPLIT_SKIP_ERRORS_DEFAULT = false;
|
||||
/** The hbase:meta region's HLog filename extension.*/
|
||||
// TODO: Implementation detail. Does not belong in here.
|
||||
String META_HLOG_FILE_EXTN = ".meta";
|
||||
|
||||
/**
|
||||
* Configuration name of HLog Trailer's warning size. If a waltrailer's size is greater than the
|
||||
* configured size, a warning is logged. This is used with Protobuf reader/writer.
|
||||
*/
|
||||
// TODO: Implementation detail. Why in here?
|
||||
String WAL_TRAILER_WARN_SIZE = "hbase.regionserver.waltrailer.warn.size";
|
||||
int DEFAULT_WAL_TRAILER_WARN_SIZE = 1024 * 1024; // 1MB
|
||||
|
||||
// TODO: Implementation detail. Why in here?
|
||||
Pattern EDITFILES_NAME_PATTERN = Pattern.compile("-?[0-9]+");
|
||||
String RECOVERED_LOG_TMPFILE_SUFFIX = ".temp";
|
||||
String SEQUENCE_ID_FILE_SUFFIX = "_seqid";
|
||||
|
||||
/**
|
||||
* WAL Reader Interface
|
||||
*/
|
||||
interface Reader {
|
||||
/**
|
||||
* @param fs File system.
|
||||
* @param path Path.
|
||||
* @param c Configuration.
|
||||
* @param s Input stream that may have been pre-opened by the caller; may be null.
|
||||
*/
|
||||
void init(FileSystem fs, Path path, Configuration c, FSDataInputStream s) throws IOException;
|
||||
|
||||
void close() throws IOException;
|
||||
|
||||
Entry next() throws IOException;
|
||||
|
||||
Entry next(Entry reuse) throws IOException;
|
||||
|
||||
void seek(long pos) throws IOException;
|
||||
|
||||
long getPosition() throws IOException;
|
||||
void reset() throws IOException;
|
||||
|
||||
/**
|
||||
* @return the WALTrailer of the current HLog. It may be null in case of legacy or corrupt WAL
|
||||
* files.
|
||||
*/
|
||||
// TODO: What we need a trailer on WAL for? It won't be present on last WAL most of the time.
|
||||
// What then?
|
||||
WALTrailer getWALTrailer();
|
||||
}
|
||||
|
||||
/**
|
||||
* WAL Writer Intrface.
|
||||
*/
|
||||
interface Writer {
|
||||
void init(FileSystem fs, Path path, Configuration c, boolean overwritable) throws IOException;
|
||||
|
||||
void close() throws IOException;
|
||||
|
||||
void sync() throws IOException;
|
||||
|
||||
void append(Entry entry) throws IOException;
|
||||
|
||||
long getLength() throws IOException;
|
||||
|
||||
/**
|
||||
* Sets HLog/WAL's WALTrailer. This trailer is appended at the end of WAL on closing.
|
||||
* @param walTrailer trailer to append to WAL.
|
||||
*/
|
||||
// TODO: Why a trailer on the log?
|
||||
void setWALTrailer(WALTrailer walTrailer);
|
||||
}
|
||||
|
||||
/**
|
||||
* Utility class that lets us keep track of the edit and it's associated key. Only used when
|
||||
* splitting logs.
|
||||
*/
|
||||
// TODO: Remove this Writable.
|
||||
// TODO: Why is this in here? Implementation detail?
|
||||
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.REPLICATION)
|
||||
class Entry implements Writable {
|
||||
private WALEdit edit;
|
||||
private HLogKey key;
|
||||
|
||||
public Entry() {
|
||||
edit = new WALEdit();
|
||||
key = new HLogKey();
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructor for both params
|
||||
*
|
||||
* @param edit log's edit
|
||||
* @param key log's key
|
||||
*/
|
||||
public Entry(HLogKey key, WALEdit edit) {
|
||||
this.key = key;
|
||||
this.edit = edit;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the edit
|
||||
*
|
||||
* @return edit
|
||||
*/
|
||||
public WALEdit getEdit() {
|
||||
return edit;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the key
|
||||
*
|
||||
* @return key
|
||||
*/
|
||||
public HLogKey getKey() {
|
||||
return key;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set compression context for this entry.
|
||||
*
|
||||
* @param compressionContext Compression context
|
||||
*/
|
||||
public void setCompressionContext(CompressionContext compressionContext) {
|
||||
edit.setCompressionContext(compressionContext);
|
||||
key.setCompressionContext(compressionContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return this.key + "=" + this.edit;
|
||||
}
|
||||
|
||||
@Override
|
||||
@SuppressWarnings("deprecation")
|
||||
public void write(DataOutput dataOutput) throws IOException {
|
||||
this.key.write(dataOutput);
|
||||
this.edit.write(dataOutput);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFields(DataInput dataInput) throws IOException {
|
||||
this.key.readFields(dataInput);
|
||||
this.edit.readFields(dataInput);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Registers WALActionsListener
|
||||
*
|
||||
* @param listener
|
||||
*/
|
||||
void registerWALActionsListener(final WALActionsListener listener);
|
||||
|
||||
/**
|
||||
* Unregisters WALActionsListener
|
||||
*
|
||||
* @param listener
|
||||
*/
|
||||
boolean unregisterWALActionsListener(final WALActionsListener listener);
|
||||
|
||||
/**
|
||||
* @return Current state of the monotonically increasing file id.
|
||||
*/
|
||||
// TODO: Remove. Implementation detail.
|
||||
long getFilenum();
|
||||
|
||||
/**
|
||||
* @return the number of HLog files
|
||||
*/
|
||||
int getNumLogFiles();
|
||||
|
||||
/**
|
||||
* @return the size of HLog files
|
||||
*/
|
||||
long getLogFileSize();
|
||||
|
||||
// TODO: Log rolling should not be in this interface.
|
||||
/**
|
||||
* Roll the log writer. That is, start writing log messages to a new file.
|
||||
*
|
||||
* <p>
|
||||
* The implementation is synchronized in order to make sure there's one rollWriter
|
||||
* running at any given time.
|
||||
*
|
||||
* @return If lots of logs, flush the returned regions so next time through we
|
||||
* can clean logs. Returns null if nothing to flush. Names are actual
|
||||
* region names as returned by {@link HRegionInfo#getEncodedName()}
|
||||
* @throws org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException
|
||||
* @throws IOException
|
||||
*/
|
||||
byte[][] rollWriter() throws FailedLogCloseException, IOException;
|
||||
|
||||
/**
|
||||
* Roll the log writer. That is, start writing log messages to a new file.
|
||||
*
|
||||
* <p>
|
||||
* The implementation is synchronized in order to make sure there's one rollWriter
|
||||
* running at any given time.
|
||||
*
|
||||
* @param force
|
||||
* If true, force creation of a new writer even if no entries have
|
||||
* been written to the current writer
|
||||
* @return If lots of logs, flush the returned regions so next time through we
|
||||
* can clean logs. Returns null if nothing to flush. Names are actual
|
||||
* region names as returned by {@link HRegionInfo#getEncodedName()}
|
||||
* @throws org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException
|
||||
* @throws IOException
|
||||
*/
|
||||
byte[][] rollWriter(boolean force) throws FailedLogCloseException, IOException;
|
||||
|
||||
/**
|
||||
* Shut down the log.
|
||||
*
|
||||
* @throws IOException
|
||||
*/
|
||||
void close() throws IOException;
|
||||
|
||||
/**
|
||||
* Shut down the log and delete the log directory.
|
||||
* Used by tests only and in rare cases where we need a log just temporarily while bootstrapping
|
||||
* a region or running migrations.
|
||||
*
|
||||
* @throws IOException
|
||||
*/
|
||||
void closeAndDelete() throws IOException;
|
||||
|
||||
/**
|
||||
* Same as {@link #appendNoSync(HRegionInfo, TableName, WALEdit, List, long, HTableDescriptor,
|
||||
* AtomicLong, boolean, long, long)}
|
||||
* except it causes a sync on the log
|
||||
* @param info
|
||||
* @param tableName
|
||||
* @param edits
|
||||
* @param now
|
||||
* @param htd
|
||||
* @param sequenceId
|
||||
* @throws IOException
|
||||
* @deprecated For tests only and even then, should use
|
||||
* {@link #appendNoSync(HTableDescriptor, HRegionInfo, HLogKey, WALEdit, AtomicLong, boolean,
|
||||
* List)} and {@link #sync()} instead.
|
||||
*/
|
||||
@Deprecated
|
||||
@VisibleForTesting
|
||||
public void append(HRegionInfo info, TableName tableName, WALEdit edits,
|
||||
final long now, HTableDescriptor htd, AtomicLong sequenceId) throws IOException;
|
||||
|
||||
/**
|
||||
* For notification post append to the writer. Used by metrics system at least.
|
||||
* @param entry
|
||||
* @param elapsedTime
|
||||
* @return Size of this append.
|
||||
*/
|
||||
long postAppend(final Entry entry, final long elapsedTime);
|
||||
|
||||
/**
|
||||
* For notification post writer sync. Used by metrics system at least.
|
||||
* @param timeInMillis How long the filesystem sync took in milliseconds.
|
||||
* @param handlerSyncs How many sync handler calls were released by this call to filesystem
|
||||
* sync.
|
||||
*/
|
||||
void postSync(final long timeInMillis, final int handlerSyncs);
|
||||
|
||||
/**
|
||||
* Append a set of edits to the WAL. WAL edits are keyed by (encoded) regionName, rowname, and
|
||||
* log-sequence-id. The WAL is not flushed/sync'd after this transaction completes BUT on return
|
||||
* this edit must have its region edit/sequence id assigned else it messes up our unification
|
||||
* of mvcc and sequenceid.
|
||||
* @param info
|
||||
* @param tableName
|
||||
* @param edits
|
||||
* @param clusterIds
|
||||
* @param now
|
||||
* @param htd
|
||||
* @param sequenceId A reference to the atomic long the <code>info</code> region is using as
|
||||
* source of its incrementing edits sequence id. Inside in this call we will increment it and
|
||||
* attach the sequence to the edit we apply the WAL.
|
||||
* @param isInMemstore Always true except for case where we are writing a compaction completion
|
||||
* record into the WAL; in this case the entry is just so we can finish an unfinished compaction
|
||||
* -- it is not an edit for memstore.
|
||||
* @param nonceGroup
|
||||
* @param nonce
|
||||
* @return Returns a 'transaction id'. Do not use. This is an internal implementation detail and
|
||||
* cannot be respected in all implementations; i.e. the append/sync machine may or may not be
|
||||
* able to sync an explicit edit only (the current default implementation syncs up to the time
|
||||
* of the sync call syncing whatever is behind the sync).
|
||||
* @throws IOException
|
||||
* @deprecated Use {@link #appendNoSync(HTableDescriptor, HRegionInfo, HLogKey, WALEdit, AtomicLong, boolean, List)}
|
||||
* instead because you can get back the region edit/sequenceid; it is set into the passed in
|
||||
* <code>key</code>.
|
||||
*/
|
||||
@Deprecated
|
||||
long appendNoSync(HRegionInfo info, TableName tableName, WALEdit edits,
|
||||
List<UUID> clusterIds, final long now, HTableDescriptor htd, AtomicLong sequenceId,
|
||||
boolean isInMemstore, long nonceGroup, long nonce) throws IOException;
|
||||
|
||||
/**
|
||||
* Append a set of edits to the WAL. The WAL is not flushed/sync'd after this transaction
|
||||
* completes BUT on return this edit must have its region edit/sequence id assigned
|
||||
* else it messes up our unification of mvcc and sequenceid. On return <code>key</code> will
|
||||
* have the region edit/sequence id filled in.
|
||||
* @param info
|
||||
* @param key Modified by this call; we add to it this edits region edit/sequence id.
|
||||
* @param edits Edits to append. MAY CONTAIN NO EDITS for case where we want to get an edit
|
||||
* sequence id that is after all currently appended edits.
|
||||
* @param htd
|
||||
* @param sequenceId A reference to the atomic long the <code>info</code> region is using as
|
||||
* source of its incrementing edits sequence id. Inside in this call we will increment it and
|
||||
* attach the sequence to the edit we apply the WAL.
|
||||
* @param inMemstore Always true except for case where we are writing a compaction completion
|
||||
* record into the WAL; in this case the entry is just so we can finish an unfinished compaction
|
||||
* -- it is not an edit for memstore.
|
||||
* @param memstoreCells list of Cells added into memstore
|
||||
* @return Returns a 'transaction id' and <code>key</code> will have the region edit/sequence id
|
||||
* in it.
|
||||
* @throws IOException
|
||||
*/
|
||||
long appendNoSync(HTableDescriptor htd, HRegionInfo info, HLogKey key, WALEdit edits,
|
||||
AtomicLong sequenceId, boolean inMemstore, List<Cell> memstoreCells) throws IOException;
|
||||
|
||||
// TODO: Do we need all these versions of sync?
|
||||
void hsync() throws IOException;
|
||||
|
||||
void hflush() throws IOException;
|
||||
|
||||
/**
|
||||
* Sync what we have in the WAL.
|
||||
* @throws IOException
|
||||
*/
|
||||
void sync() throws IOException;
|
||||
|
||||
/**
|
||||
* Sync the WAL if the txId was not already sync'd.
|
||||
* @param txid Transaction id to sync to.
|
||||
* @throws IOException
|
||||
*/
|
||||
void sync(long txid) throws IOException;
|
||||
|
||||
/**
|
||||
* WAL keeps track of the sequence numbers that were not yet flushed from memstores
|
||||
* in order to be able to do cleanup. This method tells WAL that some region is about
|
||||
* to flush memstore.
|
||||
*
|
||||
* <p>We stash the oldest seqNum for the region, and let the the next edit inserted in this
|
||||
* region be recorded in {@link #append(HRegionInfo, TableName, WALEdit, long, HTableDescriptor,
|
||||
* AtomicLong)} as new oldest seqnum.
|
||||
* In case of flush being aborted, we put the stashed value back; in case of flush succeeding,
|
||||
* the seqNum of that first edit after start becomes the valid oldest seqNum for this region.
|
||||
*
|
||||
* @return true if the flush can proceed, false in case wal is closing (ususally, when server is
|
||||
* closing) and flush couldn't be started.
|
||||
*/
|
||||
boolean startCacheFlush(final byte[] encodedRegionName);
|
||||
|
||||
/**
|
||||
* Complete the cache flush.
|
||||
* @param encodedRegionName Encoded region name.
|
||||
*/
|
||||
void completeCacheFlush(final byte[] encodedRegionName);
|
||||
|
||||
/**
|
||||
* Abort a cache flush. Call if the flush fails. Note that the only recovery
|
||||
* for an aborted flush currently is a restart of the regionserver so the
|
||||
* snapshot content dropped by the failure gets restored to the memstore.v
|
||||
* @param encodedRegionName Encoded region name.
|
||||
*/
|
||||
void abortCacheFlush(byte[] encodedRegionName);
|
||||
|
||||
/**
|
||||
* @return Coprocessor host.
|
||||
*/
|
||||
WALCoprocessorHost getCoprocessorHost();
|
||||
|
||||
/**
|
||||
* Get LowReplication-Roller status
|
||||
*
|
||||
* @return lowReplicationRollEnabled
|
||||
*/
|
||||
// TODO: This is implementation detail?
|
||||
boolean isLowReplicationRollEnabled();
|
||||
|
||||
/** Gets the earliest sequence number in the memstore for this particular region.
|
||||
* This can serve as best-effort "recent" WAL number for this region.
|
||||
* @param encodedRegionName The region to get the number for.
|
||||
* @return The number if present, HConstants.NO_SEQNUM if absent.
|
||||
*/
|
||||
long getEarliestMemstoreSeqNum(byte[] encodedRegionName);
|
||||
}
|
|
@ -1,207 +0,0 @@
|
|||
/**
|
||||
*
|
||||
* 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;
|
||||
import java.util.Arrays;
|
||||
import java.io.InterruptedIOException;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLog.Reader;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLog.Writer;
|
||||
import org.apache.hadoop.hbase.util.CancelableProgressable;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public class HLogFactory {
|
||||
private static final Log LOG = LogFactory.getLog(HLogFactory.class);
|
||||
|
||||
public static HLog createHLog(final FileSystem fs, final Path root, final String logName,
|
||||
final Configuration conf) throws IOException {
|
||||
return new FSHLog(fs, root, logName, conf);
|
||||
}
|
||||
|
||||
public static HLog createHLog(final FileSystem fs, final Path root, final String logName,
|
||||
final String oldLogName, final Configuration conf) throws IOException {
|
||||
return new FSHLog(fs, root, logName, oldLogName, conf, null, true, null, false);
|
||||
}
|
||||
|
||||
public static HLog createHLog(final FileSystem fs, final Path root, final String logName,
|
||||
final Configuration conf, final List<WALActionsListener> listeners,
|
||||
final String prefix) throws IOException {
|
||||
return new FSHLog(fs, root, logName, HConstants.HREGION_OLDLOGDIR_NAME, conf, listeners,
|
||||
true, prefix, false);
|
||||
}
|
||||
|
||||
public static HLog createMetaHLog(final FileSystem fs, final Path root, final String logName,
|
||||
final Configuration conf, final List<WALActionsListener> listeners,
|
||||
final String prefix) throws IOException {
|
||||
return new FSHLog(fs, root, logName, HConstants.HREGION_OLDLOGDIR_NAME, conf, listeners,
|
||||
false, prefix, true);
|
||||
}
|
||||
|
||||
/*
|
||||
* WAL Reader
|
||||
*/
|
||||
private static Class<? extends Reader> logReaderClass;
|
||||
|
||||
static void resetLogReaderClass() {
|
||||
logReaderClass = null;
|
||||
}
|
||||
|
||||
public static HLog.Reader createReader(final FileSystem fs,
|
||||
final Path path, Configuration conf) throws IOException {
|
||||
return createReader(fs, path, conf, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a reader for the WAL. If you are reading from a file that's being written to
|
||||
* and need to reopen it multiple times, use {@link HLog.Reader#reset()} instead of this method
|
||||
* then just seek back to the last known good position.
|
||||
* @return A WAL reader. Close when done with it.
|
||||
* @throws IOException
|
||||
*/
|
||||
public static HLog.Reader createReader(final FileSystem fs, final Path path,
|
||||
Configuration conf, CancelableProgressable reporter) throws IOException {
|
||||
return createReader(fs, path, conf, reporter, true);
|
||||
}
|
||||
|
||||
public static HLog.Reader createReader(final FileSystem fs, final Path path,
|
||||
Configuration conf, CancelableProgressable reporter, boolean allowCustom)
|
||||
throws IOException {
|
||||
if (allowCustom && (logReaderClass == null)) {
|
||||
logReaderClass = conf.getClass("hbase.regionserver.hlog.reader.impl",
|
||||
ProtobufLogReader.class, Reader.class);
|
||||
}
|
||||
Class<? extends Reader> lrClass = allowCustom ? logReaderClass : ProtobufLogReader.class;
|
||||
|
||||
try {
|
||||
// A hlog file could be under recovery, so it may take several
|
||||
// tries to get it open. Instead of claiming it is corrupted, retry
|
||||
// to open it up to 5 minutes by default.
|
||||
long startWaiting = EnvironmentEdgeManager.currentTime();
|
||||
long openTimeout = conf.getInt("hbase.hlog.open.timeout", 300000) + startWaiting;
|
||||
int nbAttempt = 0;
|
||||
while (true) {
|
||||
try {
|
||||
if (lrClass != ProtobufLogReader.class) {
|
||||
// User is overriding the WAL reader, let them.
|
||||
HLog.Reader reader = lrClass.newInstance();
|
||||
reader.init(fs, path, conf, null);
|
||||
return reader;
|
||||
} else {
|
||||
FSDataInputStream stream = fs.open(path);
|
||||
// Note that zero-length file will fail to read PB magic, and attempt to create
|
||||
// a non-PB reader and fail the same way existing code expects it to. If we get
|
||||
// rid of the old reader entirely, we need to handle 0-size files differently from
|
||||
// merely non-PB files.
|
||||
byte[] magic = new byte[ProtobufLogReader.PB_WAL_MAGIC.length];
|
||||
boolean isPbWal = (stream.read(magic) == magic.length)
|
||||
&& Arrays.equals(magic, ProtobufLogReader.PB_WAL_MAGIC);
|
||||
HLog.Reader reader =
|
||||
isPbWal ? new ProtobufLogReader() : new SequenceFileLogReader();
|
||||
reader.init(fs, path, conf, stream);
|
||||
return reader;
|
||||
}
|
||||
} catch (IOException e) {
|
||||
String msg = e.getMessage();
|
||||
if (msg != null && (msg.contains("Cannot obtain block length")
|
||||
|| msg.contains("Could not obtain the last block")
|
||||
|| msg.matches("Blocklist for [^ ]* has changed.*"))) {
|
||||
if (++nbAttempt == 1) {
|
||||
LOG.warn("Lease should have recovered. This is not expected. Will retry", e);
|
||||
}
|
||||
if (reporter != null && !reporter.progress()) {
|
||||
throw new InterruptedIOException("Operation is cancelled");
|
||||
}
|
||||
if (nbAttempt > 2 && openTimeout < EnvironmentEdgeManager.currentTime()) {
|
||||
LOG.error("Can't open after " + nbAttempt + " attempts and "
|
||||
+ (EnvironmentEdgeManager.currentTime() - startWaiting)
|
||||
+ "ms " + " for " + path);
|
||||
} else {
|
||||
try {
|
||||
Thread.sleep(nbAttempt < 3 ? 500 : 1000);
|
||||
continue; // retry
|
||||
} catch (InterruptedException ie) {
|
||||
InterruptedIOException iioe = new InterruptedIOException();
|
||||
iioe.initCause(ie);
|
||||
throw iioe;
|
||||
}
|
||||
}
|
||||
}
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
} catch (IOException ie) {
|
||||
throw ie;
|
||||
} catch (Exception e) {
|
||||
throw new IOException("Cannot get log reader", e);
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
* WAL writer
|
||||
*/
|
||||
private static Class<? extends Writer> logWriterClass;
|
||||
|
||||
static void resetLogWriterClass() {
|
||||
logWriterClass = null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a writer for the WAL.
|
||||
* @return A WAL writer. Close when done with it.
|
||||
* @throws IOException
|
||||
*/
|
||||
public static HLog.Writer createWALWriter(final FileSystem fs,
|
||||
final Path path, Configuration conf) throws IOException {
|
||||
return createWriter(fs, path, conf, false);
|
||||
}
|
||||
|
||||
public static HLog.Writer createRecoveredEditsWriter(final FileSystem fs,
|
||||
final Path path, Configuration conf) throws IOException {
|
||||
return createWriter(fs, path, conf, true);
|
||||
}
|
||||
|
||||
private static HLog.Writer createWriter(final FileSystem fs,
|
||||
final Path path, Configuration conf, boolean overwritable)
|
||||
throws IOException {
|
||||
try {
|
||||
if (logWriterClass == null) {
|
||||
logWriterClass = conf.getClass("hbase.regionserver.hlog.writer.impl",
|
||||
ProtobufLogWriter.class, Writer.class);
|
||||
}
|
||||
HLog.Writer writer = (HLog.Writer)logWriterClass.newInstance();
|
||||
writer.init(fs, path, conf, overwritable);
|
||||
return writer;
|
||||
} catch (Exception e) {
|
||||
throw new IOException("cannot get log writer", e);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -22,39 +22,22 @@ import java.io.DataInput;
|
|||
import java.io.DataOutput;
|
||||
import java.io.EOFException;
|
||||
import java.io.IOException;
|
||||
import java.io.InterruptedIOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.NavigableMap;
|
||||
import java.util.TreeMap;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
|
||||
|
||||
import org.apache.hadoop.hbase.util.ByteStringer;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FamilyScope;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.ScopeType;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALKey;
|
||||
import org.apache.hadoop.hbase.regionserver.SequenceId;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.io.WritableComparable;
|
||||
import org.apache.hadoop.hbase.wal.WALKey;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
import org.apache.hadoop.io.WritableUtils;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.protobuf.ByteString;
|
||||
|
||||
/**
|
||||
* A Key for an entry in the change log.
|
||||
|
@ -65,97 +48,29 @@ import com.google.protobuf.ByteString;
|
|||
*
|
||||
* <p>Some Transactional edits (START, COMMIT, ABORT) will not have an
|
||||
* associated row.
|
||||
* @deprecated use WALKey
|
||||
*/
|
||||
// TODO: Key and WALEdit are never used separately, or in one-to-many relation, for practical
|
||||
// purposes. They need to be merged into HLogEntry.
|
||||
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.REPLICATION)
|
||||
public class HLogKey implements WritableComparable<HLogKey>, SequenceId {
|
||||
@Deprecated
|
||||
public class HLogKey extends WALKey implements Writable {
|
||||
public static final Log LOG = LogFactory.getLog(HLogKey.class);
|
||||
|
||||
// should be < 0 (@see #readFields(DataInput))
|
||||
// version 2 supports HLog compression
|
||||
enum Version {
|
||||
UNVERSIONED(0),
|
||||
// Initial number we put on HLogKey when we introduced versioning.
|
||||
INITIAL(-1),
|
||||
// Version -2 introduced a dictionary compression facility. Only this
|
||||
// dictionary-based compression is available in version -2.
|
||||
COMPRESSED(-2);
|
||||
|
||||
final int code;
|
||||
static final Version[] byCode;
|
||||
static {
|
||||
byCode = Version.values();
|
||||
for (int i = 0; i < byCode.length; i++) {
|
||||
if (byCode[i].code != -1 * i) {
|
||||
throw new AssertionError("Values in this enum should be descending by one");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Version(int code) {
|
||||
this.code = code;
|
||||
}
|
||||
|
||||
boolean atLeast(Version other) {
|
||||
return code <= other.code;
|
||||
}
|
||||
|
||||
static Version fromCode(int code) {
|
||||
return byCode[code * -1];
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
* This is used for reading the log entries created by the previous releases
|
||||
* (0.94.11) which write the clusters information to the scopes of WALEdit.
|
||||
*/
|
||||
private static final String PREFIX_CLUSTER_KEY = ".";
|
||||
|
||||
|
||||
private static final Version VERSION = Version.COMPRESSED;
|
||||
|
||||
// The encoded region name.
|
||||
private byte [] encodedRegionName;
|
||||
private TableName tablename;
|
||||
private long logSeqNum;
|
||||
private long origLogSeqNum = 0;
|
||||
private CountDownLatch seqNumAssignedLatch = new CountDownLatch(1);
|
||||
// Time at which this edit was written.
|
||||
private long writeTime;
|
||||
|
||||
// The first element in the list is the cluster id on which the change has originated
|
||||
private List<UUID> clusterIds;
|
||||
|
||||
private NavigableMap<byte[], Integer> scopes;
|
||||
|
||||
private long nonceGroup = HConstants.NO_NONCE;
|
||||
private long nonce = HConstants.NO_NONCE;
|
||||
static final List<UUID> EMPTY_UUIDS = Collections.unmodifiableList(new ArrayList<UUID>());
|
||||
|
||||
private CompressionContext compressionContext;
|
||||
|
||||
public HLogKey() {
|
||||
init(null, null, 0L, HConstants.LATEST_TIMESTAMP,
|
||||
new ArrayList<UUID>(), HConstants.NO_NONCE, HConstants.NO_NONCE);
|
||||
super();
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public HLogKey(final byte[] encodedRegionName, final TableName tablename, long logSeqNum,
|
||||
final long now, UUID clusterId) {
|
||||
List<UUID> clusterIds = new ArrayList<UUID>();
|
||||
clusterIds.add(clusterId);
|
||||
init(encodedRegionName, tablename, logSeqNum, now, clusterIds,
|
||||
HConstants.NO_NONCE, HConstants.NO_NONCE);
|
||||
super(encodedRegionName, tablename, logSeqNum, now, clusterId);
|
||||
}
|
||||
|
||||
public HLogKey(final byte[] encodedRegionName, final TableName tablename) {
|
||||
this(encodedRegionName, tablename, System.currentTimeMillis());
|
||||
super(encodedRegionName, tablename);
|
||||
}
|
||||
|
||||
public HLogKey(final byte[] encodedRegionName, final TableName tablename, final long now) {
|
||||
init(encodedRegionName, tablename, HLog.NO_SEQUENCE_ID, now,
|
||||
EMPTY_UUIDS, HConstants.NO_NONCE, HConstants.NO_NONCE);
|
||||
super(encodedRegionName, tablename, now);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -173,7 +88,7 @@ public class HLogKey implements WritableComparable<HLogKey>, SequenceId {
|
|||
*/
|
||||
public HLogKey(final byte [] encodedRegionName, final TableName tablename,
|
||||
long logSeqNum, final long now, List<UUID> clusterIds, long nonceGroup, long nonce) {
|
||||
init(encodedRegionName, tablename, logSeqNum, now, clusterIds, nonceGroup, nonce);
|
||||
super(encodedRegionName, tablename, logSeqNum, now, clusterIds, nonceGroup, nonce);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -191,8 +106,7 @@ public class HLogKey implements WritableComparable<HLogKey>, SequenceId {
|
|||
*/
|
||||
public HLogKey(final byte [] encodedRegionName, final TableName tablename,
|
||||
final long now, List<UUID> clusterIds, long nonceGroup, long nonce) {
|
||||
init(encodedRegionName, tablename, HLog.NO_SEQUENCE_ID, now, clusterIds,
|
||||
nonceGroup, nonce);
|
||||
super(encodedRegionName, tablename, now, clusterIds, nonceGroup, nonce);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -209,240 +123,7 @@ public class HLogKey implements WritableComparable<HLogKey>, SequenceId {
|
|||
*/
|
||||
public HLogKey(final byte [] encodedRegionName, final TableName tablename, long logSeqNum,
|
||||
long nonceGroup, long nonce) {
|
||||
init(encodedRegionName, tablename, logSeqNum, EnvironmentEdgeManager.currentTime(),
|
||||
EMPTY_UUIDS, nonceGroup, nonce);
|
||||
}
|
||||
|
||||
protected void init(final byte [] encodedRegionName, final TableName tablename,
|
||||
long logSeqNum, final long now, List<UUID> clusterIds, long nonceGroup, long nonce) {
|
||||
this.logSeqNum = logSeqNum;
|
||||
this.writeTime = now;
|
||||
this.clusterIds = clusterIds;
|
||||
this.encodedRegionName = encodedRegionName;
|
||||
this.tablename = tablename;
|
||||
this.nonceGroup = nonceGroup;
|
||||
this.nonce = nonce;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param compressionContext Compression context to use
|
||||
*/
|
||||
public void setCompressionContext(CompressionContext compressionContext) {
|
||||
this.compressionContext = compressionContext;
|
||||
}
|
||||
|
||||
/** @return encoded region name */
|
||||
public byte [] getEncodedRegionName() {
|
||||
return encodedRegionName;
|
||||
}
|
||||
|
||||
/** @return table name */
|
||||
public TableName getTablename() {
|
||||
return tablename;
|
||||
}
|
||||
|
||||
/** @return log sequence number */
|
||||
public long getLogSeqNum() {
|
||||
return this.logSeqNum;
|
||||
}
|
||||
|
||||
/**
|
||||
* Allow that the log sequence id to be set post-construction and release all waiters on assigned
|
||||
* sequence number.
|
||||
* @param sequence
|
||||
*/
|
||||
void setLogSeqNum(final long sequence) {
|
||||
this.logSeqNum = sequence;
|
||||
this.seqNumAssignedLatch.countDown();
|
||||
}
|
||||
|
||||
/**
|
||||
* Used to set original seq Id for HLogKey during wal replay
|
||||
* @param seqId
|
||||
*/
|
||||
public void setOrigLogSeqNum(final long seqId) {
|
||||
this.origLogSeqNum = seqId;
|
||||
}
|
||||
|
||||
/**
|
||||
* Return a positive long if current HLogKey is created from a replay edit
|
||||
* @return original sequence number of the WALEdit
|
||||
*/
|
||||
public long getOrigLogSeqNum() {
|
||||
return this.origLogSeqNum;
|
||||
}
|
||||
|
||||
/**
|
||||
* Wait for sequence number is assigned & return the assigned value
|
||||
* @return long the new assigned sequence number
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
@Override
|
||||
public long getSequenceId() throws IOException {
|
||||
try {
|
||||
this.seqNumAssignedLatch.await();
|
||||
} catch (InterruptedException ie) {
|
||||
LOG.warn("Thread interrupted waiting for next log sequence number");
|
||||
InterruptedIOException iie = new InterruptedIOException();
|
||||
iie.initCause(ie);
|
||||
throw iie;
|
||||
}
|
||||
return this.logSeqNum;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the write time
|
||||
*/
|
||||
public long getWriteTime() {
|
||||
return this.writeTime;
|
||||
}
|
||||
|
||||
public NavigableMap<byte[], Integer> getScopes() {
|
||||
return scopes;
|
||||
}
|
||||
|
||||
/** @return The nonce group */
|
||||
public long getNonceGroup() {
|
||||
return nonceGroup;
|
||||
}
|
||||
|
||||
/** @return The nonce */
|
||||
public long getNonce() {
|
||||
return nonce;
|
||||
}
|
||||
|
||||
public void setScopes(NavigableMap<byte[], Integer> scopes) {
|
||||
this.scopes = scopes;
|
||||
}
|
||||
|
||||
public void readOlderScopes(NavigableMap<byte[], Integer> scopes) {
|
||||
if (scopes != null) {
|
||||
Iterator<Map.Entry<byte[], Integer>> iterator = scopes.entrySet()
|
||||
.iterator();
|
||||
while (iterator.hasNext()) {
|
||||
Map.Entry<byte[], Integer> scope = iterator.next();
|
||||
String key = Bytes.toString(scope.getKey());
|
||||
if (key.startsWith(PREFIX_CLUSTER_KEY)) {
|
||||
addClusterId(UUID.fromString(key.substring(PREFIX_CLUSTER_KEY
|
||||
.length())));
|
||||
iterator.remove();
|
||||
}
|
||||
}
|
||||
if (scopes.size() > 0) {
|
||||
this.scopes = scopes;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Marks that the cluster with the given clusterId has consumed the change
|
||||
*/
|
||||
public void addClusterId(UUID clusterId) {
|
||||
if (!clusterIds.contains(clusterId)) {
|
||||
clusterIds.add(clusterId);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the set of cluster Ids that have consumed the change
|
||||
*/
|
||||
public List<UUID> getClusterIds() {
|
||||
return clusterIds;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the cluster id on which the change has originated. It there is no such cluster, it
|
||||
* returns DEFAULT_CLUSTER_ID (cases where replication is not enabled)
|
||||
*/
|
||||
public UUID getOriginatingClusterId(){
|
||||
return clusterIds.isEmpty() ? HConstants.DEFAULT_CLUSTER_ID : clusterIds.get(0);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return tablename + "/" + Bytes.toString(encodedRegionName) + "/" +
|
||||
logSeqNum;
|
||||
}
|
||||
|
||||
/**
|
||||
* Produces a string map for this key. Useful for programmatic use and
|
||||
* manipulation of the data stored in an HLogKey, for example, printing
|
||||
* as JSON.
|
||||
*
|
||||
* @return a Map containing data from this key
|
||||
*/
|
||||
public Map<String, Object> toStringMap() {
|
||||
Map<String, Object> stringMap = new HashMap<String, Object>();
|
||||
stringMap.put("table", tablename);
|
||||
stringMap.put("region", Bytes.toStringBinary(encodedRegionName));
|
||||
stringMap.put("sequence", logSeqNum);
|
||||
return stringMap;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (this == obj) {
|
||||
return true;
|
||||
}
|
||||
if (obj == null || getClass() != obj.getClass()) {
|
||||
return false;
|
||||
}
|
||||
return compareTo((HLogKey)obj) == 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
int result = Bytes.hashCode(this.encodedRegionName);
|
||||
result ^= this.logSeqNum;
|
||||
result ^= this.writeTime;
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(HLogKey o) {
|
||||
int result = Bytes.compareTo(this.encodedRegionName, o.encodedRegionName);
|
||||
if (result == 0) {
|
||||
if (this.logSeqNum < o.logSeqNum) {
|
||||
result = -1;
|
||||
} else if (this.logSeqNum > o.logSeqNum) {
|
||||
result = 1;
|
||||
}
|
||||
if (result == 0) {
|
||||
if (this.writeTime < o.writeTime) {
|
||||
result = -1;
|
||||
} else if (this.writeTime > o.writeTime) {
|
||||
return 1;
|
||||
}
|
||||
}
|
||||
}
|
||||
// why isn't cluster id accounted for?
|
||||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* Drop this instance's tablename byte array and instead
|
||||
* hold a reference to the provided tablename. This is not
|
||||
* meant to be a general purpose setter - it's only used
|
||||
* to collapse references to conserve memory.
|
||||
*/
|
||||
void internTableName(TableName tablename) {
|
||||
// We should not use this as a setter - only to swap
|
||||
// in a new reference to the same table name.
|
||||
assert tablename.equals(this.tablename);
|
||||
this.tablename = tablename;
|
||||
}
|
||||
|
||||
/**
|
||||
* Drop this instance's region name byte array and instead
|
||||
* hold a reference to the provided region name. This is not
|
||||
* meant to be a general purpose setter - it's only used
|
||||
* to collapse references to conserve memory.
|
||||
*/
|
||||
void internEncodedRegionName(byte []encodedRegionName) {
|
||||
// We should not use this as a setter - only to swap
|
||||
// in a new reference to the same table name.
|
||||
assert Bytes.equals(this.encodedRegionName, encodedRegionName);
|
||||
this.encodedRegionName = encodedRegionName;
|
||||
super(encodedRegionName, tablename, logSeqNum, nonceGroup, nonce);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -486,13 +167,13 @@ public class HLogKey implements WritableComparable<HLogKey>, SequenceId {
|
|||
// encodes the length of encodedRegionName.
|
||||
// If < 0 we just read the version and the next vint is the length.
|
||||
// @see Bytes#readByteArray(DataInput)
|
||||
this.scopes = null; // writable HLogKey does not contain scopes
|
||||
setScopes(null); // writable HLogKey does not contain scopes
|
||||
int len = WritableUtils.readVInt(in);
|
||||
byte[] tablenameBytes = null;
|
||||
if (len < 0) {
|
||||
// what we just read was the version
|
||||
version = Version.fromCode(len);
|
||||
// We only compress V2 of HLogkey.
|
||||
// We only compress V2 of WALkey.
|
||||
// If compression is on, the length is handled by the dictionary
|
||||
if (compressionContext == null || !version.atLeast(Version.COMPRESSED)) {
|
||||
len = WritableUtils.readVInt(in);
|
||||
|
@ -541,87 +222,4 @@ public class HLogKey implements WritableComparable<HLogKey>, SequenceId {
|
|||
// Do not need to read the clusters information as we are using protobufs from 0.95
|
||||
}
|
||||
|
||||
public WALKey.Builder getBuilder(WALCellCodec.ByteStringCompressor compressor)
|
||||
throws IOException {
|
||||
WALKey.Builder builder = WALKey.newBuilder();
|
||||
if (compressionContext == null) {
|
||||
builder.setEncodedRegionName(ByteStringer.wrap(this.encodedRegionName));
|
||||
builder.setTableName(ByteStringer.wrap(this.tablename.getName()));
|
||||
} else {
|
||||
builder.setEncodedRegionName(compressor.compress(this.encodedRegionName,
|
||||
compressionContext.regionDict));
|
||||
builder.setTableName(compressor.compress(this.tablename.getName(),
|
||||
compressionContext.tableDict));
|
||||
}
|
||||
builder.setLogSequenceNumber(this.logSeqNum);
|
||||
builder.setWriteTime(writeTime);
|
||||
if(this.origLogSeqNum > 0) {
|
||||
builder.setOrigSequenceNumber(this.origLogSeqNum);
|
||||
}
|
||||
if (this.nonce != HConstants.NO_NONCE) {
|
||||
builder.setNonce(nonce);
|
||||
}
|
||||
if (this.nonceGroup != HConstants.NO_NONCE) {
|
||||
builder.setNonceGroup(nonceGroup);
|
||||
}
|
||||
HBaseProtos.UUID.Builder uuidBuilder = HBaseProtos.UUID.newBuilder();
|
||||
for (UUID clusterId : clusterIds) {
|
||||
uuidBuilder.setLeastSigBits(clusterId.getLeastSignificantBits());
|
||||
uuidBuilder.setMostSigBits(clusterId.getMostSignificantBits());
|
||||
builder.addClusterIds(uuidBuilder.build());
|
||||
}
|
||||
if (scopes != null) {
|
||||
for (Map.Entry<byte[], Integer> e : scopes.entrySet()) {
|
||||
ByteString family = (compressionContext == null) ? ByteStringer.wrap(e.getKey())
|
||||
: compressor.compress(e.getKey(), compressionContext.familyDict);
|
||||
builder.addScopes(FamilyScope.newBuilder()
|
||||
.setFamily(family).setScopeType(ScopeType.valueOf(e.getValue())));
|
||||
}
|
||||
}
|
||||
return builder;
|
||||
}
|
||||
|
||||
public void readFieldsFromPb(
|
||||
WALKey walKey, WALCellCodec.ByteStringUncompressor uncompressor) throws IOException {
|
||||
if (this.compressionContext != null) {
|
||||
this.encodedRegionName = uncompressor.uncompress(
|
||||
walKey.getEncodedRegionName(), compressionContext.regionDict);
|
||||
byte[] tablenameBytes = uncompressor.uncompress(
|
||||
walKey.getTableName(), compressionContext.tableDict);
|
||||
this.tablename = TableName.valueOf(tablenameBytes);
|
||||
} else {
|
||||
this.encodedRegionName = walKey.getEncodedRegionName().toByteArray();
|
||||
this.tablename = TableName.valueOf(walKey.getTableName().toByteArray());
|
||||
}
|
||||
clusterIds.clear();
|
||||
if (walKey.hasClusterId()) {
|
||||
//When we are reading the older log (0.95.1 release)
|
||||
//This is definitely the originating cluster
|
||||
clusterIds.add(new UUID(walKey.getClusterId().getMostSigBits(), walKey.getClusterId()
|
||||
.getLeastSigBits()));
|
||||
}
|
||||
for (HBaseProtos.UUID clusterId : walKey.getClusterIdsList()) {
|
||||
clusterIds.add(new UUID(clusterId.getMostSigBits(), clusterId.getLeastSigBits()));
|
||||
}
|
||||
if (walKey.hasNonceGroup()) {
|
||||
this.nonceGroup = walKey.getNonceGroup();
|
||||
}
|
||||
if (walKey.hasNonce()) {
|
||||
this.nonce = walKey.getNonce();
|
||||
}
|
||||
this.scopes = null;
|
||||
if (walKey.getScopesCount() > 0) {
|
||||
this.scopes = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
|
||||
for (FamilyScope scope : walKey.getScopesList()) {
|
||||
byte[] family = (compressionContext == null) ? scope.getFamily().toByteArray() :
|
||||
uncompressor.uncompress(scope.getFamily(), compressionContext.familyDict);
|
||||
this.scopes.put(family, scope.getScopeType().getNumber());
|
||||
}
|
||||
}
|
||||
this.logSeqNum = walKey.getLogSequenceNumber();
|
||||
this.writeTime = walKey.getWriteTime();
|
||||
if(walKey.hasOrigSequenceNumber()) {
|
||||
this.origLogSeqNum = walKey.getOrigSequenceNumber();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,7 +23,6 @@ import java.io.PrintStream;
|
|||
import java.util.ArrayList;
|
||||
import java.util.Date;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
|
@ -38,13 +37,11 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.Tag;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLog.Reader;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.wal.WALPrettyPrinter;
|
||||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
|
||||
/**
|
||||
|
@ -59,36 +56,18 @@ import org.codehaus.jackson.map.ObjectMapper;
|
|||
*
|
||||
* It can also toggle output of values.
|
||||
*
|
||||
* @deprecated use the "hbase wal" command
|
||||
*/
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Evolving
|
||||
public class HLogPrettyPrinter {
|
||||
private boolean outputValues;
|
||||
private boolean outputJSON;
|
||||
// The following enable filtering by sequence, region, and row, respectively
|
||||
private long sequence;
|
||||
private String region;
|
||||
private String row;
|
||||
// enable in order to output a single list of transactions from several files
|
||||
private boolean persistentOutput;
|
||||
private boolean firstTxn;
|
||||
// useful for programatic capture of JSON output
|
||||
private PrintStream out;
|
||||
// for JSON encoding
|
||||
private static final ObjectMapper MAPPER = new ObjectMapper();
|
||||
@Deprecated
|
||||
public class HLogPrettyPrinter extends WALPrettyPrinter {
|
||||
|
||||
/**
|
||||
* Basic constructor that simply initializes values to reasonable defaults.
|
||||
*/
|
||||
public HLogPrettyPrinter() {
|
||||
outputValues = false;
|
||||
outputJSON = false;
|
||||
sequence = -1;
|
||||
region = null;
|
||||
row = null;
|
||||
persistentOutput = false;
|
||||
firstTxn = true;
|
||||
out = System.out;
|
||||
this(false, false, -1l, null, null, false, System.out);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -119,289 +98,11 @@ public class HLogPrettyPrinter {
|
|||
public HLogPrettyPrinter(boolean outputValues, boolean outputJSON,
|
||||
long sequence, String region, String row, boolean persistentOutput,
|
||||
PrintStream out) {
|
||||
this.outputValues = outputValues;
|
||||
this.outputJSON = outputJSON;
|
||||
this.sequence = sequence;
|
||||
this.region = region;
|
||||
this.row = row;
|
||||
this.persistentOutput = persistentOutput;
|
||||
if (persistentOutput) {
|
||||
beginPersistentOutput();
|
||||
}
|
||||
this.out = out;
|
||||
this.firstTxn = true;
|
||||
}
|
||||
|
||||
/**
|
||||
* turns value output on
|
||||
*/
|
||||
public void enableValues() {
|
||||
outputValues = true;
|
||||
}
|
||||
|
||||
/**
|
||||
* turns value output off
|
||||
*/
|
||||
public void disableValues() {
|
||||
outputValues = false;
|
||||
}
|
||||
|
||||
/**
|
||||
* turns JSON output on
|
||||
*/
|
||||
public void enableJSON() {
|
||||
outputJSON = true;
|
||||
}
|
||||
|
||||
/**
|
||||
* turns JSON output off, and turns on "pretty strings" for human consumption
|
||||
*/
|
||||
public void disableJSON() {
|
||||
outputJSON = false;
|
||||
}
|
||||
|
||||
/**
|
||||
* sets the region by which output will be filtered
|
||||
*
|
||||
* @param sequence
|
||||
* when nonnegative, serves as a filter; only log entries with this
|
||||
* sequence id will be printed
|
||||
*/
|
||||
public void setSequenceFilter(long sequence) {
|
||||
this.sequence = sequence;
|
||||
}
|
||||
|
||||
/**
|
||||
* sets the region by which output will be filtered
|
||||
*
|
||||
* @param region
|
||||
* when not null, serves as a filter; only log entries from this
|
||||
* region will be printed
|
||||
*/
|
||||
public void setRegionFilter(String region) {
|
||||
this.region = region;
|
||||
}
|
||||
|
||||
/**
|
||||
* sets the region by which output will be filtered
|
||||
*
|
||||
* @param row
|
||||
* when not null, serves as a filter; only log entries from this row
|
||||
* will be printed
|
||||
*/
|
||||
public void setRowFilter(String row) {
|
||||
this.row = row;
|
||||
}
|
||||
|
||||
/**
|
||||
* enables output as a single, persistent list. at present, only relevant in
|
||||
* the case of JSON output.
|
||||
*/
|
||||
public void beginPersistentOutput() {
|
||||
if (persistentOutput)
|
||||
return;
|
||||
persistentOutput = true;
|
||||
firstTxn = true;
|
||||
if (outputJSON)
|
||||
out.print("[");
|
||||
}
|
||||
|
||||
/**
|
||||
* ends output of a single, persistent list. at present, only relevant in the
|
||||
* case of JSON output.
|
||||
*/
|
||||
public void endPersistentOutput() {
|
||||
if (!persistentOutput)
|
||||
return;
|
||||
persistentOutput = false;
|
||||
if (outputJSON)
|
||||
out.print("]");
|
||||
}
|
||||
|
||||
/**
|
||||
* reads a log file and outputs its contents, one transaction at a time, as
|
||||
* specified by the currently configured options
|
||||
*
|
||||
* @param conf
|
||||
* the HBase configuration relevant to this log file
|
||||
* @param p
|
||||
* the path of the log file to be read
|
||||
* @throws IOException
|
||||
* may be unable to access the configured filesystem or requested
|
||||
* file.
|
||||
*/
|
||||
public void processFile(final Configuration conf, final Path p)
|
||||
throws IOException {
|
||||
FileSystem fs = FileSystem.get(conf);
|
||||
if (!fs.exists(p)) {
|
||||
throw new FileNotFoundException(p.toString());
|
||||
}
|
||||
if (!fs.isFile(p)) {
|
||||
throw new IOException(p + " is not a file");
|
||||
}
|
||||
if (outputJSON && !persistentOutput) {
|
||||
out.print("[");
|
||||
firstTxn = true;
|
||||
}
|
||||
Reader log = HLogFactory.createReader(fs, p, conf);
|
||||
try {
|
||||
FSHLog.Entry entry;
|
||||
while ((entry = log.next()) != null) {
|
||||
HLogKey key = entry.getKey();
|
||||
WALEdit edit = entry.getEdit();
|
||||
// begin building a transaction structure
|
||||
Map<String, Object> txn = key.toStringMap();
|
||||
long writeTime = key.getWriteTime();
|
||||
// check output filters
|
||||
if (sequence >= 0 && ((Long) txn.get("sequence")) != sequence)
|
||||
continue;
|
||||
if (region != null && !((String) txn.get("region")).equals(region))
|
||||
continue;
|
||||
// initialize list into which we will store atomic actions
|
||||
List<Map> actions = new ArrayList<Map>();
|
||||
for (Cell cell : edit.getCells()) {
|
||||
// add atomic operation to txn
|
||||
Map<String, Object> op = new HashMap<String, Object>(toStringMap(cell));
|
||||
if (outputValues) op.put("value", Bytes.toStringBinary(cell.getValue()));
|
||||
// check row output filter
|
||||
if (row == null || ((String) op.get("row")).equals(row))
|
||||
actions.add(op);
|
||||
}
|
||||
if (actions.size() == 0)
|
||||
continue;
|
||||
txn.put("actions", actions);
|
||||
if (outputJSON) {
|
||||
// JSON output is a straightforward "toString" on the txn object
|
||||
if (firstTxn)
|
||||
firstTxn = false;
|
||||
else
|
||||
out.print(",");
|
||||
// encode and print JSON
|
||||
out.print(MAPPER.writeValueAsString(txn));
|
||||
} else {
|
||||
// Pretty output, complete with indentation by atomic action
|
||||
out.println("Sequence " + txn.get("sequence") + " "
|
||||
+ "from region " + txn.get("region") + " " + "in table "
|
||||
+ txn.get("table") + " at write timestamp: " + new Date(writeTime));
|
||||
for (int i = 0; i < actions.size(); i++) {
|
||||
Map op = actions.get(i);
|
||||
out.println(" Action:");
|
||||
out.println(" row: " + op.get("row"));
|
||||
out.println(" column: " + op.get("family") + ":"
|
||||
+ op.get("qualifier"));
|
||||
out.println(" timestamp: "
|
||||
+ (new Date((Long) op.get("timestamp"))));
|
||||
if(op.get("tag") != null) {
|
||||
out.println(" tag: " + op.get("tag"));
|
||||
}
|
||||
if (outputValues)
|
||||
out.println(" value: " + op.get("value"));
|
||||
}
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
log.close();
|
||||
}
|
||||
if (outputJSON && !persistentOutput) {
|
||||
out.print("]");
|
||||
}
|
||||
}
|
||||
|
||||
private static Map<String, Object> toStringMap(Cell cell) {
|
||||
Map<String, Object> stringMap = new HashMap<String, Object>();
|
||||
stringMap.put("row",
|
||||
Bytes.toStringBinary(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()));
|
||||
stringMap.put("family", Bytes.toStringBinary(cell.getFamilyArray(), cell.getFamilyOffset(),
|
||||
cell.getFamilyLength()));
|
||||
stringMap.put("qualifier",
|
||||
Bytes.toStringBinary(cell.getQualifierArray(), cell.getQualifierOffset(),
|
||||
cell.getQualifierLength()));
|
||||
stringMap.put("timestamp", cell.getTimestamp());
|
||||
stringMap.put("vlen", cell.getValueLength());
|
||||
if (cell.getTagsLength() > 0) {
|
||||
List<String> tagsString = new ArrayList<String>();
|
||||
Iterator<Tag> tagsIterator = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(),
|
||||
cell.getTagsLength());
|
||||
while (tagsIterator.hasNext()) {
|
||||
Tag tag = tagsIterator.next();
|
||||
tagsString.add((tag.getType()) + ":"
|
||||
+ Bytes.toStringBinary(tag.getBuffer(), tag.getTagOffset(), tag.getTagLength()));
|
||||
}
|
||||
stringMap.put("tag", tagsString);
|
||||
}
|
||||
return stringMap;
|
||||
super(outputValues, outputJSON, sequence, region, row, persistentOutput, out);
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws IOException {
|
||||
run(args);
|
||||
WALPrettyPrinter.main(args);
|
||||
}
|
||||
|
||||
/**
|
||||
* Pass one or more log file names and formatting options and it will dump out
|
||||
* a text version of the contents on <code>stdout</code>.
|
||||
*
|
||||
* @param args
|
||||
* Command line arguments
|
||||
* @throws IOException
|
||||
* Thrown upon file system errors etc.
|
||||
* @throws ParseException
|
||||
* Thrown if command-line parsing fails.
|
||||
*/
|
||||
public static void run(String[] args) throws IOException {
|
||||
// create options
|
||||
Options options = new Options();
|
||||
options.addOption("h", "help", false, "Output help message");
|
||||
options.addOption("j", "json", false, "Output JSON");
|
||||
options.addOption("p", "printvals", false, "Print values");
|
||||
options.addOption("r", "region", true,
|
||||
"Region to filter by. Pass region name; e.g. 'hbase:meta,,1'");
|
||||
options.addOption("s", "sequence", true,
|
||||
"Sequence to filter by. Pass sequence number.");
|
||||
options.addOption("w", "row", true, "Row to filter by. Pass row name.");
|
||||
|
||||
HLogPrettyPrinter printer = new HLogPrettyPrinter();
|
||||
CommandLineParser parser = new PosixParser();
|
||||
List files = null;
|
||||
try {
|
||||
CommandLine cmd = parser.parse(options, args);
|
||||
files = cmd.getArgList();
|
||||
if (files.size() == 0 || cmd.hasOption("h")) {
|
||||
HelpFormatter formatter = new HelpFormatter();
|
||||
formatter.printHelp("HLog <filename...>", options, true);
|
||||
System.exit(-1);
|
||||
}
|
||||
// configure the pretty printer using command line options
|
||||
if (cmd.hasOption("p"))
|
||||
printer.enableValues();
|
||||
if (cmd.hasOption("j"))
|
||||
printer.enableJSON();
|
||||
if (cmd.hasOption("r"))
|
||||
printer.setRegionFilter(cmd.getOptionValue("r"));
|
||||
if (cmd.hasOption("s"))
|
||||
printer.setSequenceFilter(Long.parseLong(cmd.getOptionValue("s")));
|
||||
if (cmd.hasOption("w"))
|
||||
printer.setRowFilter(cmd.getOptionValue("w"));
|
||||
} catch (ParseException e) {
|
||||
e.printStackTrace();
|
||||
HelpFormatter formatter = new HelpFormatter();
|
||||
formatter.printHelp("HFile filename(s) ", options, true);
|
||||
System.exit(-1);
|
||||
}
|
||||
// get configuration, file system, and process the given files
|
||||
Configuration conf = HBaseConfiguration.create();
|
||||
FSUtils.setFsDefault(conf, FSUtils.getRootDir(conf));
|
||||
|
||||
// begin output
|
||||
printer.beginPersistentOutput();
|
||||
for (Object f : files) {
|
||||
Path file = new Path((String) f);
|
||||
FileSystem fs = file.getFileSystem(conf);
|
||||
if (!fs.exists(file)) {
|
||||
System.err.println("ERROR, file doesnt exist: " + file);
|
||||
return;
|
||||
}
|
||||
printer.processFile(conf, file);
|
||||
}
|
||||
printer.endPersistentOutput();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,367 +0,0 @@
|
|||
/**
|
||||
*
|
||||
* 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;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.NavigableSet;
|
||||
import java.util.TreeSet;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.regex.Matcher;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.PathFilter;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
|
||||
import com.google.protobuf.TextFormat;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public class HLogUtil {
|
||||
static final Log LOG = LogFactory.getLog(HLogUtil.class);
|
||||
|
||||
/**
|
||||
* Pattern used to validate a HLog file name
|
||||
*/
|
||||
private static final Pattern pattern =
|
||||
Pattern.compile(".*\\.\\d*("+HLog.META_HLOG_FILE_EXTN+")*");
|
||||
|
||||
/**
|
||||
* @param filename
|
||||
* name of the file to validate
|
||||
* @return <tt>true</tt> if the filename matches an HLog, <tt>false</tt>
|
||||
* otherwise
|
||||
*/
|
||||
public static boolean validateHLogFilename(String filename) {
|
||||
return pattern.matcher(filename).matches();
|
||||
}
|
||||
|
||||
/**
|
||||
* Construct the HLog directory name
|
||||
*
|
||||
* @param serverName
|
||||
* Server name formatted as described in {@link ServerName}
|
||||
* @return the relative HLog directory name, e.g.
|
||||
* <code>.logs/1.example.org,60030,12345</code> if
|
||||
* <code>serverName</code> passed is
|
||||
* <code>1.example.org,60030,12345</code>
|
||||
*/
|
||||
public static String getHLogDirectoryName(final String serverName) {
|
||||
StringBuilder dirName = new StringBuilder(HConstants.HREGION_LOGDIR_NAME);
|
||||
dirName.append("/");
|
||||
dirName.append(serverName);
|
||||
return dirName.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* @param regiondir
|
||||
* This regions directory in the filesystem.
|
||||
* @return The directory that holds recovered edits files for the region
|
||||
* <code>regiondir</code>
|
||||
*/
|
||||
public static Path getRegionDirRecoveredEditsDir(final Path regiondir) {
|
||||
return new Path(regiondir, HConstants.RECOVERED_EDITS_DIR);
|
||||
}
|
||||
|
||||
/**
|
||||
* Move aside a bad edits file.
|
||||
*
|
||||
* @param fs
|
||||
* @param edits
|
||||
* Edits file to move aside.
|
||||
* @return The name of the moved aside file.
|
||||
* @throws IOException
|
||||
*/
|
||||
public static Path moveAsideBadEditsFile(final FileSystem fs, final Path edits)
|
||||
throws IOException {
|
||||
Path moveAsideName = new Path(edits.getParent(), edits.getName() + "."
|
||||
+ System.currentTimeMillis());
|
||||
if (!fs.rename(edits, moveAsideName)) {
|
||||
LOG.warn("Rename failed from " + edits + " to " + moveAsideName);
|
||||
}
|
||||
return moveAsideName;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param path
|
||||
* - the path to analyze. Expected format, if it's in hlog directory:
|
||||
* / [base directory for hbase] / hbase / .logs / ServerName /
|
||||
* logfile
|
||||
* @return null if it's not a log file. Returns the ServerName of the region
|
||||
* server that created this log file otherwise.
|
||||
*/
|
||||
public static ServerName getServerNameFromHLogDirectoryName(
|
||||
Configuration conf, String path) throws IOException {
|
||||
if (path == null
|
||||
|| path.length() <= HConstants.HREGION_LOGDIR_NAME.length()) {
|
||||
return null;
|
||||
}
|
||||
|
||||
if (conf == null) {
|
||||
throw new IllegalArgumentException("parameter conf must be set");
|
||||
}
|
||||
|
||||
final String rootDir = conf.get(HConstants.HBASE_DIR);
|
||||
if (rootDir == null || rootDir.isEmpty()) {
|
||||
throw new IllegalArgumentException(HConstants.HBASE_DIR
|
||||
+ " key not found in conf.");
|
||||
}
|
||||
|
||||
final StringBuilder startPathSB = new StringBuilder(rootDir);
|
||||
if (!rootDir.endsWith("/"))
|
||||
startPathSB.append('/');
|
||||
startPathSB.append(HConstants.HREGION_LOGDIR_NAME);
|
||||
if (!HConstants.HREGION_LOGDIR_NAME.endsWith("/"))
|
||||
startPathSB.append('/');
|
||||
final String startPath = startPathSB.toString();
|
||||
|
||||
String fullPath;
|
||||
try {
|
||||
fullPath = FileSystem.get(conf).makeQualified(new Path(path)).toString();
|
||||
} catch (IllegalArgumentException e) {
|
||||
LOG.info("Call to makeQualified failed on " + path + " " + e.getMessage());
|
||||
return null;
|
||||
}
|
||||
|
||||
if (!fullPath.startsWith(startPath)) {
|
||||
return null;
|
||||
}
|
||||
|
||||
final String serverNameAndFile = fullPath.substring(startPath.length());
|
||||
|
||||
if (serverNameAndFile.indexOf('/') < "a,0,0".length()) {
|
||||
// Either it's a file (not a directory) or it's not a ServerName format
|
||||
return null;
|
||||
}
|
||||
|
||||
Path p = new Path(path);
|
||||
return getServerNameFromHLogDirectoryName(p);
|
||||
}
|
||||
|
||||
/**
|
||||
* This function returns region server name from a log file name which is in either format:
|
||||
* hdfs://<name node>/hbase/.logs/<server name>-splitting/... or hdfs://<name
|
||||
* node>/hbase/.logs/<server name>/...
|
||||
* @param logFile
|
||||
* @return null if the passed in logFile isn't a valid HLog file path
|
||||
*/
|
||||
public static ServerName getServerNameFromHLogDirectoryName(Path logFile) {
|
||||
Path logDir = logFile.getParent();
|
||||
String logDirName = logDir.getName();
|
||||
if (logDirName.equals(HConstants.HREGION_LOGDIR_NAME)) {
|
||||
logDir = logFile;
|
||||
logDirName = logDir.getName();
|
||||
}
|
||||
ServerName serverName = null;
|
||||
if (logDirName.endsWith(HLog.SPLITTING_EXT)) {
|
||||
logDirName = logDirName.substring(0, logDirName.length() - HLog.SPLITTING_EXT.length());
|
||||
}
|
||||
try {
|
||||
serverName = ServerName.parseServerName(logDirName);
|
||||
} catch (IllegalArgumentException ex) {
|
||||
serverName = null;
|
||||
LOG.warn("Cannot parse a server name from path=" + logFile + "; " + ex.getMessage());
|
||||
}
|
||||
if (serverName != null && serverName.getStartcode() < 0) {
|
||||
LOG.warn("Invalid log file path=" + logFile);
|
||||
return null;
|
||||
}
|
||||
return serverName;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns sorted set of edit files made by wal-log splitter, excluding files
|
||||
* with '.temp' suffix.
|
||||
*
|
||||
* @param fs
|
||||
* @param regiondir
|
||||
* @return Files in passed <code>regiondir</code> as a sorted set.
|
||||
* @throws IOException
|
||||
*/
|
||||
public static NavigableSet<Path> getSplitEditFilesSorted(final FileSystem fs,
|
||||
final Path regiondir) throws IOException {
|
||||
NavigableSet<Path> filesSorted = new TreeSet<Path>();
|
||||
Path editsdir = HLogUtil.getRegionDirRecoveredEditsDir(regiondir);
|
||||
if (!fs.exists(editsdir))
|
||||
return filesSorted;
|
||||
FileStatus[] files = FSUtils.listStatus(fs, editsdir, new PathFilter() {
|
||||
@Override
|
||||
public boolean accept(Path p) {
|
||||
boolean result = false;
|
||||
try {
|
||||
// Return files and only files that match the editfile names pattern.
|
||||
// There can be other files in this directory other than edit files.
|
||||
// In particular, on error, we'll move aside the bad edit file giving
|
||||
// it a timestamp suffix. See moveAsideBadEditsFile.
|
||||
Matcher m = HLog.EDITFILES_NAME_PATTERN.matcher(p.getName());
|
||||
result = fs.isFile(p) && m.matches();
|
||||
// Skip the file whose name ends with RECOVERED_LOG_TMPFILE_SUFFIX,
|
||||
// because it means splithlog thread is writting this file.
|
||||
if (p.getName().endsWith(HLog.RECOVERED_LOG_TMPFILE_SUFFIX)) {
|
||||
result = false;
|
||||
}
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Failed isFile check on " + p);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
});
|
||||
if (files == null)
|
||||
return filesSorted;
|
||||
for (FileStatus status : files) {
|
||||
filesSorted.add(status.getPath());
|
||||
}
|
||||
return filesSorted;
|
||||
}
|
||||
|
||||
public static boolean isMetaFile(Path p) {
|
||||
return isMetaFile(p.getName());
|
||||
}
|
||||
|
||||
public static boolean isMetaFile(String p) {
|
||||
if (p != null && p.endsWith(HLog.META_HLOG_FILE_EXTN)) {
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Write the marker that a compaction has succeeded and is about to be committed.
|
||||
* This provides info to the HMaster to allow it to recover the compaction if
|
||||
* this regionserver dies in the middle (This part is not yet implemented). It also prevents
|
||||
* the compaction from finishing if this regionserver has already lost its lease on the log.
|
||||
* @param sequenceId Used by HLog to get sequence Id for the waledit.
|
||||
*/
|
||||
public static void writeCompactionMarker(HLog log, HTableDescriptor htd, HRegionInfo info,
|
||||
final CompactionDescriptor c, AtomicLong sequenceId) throws IOException {
|
||||
TableName tn = TableName.valueOf(c.getTableName().toByteArray());
|
||||
HLogKey key = new HLogKey(info.getEncodedNameAsBytes(), tn);
|
||||
log.appendNoSync(htd, info, key, WALEdit.createCompaction(info, c), sequenceId, false, null);
|
||||
log.sync();
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("Appended compaction marker " + TextFormat.shortDebugString(c));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Write a flush marker indicating a start / abort or a complete of a region flush
|
||||
*/
|
||||
public static long writeFlushMarker(HLog log, HTableDescriptor htd, HRegionInfo info,
|
||||
final FlushDescriptor f, AtomicLong sequenceId, boolean sync) throws IOException {
|
||||
TableName tn = TableName.valueOf(f.getTableName().toByteArray());
|
||||
HLogKey key = new HLogKey(info.getEncodedNameAsBytes(), tn);
|
||||
long trx = log.appendNoSync(htd, info, key, WALEdit.createFlushWALEdit(info, f), sequenceId, false, null);
|
||||
if (sync) log.sync(trx);
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("Appended flush marker " + TextFormat.shortDebugString(f));
|
||||
}
|
||||
return trx;
|
||||
}
|
||||
|
||||
/**
|
||||
* Write a region open marker indicating that the region is opened
|
||||
*/
|
||||
public static long writeRegionEventMarker(HLog log, HTableDescriptor htd, HRegionInfo info,
|
||||
final RegionEventDescriptor r, AtomicLong sequenceId) throws IOException {
|
||||
TableName tn = TableName.valueOf(r.getTableName().toByteArray());
|
||||
HLogKey key = new HLogKey(info.getEncodedNameAsBytes(), tn);
|
||||
long trx = log.appendNoSync(htd, info, key, WALEdit.createRegionEventWALEdit(info, r),
|
||||
sequenceId, false, null);
|
||||
log.sync(trx);
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("Appended region event marker " + TextFormat.shortDebugString(r));
|
||||
}
|
||||
return trx;
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a file with name as region open sequence id
|
||||
*
|
||||
* @param fs
|
||||
* @param regiondir
|
||||
* @param newSeqId
|
||||
* @param saftyBumper
|
||||
* @return long new sequence Id value
|
||||
* @throws IOException
|
||||
*/
|
||||
public static long writeRegionOpenSequenceIdFile(final FileSystem fs, final Path regiondir,
|
||||
long newSeqId, long saftyBumper) throws IOException {
|
||||
|
||||
Path editsdir = HLogUtil.getRegionDirRecoveredEditsDir(regiondir);
|
||||
long maxSeqId = 0;
|
||||
FileStatus[] files = null;
|
||||
if (fs.exists(editsdir)) {
|
||||
files = FSUtils.listStatus(fs, editsdir, new PathFilter() {
|
||||
@Override
|
||||
public boolean accept(Path p) {
|
||||
if (p.getName().endsWith(HLog.SEQUENCE_ID_FILE_SUFFIX)) {
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
});
|
||||
if (files != null) {
|
||||
for (FileStatus status : files) {
|
||||
String fileName = status.getPath().getName();
|
||||
try {
|
||||
Long tmpSeqId = Long.parseLong(fileName.substring(0, fileName.length()
|
||||
- HLog.SEQUENCE_ID_FILE_SUFFIX.length()));
|
||||
maxSeqId = Math.max(tmpSeqId, maxSeqId);
|
||||
} catch (NumberFormatException ex) {
|
||||
LOG.warn("Invalid SeqId File Name=" + fileName);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
if (maxSeqId > newSeqId) {
|
||||
newSeqId = maxSeqId;
|
||||
}
|
||||
newSeqId += saftyBumper; // bump up SeqId
|
||||
|
||||
// write a new seqId file
|
||||
Path newSeqIdFile = new Path(editsdir, newSeqId + HLog.SEQUENCE_ID_FILE_SUFFIX);
|
||||
if (!fs.createNewFile(newSeqIdFile)) {
|
||||
throw new IOException("Failed to create SeqId file:" + newSeqIdFile);
|
||||
}
|
||||
// remove old ones
|
||||
if(files != null) {
|
||||
for (FileStatus status : files) {
|
||||
fs.delete(status.getPath(), false);
|
||||
}
|
||||
}
|
||||
return newSeqId;
|
||||
}
|
||||
|
||||
}
|
|
@ -31,7 +31,7 @@ import org.apache.hadoop.util.StringUtils;
|
|||
* single function call and turn it into multiple manipulations of the hadoop metrics system.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class MetricsWAL {
|
||||
public class MetricsWAL extends WALActionsListener.Base {
|
||||
static final Log LOG = LogFactory.getLog(MetricsWAL.class);
|
||||
|
||||
private final MetricsWALSource source;
|
||||
|
@ -40,19 +40,20 @@ public class MetricsWAL {
|
|||
source = CompatibilitySingletonFactory.getInstance(MetricsWALSource.class);
|
||||
}
|
||||
|
||||
public void finishSync(long time) {
|
||||
source.incrementSyncTime(time);
|
||||
@Override
|
||||
public void postSync(final long timeInNanos, final int handlerSyncs) {
|
||||
source.incrementSyncTime(timeInNanos/1000000l);
|
||||
}
|
||||
|
||||
public void finishAppend(long time, long size) {
|
||||
|
||||
@Override
|
||||
public void postAppend(final long size, final long time) {
|
||||
source.incrementAppendCount();
|
||||
source.incrementAppendTime(time);
|
||||
source.incrementAppendSize(size);
|
||||
|
||||
if (time > 1000) {
|
||||
source.incrementSlowAppendCount();
|
||||
LOG.warn(String.format("%s took %d ms appending an edit to hlog; len~=%s",
|
||||
LOG.warn(String.format("%s took %d ms appending an edit to wal; len~=%s",
|
||||
Thread.currentThread().getName(),
|
||||
time,
|
||||
StringUtils.humanReadableInt(size)));
|
||||
|
|
|
@ -31,6 +31,8 @@ import org.apache.commons.logging.Log;
|
|||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.hbase.codec.Codec;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
|
@ -40,6 +42,7 @@ import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALHeader.Builder;
|
|||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALKey;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALTrailer;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.wal.WAL.Entry;
|
||||
|
||||
import com.google.protobuf.CodedInputStream;
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
|
@ -58,17 +61,32 @@ import com.google.protobuf.InvalidProtocolBufferException;
|
|||
@InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX, HBaseInterfaceAudience.CONFIG})
|
||||
public class ProtobufLogReader extends ReaderBase {
|
||||
private static final Log LOG = LogFactory.getLog(ProtobufLogReader.class);
|
||||
static final byte[] PB_WAL_MAGIC = Bytes.toBytes("PWAL");
|
||||
static final byte[] PB_WAL_COMPLETE_MAGIC = Bytes.toBytes("LAWP");
|
||||
// public for WALFactory until we move everything to o.a.h.h.wal
|
||||
@InterfaceAudience.Private
|
||||
public static final byte[] PB_WAL_MAGIC = Bytes.toBytes("PWAL");
|
||||
// public for TestWALSplit
|
||||
@InterfaceAudience.Private
|
||||
public static final byte[] PB_WAL_COMPLETE_MAGIC = Bytes.toBytes("LAWP");
|
||||
/**
|
||||
* Configuration name of WAL Trailer's warning size. If a waltrailer's size is greater than the
|
||||
* configured size, providers should log a warning. e.g. this is used with Protobuf reader/writer.
|
||||
*/
|
||||
static final String WAL_TRAILER_WARN_SIZE = "hbase.regionserver.waltrailer.warn.size";
|
||||
static final int DEFAULT_WAL_TRAILER_WARN_SIZE = 1024 * 1024; // 1MB
|
||||
|
||||
protected FSDataInputStream inputStream;
|
||||
protected Codec.Decoder cellDecoder;
|
||||
protected WALCellCodec.ByteStringUncompressor byteStringUncompressor;
|
||||
protected boolean hasCompression = false;
|
||||
protected boolean hasTagCompression = false;
|
||||
// walEditsStopOffset is the position of the last byte to read. After reading the last WALEdit entry
|
||||
// in the hlog, the inputstream's position is equal to walEditsStopOffset.
|
||||
// in the wal, the inputstream's position is equal to walEditsStopOffset.
|
||||
private long walEditsStopOffset;
|
||||
private boolean trailerPresent;
|
||||
protected WALTrailer trailer;
|
||||
// maximum size of the wal Trailer in bytes. If a user writes/reads a trailer with size larger
|
||||
// than this size, it is written/read respectively, with a WARN message in the log.
|
||||
protected int trailerWarnSize;
|
||||
private static List<String> writerClsNames = new ArrayList<String>();
|
||||
static {
|
||||
writerClsNames.add(ProtobufLogWriter.class.getSimpleName());
|
||||
|
@ -120,6 +138,13 @@ public class ProtobufLogReader extends ReaderBase {
|
|||
initAfterCompression(clsName); // We need a new decoder (at least).
|
||||
}
|
||||
|
||||
@Override
|
||||
public void init(FileSystem fs, Path path, Configuration conf, FSDataInputStream stream)
|
||||
throws IOException {
|
||||
this.trailerWarnSize = conf.getInt(WAL_TRAILER_WARN_SIZE, DEFAULT_WAL_TRAILER_WARN_SIZE);
|
||||
super.init(fs, path, conf, stream);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String initReader(FSDataInputStream stream) throws IOException {
|
||||
return initInternal(stream, true);
|
||||
|
@ -268,7 +293,7 @@ public class ProtobufLogReader extends ReaderBase {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected boolean readNext(HLog.Entry entry) throws IOException {
|
||||
protected boolean readNext(Entry entry) throws IOException {
|
||||
while (true) {
|
||||
// OriginalPosition might be < 0 on local fs; if so, it is useless to us.
|
||||
long originalPosition = this.inputStream.getPos();
|
||||
|
@ -332,7 +357,7 @@ public class ProtobufLogReader extends ReaderBase {
|
|||
initCause(realEofEx != null ? realEofEx : ex);
|
||||
}
|
||||
if (trailerPresent && this.inputStream.getPos() > this.walEditsStopOffset) {
|
||||
LOG.error("Read WALTrailer while reading WALEdits. hlog: " + this.path
|
||||
LOG.error("Read WALTrailer while reading WALEdits. wal: " + this.path
|
||||
+ ", inputStream.getPos(): " + this.inputStream.getPos() + ", walEditsStopOffset: "
|
||||
+ this.walEditsStopOffset);
|
||||
throw new EOFException("Read WALTrailer while reading WALEdits");
|
||||
|
@ -369,11 +394,6 @@ public class ProtobufLogReader extends ReaderBase {
|
|||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public WALTrailer getWALTrailer() {
|
||||
return trailer;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void seekOnFs(long pos) throws IOException {
|
||||
this.inputStream.seek(pos);
|
||||
|
|
|
@ -34,6 +34,10 @@ import org.apache.hadoop.hbase.codec.Codec;
|
|||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALHeader;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALTrailer;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.wal.WAL.Entry;
|
||||
|
||||
import static org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader.WAL_TRAILER_WARN_SIZE;
|
||||
import static org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader.DEFAULT_WAL_TRAILER_WARN_SIZE;
|
||||
|
||||
/**
|
||||
* Writer for protobuf-based WAL.
|
||||
|
@ -77,8 +81,7 @@ public class ProtobufLogWriter extends WriterBase {
|
|||
super.init(fs, path, conf, overwritable);
|
||||
assert this.output == null;
|
||||
boolean doCompress = initializeCompressionContext(conf, path);
|
||||
this.trailerWarnSize = conf.getInt(HLog.WAL_TRAILER_WARN_SIZE,
|
||||
HLog.DEFAULT_WAL_TRAILER_WARN_SIZE);
|
||||
this.trailerWarnSize = conf.getInt(WAL_TRAILER_WARN_SIZE, DEFAULT_WAL_TRAILER_WARN_SIZE);
|
||||
int bufferSize = FSUtils.getDefaultBufferSize(fs);
|
||||
short replication = (short)conf.getInt(
|
||||
"hbase.regionserver.hlog.replication", FSUtils.getDefaultReplication(fs, path));
|
||||
|
@ -110,7 +113,7 @@ public class ProtobufLogWriter extends WriterBase {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void append(HLog.Entry entry) throws IOException {
|
||||
public void append(Entry entry) throws IOException {
|
||||
entry.setCompressionContext(compressionContext);
|
||||
entry.getKey().getBuilder(compressor).
|
||||
setFollowingKvCount(entry.getEdit().size()).build().writeDelimitedTo(output);
|
||||
|
@ -134,7 +137,7 @@ public class ProtobufLogWriter extends WriterBase {
|
|||
}
|
||||
}
|
||||
|
||||
protected WALTrailer buildWALTrailer(WALTrailer.Builder builder) {
|
||||
WALTrailer buildWALTrailer(WALTrailer.Builder builder) {
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
|
@ -188,8 +191,7 @@ public class ProtobufLogWriter extends WriterBase {
|
|||
return this.output;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setWALTrailer(WALTrailer walTrailer) {
|
||||
void setWALTrailer(WALTrailer walTrailer) {
|
||||
this.trailer = walTrailer;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -31,21 +31,19 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.io.util.LRUDictionary;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALTrailer;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
|
||||
import org.apache.hadoop.hbase.wal.DefaultWALProvider;
|
||||
import org.apache.hadoop.hbase.wal.WAL.Entry;
|
||||
|
||||
@InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
|
||||
public abstract class ReaderBase implements HLog.Reader {
|
||||
public abstract class ReaderBase implements DefaultWALProvider.Reader {
|
||||
private static final Log LOG = LogFactory.getLog(ReaderBase.class);
|
||||
protected Configuration conf;
|
||||
protected FileSystem fs;
|
||||
protected Path path;
|
||||
protected long edit = 0;
|
||||
protected long fileLength;
|
||||
protected WALTrailer trailer;
|
||||
// maximum size of the wal Trailer in bytes. If a user writes/reads a trailer with size larger
|
||||
// than this size, it is written/read respectively, with a WARN message in the log.
|
||||
protected int trailerWarnSize;
|
||||
/**
|
||||
* Compression context to use reading. Can be null if no compression.
|
||||
*/
|
||||
|
@ -65,8 +63,6 @@ public abstract class ReaderBase implements HLog.Reader {
|
|||
this.path = path;
|
||||
this.fs = fs;
|
||||
this.fileLength = this.fs.getFileStatus(path).getLen();
|
||||
this.trailerWarnSize = conf.getInt(HLog.WAL_TRAILER_WARN_SIZE,
|
||||
HLog.DEFAULT_WAL_TRAILER_WARN_SIZE);
|
||||
String cellCodecClsName = initReader(stream);
|
||||
|
||||
boolean compression = hasCompression();
|
||||
|
@ -87,15 +83,17 @@ public abstract class ReaderBase implements HLog.Reader {
|
|||
}
|
||||
|
||||
@Override
|
||||
public HLog.Entry next() throws IOException {
|
||||
public Entry next() throws IOException {
|
||||
return next(null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public HLog.Entry next(HLog.Entry reuse) throws IOException {
|
||||
HLog.Entry e = reuse;
|
||||
public Entry next(Entry reuse) throws IOException {
|
||||
Entry e = reuse;
|
||||
if (e == null) {
|
||||
e = new HLog.Entry(new HLogKey(), new WALEdit());
|
||||
// we use HLogKey here instead of WALKey directly to support legacy coprocessors,
|
||||
// seqencefile based readers, and HLogInputFormat.
|
||||
e = new Entry(new HLogKey(), new WALEdit());
|
||||
}
|
||||
if (compressionContext != null) {
|
||||
e.setCompressionContext(compressionContext);
|
||||
|
@ -165,15 +163,10 @@ public abstract class ReaderBase implements HLog.Reader {
|
|||
* @param e The entry to read into.
|
||||
* @return Whether there was anything to read.
|
||||
*/
|
||||
protected abstract boolean readNext(HLog.Entry e) throws IOException;
|
||||
protected abstract boolean readNext(Entry e) throws IOException;
|
||||
|
||||
/**
|
||||
* Performs a filesystem-level seek to a certain position in an underlying file.
|
||||
*/
|
||||
protected abstract void seekOnFs(long pos) throws IOException;
|
||||
|
||||
@Override
|
||||
public WALTrailer getWALTrailer() {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -52,10 +52,10 @@ public class SecureProtobufLogWriter extends ProtobufLogWriter {
|
|||
builder.setWriterClsName(SecureProtobufLogWriter.class.getSimpleName());
|
||||
if (conf.getBoolean(HConstants.ENABLE_WAL_ENCRYPTION, false)) {
|
||||
// Get an instance of our cipher
|
||||
Cipher cipher = Encryption.getCipher(conf,
|
||||
conf.get(HConstants.CRYPTO_WAL_ALGORITHM_CONF_KEY, DEFAULT_CIPHER));
|
||||
final String cipherName = conf.get(HConstants.CRYPTO_WAL_ALGORITHM_CONF_KEY, DEFAULT_CIPHER);
|
||||
Cipher cipher = Encryption.getCipher(conf, cipherName);
|
||||
if (cipher == null) {
|
||||
throw new RuntimeException("Cipher '" + cipher + "' is not available");
|
||||
throw new RuntimeException("Cipher '" + cipherName + "' is not available");
|
||||
}
|
||||
|
||||
// Generate an encryption key for this WAL
|
||||
|
|
|
@ -33,7 +33,7 @@ import org.apache.hadoop.fs.FSDataInputStream;
|
|||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLog.Entry;
|
||||
import org.apache.hadoop.hbase.wal.WAL.Entry;
|
||||
import org.apache.hadoop.io.SequenceFile;
|
||||
import org.apache.hadoop.io.SequenceFile.Metadata;
|
||||
import org.apache.hadoop.io.Text;
|
||||
|
@ -222,10 +222,27 @@ public class SequenceFileLogReader extends ReaderBase {
|
|||
}
|
||||
|
||||
|
||||
/**
|
||||
* fill in the passed entry with teh next key/value.
|
||||
* Note that because this format deals with our legacy storage, the provided
|
||||
* Entery MUST use an {@link HLogKey} for the key.
|
||||
* @return boolean indicating if the contents of Entry have been filled in.
|
||||
*/
|
||||
@Override
|
||||
protected boolean readNext(Entry e) throws IOException {
|
||||
try {
|
||||
boolean hasNext = this.reader.next(e.getKey(), e.getEdit());
|
||||
if (!(e.getKey() instanceof HLogKey)) {
|
||||
final IllegalArgumentException exception = new IllegalArgumentException(
|
||||
"SequenceFileLogReader only works when given entries that have HLogKey for keys. This" +
|
||||
" one had '" + e.getKey().getClass() + "'");
|
||||
LOG.error("We need to use the legacy SequenceFileLogReader to handle a " +
|
||||
" pre-0.96 style WAL, but HBase internals failed to use the deprecated HLogKey class." +
|
||||
" This is a bug; please file an issue or email the developer mailing list. You will " +
|
||||
"need the following exception details when seeking help from the HBase community.",
|
||||
exception);
|
||||
throw exception;
|
||||
}
|
||||
boolean hasNext = this.reader.next((HLogKey)e.getKey(), e.getEdit());
|
||||
if (!hasNext) return false;
|
||||
// Scopes are probably in WAL edit, move to key
|
||||
NavigableMap<byte[], Integer> scopes = e.getEdit().getAndRemoveScopes();
|
||||
|
|
|
@ -25,8 +25,10 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
|
||||
import org.apache.hadoop.hbase.wal.WALKey;
|
||||
|
||||
/**
|
||||
* Get notification of {@link FSHLog}/WAL log events. The invocations are inline
|
||||
* Get notification of WAL events. The invocations are inline
|
||||
* so make sure your implementation is fast else you'll slow hbase.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
|
@ -35,30 +37,30 @@ public interface WALActionsListener {
|
|||
/**
|
||||
* The WAL is going to be rolled. The oldPath can be null if this is
|
||||
* the first log file from the regionserver.
|
||||
* @param oldPath the path to the old hlog
|
||||
* @param newPath the path to the new hlog
|
||||
* @param oldPath the path to the old wal
|
||||
* @param newPath the path to the new wal
|
||||
*/
|
||||
void preLogRoll(Path oldPath, Path newPath) throws IOException;
|
||||
|
||||
/**
|
||||
* The WAL has been rolled. The oldPath can be null if this is
|
||||
* the first log file from the regionserver.
|
||||
* @param oldPath the path to the old hlog
|
||||
* @param newPath the path to the new hlog
|
||||
* @param oldPath the path to the old wal
|
||||
* @param newPath the path to the new wal
|
||||
*/
|
||||
void postLogRoll(Path oldPath, Path newPath) throws IOException;
|
||||
|
||||
/**
|
||||
* The WAL is going to be archived.
|
||||
* @param oldPath the path to the old hlog
|
||||
* @param newPath the path to the new hlog
|
||||
* @param oldPath the path to the old wal
|
||||
* @param newPath the path to the new wal
|
||||
*/
|
||||
void preLogArchive(Path oldPath, Path newPath) throws IOException;
|
||||
|
||||
/**
|
||||
* The WAL has been archived.
|
||||
* @param oldPath the path to the old hlog
|
||||
* @param newPath the path to the new hlog
|
||||
* @param oldPath the path to the old wal
|
||||
* @param newPath the path to the new wal
|
||||
*/
|
||||
void postLogArchive(Path oldPath, Path newPath) throws IOException;
|
||||
|
||||
|
@ -79,7 +81,7 @@ public interface WALActionsListener {
|
|||
* @param logEdit
|
||||
*/
|
||||
void visitLogEntryBeforeWrite(
|
||||
HRegionInfo info, HLogKey logKey, WALEdit logEdit
|
||||
HRegionInfo info, WALKey logKey, WALEdit logEdit
|
||||
);
|
||||
|
||||
/**
|
||||
|
@ -87,11 +89,59 @@ public interface WALActionsListener {
|
|||
* @param htd
|
||||
* @param logKey
|
||||
* @param logEdit
|
||||
* TODO: Retire this in favor of {@link #visitLogEntryBeforeWrite(HRegionInfo, HLogKey, WALEdit)}
|
||||
* It only exists to get scope when replicating. Scope should be in the HLogKey and not need
|
||||
* TODO: Retire this in favor of {@link #visitLogEntryBeforeWrite(HRegionInfo, WALKey, WALEdit)}
|
||||
* It only exists to get scope when replicating. Scope should be in the WALKey and not need
|
||||
* us passing in a <code>htd</code>.
|
||||
*/
|
||||
void visitLogEntryBeforeWrite(
|
||||
HTableDescriptor htd, HLogKey logKey, WALEdit logEdit
|
||||
HTableDescriptor htd, WALKey logKey, WALEdit logEdit
|
||||
);
|
||||
|
||||
/**
|
||||
* For notification post append to the writer. Used by metrics system at least.
|
||||
* TODO: Combine this with above.
|
||||
* @param entryLen approx length of cells in this append.
|
||||
* @param elapsedTimeMillis elapsed time in milliseconds.
|
||||
*/
|
||||
void postAppend(final long entryLen, final long elapsedTimeMillis);
|
||||
|
||||
/**
|
||||
* For notification post writer sync. Used by metrics system at least.
|
||||
* @param timeInNanos How long the filesystem sync took in nanoseconds.
|
||||
* @param handlerSyncs How many sync handler calls were released by this call to filesystem
|
||||
* sync.
|
||||
*/
|
||||
void postSync(final long timeInNanos, final int handlerSyncs);
|
||||
|
||||
static class Base implements WALActionsListener {
|
||||
@Override
|
||||
public void preLogRoll(Path oldPath, Path newPath) throws IOException {}
|
||||
|
||||
@Override
|
||||
public void postLogRoll(Path oldPath, Path newPath) throws IOException {}
|
||||
|
||||
@Override
|
||||
public void preLogArchive(Path oldPath, Path newPath) throws IOException {}
|
||||
|
||||
@Override
|
||||
public void postLogArchive(Path oldPath, Path newPath) throws IOException {}
|
||||
|
||||
@Override
|
||||
public void logRollRequested() {}
|
||||
|
||||
@Override
|
||||
public void logCloseRequested() {}
|
||||
|
||||
@Override
|
||||
public void visitLogEntryBeforeWrite(HRegionInfo info, WALKey logKey, WALEdit logEdit) {}
|
||||
|
||||
@Override
|
||||
public void visitLogEntryBeforeWrite(HTableDescriptor htd, WALKey logKey, WALEdit logEdit) {}
|
||||
|
||||
@Override
|
||||
public void postAppend(final long entryLen, final long elapsedTimeMillis) {}
|
||||
|
||||
@Override
|
||||
public void postSync(final long timeInNanos, final int handlerSyncs) {}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -131,7 +131,7 @@ public class WALCellCodec implements Codec {
|
|||
byte[] uncompress(ByteString data, Dictionary dict) throws IOException;
|
||||
}
|
||||
|
||||
// TODO: it sucks that compression context is in HLog.Entry. It'd be nice if it was here.
|
||||
// TODO: it sucks that compression context is in WAL.Entry. It'd be nice if it was here.
|
||||
// Dictionary could be gotten by enum; initially, based on enum, context would create
|
||||
// an array of dictionaries.
|
||||
static class BaosAndCompressor extends ByteArrayOutputStream implements ByteStringCompressor {
|
||||
|
|
|
@ -28,9 +28,12 @@ import org.apache.hadoop.hbase.coprocessor.*;
|
|||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
||||
import org.apache.hadoop.hbase.wal.WAL;
|
||||
import org.apache.hadoop.hbase.wal.WALKey;
|
||||
|
||||
/**
|
||||
* Implements the coprocessor environment and runtime support for coprocessors
|
||||
* loaded within a {@link FSHLog}.
|
||||
* loaded within a {@link WAL}.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class WALCoprocessorHost
|
||||
|
@ -42,10 +45,13 @@ public class WALCoprocessorHost
|
|||
static class WALEnvironment extends CoprocessorHost.Environment
|
||||
implements WALCoprocessorEnvironment {
|
||||
|
||||
private FSHLog wal;
|
||||
private final WAL wal;
|
||||
|
||||
final boolean useLegacyPre;
|
||||
final boolean useLegacyPost;
|
||||
|
||||
@Override
|
||||
public FSHLog getWAL() {
|
||||
public WAL getWAL() {
|
||||
return wal;
|
||||
}
|
||||
|
||||
|
@ -56,23 +62,32 @@ public class WALCoprocessorHost
|
|||
* @param priority chaining priority
|
||||
* @param seq load sequence
|
||||
* @param conf configuration
|
||||
* @param hlog HLog
|
||||
* @param wal WAL
|
||||
*/
|
||||
public WALEnvironment(Class<?> implClass, final Coprocessor impl,
|
||||
final int priority, final int seq, final Configuration conf,
|
||||
final FSHLog hlog) {
|
||||
final WAL wal) {
|
||||
super(impl, priority, seq, conf);
|
||||
this.wal = hlog;
|
||||
this.wal = wal;
|
||||
// Pick which version of the API we'll call.
|
||||
// This way we avoid calling the new version on older WALObservers so
|
||||
// we can maintain binary compatibility.
|
||||
// See notes in javadoc for WALObserver
|
||||
useLegacyPre = useLegacyMethod(impl.getClass(), "preWALWrite", ObserverContext.class,
|
||||
HRegionInfo.class, WALKey.class, WALEdit.class);
|
||||
useLegacyPost = useLegacyMethod(impl.getClass(), "postWALWrite", ObserverContext.class,
|
||||
HRegionInfo.class, WALKey.class, WALEdit.class);
|
||||
}
|
||||
}
|
||||
|
||||
FSHLog wal;
|
||||
private final WAL wal;
|
||||
|
||||
/**
|
||||
* Constructor
|
||||
* @param log the write ahead log
|
||||
* @param conf the configuration
|
||||
*/
|
||||
public WALCoprocessorHost(final FSHLog log, final Configuration conf) {
|
||||
public WALCoprocessorHost(final WAL log, final Configuration conf) {
|
||||
// We don't want to require an Abortable passed down through (FS)HLog, so
|
||||
// this means that a failure to load of a WAL coprocessor won't abort the
|
||||
// server. This isn't ideal, and means that security components that
|
||||
|
@ -100,21 +115,29 @@ public class WALCoprocessorHost
|
|||
* @return true if default behavior should be bypassed, false otherwise
|
||||
* @throws IOException
|
||||
*/
|
||||
public boolean preWALWrite(final HRegionInfo info, final HLogKey logKey, final WALEdit logEdit)
|
||||
public boolean preWALWrite(final HRegionInfo info, final WALKey logKey, final WALEdit logEdit)
|
||||
throws IOException {
|
||||
boolean bypass = false;
|
||||
if (this.coprocessors == null || this.coprocessors.isEmpty()) return bypass;
|
||||
ObserverContext<WALCoprocessorEnvironment> ctx = null;
|
||||
for (WALEnvironment env: coprocessors) {
|
||||
if (env.getInstance() instanceof
|
||||
org.apache.hadoop.hbase.coprocessor.WALObserver) {
|
||||
if (env.getInstance() instanceof WALObserver) {
|
||||
final WALObserver observer = (WALObserver)env.getInstance();
|
||||
ctx = ObserverContext.createAndPrepare(env, ctx);
|
||||
Thread currentThread = Thread.currentThread();
|
||||
ClassLoader cl = currentThread.getContextClassLoader();
|
||||
try {
|
||||
currentThread.setContextClassLoader(env.getClassLoader());
|
||||
((org.apache.hadoop.hbase.coprocessor.WALObserver)env.getInstance()).
|
||||
preWALWrite(ctx, info, logKey, logEdit);
|
||||
if (env.useLegacyPre) {
|
||||
if (logKey instanceof HLogKey) {
|
||||
observer.preWALWrite(ctx, info, (HLogKey)logKey, logEdit);
|
||||
} else {
|
||||
legacyWarning(observer.getClass(),
|
||||
"There are wal keys present that are not HLogKey.");
|
||||
}
|
||||
} else {
|
||||
observer.preWALWrite(ctx, info, logKey, logEdit);
|
||||
}
|
||||
} catch (Throwable e) {
|
||||
handleCoprocessorThrowable(env, e);
|
||||
} finally {
|
||||
|
@ -135,20 +158,28 @@ public class WALCoprocessorHost
|
|||
* @param logEdit
|
||||
* @throws IOException
|
||||
*/
|
||||
public void postWALWrite(final HRegionInfo info, final HLogKey logKey, final WALEdit logEdit)
|
||||
public void postWALWrite(final HRegionInfo info, final WALKey logKey, final WALEdit logEdit)
|
||||
throws IOException {
|
||||
if (this.coprocessors == null || this.coprocessors.isEmpty()) return;
|
||||
ObserverContext<WALCoprocessorEnvironment> ctx = null;
|
||||
for (WALEnvironment env: coprocessors) {
|
||||
if (env.getInstance() instanceof
|
||||
org.apache.hadoop.hbase.coprocessor.WALObserver) {
|
||||
if (env.getInstance() instanceof WALObserver) {
|
||||
final WALObserver observer = (WALObserver)env.getInstance();
|
||||
ctx = ObserverContext.createAndPrepare(env, ctx);
|
||||
Thread currentThread = Thread.currentThread();
|
||||
ClassLoader cl = currentThread.getContextClassLoader();
|
||||
try {
|
||||
currentThread.setContextClassLoader(env.getClassLoader());
|
||||
((org.apache.hadoop.hbase.coprocessor.WALObserver)env.getInstance()).
|
||||
postWALWrite(ctx, info, logKey, logEdit);
|
||||
if (env.useLegacyPost) {
|
||||
if (logKey instanceof HLogKey) {
|
||||
observer.postWALWrite(ctx, info, (HLogKey)logKey, logEdit);
|
||||
} else {
|
||||
legacyWarning(observer.getClass(),
|
||||
"There are wal keys present that are not HLogKey.");
|
||||
}
|
||||
} else {
|
||||
observer.postWALWrite(ctx, info, logKey, logEdit);
|
||||
}
|
||||
} catch (Throwable e) {
|
||||
handleCoprocessorThrowable(env, e);
|
||||
} finally {
|
||||
|
|
|
@ -52,7 +52,7 @@ import org.apache.hadoop.io.Writable;
|
|||
* for serializing/deserializing a set of KeyValue items.
|
||||
*
|
||||
* Previously, if a transaction contains 3 edits to c1, c2, c3 for a row R,
|
||||
* the HLog would have three log entries as follows:
|
||||
* the WAL would have three log entries as follows:
|
||||
*
|
||||
* <logseq1-for-edit1>:<KeyValue-for-edit-c1>
|
||||
* <logseq2-for-edit2>:<KeyValue-for-edit-c2>
|
||||
|
@ -73,7 +73,7 @@ import org.apache.hadoop.io.Writable;
|
|||
* <-1, 3, <Keyvalue-for-edit-c1>, <KeyValue-for-edit-c2>, <KeyValue-for-edit-c3>>
|
||||
*
|
||||
* The -1 marker is just a special way of being backward compatible with
|
||||
* an old HLog which would have contained a single <KeyValue>.
|
||||
* an old WAL which would have contained a single <KeyValue>.
|
||||
*
|
||||
* The deserializer for WALEdit backward compatibly detects if the record
|
||||
* is an old style KeyValue or the new style WALEdit.
|
||||
|
@ -168,7 +168,7 @@ public class WALEdit implements Writable, HeapSize {
|
|||
int versionOrLength = in.readInt();
|
||||
// TODO: Change version when we protobuf. Also, change way we serialize KV! Pb it too.
|
||||
if (versionOrLength == VERSION_2) {
|
||||
// this is new style HLog entry containing multiple KeyValues.
|
||||
// this is new style WAL entry containing multiple KeyValues.
|
||||
int numEdits = in.readInt();
|
||||
for (int idx = 0; idx < numEdits; idx++) {
|
||||
if (compressionContext != null) {
|
||||
|
@ -189,7 +189,7 @@ public class WALEdit implements Writable, HeapSize {
|
|||
}
|
||||
}
|
||||
} else {
|
||||
// this is an old style HLog entry. The int that we just
|
||||
// this is an old style WAL entry. The int that we just
|
||||
// read is actually the length of a single KeyValue
|
||||
this.add(KeyValue.create(versionOrLength, in));
|
||||
}
|
||||
|
|
|
@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
|
|||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.wal.WAL.Entry;
|
||||
|
||||
import com.google.protobuf.ServiceException;
|
||||
|
||||
|
@ -96,17 +97,17 @@ public class WALEditsReplaySink {
|
|||
* @param entries
|
||||
* @throws IOException
|
||||
*/
|
||||
public void replayEntries(List<Pair<HRegionLocation, HLog.Entry>> entries) throws IOException {
|
||||
public void replayEntries(List<Pair<HRegionLocation, Entry>> entries) throws IOException {
|
||||
if (entries.size() == 0) {
|
||||
return;
|
||||
}
|
||||
|
||||
int batchSize = entries.size();
|
||||
Map<HRegionInfo, List<HLog.Entry>> entriesByRegion =
|
||||
new HashMap<HRegionInfo, List<HLog.Entry>>();
|
||||
Map<HRegionInfo, List<Entry>> entriesByRegion =
|
||||
new HashMap<HRegionInfo, List<Entry>>();
|
||||
HRegionLocation loc = null;
|
||||
HLog.Entry entry = null;
|
||||
List<HLog.Entry> regionEntries = null;
|
||||
Entry entry = null;
|
||||
List<Entry> regionEntries = null;
|
||||
// Build the action list.
|
||||
for (int i = 0; i < batchSize; i++) {
|
||||
loc = entries.get(i).getFirst();
|
||||
|
@ -114,7 +115,7 @@ public class WALEditsReplaySink {
|
|||
if (entriesByRegion.containsKey(loc.getRegionInfo())) {
|
||||
regionEntries = entriesByRegion.get(loc.getRegionInfo());
|
||||
} else {
|
||||
regionEntries = new ArrayList<HLog.Entry>();
|
||||
regionEntries = new ArrayList<Entry>();
|
||||
entriesByRegion.put(loc.getRegionInfo(), regionEntries);
|
||||
}
|
||||
regionEntries.add(entry);
|
||||
|
@ -123,9 +124,9 @@ public class WALEditsReplaySink {
|
|||
long startTime = EnvironmentEdgeManager.currentTime();
|
||||
|
||||
// replaying edits by region
|
||||
for (Map.Entry<HRegionInfo, List<HLog.Entry>> _entry : entriesByRegion.entrySet()) {
|
||||
for (Map.Entry<HRegionInfo, List<Entry>> _entry : entriesByRegion.entrySet()) {
|
||||
HRegionInfo curRegion = _entry.getKey();
|
||||
List<HLog.Entry> allActions = _entry.getValue();
|
||||
List<Entry> allActions = _entry.getValue();
|
||||
// send edits in chunks
|
||||
int totalActions = allActions.size();
|
||||
int replayedActions = 0;
|
||||
|
@ -159,7 +160,7 @@ public class WALEditsReplaySink {
|
|||
}
|
||||
|
||||
private void replayEdits(final HRegionLocation regionLoc, final HRegionInfo regionInfo,
|
||||
final List<HLog.Entry> entries) throws IOException {
|
||||
final List<Entry> entries) throws IOException {
|
||||
try {
|
||||
RpcRetryingCallerFactory factory = RpcRetryingCallerFactory.instantiate(conf);
|
||||
ReplayServerCallable<ReplicateWALEntryResponse> callable =
|
||||
|
@ -182,11 +183,11 @@ public class WALEditsReplaySink {
|
|||
*/
|
||||
class ReplayServerCallable<R> extends RegionServerCallable<ReplicateWALEntryResponse> {
|
||||
private HRegionInfo regionInfo;
|
||||
private List<HLog.Entry> entries;
|
||||
private List<Entry> entries;
|
||||
|
||||
ReplayServerCallable(final HConnection connection, final TableName tableName,
|
||||
final HRegionLocation regionLoc, final HRegionInfo regionInfo,
|
||||
final List<HLog.Entry> entries) {
|
||||
final List<Entry> entries) {
|
||||
super(connection, tableName, null);
|
||||
this.entries = entries;
|
||||
this.regionInfo = regionInfo;
|
||||
|
@ -203,11 +204,11 @@ public class WALEditsReplaySink {
|
|||
return null;
|
||||
}
|
||||
|
||||
private void replayToServer(HRegionInfo regionInfo, List<HLog.Entry> entries)
|
||||
private void replayToServer(HRegionInfo regionInfo, List<Entry> entries)
|
||||
throws IOException, ServiceException {
|
||||
if (entries.isEmpty()) return;
|
||||
|
||||
HLog.Entry[] entriesArray = new HLog.Entry[entries.size()];
|
||||
Entry[] entriesArray = new Entry[entries.size()];
|
||||
entriesArray = entries.toArray(entriesArray);
|
||||
AdminService.BlockingInterface remoteSvr = conn.getAdmin(getLocation().getServerName());
|
||||
|
||||
|
@ -228,11 +229,11 @@ public class WALEditsReplaySink {
|
|||
// if not due to connection issue, the following code should run fast because it uses
|
||||
// cached location
|
||||
boolean skip = false;
|
||||
for (HLog.Entry entry : this.entries) {
|
||||
for (Entry entry : this.entries) {
|
||||
WALEdit edit = entry.getEdit();
|
||||
List<Cell> cells = edit.getCells();
|
||||
for (Cell cell : cells) {
|
||||
// filtering HLog meta entries
|
||||
// filtering WAL meta entries
|
||||
setLocation(conn.locateRegion(tableName, cell.getRow()));
|
||||
skip = true;
|
||||
break;
|
||||
|
|
|
@ -28,12 +28,14 @@ import org.apache.hadoop.hbase.HConstants;
|
|||
import org.apache.hadoop.hbase.io.util.LRUDictionary;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
|
||||
import org.apache.hadoop.hbase.wal.DefaultWALProvider;
|
||||
|
||||
/**
|
||||
* Context used by our wal dictionary compressor. Null if we're not to do our
|
||||
* custom dictionary compression.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public abstract class WriterBase implements HLog.Writer {
|
||||
public abstract class WriterBase implements DefaultWALProvider.Writer {
|
||||
|
||||
protected CompressionContext compressionContext;
|
||||
protected Configuration conf;
|
||||
|
|
|
@ -24,7 +24,7 @@ import java.util.List;
|
|||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLog.Entry;
|
||||
import org.apache.hadoop.hbase.wal.WAL.Entry;
|
||||
|
||||
/**
|
||||
* A {@link WALEntryFilter} which contains multiple filters and applies them
|
||||
|
|
|
@ -26,7 +26,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLog;
|
||||
import org.apache.hadoop.hbase.wal.WAL.Entry;
|
||||
import org.apache.hadoop.hbase.replication.regionserver.MetricsSource;
|
||||
|
||||
import com.google.common.util.concurrent.Service;
|
||||
|
@ -128,13 +128,13 @@ public interface ReplicationEndpoint extends Service {
|
|||
*/
|
||||
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.REPLICATION)
|
||||
class ReplicateContext {
|
||||
List<HLog.Entry> entries;
|
||||
List<Entry> entries;
|
||||
int size;
|
||||
@InterfaceAudience.Private
|
||||
public ReplicateContext() {
|
||||
}
|
||||
|
||||
public ReplicateContext setEntries(List<HLog.Entry> entries) {
|
||||
public ReplicateContext setEntries(List<Entry> entries) {
|
||||
this.entries = entries;
|
||||
return this;
|
||||
}
|
||||
|
@ -142,7 +142,7 @@ public interface ReplicationEndpoint extends Service {
|
|||
this.size = size;
|
||||
return this;
|
||||
}
|
||||
public List<HLog.Entry> getEntries() {
|
||||
public List<Entry> getEntries() {
|
||||
return entries;
|
||||
}
|
||||
public int getSize() {
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue