@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)
. *
Modifier and Type | Class and Description |
---|---|
private static class |
AbstractFSWAL.WalProps |
WAL.Entry, WAL.Reader
Modifier and Type | Field and Description |
---|---|
protected Abortable |
abortable |
private int |
archiveRetries |
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_ROLL_ON_SYNC_TIME_MS |
protected static int |
DEFAULT_SLOW_SYNC_ROLL_INTERVAL_MS |
protected static int |
DEFAULT_SLOW_SYNC_ROLL_THRESHOLD |
protected static int |
DEFAULT_SLOW_SYNC_TIME_MS |
protected static int |
DEFAULT_WAL_SYNC_TIMEOUT_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 Map<String,W> |
inflightWALClosures
Tracks the logs in the process of being closed.
|
private long |
lastTimeCheckLowReplication |
private long |
lastTimeCheckSlowSync |
protected List<WALActionsListener> |
listeners
Listeners that are called on WAL events.
|
private static org.slf4j.Logger |
LOG |
(package private) Comparator<org.apache.hadoop.fs.Path> |
LOG_NAME_COMPARATOR
WAL Comparator; it compares the timestamp (log filenum), present in the log file name.
|
private ExecutorService |
logArchiveExecutor |
protected long |
logrollsize |
static String |
MAX_LOGS |
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.
|
static String |
RING_BUFFER_SLOT_COUNT |
protected static String |
ROLL_ON_SYNC_TIME_MS |
protected long |
rollOnSyncNs |
protected AtomicBoolean |
rollRequested |
protected ReentrantLock |
rollWriterLock
This lock makes sure only one log roll runs at a time.
|
protected SequenceIdAccounting |
sequenceIdAccounting
Class that does accounting of sequenceids in WAL subsystem.
|
protected AtomicBoolean |
shutdown |
protected static String |
SLOW_SYNC_ROLL_INTERVAL_MS |
protected static String |
SLOW_SYNC_ROLL_THRESHOLD |
protected static String |
SLOW_SYNC_TIME_MS |
protected int |
slowSyncCheckInterval |
protected AtomicInteger |
slowSyncCount |
protected long |
slowSyncNs |
protected int |
slowSyncRollThreshold |
protected SyncFutureCache |
syncFutureCache
A cache of sync futures reused by threads.
|
protected AtomicLong |
totalLogSize
The total size of wal
|
protected boolean |
useHsync |
static String |
WAL_ROLL_MULTIPLIER |
protected static String |
WAL_SYNC_TIMEOUT_MS |
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,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
|
private long |
walSyncTimeoutNs |
(package private) W |
writer
Current log file.
|
Modifier | Constructor and Description |
---|---|
protected |
AbstractFSWAL(org.apache.hadoop.fs.FileSystem fs,
Abortable abortable,
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) |
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.
|
protected abstract long |
append(RegionInfo info,
WALKeyImpl key,
WALEdit edits,
boolean inMemstore)
Append a set of edits to the WAL.
|
long |
appendData(RegionInfo info,
WALKeyImpl key,
WALEdit edits)
Append a set of data edits to the WAL.
|
protected boolean |
appendEntry(W writer,
FSWALEntry entry) |
long |
appendMarker(RegionInfo info,
WALKeyImpl key,
WALEdit edits)
Append an operational 'meta' event marker edit to the WAL.
|
protected void |
archive(Pair<org.apache.hadoop.fs.Path,Long> log) |
protected void |
archiveLogFile(org.apache.hadoop.fs.Path p) |
protected void |
atHeadOfRingBufferEventHandlerAppend()
Exposed for testing only.
|
protected void |
blockOnSync(SyncFuture syncFuture) |
private int |
calculateMaxLogFiles(org.apache.hadoop.conf.Configuration conf,
long logRollSize) |
void |
checkLogLowReplication(long checkInterval) |
private void |
cleanOldLogs()
Archive old logs.
|
void |
close()
Caller no longer needs any edits from this WAL.
|
void |
completeCacheFlush(byte[] encodedRegionName,
long maxFlushedSeqId)
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.
|
private IOException |
convertInterruptedExceptionToIOException(InterruptedException ie) |
private io.opentelemetry.api.trace.Span |
createSpan(String name) |
protected abstract W |
createWriterInstance(org.apache.hadoop.fs.Path path) |
protected abstract void |
doAppend(W writer,
FSWALEntry entry) |
protected abstract boolean |
doCheckLogLowReplication() |
protected boolean |
doCheckSlowSync()
Returns true if we exceeded the slow sync roll threshold over the last check interval
|
protected abstract void |
doReplaceWriter(org.apache.hadoop.fs.Path oldPath,
org.apache.hadoop.fs.Path newPath,
W nextWriter)
Notice that you need to clear the
rollRequested flag in this method, as the new writer
will begin to work before returning from this method. |
protected abstract void |
doShutdown() |
protected abstract void |
doSync(boolean forceSync) |
protected abstract void |
doSync(long txid,
boolean forceSync) |
private static IOException |
ensureIOException(Throwable t) |
(package private) Map<byte[],List<byte[]>> |
findRegionsToForceFlush()
If the number of un-archived WAL files ('live' WALs) is greater than maximum allowed, check the
first (oldest) WAL, and return those regions which should be flushed so that it can be
let-go/'archived'.
|
WALCoprocessorHost |
getCoprocessorHost()
Returns Coprocessor host.
|
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 n
|
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 . |
(package private) org.apache.hadoop.fs.FileStatus[] |
getFiles()
Get the backing files associated with this WAL.
|
int |
getInflightWALCloseCount()
Returns number of WALs currently in the process of closing.
|
long |
getLogFileSize()
Returns the size of log files in use
|
OptionalLong |
getLogFileSizeIfBeingWritten(org.apache.hadoop.fs.Path path)
if the given
path is being written currently, then return its length. |
(package private) abstract int |
getLogReplication()
This method gets the datanode replication count for the current WAL.
|
private org.apache.hadoop.fs.Path |
getNewPath()
retrieve the next path to use for writing.
|
int |
getNumLogFiles()
Returns the number of log files in use
|
int |
getNumRolledLogFiles()
Returns the number of rolled log files
|
(package private) org.apache.hadoop.fs.Path |
getOldPath() |
(package private) abstract org.apache.hadoop.hdfs.protocol.DatanodeInfo[] |
getPipeline()
This method gets the pipeline for the current WAL.
|
protected int |
getPreallocatedEventCount() |
protected SyncFuture |
getSyncFuture(long sequence,
boolean forceSync) |
(package private) long |
getUnflushedEntriesCount() |
static org.apache.hadoop.fs.Path |
getWALArchivePath(org.apache.hadoop.fs.Path archiveDir,
org.apache.hadoop.fs.Path p) |
void |
init()
Used to initialize the WAL.
|
protected boolean |
isLogRollRequested() |
(package private) boolean |
isUnflushedEntries() |
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. |
private long |
postAppend(WAL.Entry e,
long elapsedTime) |
protected void |
postSync(long timeInNanos,
int handlerSyncs) |
void |
registerWALActionsListener(WALActionsListener listener)
Registers WALActionsListener
|
(package private) org.apache.hadoop.fs.Path |
replaceWriter(org.apache.hadoop.fs.Path oldPath,
org.apache.hadoop.fs.Path newPath,
W nextWriter)
Cleans up current writer closing it and then puts in place the passed in
nextWriter . |
void |
requestLogRoll() |
protected void |
requestLogRoll(WALActionsListener.RollRequestReason reason) |
Map<byte[],List<byte[]>> |
rollWriter()
Roll the log writer.
|
Map<byte[],List<byte[]>> |
rollWriter(boolean force)
Roll the log writer.
|
private Map<byte[],List<byte[]>> |
rollWriterInternal(boolean force) |
void |
shutdown()
Stop accepting new writes.
|
private static void |
split(org.apache.hadoop.conf.Configuration conf,
org.apache.hadoop.fs.Path p) |
protected long |
stampSequenceIdAndPublishToRingBuffer(RegionInfo hri,
WALKeyImpl key,
WALEdit edits,
boolean inMemstore,
com.lmax.disruptor.RingBuffer<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.
|
void |
sync()
Sync what we have in the WAL.
|
void |
sync(boolean forceSync) |
void |
sync(long txid)
Sync the WAL if the txId was not already sync'd.
|
void |
sync(long txid,
boolean forceSync) |
private void |
tellListenersAboutPostLogRoll(org.apache.hadoop.fs.Path oldPath,
org.apache.hadoop.fs.Path newPath)
Tell listeners about post log roll.
|
private void |
tellListenersAboutPreLogRoll(org.apache.hadoop.fs.Path oldPath,
org.apache.hadoop.fs.Path newPath)
Tell listeners about pre log roll.
|
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.
|
private static void |
usage() |
clone, equals, finalize, getClass, hashCode, notify, notifyAll, wait, wait, wait
getTimestamp
private static final org.slf4j.Logger LOG
protected static final String SLOW_SYNC_TIME_MS
protected static final int DEFAULT_SLOW_SYNC_TIME_MS
protected static final String ROLL_ON_SYNC_TIME_MS
protected static final int DEFAULT_ROLL_ON_SYNC_TIME_MS
protected static final String SLOW_SYNC_ROLL_THRESHOLD
protected static final int DEFAULT_SLOW_SYNC_ROLL_THRESHOLD
protected static final String SLOW_SYNC_ROLL_INTERVAL_MS
protected static final int DEFAULT_SLOW_SYNC_ROLL_INTERVAL_MS
protected static final String WAL_SYNC_TIMEOUT_MS
protected static final int DEFAULT_WAL_SYNC_TIMEOUT_MS
public static final String WAL_ROLL_MULTIPLIER
public static final String MAX_LOGS
public static final String RING_BUFFER_SLOT_COUNT
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 Map<String,W extends WALProvider.WriterBase> inflightWALClosures
protected final SequenceIdAccounting sequenceIdAccounting
protected final long slowSyncNs
protected final long rollOnSyncNs
protected final int slowSyncRollThreshold
protected final int slowSyncCheckInterval
protected final AtomicInteger slowSyncCount
private final long walSyncTimeoutNs
protected final long logrollsize
protected final long blocksize
protected final int maxLogs
protected final boolean useHsync
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
volatile W extends WALProvider.WriterBase writer
private volatile long lastTimeCheckLowReplication
private volatile long lastTimeCheckSlowSync
protected volatile boolean closed
protected final AtomicBoolean shutdown
final Comparator<org.apache.hadoop.fs.Path> LOG_NAME_COMPARATOR
protected ConcurrentNavigableMap<org.apache.hadoop.fs.Path,AbstractFSWAL.WalProps> walFile2Props
protected final SyncFutureCache syncFutureCache
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 final AtomicBoolean rollRequested
private final ExecutorService logArchiveExecutor
private final int archiveRetries
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
protected AbstractFSWAL(org.apache.hadoop.fs.FileSystem fs, Abortable abortable, 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.private int calculateMaxLogFiles(org.apache.hadoop.conf.Configuration conf, long logRollSize)
protected final int getPreallocatedEventCount()
public void init() throws IOException
IOException
public void registerWALActionsListener(WALActionsListener listener)
WAL
registerWALActionsListener
in interface WAL
public boolean unregisterWALActionsListener(WALActionsListener listener)
WAL
unregisterWALActionsListener
in interface WAL
public WALCoprocessorHost getCoprocessorHost()
WAL
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[], long)
,
WAL.abortCacheFlush(byte[])
public Long startCacheFlush(byte[] encodedRegionName, Map<byte[],Long> familyToSeq)
startCacheFlush
in interface WAL
public void completeCacheFlush(byte[] encodedRegionName, long maxFlushedSeqId)
WAL
completeCacheFlush
in interface WAL
encodedRegionName
- Encoded region name.maxFlushedSeqId
- The maxFlushedSeqId for this flush. There is no edit in memory that is
less that this sequence id.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 Map<byte[],List<byte[]>> rollWriter() throws FailedLogCloseException, IOException
WAL
rollWriter
in interface WAL
RegionInfo.getEncodedName()
FailedLogCloseException
IOException
public final void sync() throws IOException
WAL
sync
in interface WAL
IOException
public final void sync(long txid) throws IOException
WAL
sync
in interface WAL
txid
- Transaction id to sync to.IOException
public final void sync(boolean forceSync) throws IOException
sync
in interface WAL
forceSync
- Flag to force sync rather than flushing to the buffer. Example - Hadoop hflush
vs hsync.IOException
public final void sync(long txid, boolean forceSync) throws IOException
sync
in interface WAL
txid
- Transaction id to sync to.forceSync
- Flag to force sync rather than flushing to the buffer. Example - Hadoop hflush
vs hsync.IOException
protected abstract void doSync(boolean forceSync) throws IOException
IOException
protected abstract void doSync(long txid, boolean forceSync) throws IOException
IOException
protected org.apache.hadoop.fs.Path computeFilename(long filenum)
filenum
- to use npublic org.apache.hadoop.fs.Path getCurrentFileName()
private org.apache.hadoop.fs.Path getNewPath() throws IOException
IOException
org.apache.hadoop.fs.Path getOldPath()
private void tellListenersAboutPreLogRoll(org.apache.hadoop.fs.Path oldPath, org.apache.hadoop.fs.Path newPath) throws IOException
IOException
private void tellListenersAboutPostLogRoll(org.apache.hadoop.fs.Path oldPath, org.apache.hadoop.fs.Path newPath) throws IOException
IOException
public int getNumRolledLogFiles()
public int getNumLogFiles()
Map<byte[],List<byte[]>> findRegionsToForceFlush() throws IOException
IOException
private void cleanOldLogs() throws IOException
IOException
public static org.apache.hadoop.fs.Path getWALArchivePath(org.apache.hadoop.fs.Path archiveDir, org.apache.hadoop.fs.Path p)
protected void archiveLogFile(org.apache.hadoop.fs.Path p) throws IOException
IOException
protected final void logRollAndSetupWalProps(org.apache.hadoop.fs.Path oldPath, org.apache.hadoop.fs.Path newPath, long oldFileLen)
private io.opentelemetry.api.trace.Span createSpan(String name)
org.apache.hadoop.fs.Path replaceWriter(org.apache.hadoop.fs.Path oldPath, org.apache.hadoop.fs.Path newPath, W nextWriter) throws IOException
nextWriter
.
oldPath
- may be nullnewPath
- may be nullnextWriter
- may be nullnewPath
IOException
- if there is a problem flushing or closing the underlying FSprotected final void blockOnSync(SyncFuture syncFuture) throws IOException
IOException
private static IOException ensureIOException(Throwable t)
private IOException convertInterruptedExceptionToIOException(InterruptedException ie)
private Map<byte[],List<byte[]>> rollWriterInternal(boolean force) throws IOException
IOException
public Map<byte[],List<byte[]>> rollWriter(boolean force) throws IOException
WAL
rollWriter
in interface WAL
RegionInfo.getEncodedName()
IOException
public long getLogFileSize()
public void requestLogRoll()
org.apache.hadoop.fs.FileStatus[] getFiles() throws IOException
IOException
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 int getInflightWALCloseCount()
public void updateStore(byte[] encodedRegionName, byte[] familyName, Long sequenceid, boolean onlyIfGreater)
updateStore
in interface WAL
protected final SyncFuture getSyncFuture(long sequence, boolean forceSync)
protected boolean isLogRollRequested()
protected final void requestLogRoll(WALActionsListener.RollRequestReason reason)
long getUnflushedEntriesCount()
boolean isUnflushedEntries()
protected void atHeadOfRingBufferEventHandlerAppend()
protected final boolean appendEntry(W writer, FSWALEntry entry) throws IOException
IOException
private long postAppend(WAL.Entry e, long elapsedTime) 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<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 long appendData(RegionInfo info, WALKeyImpl key, WALEdit edits) throws IOException
WAL
key
will have the region edit/sequence id filled in.appendData
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.key
will have the region edit/sequence id
in it.IOException
WAL.appendMarker(RegionInfo, WALKeyImpl, WALEdit)
public long appendMarker(RegionInfo info, WALKeyImpl key, WALEdit edits) throws IOException
WAL
WAL.appendData(RegionInfo, WALKeyImpl, WALEdit)
is that a marker will not have transitioned
through the memstore.
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 key
will have the region edit/sequence id filled in.appendMarker
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.key
will have the region edit/sequence id
in it.IOException
WAL.appendData(RegionInfo, WALKeyImpl, WALEdit)
protected abstract long append(RegionInfo info, WALKeyImpl key, WALEdit edits, boolean inMemstore) throws IOException
key
will have the region edit/sequence id filled in.
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. At assignment
time, we stamp all the passed in Cells inside WALEdit with their sequenceId. You must
'complete' the transaction this mvcc transaction by calling
MultiVersionConcurrencyControl#complete(...) or a variant otherwise mvcc will get stuck. Do it
in the finally of a try/finally block within which this append lives and any subsequent
operations like sync or update of memstore, etc. Get the WriteEntry to pass mvcc out of the
passed in WALKey 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.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 region event meta marker
edit, for example, a compaction completion record into the WAL or noting a
Region Open event. In these cases the entry is just so we can finish an
unfinished compaction after a crash when the new Server reads the WAL on
recovery, etc. These transition event 'Markers' do not go via the memstore.
When memstore is false, we presume a Marker event edit.key
will have the region edit/sequence id
in it.IOException
protected abstract void doAppend(W writer, 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
rollRequested
flag in this method, as the new writer
will begin to work before returning from this method. If we clear the flag after returning from
this call, we may miss a roll request. The implementation class should choose a proper place to
clear the rollRequested
flag so we do not miss a roll request, typically before you
start writing to the new writer.IOException
protected abstract void doShutdown() throws IOException
IOException
protected abstract boolean doCheckLogLowReplication()
protected boolean doCheckSlowSync()
public void checkLogLowReplication(long checkInterval)
abstract org.apache.hadoop.hdfs.protocol.DatanodeInfo[] getPipeline()
abstract int getLogReplication()
private static void split(org.apache.hadoop.conf.Configuration conf, org.apache.hadoop.fs.Path p) throws IOException
IOException
private static void usage()
public static void main(String[] args) throws IOException
stdout
or split the specified log files.IOException
Copyright © 2007–2020 The Apache Software Foundation. All rights reserved.