HBASE-10378 Refactor write-ahead-log implementation -- ADDEDNUM

This commit is contained in:
stack 2014-11-18 12:12:35 -08:00
parent f5e05eb836
commit b94e6738d9
204 changed files with 2936 additions and 9161 deletions

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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().

View File

@ -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

View File

@ -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;

View File

@ -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
*

View File

@ -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);

View File

@ -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);

View File

@ -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;

View File

@ -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 servers 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) {

View File

@ -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();

View File

@ -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 */

View File

@ -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.
*

View File

@ -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>

View File

@ -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);

View File

@ -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";

View File

@ -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.

View File

@ -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

View File

@ -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);

View File

@ -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);
}
}

View File

@ -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())

View File

@ -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 {

View File

@ -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();

View File

@ -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;

View File

@ -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>

View File

@ -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() {

View File

@ -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

View File

@ -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

View File

@ -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;

View File

@ -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;
}

View File

@ -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 {
}

View File

@ -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;

View File

@ -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>

View File

@ -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);

View File

@ -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;

View File

@ -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.

View File

@ -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 {

View File

@ -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

View File

@ -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

View File

@ -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);
}
}

View File

@ -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);
}
}
}

View File

@ -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;

View File

@ -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();
}

View File

@ -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;
}

View File

@ -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

View File

@ -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);
}
}

View File

@ -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;

View File

@ -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();
}
}

View File

@ -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;

View File

@ -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

View File

@ -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"

View File

@ -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;

View File

@ -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
*/

View File

@ -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);

View File

@ -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.
*

View File

@ -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() {

View File

@ -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);

View File

@ -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());
}
}

View File

@ -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)

View File

@ -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

View File

@ -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 {

View File

@ -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) {

View File

@ -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();
}

View File

@ -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()

View File

@ -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;

View File

@ -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());
}

View File

@ -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
}
}

View File

@ -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;

View File

@ -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);
}
}

View File

@ -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;

View File

@ -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);

View File

@ -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

View File

@ -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);

View File

@ -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.

View File

@ -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.

View File

@ -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();
}
}
}

View File

@ -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

View File

@ -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();

View File

@ -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;
}

View File

@ -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);
}

View File

@ -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);
}
}
}

View File

@ -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();
}
}
}

View File

@ -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();
}
}

View File

@ -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;
}
}

View File

@ -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)));

View File

@ -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);

View File

@ -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;
}
}

View File

@ -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;
}
}

View File

@ -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

View File

@ -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();

View File

@ -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) {}
}
}

View File

@ -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 {

View File

@ -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 {

View File

@ -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));
}

View File

@ -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;

View File

@ -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;

View File

@ -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

View File

@ -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