@InterfaceAudience.Private public abstract class AbstractFSWAL<W extends WALProvider.WriterBase> extends Object implements WAL
WAL
to go against FileSystem
; i.e. keep WALs in HDFS. Only one
WAL is ever being written at a time. When a WAL hits a configured maximum size, it is rolled.
This is done internal to the implementation.
As data is flushed from the MemStore to other on-disk structures (files sorted by key, hfiles), a WAL becomes obsolete. We can let go of all the log edits/entries for a given HRegion-sequence id. A bunch of work in the below is done keeping account of these region sequence ids -- what is flushed out to hfiles, and what is yet in WAL and in memory only.
It is only practical to delete entire files. Thus, we delete an entire on-disk file
F
when all of the edits in F
have a log-sequence-id that's older
(smaller) than the most-recent flush.
To read an WAL, call
WALFactory.createReader(org.apache.hadoop.fs.FileSystem, org.apache.hadoop.fs.Path)
. *
WAL.Entry, WAL.Reader
Modifier and Type | Field and Description |
---|---|
protected long |
blocksize
Block size to use writing files.
|
protected boolean |
closed |
protected org.apache.hadoop.conf.Configuration |
conf
conf object
|
protected WALCoprocessorHost |
coprocessorHost |
protected static int |
DEFAULT_SLOW_SYNC_TIME_MS |
protected AtomicLong |
filenum |
protected org.apache.hadoop.fs.FileSystem |
fs
file system instance
|
protected AtomicLong |
highestSyncedTxid
Updated to the transaction id of the last successful sync call.
|
protected long |
highestUnsyncedTxid
The highest known outstanding unsync'd WALEdit transaction id.
|
protected String |
implClassName
The class name of the runtime implementation, used as prefix for logging/tracing.
|
protected List<WALActionsListener> |
listeners
Listeners that are called on WAL events.
|
protected long |
logrollsize |
protected int |
maxLogs |
protected AtomicInteger |
numEntries |
protected org.apache.hadoop.fs.PathFilter |
ourFiles
Matches just those wal files that belong to this wal instance.
|
protected String |
prefixPathStr
Prefix used when checking for wal membership.
|
protected ReentrantLock |
rollWriterLock
This lock makes sure only one log roll runs at a time.
|
protected org.apache.hadoop.hbase.regionserver.wal.SequenceIdAccounting |
sequenceIdAccounting
Class that does accounting of sequenceids in WAL subsystem.
|
protected AtomicBoolean |
shutdown |
protected long |
slowSyncNs |
protected AtomicLong |
totalLogSize
The total size of wal
|
protected org.apache.hadoop.fs.Path |
walArchiveDir
dir path where old logs are kept.
|
protected org.apache.hadoop.fs.Path |
walDir
WAL directory, where all WAL files would be placed.
|
protected ConcurrentNavigableMap<org.apache.hadoop.fs.Path,org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL.WalProps> |
walFile2Props
Map of WAL log file to properties.
|
protected String |
walFilePrefix
Prefix of a WAL file, usually the region server name it is hosted on.
|
protected String |
walFileSuffix
Suffix included on generated wal file names
|
Modifier | Constructor and Description |
---|---|
protected |
AbstractFSWAL(org.apache.hadoop.fs.FileSystem fs,
org.apache.hadoop.fs.Path rootDir,
String logDir,
String archiveDir,
org.apache.hadoop.conf.Configuration conf,
List<WALActionsListener> listeners,
boolean failIfWALExists,
String prefix,
String suffix) |
Modifier and Type | Method and Description |
---|---|
void |
abortCacheFlush(byte[] encodedRegionName)
Abort a cache flush.
|
abstract long |
append(RegionInfo info,
WALKeyImpl key,
WALEdit edits,
boolean inMemstore)
NOTE: This append, at a time that is usually after this call returns, starts an mvcc
transaction by calling 'begin' wherein which we assign this update a sequenceid.
|
protected boolean |
append(W writer,
org.apache.hadoop.hbase.regionserver.wal.FSWALEntry entry) |
protected void |
atHeadOfRingBufferEventHandlerAppend()
Exposed for testing only.
|
protected void |
blockOnSync(org.apache.hadoop.hbase.regionserver.wal.SyncFuture syncFuture) |
void |
checkLogLowReplication(long checkInterval) |
void |
close()
Caller no longer needs any edits from this WAL.
|
void |
completeCacheFlush(byte[] encodedRegionName)
Complete the cache flush.
|
protected org.apache.hadoop.fs.Path |
computeFilename(long filenum)
This is a convenience method that computes a new filename with a given file-number.
|
protected abstract W |
createWriterInstance(org.apache.hadoop.fs.Path path) |
protected abstract void |
doAppend(W writer,
org.apache.hadoop.hbase.regionserver.wal.FSWALEntry entry) |
protected abstract boolean |
doCheckLogLowReplication() |
protected abstract void |
doReplaceWriter(org.apache.hadoop.fs.Path oldPath,
org.apache.hadoop.fs.Path newPath,
W nextWriter) |
protected abstract void |
doShutdown() |
WALCoprocessorHost |
getCoprocessorHost() |
org.apache.hadoop.fs.Path |
getCurrentFileName()
This is a convenience method that computes a new filename with a given using the current WAL
file-number
|
long |
getEarliestMemStoreSeqNum(byte[] encodedRegionName)
Gets the earliest unflushed sequence id in the memstore for the region.
|
long |
getEarliestMemStoreSeqNum(byte[] encodedRegionName,
byte[] familyName)
Gets the earliest unflushed sequence id in the memstore for the store.
|
long |
getFilenum() |
protected long |
getFileNumFromFileName(org.apache.hadoop.fs.Path fileName)
A log file has a creation timestamp (in ms) in its file name (
filenum . |
long |
getLogFileSize() |
OptionalLong |
getLogFileSizeIfBeingWritten(org.apache.hadoop.fs.Path path)
if the given
path is being written currently, then return its length. |
int |
getNumLogFiles() |
int |
getNumRolledLogFiles() |
protected int |
getPreallocatedEventCount() |
protected org.apache.hadoop.hbase.regionserver.wal.SyncFuture |
getSyncFuture(long sequence) |
static org.apache.hadoop.fs.Path |
getWALArchivePath(org.apache.hadoop.fs.Path archiveDir,
org.apache.hadoop.fs.Path p) |
protected void |
logRollAndSetupWalProps(org.apache.hadoop.fs.Path oldPath,
org.apache.hadoop.fs.Path newPath,
long oldFileLen) |
static void |
main(String[] args)
Pass one or more log file names and it will either dump out a text version on
stdout or split the specified log files. |
protected void |
postSync(long timeInNanos,
int handlerSyncs) |
void |
registerWALActionsListener(WALActionsListener listener)
Registers WALActionsListener
|
void |
requestLogRoll() |
protected void |
requestLogRoll(boolean tooFewReplicas) |
byte[][] |
rollWriter()
Roll the log writer.
|
byte[][] |
rollWriter(boolean force)
Roll the log writer.
|
void |
shutdown()
Stop accepting new writes.
|
protected long |
stampSequenceIdAndPublishToRingBuffer(RegionInfo hri,
WALKeyImpl key,
WALEdit edits,
boolean inMemstore,
com.lmax.disruptor.RingBuffer<org.apache.hadoop.hbase.regionserver.wal.RingBufferTruck> ringBuffer) |
Long |
startCacheFlush(byte[] encodedRegionName,
Map<byte[],Long> familyToSeq) |
Long |
startCacheFlush(byte[] encodedRegionName,
Set<byte[]> families)
WAL keeps track of the sequence numbers that are as yet not flushed im memstores
in order to be able to do accounting to figure which WALs can be let go.
|
String |
toString()
Human readable identifying information about the state of this WAL.
|
boolean |
unregisterWALActionsListener(WALActionsListener listener)
Unregisters WALActionsListener
|
void |
updateStore(byte[] encodedRegionName,
byte[] familyName,
Long sequenceid,
boolean onlyIfGreater)
updates the sequence number of a specific store.
|
protected static final int DEFAULT_SLOW_SYNC_TIME_MS
protected final org.apache.hadoop.fs.FileSystem fs
protected final org.apache.hadoop.fs.Path walDir
protected final org.apache.hadoop.fs.Path walArchiveDir
protected final org.apache.hadoop.fs.PathFilter ourFiles
protected final String walFilePrefix
protected final String walFileSuffix
protected final String prefixPathStr
protected final WALCoprocessorHost coprocessorHost
protected final org.apache.hadoop.conf.Configuration conf
protected final List<WALActionsListener> listeners
protected final org.apache.hadoop.hbase.regionserver.wal.SequenceIdAccounting sequenceIdAccounting
protected final long slowSyncNs
protected final long logrollsize
protected final long blocksize
protected final int maxLogs
protected final ReentrantLock rollWriterLock
protected final AtomicLong filenum
protected final AtomicInteger numEntries
protected volatile long highestUnsyncedTxid
protected final AtomicLong highestSyncedTxid
highestUnsyncedTxid
for case where we have an append where a sync has not yet come in
for it.protected final AtomicLong totalLogSize
protected volatile boolean closed
protected final AtomicBoolean shutdown
protected ConcurrentNavigableMap<org.apache.hadoop.fs.Path,org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL.WalProps> walFile2Props
protected final String implClassName
Performance testing shows getClass().getSimpleName() might be a bottleneck so we store it here, refer to HBASE-17676 for more details
protected AbstractFSWAL(org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.fs.Path rootDir, String logDir, String archiveDir, org.apache.hadoop.conf.Configuration conf, List<WALActionsListener> listeners, boolean failIfWALExists, String prefix, String suffix) throws FailedLogCloseException, IOException
FailedLogCloseException
IOException
public long getFilenum()
protected long getFileNumFromFileName(org.apache.hadoop.fs.Path fileName)
filenum
. This helper
method returns the creation timestamp from a given log file. It extracts the timestamp assuming
the filename is created with the computeFilename(long filenum)
method.protected final int getPreallocatedEventCount()
public void registerWALActionsListener(WALActionsListener listener)
WAL
registerWALActionsListener
in interface WAL
public boolean unregisterWALActionsListener(WALActionsListener listener)
WAL
unregisterWALActionsListener
in interface WAL
public WALCoprocessorHost getCoprocessorHost()
getCoprocessorHost
in interface WAL
public Long startCacheFlush(byte[] encodedRegionName, Set<byte[]> families)
WAL
Currently, it is expected that the update lock is held for the region; i.e. no concurrent appends while we set up cache flush.
startCacheFlush
in interface WAL
families
- Families to flush. May be a subset of all families in the region.HConstants.NO_SEQNUM
if we are flushing the whole region OR if
we are flushing a subset of all families but there are no edits in those families not
being flushed; in other words, this is effectively same as a flush of all of the region
though we were passed a subset of regions. Otherwise, it returns the sequence id of the
oldest/lowest outstanding edit.WAL.completeCacheFlush(byte[])
,
WAL.abortCacheFlush(byte[])
public Long startCacheFlush(byte[] encodedRegionName, Map<byte[],Long> familyToSeq)
startCacheFlush
in interface WAL
public void completeCacheFlush(byte[] encodedRegionName)
WAL
completeCacheFlush
in interface WAL
encodedRegionName
- Encoded region name.WAL.startCacheFlush(byte[], Set)
,
WAL.abortCacheFlush(byte[])
public void abortCacheFlush(byte[] encodedRegionName)
WAL
abortCacheFlush
in interface WAL
encodedRegionName
- Encoded region name.public long getEarliestMemStoreSeqNum(byte[] encodedRegionName)
WAL
getEarliestMemStoreSeqNum
in interface WAL
encodedRegionName
- The region to get the number for.public long getEarliestMemStoreSeqNum(byte[] encodedRegionName, byte[] familyName)
WAL
getEarliestMemStoreSeqNum
in interface WAL
encodedRegionName
- The region to get the number for.familyName
- The family to get the number for.public byte[][] rollWriter() throws FailedLogCloseException, IOException
WAL
The implementation is synchronized in order to make sure there's one rollWriter running at any given time.
rollWriter
in interface WAL
RegionInfo.getEncodedName()
FailedLogCloseException
IOException
protected org.apache.hadoop.fs.Path computeFilename(long filenum)
filenum
- to usepublic org.apache.hadoop.fs.Path getCurrentFileName()
public int getNumRolledLogFiles()
public int getNumLogFiles()
public static org.apache.hadoop.fs.Path getWALArchivePath(org.apache.hadoop.fs.Path archiveDir, org.apache.hadoop.fs.Path p)
protected final void logRollAndSetupWalProps(org.apache.hadoop.fs.Path oldPath, org.apache.hadoop.fs.Path newPath, long oldFileLen)
protected final void blockOnSync(org.apache.hadoop.hbase.regionserver.wal.SyncFuture syncFuture) throws IOException
IOException
public byte[][] rollWriter(boolean force) throws FailedLogCloseException, IOException
WAL
The implementation is synchronized in order to make sure there's one rollWriter running at any given time.
rollWriter
in interface WAL
force
- If true, force creation of a new writer even if no entries have
been written to the current writerRegionInfo.getEncodedName()
FailedLogCloseException
IOException
public long getLogFileSize()
public void requestLogRoll()
public void shutdown() throws IOException
WAL
shutdown
in interface WAL
IOException
public void close() throws IOException
WAL
close
in interface Closeable
close
in interface AutoCloseable
close
in interface WAL
IOException
public void updateStore(byte[] encodedRegionName, byte[] familyName, Long sequenceid, boolean onlyIfGreater)
updateStore
in interface WAL
encodedRegionName
- familyName
- sequenceid
- onlyIfGreater
- protected final org.apache.hadoop.hbase.regionserver.wal.SyncFuture getSyncFuture(long sequence)
protected final void requestLogRoll(boolean tooFewReplicas)
protected void atHeadOfRingBufferEventHandlerAppend()
protected final boolean append(W writer, org.apache.hadoop.hbase.regionserver.wal.FSWALEntry entry) throws IOException
IOException
protected final void postSync(long timeInNanos, int handlerSyncs)
protected final long stampSequenceIdAndPublishToRingBuffer(RegionInfo hri, WALKeyImpl key, WALEdit edits, boolean inMemstore, com.lmax.disruptor.RingBuffer<org.apache.hadoop.hbase.regionserver.wal.RingBufferTruck> ringBuffer) throws IOException
IOException
public String toString()
WAL
public OptionalLong getLogFileSizeIfBeingWritten(org.apache.hadoop.fs.Path path)
path
is being written currently, then return its length.
This is used by replication to prevent replicating unacked log entries. See https://issues.apache.org/jira/browse/HBASE-14004 for more details.
getLogFileSizeIfBeingWritten
in interface WALFileLengthProvider
public abstract long append(RegionInfo info, WALKeyImpl key, WALEdit edits, boolean inMemstore) throws IOException
walKey
parameter. Be warned that the WriteEntry is not
immediately available on return from this method. It WILL be available subsequent to a sync of
this append; otherwise, you will just have to wait on the WriteEntry to get filled in.append
in interface WAL
info
- the regioninfo associated with appendkey
- Modified by this call; we add to it this edits region edit/sequence id.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.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.key
will have the region edit/sequence id
in it.IOException
protected abstract void doAppend(W writer, org.apache.hadoop.hbase.regionserver.wal.FSWALEntry entry) throws IOException
IOException
protected abstract W createWriterInstance(org.apache.hadoop.fs.Path path) throws IOException, CommonFSUtils.StreamLacksCapabilityException
protected abstract void doReplaceWriter(org.apache.hadoop.fs.Path oldPath, org.apache.hadoop.fs.Path newPath, W nextWriter) throws IOException
IOException
protected abstract void doShutdown() throws IOException
IOException
protected abstract boolean doCheckLogLowReplication()
public void checkLogLowReplication(long checkInterval)
public static void main(String[] args) throws IOException
stdout
or split the specified log files.IOException
Copyright © 2007–2019 Cloudera. All rights reserved.