@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 long |
blocksize
Block size to use writing files.
|
private ThreadLocal<SyncFuture> |
cachedSyncFutures
Map of
SyncFutures owned by Thread objects. |
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.
|
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.
|
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 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,
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.
|
private 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)
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) |
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() |
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() |
private static IOException |
ensureIOException(Throwable t) |
(package private) 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() |
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. |
(package private) org.apache.hadoop.fs.FileStatus[] |
getFiles()
Get the backing files associated with this WAL.
|
long |
getLogFileSize() |
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() |
int |
getNumRolledLogFiles() |
(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) |
byte[][] |
rollWriter()
Roll the log writer.
|
byte[][] |
rollWriter(boolean force)
Roll the log writer.
|
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.
|
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() |
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 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
private final ThreadLocal<SyncFuture> cachedSyncFutures
SyncFutures owned by Thread objects. Used so we reuse SyncFutures.
Thread local is used so JVM can GC the terminated thread for us. See HBASE-21228
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
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
FailedLogCloseExceptionIOExceptionpublic 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
IOExceptionpublic void registerWALActionsListener(WALActionsListener listener)
WALregisterWALActionsListener in interface WALpublic boolean unregisterWALActionsListener(WALActionsListener listener)
WALunregisterWALActionsListener in interface WALpublic WALCoprocessorHost getCoprocessorHost()
getCoprocessorHost in interface WALpublic Long startCacheFlush(byte[] encodedRegionName, Set<byte[]> families)
WALCurrently, 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 WALfamilies - 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 WALpublic void completeCacheFlush(byte[] encodedRegionName)
WALcompleteCacheFlush in interface WALencodedRegionName - Encoded region name.WAL.startCacheFlush(byte[], Set),
WAL.abortCacheFlush(byte[])public void abortCacheFlush(byte[] encodedRegionName)
WALabortCacheFlush in interface WALencodedRegionName - Encoded region name.public long getEarliestMemStoreSeqNum(byte[] encodedRegionName)
WALgetEarliestMemStoreSeqNum in interface WALencodedRegionName - The region to get the number for.public long getEarliestMemStoreSeqNum(byte[] encodedRegionName, byte[] familyName)
WALgetEarliestMemStoreSeqNum in interface WALencodedRegionName - The region to get the number for.familyName - The family to get the number for.public byte[][] rollWriter() throws FailedLogCloseException, IOException
WALrollWriter in interface WALRegionInfo.getEncodedName()FailedLogCloseExceptionIOExceptionprotected org.apache.hadoop.fs.Path computeFilename(long filenum)
filenum - to usepublic org.apache.hadoop.fs.Path getCurrentFileName()
private org.apache.hadoop.fs.Path getNewPath() throws IOException
IOExceptionorg.apache.hadoop.fs.Path getOldPath()
private void tellListenersAboutPreLogRoll(org.apache.hadoop.fs.Path oldPath, org.apache.hadoop.fs.Path newPath) throws IOException
IOExceptionprivate void tellListenersAboutPostLogRoll(org.apache.hadoop.fs.Path oldPath, org.apache.hadoop.fs.Path newPath) throws IOException
IOExceptionpublic int getNumRolledLogFiles()
public int getNumLogFiles()
byte[][] findRegionsToForceFlush() throws IOException
IOExceptionprivate void cleanOldLogs() throws IOException
IOExceptionpublic static org.apache.hadoop.fs.Path getWALArchivePath(org.apache.hadoop.fs.Path archiveDir, org.apache.hadoop.fs.Path p)
private void archiveLogFile(org.apache.hadoop.fs.Path p) throws IOException
IOExceptionprotected final void logRollAndSetupWalProps(org.apache.hadoop.fs.Path oldPath, org.apache.hadoop.fs.Path newPath, long oldFileLen)
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 nullnewPathIOException - if there is a problem flushing or closing the underlying FSprotected final void blockOnSync(SyncFuture syncFuture) throws IOException
IOExceptionprivate static IOException ensureIOException(Throwable t)
private IOException convertInterruptedExceptionToIOException(InterruptedException ie)
public byte[][] rollWriter(boolean force) throws FailedLogCloseException, IOException
WALrollWriter in interface WALforce - If true, force creation of a new writer even if no entries have
been written to the current writerRegionInfo.getEncodedName()FailedLogCloseExceptionIOExceptionpublic long getLogFileSize()
public void requestLogRoll()
org.apache.hadoop.fs.FileStatus[] getFiles() throws IOException
IOExceptionpublic void shutdown() throws IOException
WALshutdown in interface WALIOExceptionpublic void close() throws IOException
WALclose in interface Closeableclose in interface AutoCloseableclose in interface WALIOExceptionpublic void updateStore(byte[] encodedRegionName, byte[] familyName, Long sequenceid, boolean onlyIfGreater)
updateStore in interface WALprotected 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
IOExceptionprivate long postAppend(WAL.Entry e, long elapsedTime) throws IOException
IOExceptionprotected 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
IOExceptionpublic String toString()
WALpublic 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 WALFileLengthProviderpublic long appendData(RegionInfo info, WALKeyImpl key, WALEdit edits) throws IOException
WALkey will have the region edit/sequence id filled in.appendData in interface WALinfo - 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.IOExceptionWAL.appendMarker(RegionInfo, WALKeyImpl, WALEdit)public long appendMarker(RegionInfo info, WALKeyImpl key, WALEdit edits) throws IOException
WALWAL.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 WALinfo - 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.IOExceptionWAL.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.IOExceptionprotected abstract void doAppend(W writer, FSWALEntry entry) throws IOException
IOExceptionprotected 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.IOExceptionprotected abstract void doShutdown() throws IOException
IOExceptionprotected 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
IOExceptionprivate static void usage()
public static void main(String[] args) throws IOException
stdout or split the specified log files.IOExceptionCopyright © 2007–2020 The Apache Software Foundation. All rights reserved.