From 74b5a394f4faa2c8536e66b7f15b6d942ae345ee Mon Sep 17 00:00:00 2001 From: Michael Stack Date: Sun, 26 Jan 2014 04:41:39 +0000 Subject: [PATCH] HBASE-10156 FSHLog Refactor (WAS -> Fix up the HBASE-8755 slowdown when low contention) git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1561450 13f79535-47bb-0310-9956-ffa450edef68 --- .../apache/hadoop/hbase/HTableDescriptor.java | 4 +- .../wal/FailedLogCloseException.java | 19 +- .../FailedSyncBeforeLogCloseException.java | 51 + hbase-server/pom.xml | 6 +- .../hadoop/hbase/regionserver/wal/FSHLog.java | 2001 ++++++++++------- .../hbase/regionserver/wal/FSWALEntry.java | 80 + .../hadoop/hbase/regionserver/wal/HLog.java | 103 +- .../hbase/regionserver/wal/HLogFactory.java | 4 +- .../hbase/regionserver/wal/HLogKey.java | 21 +- .../hbase/regionserver/wal/HLogUtil.java | 8 +- .../regionserver/wal/ProtobufLogWriter.java | 7 +- .../regionserver/wal/RingBufferTruck.java | 77 + .../hbase/regionserver/wal/SyncFuture.java | 152 ++ .../regionserver/wal/WALCoprocessorHost.java | 2 + .../hbase/regionserver/wal/WALEdit.java | 8 +- .../hbase/mapreduce/TestHLogRecordReader.java | 10 + .../master/TestDistributedLogSplitting.java | 2 +- .../hbase/regionserver/TestParallelPut.java | 49 +- .../wal/HLogPerformanceEvaluation.java | 119 +- .../regionserver/wal/TestDurability.java | 4 +- .../hbase/regionserver/wal/TestHLog.java | 10 +- .../hbase/regionserver/wal/TestHLogSplit.java | 2 +- .../regionserver/wal/TestLogRollAbort.java | 75 +- .../regionserver/wal/TestLogRolling.java | 31 +- .../wal/TestLogRollingNoCluster.java | 14 +- pom.xml | 6 + 26 files changed, 1813 insertions(+), 1052 deletions(-) create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FailedSyncBeforeLogCloseException.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/RingBufferTruck.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SyncFuture.java diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java index a7fbb59949c..8bd2d13adc8 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java @@ -600,8 +600,8 @@ public class HTableDescriptor implements WritableComparable { * Check if async log edits are enabled on the table. * * @return true if that async log flush is enabled on the table - * - * @see #setAsyncLogFlush(boolean) + * @deprecated Since 0.96 we no longer have an explicity deferred log flush/sync functionality. + * Use {@link #getDurability()}. */ public synchronized boolean isAsyncLogFlush() { return getDurability() == Durability.ASYNC_WAL; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FailedLogCloseException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FailedLogCloseException.java index 22a89a0eb11..26b68f385d6 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FailedLogCloseException.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FailedLogCloseException.java @@ -32,17 +32,22 @@ import java.io.IOException; public class FailedLogCloseException extends IOException { private static final long serialVersionUID = 1759152841462990925L; - /** - * - */ public FailedLogCloseException() { super(); } /** - * @param arg0 + * @param msg */ - public FailedLogCloseException(String arg0) { - super(arg0); + public FailedLogCloseException(String msg) { + super(msg); } -} + + public FailedLogCloseException(final String msg, final Throwable t) { + super(msg, t); + } + + public FailedLogCloseException(final Throwable t) { + super(t); + } +} \ No newline at end of file diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FailedSyncBeforeLogCloseException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FailedSyncBeforeLogCloseException.java new file mode 100644 index 00000000000..997456e4aed --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FailedSyncBeforeLogCloseException.java @@ -0,0 +1,51 @@ +/** + * + * 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.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Thrown when we fail close of the write-ahead-log file. + * Package private. Only used inside this package. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public class FailedSyncBeforeLogCloseException extends FailedLogCloseException { + private static final long serialVersionUID = 1759152841462990925L; + + public FailedSyncBeforeLogCloseException() { + super(); + } + + /** + * @param msg + */ + public FailedSyncBeforeLogCloseException(String msg) { + super(msg); + } + + public FailedSyncBeforeLogCloseException(final String msg, final Throwable t) { + super(msg, t); + } + + public FailedSyncBeforeLogCloseException(final Throwable t) { + super(t); + } +} \ No newline at end of file diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml index f5723b25c5a..ea5660cd3f8 100644 --- a/hbase-server/pom.xml +++ b/hbase-server/pom.xml @@ -446,7 +446,11 @@ org.cloudera.htrace htrace-core - + + + com.lmax + disruptor + diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java index 190bb4ef40a..c479f16810a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java @@ -1,5 +1,4 @@ /** - * * 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 @@ -20,6 +19,7 @@ package org.apache.hadoop.hbase.regionserver.wal; import java.io.FileNotFoundException; import java.io.IOException; +import java.io.InterruptedIOException; import java.io.OutputStream; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; @@ -28,15 +28,21 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Comparator; import java.util.HashMap; -import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.NavigableMap; import java.util.TreeMap; import java.util.UUID; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.ReentrantLock; @@ -62,157 +68,188 @@ import org.apache.hadoop.hbase.util.DrainBarrier; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.HasThread; +import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.util.StringUtils; import org.cloudera.htrace.Trace; import org.cloudera.htrace.TraceScope; import com.google.common.annotations.VisibleForTesting; +import com.lmax.disruptor.BlockingWaitStrategy; +import com.lmax.disruptor.EventHandler; +import com.lmax.disruptor.ExceptionHandler; +import com.lmax.disruptor.LifecycleAware; +import com.lmax.disruptor.TimeoutException; +import com.lmax.disruptor.dsl.Disruptor; +import com.lmax.disruptor.dsl.ProducerType; /** - * HLog stores all the edits to the HStore. Its the hbase write-ahead-log - * implementation. + * Implementation of {@link HLog} to go against {@link FileSystem}; i.e. keep WALs in HDFS. + * Only one HLog/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, so external + * callers do not have to be concerned with log rolling. * - * It performs logfile-rolling, so external callers are not aware that the - * underlying file is being rolled. + *

As data is flushed from the MemStore to other (better) 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-id up to the most-recent CACHEFLUSH message from that HRegion. 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. * - *

- * There is one HLog per RegionServer. All edits for all Regions carried by - * a particular RegionServer are entered first in the HLog. - * - *

- * Each HRegion is identified by a unique long int. HRegions do - * not need to declare themselves before using the HLog; they simply include - * their HRegion-id in the append or - * completeCacheFlush calls. - * - *

- * An HLog consists of multiple on-disk files, which have a chronological order. - * As data is flushed to other (better) on-disk structures, the log becomes - * obsolete. We can destroy all the log messages for a given HRegion-id up to - * the most-recent CACHEFLUSH message from that HRegion. - * - *

- * It's only practical to delete entire files. Thus, we delete an entire on-disk - * file F when all of the messages in F have a log-sequence-id that's older - * (smaller) than the most-recent CACHEFLUSH message for every HRegion that has - * a message in F. - * - *

- * Synchronized methods can never execute in parallel. However, between the - * start of a cache flush and the completion point, appends are allowed but log - * rolling is not. To prevent log rolling taking place during this period, a - * separate reentrant lock is used. + *

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 CACHEFLUSH message for every HRegion that has an edit in + * F. * *

To read an HLog, call {@link HLogFactory#createReader(org.apache.hadoop.fs.FileSystem, * org.apache.hadoop.fs.Path, org.apache.hadoop.conf.Configuration)}. - * */ @InterfaceAudience.Private class FSHLog implements HLog, Syncable { + // IMPLEMENTATION NOTES: + // + // At the core is a ring buffer. Our ring buffer is the LMAX Disruptor. It tries to + // minimize synchronizations and volatile writes when multiple contending threads as is the case + // here appending and syncing on a single WAL. The Disruptor is configured to handle multiple + // producers but it has one consumer only (the producers in HBase are IPC Handlers calling append + // and then sync). The single consumer/writer pulls the appends and syncs off the ring buffer. + // The appends are added to the WAL immediately without pause or batching (there may be a slight + // benefit batching appends but it complicates the implementation -- the gain is not worth + // the added complication). When a producer calls sync, it is given back a future. The producer + // 'blocks' on the future so it does not return until the sync completes. The future is passed + // over the ring buffer from the producer to the consumer thread where it does its best to batch + // up the producer syncs so one WAL sync actually spans multiple producer sync invocations. How + // well the batching works depends on the write rate; i.e. we tend to batch more in times of + // high writes/syncs. + // + //

The consumer thread pass the syncs off to muliple syncing threads in a round robin fashion + // to ensure we keep up back-to-back FS sync calls (FS sync calls are the long poll writing the + // WAL). The consumer thread passes the futures to the sync threads for it to complete + // the futures when done. + // + //

The 'sequence' in the below is the sequence of the append/sync on the ringbuffer. It + // acts as a sort-of transaction id. It is always incrementing. + // + //

The RingBufferEventHandler class hosts the ring buffer consuming code. The threads that + // do the actual FS sync are implementations of SyncRunner. SafePointZigZagLatch is a + // synchronization class used to halt the consumer at a safe point -- just after all outstanding + // syncs and appends have completed -- so the log roller can swap the WAL out under it. + static final Log LOG = LogFactory.getLog(FSHLog.class); - private final FileSystem fs; - private final Path rootDir; - private final Path dir; - private final Configuration conf; - // Listeners that are called on WAL events. - private List listeners = - new CopyOnWriteArrayList(); - private final long blocksize; - private final String prefix; - private final AtomicLong unflushedEntries = new AtomicLong(0); - private final AtomicLong syncedTillHere = new AtomicLong(0); - private long lastUnSyncedTxid; - private final Path oldLogDir; + /** + * Disruptor is a fancy ring buffer. This disruptor/ring buffer is used to take edits and sync + * calls from the Handlers and passes them to the append and sync executors with minimal + * contention. + */ + private final Disruptor disruptor; - // all writes pending on AsyncWriter/AsyncSyncer thread with - // txid <= failedTxid will fail by throwing asyncIOE - private final AtomicLong failedTxid = new AtomicLong(0); - private volatile IOException asyncIOE = null; + /** + * An executorservice that runs the AppendEventHandler append executor. + */ + private final ExecutorService appendExecutor; + + /** + * This fellow is run by the above appendExecutor service but it is all about batching up appends + * and syncs; it may shutdown without cleaning out the last few appends or syncs. To guard + * against this, keep a reference to this handler and do explicit close on way out to make sure + * all flushed out before we exit. + */ + private final RingBufferEventHandler ringBufferEventHandler; + + /** + * Map of {@link SyncFuture}s keyed by Handler objects. Used so we reuse SyncFutures. + */ + private final Map syncFuturesByHandler; + + private final FileSystem fs; + private final Path fullPathLogDir; + private final Path fullPathOldLogDir; + private final Configuration conf; + private final String logFilePrefix; + + /** + * The highest known outstanding unsync'd WALEdit sequence number where sequence number is the + * ring buffer sequence. + */ + private volatile long highestUnsyncedSequence = -1; + + /** + * Updated to the ring buffer sequence of the last successful sync call. This can be less than + * {@link #highestUnsyncedSequence} for case where we have an append where a sync has not yet + * come in for it. + */ + private final AtomicLong highestSyncedSequence = new AtomicLong(0); private WALCoprocessorHost coprocessorHost; - private FSDataOutputStream hdfs_out; // FSDataOutputStream associated with the current SequenceFile.writer - // Minimum tolerable replicas, if the actual value is lower than it, - // rollWriter will be triggered - private int minTolerableReplication; - private Method getNumCurrentReplicas; // refers to DFSOutputStream.getNumCurrentReplicas - final static Object [] NO_ARGS = new Object []{}; - - /** The barrier used to ensure that close() waits for all log rolls and flushes to finish. */ - private DrainBarrier closeBarrier = new DrainBarrier(); - /** - * Current log file. + * FSDataOutputStream associated with the current SequenceFile.writer */ - Writer writer; + private FSDataOutputStream hdfs_out; - /** - * This lock synchronizes all operations on oldestUnflushedSeqNums and oldestFlushingSeqNums, - * with the exception of append's putIfAbsent into oldestUnflushedSeqNums. - * We only use these to find out the low bound seqNum, or to find regions with old seqNums to - * force flush them, so we don't care about these numbers messing with anything. */ - private final Object oldestSeqNumsLock = new Object(); - - /** - * This lock makes sure only one log roll runs at the same time. Should not be taken while - * any other lock is held. We don't just use synchronized because that results in bogus and - * tedious findbugs warning when it thinks synchronized controls writer thread safety */ - private final ReentrantLock rollWriterLock = new ReentrantLock(true); - - /** - * Map of encoded region names to their most recent sequence/edit id in their memstore. - */ - private final ConcurrentSkipListMap oldestUnflushedSeqNums = - new ConcurrentSkipListMap(Bytes.BYTES_COMPARATOR); - /** - * Map of encoded region names to their most recent sequence/edit id in their memstore; - * contains the regions that are currently flushing. That way we can store two numbers for - * flushing and non-flushing (oldestUnflushedSeqNums) memstore for the same region. - */ - private final Map oldestFlushingSeqNums = - new TreeMap(Bytes.BYTES_COMPARATOR); - - private volatile boolean closed = false; - - private boolean forMeta = false; - - // The timestamp (in ms) when the log file was created. - private volatile long filenum = -1; - - //number of transactions in the current Hlog. - private final AtomicInteger numEntries = new AtomicInteger(0); + // All about log rolling if not enough replicas outstanding. + // Minimum tolerable replicas, if the actual value is lower than it, rollWriter will be triggered + private final int minTolerableReplication; + // DFSOutputStream.getNumCurrentReplicas method instance gotten via reflection. + private final Method getNumCurrentReplicas; + private final static Object [] NO_ARGS = new Object []{}; // If live datanode count is lower than the default replicas value, // RollWriter will be triggered in each sync(So the RollWriter will be // triggered one by one in a short time). Using it as a workaround to slow // down the roll frequency triggered by checkLowReplication(). - private AtomicInteger consecutiveLogRolls = new AtomicInteger(0); + private final AtomicInteger consecutiveLogRolls = new AtomicInteger(0); private final int lowReplicationRollLimit; - // If consecutiveLogRolls is larger than lowReplicationRollLimit, // then disable the rolling in checkLowReplication(). // Enable it if the replications recover. private volatile boolean lowReplicationRollEnabled = true; - // If > than this size, roll the log. This is typically 0.95 times the size - // of the default Hdfs block size. + /** + * Current log file. + */ + volatile Writer writer; + + /** The barrier used to ensure that close() waits for all log rolls and flushes to finish. */ + private final DrainBarrier closeBarrier = new DrainBarrier(); + + /** + * This lock makes sure only one log roll runs at a time. Should not be taken while any other + * lock is held. We don't just use synchronized because that results in bogus and tedious + * findbugs warning when it thinks synchronized controls writer thread safety. It is held when + * we are actually rolling the log. It is checked when we are looking to see if we should roll + * the log or not. + */ + private final ReentrantLock rollWriterLock = new ReentrantLock(true); + + // Listeners that are called on WAL events. + private final List listeners = + new CopyOnWriteArrayList(); + + private volatile boolean closed = false; + + /** + * Set when this WAL is for meta only (we run a WAL for all regions except meta -- it has its + * own dedicated WAL). + */ + private final boolean forMeta; + + // The timestamp (in ms) when the log file was created. + private final AtomicLong filenum = new AtomicLong(-1); + + // Number of transactions in the current Hlog. + private final AtomicInteger numEntries = new AtomicInteger(0); + + // If > than this size, roll the log. private final long logrollsize; - - /** size of current log */ - private long curLogSize = 0; /** * The total size of hlog */ private AtomicLong totalLogSize = new AtomicLong(0); - - // We synchronize on updateLock to prevent updates and to prevent a log roll - // during an update - // locked during appends - private final Object updateLock = new Object(); - private final Object pendingWritesLock = new Object(); + /** + * If WAL is enabled. + */ private final boolean enabled; /* @@ -222,39 +259,57 @@ class FSHLog implements HLog, Syncable { */ private final int maxLogs; - // List of pending writes to the HLog. There corresponds to transactions - // that have not yet returned to the client. We keep them cached here - // instead of writing them to HDFS piecemeal. The goal is to increase - // the batchsize for writing-to-hdfs as well as sync-to-hdfs, so that - // we can get better system throughput. - private List pendingWrites = new LinkedList(); - - private final AsyncWriter asyncWriter; - // since AsyncSyncer takes much longer than other phase(add WALEdits to local - // buffer, write local buffer to HDFS, notify pending write handler threads), - // when a sync is ongoing, all other phase pend, we use multiple parallel - // AsyncSyncer threads to improve overall throughput. - private final AsyncSyncer[] asyncSyncers; - private final AsyncNotifier asyncNotifier; - /** Number of log close errors tolerated before we abort */ private final int closeErrorsTolerated; private final AtomicInteger closeErrorCount = new AtomicInteger(); private final MetricsWAL metrics; -/** - * Map of region encoded names to the latest sequence num obtained from them while appending - * WALEdits to the wal. We create one map for each WAL file at the time it is rolled. - *

- * When deciding whether to archive a WAL file, we compare the sequence IDs in this map to - * {@link #oldestFlushingSeqNums} and {@link #oldestUnflushedSeqNums}. - * See {@link FSHLog#areAllRegionsFlushed(Map, Map, Map)} for more info. - *

- * This map uses byte[] as the key, and uses reference equality. It works in our use case as we - * use {@link HRegionInfo#getEncodedNameAsBytes()} as keys. For a given region, it always returns - * the same array. - */ - private Map latestSequenceNums = new HashMap(); + + // Region sequence id accounting across flushes and for knowing when we can GC a WAL. These + // sequence id numbers are by region and unrelated to the ring buffer sequence number accounting + // done above in failedSequence, highest sequence, etc. + /** + * This lock ties all operations on oldestFlushingRegionSequenceIds and + * oldestFlushedRegionSequenceIds Maps with the exception of append's putIfAbsent call into + * oldestUnflushedSeqNums. We use these Maps to find out the low bound seqNum, or to find regions + * with old seqNums to force flush; we are interested in old stuff not the new additions + * (TODO: IS THIS SAFE? CHECK!). + */ + private final Object regionSequenceIdLock = new Object(); + + /** + * Map of encoded region names to their OLDEST -- i.e. their first, the longest-lived -- + * sequence id in memstore. Note that this sequenceid is the region sequence id. This is not + * related to the id we use above for {@link #highestSyncedSequence} and + * {@link #highestUnsyncedSequence} which is the sequence from the disruptor ring buffer, an + * internal detail. + */ + private final ConcurrentSkipListMap oldestUnflushedRegionSequenceIds = + new ConcurrentSkipListMap(Bytes.BYTES_COMPARATOR); + + /** + * Map of encoded region names to their lowest or OLDEST sequence/edit id in memstore currently + * being flushed out to hfiles. Entries are moved here from + * {@link #oldestUnflushedRegionSequenceIds} while the lock {@link #regionSequenceIdLock} is held + * (so movement between the Maps is atomic). This is not related to the id we use above for + * {@link #highestSyncedSequence} and {@link #highestUnsyncedSequence} which is the sequence from + * the disruptor ring buffer, an internal detail. + */ + private final Map lowestFlushingRegionSequenceIds = + new TreeMap(Bytes.BYTES_COMPARATOR); + + /** + * Map of region encoded names to the latest region sequence id. Updated on each append of + * WALEdits to the WAL. We create one map for each WAL file at the time it is rolled. + *

When deciding whether to archive a WAL file, we compare the sequence IDs in this map to + * {@link #lowestFlushingRegionSequenceIds} and {@link #oldestUnflushedRegionSequenceIds}. + * See {@link FSHLog#areAllRegionsFlushed(Map, Map, Map)} for more info. + *

+ * This map uses byte[] as the key, and uses reference equality. It works in our use case as we + * use {@link HRegionInfo#getEncodedNameAsBytes()} as keys. For a given region, it always returns + * the same array. + */ + private Map highestRegionSequenceIds = new HashMap(); /** * WAL Comparator; it compares the timestamp (log filenum), present in the log file name. @@ -270,12 +325,36 @@ class FSHLog implements HLog, Syncable { }; /** - * Map of log file to the latest sequence nums of all regions it has entries of. + * Map of wal log file to the latest sequence nums of all regions it has entries of. * The map is sorted by the log file creation timestamp (contained in the log file name). */ - private NavigableMap> hlogSequenceNums = + private NavigableMap> byWalRegionSequenceIds = new ConcurrentSkipListMap>(LOG_NAME_COMPARATOR); + /** + * Exception handler to pass the disruptor ringbuffer. Same as native implemenation only it + * logs using our logger instead of java native logger. + */ + static class RingBufferExceptionHandler implements ExceptionHandler { + @Override + public void handleEventException(Throwable ex, long sequence, Object event) { + LOG.error("Sequence=" + sequence + ", event=" + event, ex); + throw new RuntimeException(ex); + } + + @Override + public void handleOnStartException(Throwable ex) { + LOG.error(ex); + throw new RuntimeException(ex); + } + + @Override + public void handleOnShutdownException(Throwable ex) { + LOG.error(ex); + throw new RuntimeException(ex); + } + } + /** * Constructor. * @@ -286,10 +365,9 @@ class FSHLog implements HLog, Syncable { * @throws IOException */ public FSHLog(final FileSystem fs, final Path root, final String logDir, - final Configuration conf) + final Configuration conf) throws IOException { - this(fs, root, logDir, HConstants.HREGION_OLDLOGDIR_NAME, - conf, null, true, null, false); + this(fs, root, logDir, HConstants.HREGION_OLDLOGDIR_NAME, conf, null, true, null, false); } /** @@ -302,11 +380,10 @@ class FSHLog implements HLog, Syncable { * @param conf configuration to use * @throws IOException */ - public FSHLog(final FileSystem fs, final Path root, final String logDir, - final String oldLogDir, final Configuration conf) + public FSHLog(final FileSystem fs, final Path root, final String logDir, final String oldLogDir, + final Configuration conf) throws IOException { - this(fs, root, logDir, oldLogDir, - conf, null, true, null, false); + this(fs, root, logDir, oldLogDir, conf, null, true, null, false); } /** @@ -331,8 +408,8 @@ class FSHLog implements HLog, Syncable { public FSHLog(final FileSystem fs, final Path root, final String logDir, final Configuration conf, final List listeners, final String prefix) throws IOException { - this(fs, root, logDir, HConstants.HREGION_OLDLOGDIR_NAME, - conf, listeners, true, prefix, false); + this(fs, root, logDir, HConstants.HREGION_OLDLOGDIR_NAME, conf, listeners, true, prefix, + false); } /** @@ -343,7 +420,7 @@ class FSHLog implements HLog, Syncable { * HLog object is started up. * * @param fs filesystem handle - * @param root path to where logs and oldlogs + * @param rootDir path to where logs and oldlogs * @param logDir dir where hlogs are stored * @param oldLogDir dir where hlogs are archived * @param conf configuration to use @@ -357,116 +434,121 @@ class FSHLog implements HLog, Syncable { * @param forMeta if this hlog is meant for meta updates * @throws IOException */ - public FSHLog(final FileSystem fs, final Path root, final String logDir, + public FSHLog(final FileSystem fs, final Path rootDir, final String logDir, final String oldLogDir, final Configuration conf, final List listeners, final boolean failIfLogDirExists, final String prefix, boolean forMeta) throws IOException { super(); this.fs = fs; - this.rootDir = root; - this.dir = new Path(this.rootDir, logDir); - this.oldLogDir = new Path(this.rootDir, oldLogDir); + this.fullPathLogDir = new Path(rootDir, logDir); + this.fullPathOldLogDir = new Path(rootDir, oldLogDir); this.forMeta = forMeta; this.conf = conf; + // Register listeners. if (listeners != null) { for (WALActionsListener i: listeners) { registerWALActionsListener(i); } } - this.blocksize = this.conf.getLong("hbase.regionserver.hlog.blocksize", - FSUtils.getDefaultBlockSize(this.fs, this.dir)); - // Roll at 95% of block size. - float multi = conf.getFloat("hbase.regionserver.logroll.multiplier", 0.95f); - this.logrollsize = (long)(this.blocksize * multi); + // Get size to roll log at. Roll at 95% of HDFS block size so we avoid crossing HDFS blocks + // (it costs x'ing bocks) + long blocksize = this.conf.getLong("hbase.regionserver.hlog.blocksize", + FSUtils.getDefaultBlockSize(this.fs, this.fullPathLogDir)); + this.logrollsize = + (long)(blocksize * conf.getFloat("hbase.regionserver.logroll.multiplier", 0.95f)); this.maxLogs = conf.getInt("hbase.regionserver.maxlogs", 32); - this.minTolerableReplication = conf.getInt( - "hbase.regionserver.hlog.tolerable.lowreplication", - FSUtils.getDefaultReplication(fs, this.dir)); - this.lowReplicationRollLimit = conf.getInt( - "hbase.regionserver.hlog.lowreplication.rolllimit", 5); + this.minTolerableReplication = conf.getInt( "hbase.regionserver.hlog.tolerable.lowreplication", + FSUtils.getDefaultReplication(fs, this.fullPathLogDir)); + this.lowReplicationRollLimit = + conf.getInt("hbase.regionserver.hlog.lowreplication.rolllimit", 5); this.enabled = conf.getBoolean("hbase.regionserver.hlog.enabled", true); - this.closeErrorsTolerated = conf.getInt( - "hbase.regionserver.logroll.errors.tolerated", 0); - - - LOG.info("WAL/HLog configuration: blocksize=" + - StringUtils.byteDesc(this.blocksize) + - ", rollsize=" + StringUtils.byteDesc(this.logrollsize) + - ", enabled=" + this.enabled); + this.closeErrorsTolerated = conf.getInt("hbase.regionserver.logroll.errors.tolerated", 0); // If prefix is null||empty then just name it hlog - this.prefix = prefix == null || prefix.isEmpty() ? - "hlog" : URLEncoder.encode(prefix, "UTF8"); + this.logFilePrefix = + prefix == null || prefix.isEmpty() ? "hlog" : URLEncoder.encode(prefix, "UTF8"); + int maxHandlersCount = conf.getInt(HConstants.REGION_SERVER_HANDLER_COUNT, 200); + + LOG.info("WAL configuration: blocksize=" + StringUtils.byteDesc(blocksize) + + ", rollsize=" + StringUtils.byteDesc(this.logrollsize) + + ", enabled=" + this.enabled + ", prefix=" + this.logFilePrefix + ", logDir=" + + this.fullPathLogDir + ", oldLogDir=" + this.fullPathOldLogDir); boolean dirExists = false; - if (failIfLogDirExists && (dirExists = this.fs.exists(dir))) { - throw new IOException("Target HLog directory already exists: " + dir); + if (failIfLogDirExists && (dirExists = this.fs.exists(fullPathLogDir))) { + throw new IOException("Target HLog directory already exists: " + fullPathLogDir); } - if (!dirExists && !fs.mkdirs(dir)) { - throw new IOException("Unable to mkdir " + dir); + if (!dirExists && !fs.mkdirs(fullPathLogDir)) { + throw new IOException("Unable to mkdir " + fullPathLogDir); } - if (!fs.exists(this.oldLogDir)) { - if (!fs.mkdirs(this.oldLogDir)) { - throw new IOException("Unable to mkdir " + this.oldLogDir); + if (!fs.exists(this.fullPathOldLogDir)) { + if (!fs.mkdirs(this.fullPathOldLogDir)) { + throw new IOException("Unable to mkdir " + this.fullPathOldLogDir); } } + // rollWriter sets this.hdfs_out if it can. rollWriter(); // handle the reflection necessary to call getNumCurrentReplicas() this.getNumCurrentReplicas = getGetNumCurrentReplicas(this.hdfs_out); - final String n = Thread.currentThread().getName(); - - - asyncWriter = new AsyncWriter(n + "-WAL.AsyncWriter"); - asyncWriter.start(); - - int syncerNums = conf.getInt("hbase.hlog.asyncer.number", 5); - asyncSyncers = new AsyncSyncer[syncerNums]; - for (int i = 0; i < asyncSyncers.length; ++i) { - asyncSyncers[i] = new AsyncSyncer(n + "-WAL.AsyncSyncer" + i); - asyncSyncers[i].start(); - } - - asyncNotifier = new AsyncNotifier(n + "-WAL.AsyncNotifier"); - asyncNotifier.start(); - - coprocessorHost = new WALCoprocessorHost(this, conf); - + this.coprocessorHost = new WALCoprocessorHost(this, conf); this.metrics = new MetricsWAL(); + + // This is the 'writer' -- a single threaded executor. This single thread 'consumes' what is + // put on the ring buffer. + String hostingThreadName = Thread.currentThread().getName(); + this.appendExecutor = Executors. + newSingleThreadExecutor(Threads.getNamedThreadFactory(hostingThreadName + ".append")); + // Preallocate objects to use on the ring buffer. The way that appends and syncs work, we will + // be stuck and make no progress if the buffer is filled with appends only and there is no + // sync. If no sync, then the handlers will be outstanding just waiting on sync completion + // before they return. + final int preallocatedEventCount = + this.conf.getInt("hbase.regionserver.wal.disruptor.event.count", 1024 * 16); + // Using BlockingWaitStrategy. Stuff that is going on here takes so long it makes no sense + // spinning as other strategies do. + this.disruptor = + new Disruptor(RingBufferTruck.EVENT_FACTORY, preallocatedEventCount, + this.appendExecutor, ProducerType.MULTI, new BlockingWaitStrategy()); + this.ringBufferEventHandler = + new RingBufferEventHandler(conf.getInt("hbase.regionserver.hlog.syncer.count", 5), + maxHandlersCount); + this.disruptor.handleExceptionsWith(new RingBufferExceptionHandler()); + this.disruptor.handleEventsWith(new RingBufferEventHandler [] {this.ringBufferEventHandler}); + // Presize our map of SyncFutures by handler objects. + this.syncFuturesByHandler = new ConcurrentHashMap(maxHandlersCount); + // Starting up threads in constructor is a no no; Interface should have an init call. + this.disruptor.start(); } /** * Find the 'getNumCurrentReplicas' on the passed os stream. * @return Method or null. */ - private Method getGetNumCurrentReplicas(final FSDataOutputStream os) { + private static Method getGetNumCurrentReplicas(final FSDataOutputStream os) { Method m = null; if (os != null) { - Class wrappedStreamClass = os.getWrappedStream() - .getClass(); + Class wrappedStreamClass = os.getWrappedStream().getClass(); try { - m = wrappedStreamClass.getDeclaredMethod("getNumCurrentReplicas", - new Class[] {}); + m = wrappedStreamClass.getDeclaredMethod("getNumCurrentReplicas", new Class[] {}); m.setAccessible(true); } catch (NoSuchMethodException e) { - LOG.info("FileSystem's output stream doesn't support" - + " getNumCurrentReplicas; --HDFS-826 not available; fsOut=" - + wrappedStreamClass.getName()); + LOG.info("FileSystem's output stream doesn't support getNumCurrentReplicas; " + + "HDFS-826 not available; fsOut=" + wrappedStreamClass.getName()); } catch (SecurityException e) { - LOG.info("Doesn't have access to getNumCurrentReplicas on " - + "FileSystems's output stream --HDFS-826 not available; fsOut=" - + wrappedStreamClass.getName(), e); + LOG.info("No access to getNumCurrentReplicas on FileSystems's output stream; HDFS-826 " + + "not available; fsOut=" + wrappedStreamClass.getName(), e); m = null; // could happen on setAccessible() } } if (m != null) { - if (LOG.isTraceEnabled()) LOG.trace("Using getNumCurrentReplicas--HDFS-826"); + if (LOG.isTraceEnabled()) LOG.trace("Using getNumCurrentReplicas"); } return m; } @@ -483,7 +565,7 @@ class FSHLog implements HLog, Syncable { @Override public long getFilenum() { - return this.filenum; + return this.filenum.get(); } /** @@ -503,92 +585,99 @@ class FSHLog implements HLog, Syncable { return rollWriter(false); } + private Path getNewPath() throws IOException { + this.filenum.set(System.currentTimeMillis()); + Path newPath = computeFilename(); + while (fs.exists(newPath)) { + this.filenum.incrementAndGet(); + newPath = computeFilename(); + } + return newPath; + } + + Path getOldPath() { + long currentFilenum = this.filenum.get(); + Path oldPath = null; + if (currentFilenum > 0) { + // ComputeFilename will take care of meta hlog filename + oldPath = computeFilename(currentFilenum); + } // I presume if currentFilenum is <= 0, this is first file and null for oldPath if fine? + return oldPath; + } + + /** + * Tell listeners about pre log roll. + * @throws IOException + */ + private void tellListenersAboutPreLogRoll(final Path oldPath, final Path newPath) + throws IOException { + if (!this.listeners.isEmpty()) { + for (WALActionsListener i : this.listeners) { + i.preLogRoll(oldPath, newPath); + } + } + } + + /** + * Tell listeners about post log roll. + * @throws IOException + */ + private void tellListenersAboutPostLogRoll(final Path oldPath, final Path newPath) + throws IOException { + if (!this.listeners.isEmpty()) { + for (WALActionsListener i : this.listeners) { + i.postLogRoll(oldPath, newPath); + } + } + } + + /** + * Run a sync after opening to set up the pipeline. + * @param nextWriter + * @param startTimeNanos + */ + private void preemptiveSync(final ProtobufLogWriter nextWriter) { + long startTimeNanos = System.nanoTime(); + try { + nextWriter.sync(); + postSync(System.nanoTime() - startTimeNanos, 0); + } catch (IOException e) { + // optimization failed, no need to abort here. + LOG.warn("pre-sync failed but an optimization so keep going", e); + } + } + @Override - public byte [][] rollWriter(boolean force) - throws FailedLogCloseException, IOException { + public byte [][] rollWriter(boolean force) throws FailedLogCloseException, IOException { rollWriterLock.lock(); try { // Return if nothing to flush. - if (!force && this.writer != null && this.numEntries.get() <= 0) { - return null; - } + if (!force && (this.writer != null && this.numEntries.get() <= 0)) return null; byte [][] regionsToFlush = null; - if (closed) { + if (this.closed) { LOG.debug("HLog closed. Skipping rolling of writer"); - return null; + return regionsToFlush; + } + if (!closeBarrier.beginOp()) { + LOG.debug("HLog closing. Skipping rolling of writer"); + return regionsToFlush; } try { - if (!closeBarrier.beginOp()) { - LOG.debug("HLog closing. Skipping rolling of writer"); - return regionsToFlush; - } - // Do all the preparation outside of the updateLock to block - // as less as possible the incoming writes - long currentFilenum = this.filenum; - Path oldPath = null; - if (currentFilenum > 0) { - //computeFilename will take care of meta hlog filename - oldPath = computeFilename(currentFilenum); - } - this.filenum = System.currentTimeMillis(); - Path newPath = computeFilename(); - while (fs.exists(newPath)) { - this.filenum++; - newPath = computeFilename(); - } - - // Tell our listeners that a new log is about to be created - if (!this.listeners.isEmpty()) { - for (WALActionsListener i : this.listeners) { - i.preLogRoll(oldPath, newPath); - } - } + Path oldPath = getOldPath(); + Path newPath = getNewPath(); + // Any exception from here on is catastrophic, non-recoverable so we currently abort. FSHLog.Writer nextWriter = this.createWriterInstance(fs, newPath, conf); - // Can we get at the dfsclient outputstream? FSDataOutputStream nextHdfsOut = null; if (nextWriter instanceof ProtobufLogWriter) { nextHdfsOut = ((ProtobufLogWriter)nextWriter).getStream(); - // perform the costly sync before we get the lock to roll writers. - try { - nextWriter.sync(); - postSync(); - } catch (IOException e) { - // optimization failed, no need to abort here. - LOG.warn("pre-sync failed", e); - } + // If a ProtobufLogWriter, go ahead and try and sync to force setup of pipeline. + // If this fails, we just keep going.... it is an optimization, not the end of the world. + preemptiveSync((ProtobufLogWriter)nextWriter); } - - Path oldFile = null; - int oldNumEntries = 0; - synchronized (updateLock) { - // Clean up current writer. - oldNumEntries = this.numEntries.get(); - oldFile = cleanupCurrentWriter(currentFilenum); - this.writer = nextWriter; - this.hdfs_out = nextHdfsOut; - this.numEntries.set(0); - if (oldFile != null) { - this.hlogSequenceNums.put(oldFile, this.latestSequenceNums); - this.latestSequenceNums = new HashMap(); - } - } - if (oldFile == null) LOG.info("New WAL " + FSUtils.getPath(newPath)); - else { - long oldFileLen = this.fs.getFileStatus(oldFile).getLen(); - this.totalLogSize.addAndGet(oldFileLen); - LOG.info("Rolled WAL " + FSUtils.getPath(oldFile) + " with entries=" - + oldNumEntries + ", filesize=" - + StringUtils.humanReadableInt(oldFileLen) + "; new WAL " - + FSUtils.getPath(newPath)); - } - - // Tell our listeners that a new log was created - if (!this.listeners.isEmpty()) { - for (WALActionsListener i : this.listeners) { - i.postLogRoll(oldPath, newPath); - } - } - + tellListenersAboutPreLogRoll(oldPath, newPath); + // NewPath could be equal to oldPath if replaceWriter fails. + newPath = replaceWriter(oldPath, newPath, nextWriter, nextHdfsOut); + tellListenersAboutPostLogRoll(oldPath, newPath); // Can we delete any of the old log files? if (getNumRolledLogFiles() > 0) { cleanOldLogs(); @@ -623,13 +712,13 @@ class FSHLog implements HLog, Syncable { /** * Archive old logs that could be archived: a log is eligible for archiving if all its WALEdits - * are already flushed by the corresponding regions. + * have been flushed to hfiles. *

* For each log file, it compares its region to sequenceId map - * (@link {@link FSHLog#latestSequenceNums} with corresponding region entries in - * {@link FSHLog#oldestFlushingSeqNums} and {@link FSHLog#oldestUnflushedSeqNums}. - * If all the regions in the map are flushed past of their value, then the wal is eligible for - * archiving. + * (@link {@link FSHLog#highestRegionSequenceIds} with corresponding region entries in + * {@link FSHLog#lowestFlushingRegionSequenceIds} and + * {@link FSHLog#oldestUnflushedRegionSequenceIds}. If all the regions in the map are flushed + * past of their value, then the wal is eligible for archiving. * @throws IOException */ private void cleanOldLogs() throws IOException { @@ -637,34 +726,35 @@ class FSHLog implements HLog, Syncable { Map oldestUnflushedSeqNumsLocal = null; List logsToArchive = new ArrayList(); // make a local copy so as to avoid locking when we iterate over these maps. - synchronized (oldestSeqNumsLock) { - oldestFlushingSeqNumsLocal = new HashMap(this.oldestFlushingSeqNums); - oldestUnflushedSeqNumsLocal = new HashMap(this.oldestUnflushedSeqNums); + synchronized (regionSequenceIdLock) { + oldestFlushingSeqNumsLocal = new HashMap(this.lowestFlushingRegionSequenceIds); + oldestUnflushedSeqNumsLocal = + new HashMap(this.oldestUnflushedRegionSequenceIds); } - for (Map.Entry> e : hlogSequenceNums.entrySet()) { + for (Map.Entry> e : byWalRegionSequenceIds.entrySet()) { // iterate over the log file. Path log = e.getKey(); Map sequenceNums = e.getValue(); // iterate over the map for this log file, and tell whether it should be archive or not. if (areAllRegionsFlushed(sequenceNums, oldestFlushingSeqNumsLocal, - oldestUnflushedSeqNumsLocal)) { + oldestUnflushedSeqNumsLocal)) { logsToArchive.add(log); - LOG.debug("log file is ready for archiving " + log); + LOG.debug("WAL file ready for archiving " + log); } } for (Path p : logsToArchive) { this.totalLogSize.addAndGet(-this.fs.getFileStatus(p).getLen()); archiveLogFile(p); - this.hlogSequenceNums.remove(p); + this.byWalRegionSequenceIds.remove(p); } } /** * Takes a region:sequenceId map for a WAL file, and checks whether the file can be archived. * It compares the region entries present in the passed sequenceNums map with the local copy of - * {@link #oldestUnflushedSeqNums} and {@link #oldestFlushingSeqNums}. If, for all regions, - * the value is lesser than the minimum of values present in the oldestFlushing/UnflushedSeqNums, - * then the wal file is eligible for archiving. + * {@link #oldestUnflushedRegionSequenceIds} and {@link #lowestFlushingRegionSequenceIds}. If, + * for all regions, the value is lesser than the minimum of values present in the + * oldestFlushing/UnflushedSeqNums, then the wal file is eligible for archiving. * @param sequenceNums for a HLog, at the time when it was rolled. * @param oldestFlushingMap * @param oldestUnflushedMap @@ -673,7 +763,7 @@ class FSHLog implements HLog, Syncable { static boolean areAllRegionsFlushed(Map sequenceNums, Map oldestFlushingMap, Map oldestUnflushedMap) { for (Map.Entry regionSeqIdEntry : sequenceNums.entrySet()) { - // find region entries in the flushing/unflushed map. If there is no entry, it means + // find region entries in the flushing/unflushed map. If there is no entry, it meansj // a region doesn't have any unflushed entry. long oldestFlushing = oldestFlushingMap.containsKey(regionSeqIdEntry.getKey()) ? oldestFlushingMap.get(regionSeqIdEntry.getKey()) : Long.MAX_VALUE; @@ -688,10 +778,11 @@ class FSHLog implements HLog, Syncable { /** * Iterates over the given map of regions, and compares their sequence numbers with corresponding - * entries in {@link #oldestUnflushedSeqNums}. If the sequence number is greater or equal, the - * region is eligible to flush, otherwise, there is no benefit to flush (from the perspective of - * passed regionsSequenceNums map), because the region has already flushed the entries present - * in the WAL file for which this method is called for (typically, the oldest wal file). + * entries in {@link #oldestUnflushedRegionSequenceIds}. If the sequence number is greater or + * equal, the region is eligible to flush, otherwise, there is no benefit to flush (from the + * perspective of passed regionsSequenceNums map), because the region has already flushed the + * entries present in the WAL file for which this method is called for (typically, the oldest + * wal file). * @param regionsSequenceNums * @return regions which should be flushed (whose sequence numbers are larger than their * corresponding un-flushed entries. @@ -699,9 +790,9 @@ class FSHLog implements HLog, Syncable { private byte[][] findEligibleMemstoresToFlush(Map regionsSequenceNums) { List regionsToFlush = null; // Keeping the old behavior of iterating unflushedSeqNums under oldestSeqNumsLock. - synchronized (oldestSeqNumsLock) { + synchronized (regionSequenceIdLock) { for (Map.Entry e : regionsSequenceNums.entrySet()) { - Long unFlushedVal = this.oldestUnflushedSeqNums.get(e.getKey()); + Long unFlushedVal = this.oldestUnflushedRegionSequenceIds.get(e.getKey()); if (unFlushedVal != null && unFlushedVal <= e.getValue()) { if (regionsToFlush == null) regionsToFlush = new ArrayList(); regionsToFlush.add(e.getKey()); @@ -724,7 +815,7 @@ class FSHLog implements HLog, Syncable { int logCount = getNumRolledLogFiles(); if (logCount > this.maxLogs && logCount > 0) { Map.Entry> firstWALEntry = - this.hlogSequenceNums.firstEntry(); + this.byWalRegionSequenceIds.firstEntry(); regions = findEligibleMemstoresToFlush(firstWALEntry.getValue()); } if (regions != null) { @@ -740,56 +831,103 @@ class FSHLog implements HLog, Syncable { return regions; } - /* - * Cleans up current writer closing. - * Presumes we're operating inside an updateLock scope. - * @return Path to current writer or null if none. + /** + * Cleans up current writer closing it and then puts in place the passed in + * nextWriter + * + * @param oldPath + * @param newPath + * @param nextWriter + * @param nextHdfsOut + * @return newPath * @throws IOException */ - Path cleanupCurrentWriter(final long currentfilenum) throws IOException { - Path oldFile = null; - if (this.writer != null) { - // Close the current writer, get a new one. + Path replaceWriter(final Path oldPath, final Path newPath, FSHLog.Writer nextWriter, + final FSDataOutputStream nextHdfsOut) + throws IOException { + // Ask the ring buffer writer to pause at a safe point. Once we do this, the writer + // thread will eventually pause. An error hereafter needs to release the writer thread + // regardless -- hence the finally block below. Note, this method is called from the FSHLog + // constructor BEFORE the ring buffer is set running so it is null on first time through + // here; allow for that. + SyncFuture syncFuture = null; + SafePointZigZagLatch zigzagLatch = (this.ringBufferEventHandler == null)? + null: this.ringBufferEventHandler.attainSafePoint(); + try { + // Wait on the safe point to be achieved. Send in a sync in case nothing has hit the + // ring buffer between the above notification of writer that we want it to go to + // 'safe point' and then here where we are waiting on it to attain safe point. Use + // 'sendSync' instead of 'sync' because we do not want this thread to block waiting on it + // to come back. Cleanup this syncFuture down below after we are ready to run again. try { - // Wait till all current transactions are written to the hlog. - // No new transactions can occur because we have the updatelock. - if (this.unflushedEntries.get() != this.syncedTillHere.get()) { - LOG.debug("cleanupCurrentWriter " + - " waiting for transactions to get synced " + - " total " + this.unflushedEntries.get() + - " synced till here " + this.syncedTillHere.get()); - sync(); - } - this.writer.close(); - this.writer = null; - closeErrorCount.set(0); - } catch (IOException e) { - LOG.error("Failed close of HLog writer", e); + if (zigzagLatch != null) syncFuture = zigzagLatch.waitSafePoint(publishSyncOnRingBuffer()); + } catch (FailedSyncBeforeLogCloseException e) { + if (isUnflushedEntries()) throw e; + // Else, let is pass through to the close. + LOG.warn("Failed last sync but no outstanding unsync edits so falling through to close; " + + e.getMessage()); + } + + // It is at the safe point. Swap out writer from under the blocked writer thread. + // TODO: This is close is inline with critical section. Should happen in background? + try { + if (this.writer != null) this.writer.close(); + this.closeErrorCount.set(0); + } catch (IOException ioe) { int errors = closeErrorCount.incrementAndGet(); - if (errors <= closeErrorsTolerated && !hasUnSyncedEntries()) { - LOG.warn("Riding over HLog close failure! error count="+errors); + if (!isUnflushedEntries() && (errors <= this.closeErrorsTolerated)) { + LOG.warn("Riding over failed WAL close of " + oldPath + ", cause=\"" + + ioe.getMessage() + "\", errors=" + errors + + "; THIS FILE WAS NOT CLOSED BUT ALL EDITS SYNCED SO SHOULD BE OK"); } else { - if (hasUnSyncedEntries()) { - LOG.error("Aborting due to unflushed edits in HLog"); - } - // Failed close of log file. Means we're losing edits. For now, - // shut ourselves down to minimize loss. Alternative is to try and - // keep going. See HBASE-930. - FailedLogCloseException flce = - new FailedLogCloseException("#" + currentfilenum); - flce.initCause(e); - throw flce; + throw ioe; } } - if (currentfilenum >= 0) { - oldFile = computeFilename(currentfilenum); + this.writer = nextWriter; + this.hdfs_out = nextHdfsOut; + int oldNumEntries = this.numEntries.get(); + this.numEntries.set(0); + if (oldPath != null) { + this.byWalRegionSequenceIds.put(oldPath, this.highestRegionSequenceIds); + this.highestRegionSequenceIds = new HashMap(); + long oldFileLen = this.fs.getFileStatus(oldPath).getLen(); + this.totalLogSize.addAndGet(oldFileLen); + LOG.info("Rolled WAL " + FSUtils.getPath(oldPath) + " with entries=" + oldNumEntries + + ", filesize=" + StringUtils.humanReadableInt(oldFileLen) + "; new WAL " + + FSUtils.getPath(newPath)); + } else { + LOG.info("New WAL " + FSUtils.getPath(newPath)); + } + } catch (InterruptedException ie) { + // Perpetuate the interrupt + Thread.currentThread().interrupt(); + } catch (IOException e) { + long count = getUnflushedEntriesCount(); + LOG.error("Failed close of HLog writer " + oldPath + ", unflushedEntries=" + count, e); + throw new FailedLogCloseException(oldPath + ", unflushedEntries=" + count, e); + } finally { + // Let the writer thread go regardless, whether error or not. + if (zigzagLatch != null) { + zigzagLatch.releaseSafePoint(); + // It will be null if we failed our wait on safe point above. + if (syncFuture != null) blockOnSync(syncFuture); } } - return oldFile; + return newPath; + } + + long getUnflushedEntriesCount() { + long highestSynced = this.highestSyncedSequence.get(); + return highestSynced > this.highestUnsyncedSequence? + 0: this.highestUnsyncedSequence - highestSynced; + } + + boolean isUnflushedEntries() { + return getUnflushedEntriesCount() > 0; } private void archiveLogFile(final Path p) throws IOException { - Path newPath = getHLogArchivePath(this.oldLogDir, p); + Path newPath = getHLogArchivePath(this.fullPathOldLogDir, p); // Tell our listeners that a log is going to be archived. if (!this.listeners.isEmpty()) { for (WALActionsListener i : this.listeners) { @@ -807,30 +945,31 @@ class FSHLog implements HLog, Syncable { } } - /** - * This is a convenience method that computes a new filename with a given - * using the current HLog file-number - * @return Path - */ - protected Path computeFilename() { - return computeFilename(this.filenum); - } - /** * This is a convenience method that computes a new filename with a given * file-number. * @param filenum to use * @return Path */ - protected Path computeFilename(long filenum) { - if (filenum < 0) { + protected Path computeFilename(final long filenum) { + this.filenum.set(filenum); + return computeFilename(); + } + + /** + * This is a convenience method that computes a new filename with a given + * using the current HLog file-number + * @return Path + */ + protected Path computeFilename() { + if (this.filenum.get() < 0) { throw new RuntimeException("hlog file number can't be < 0"); } - String child = prefix + "." + filenum; + String child = logFilePrefix + "." + filenum; if (forMeta) { child += HLog.META_HLOG_FILE_EXTN; } - return new Path(dir, child); + return new Path(fullPathLogDir, child); } /** @@ -844,7 +983,7 @@ class FSHLog implements HLog, Syncable { protected long getFileNumFromFileName(Path fileName) { if (fileName == null) throw new IllegalArgumentException("file name can't be null"); // The path should start with dir/. - String prefixPathStr = new Path(dir, prefix + ".").toString(); + String prefixPathStr = new Path(fullPathLogDir, logFilePrefix + ".").toString(); if (!fileName.toString().startsWith(prefixPathStr)) { throw new IllegalArgumentException("The log file " + fileName + " doesn't belong to" + " this regionserver " + prefixPathStr); @@ -857,12 +996,12 @@ class FSHLog implements HLog, Syncable { @Override public void closeAndDelete() throws IOException { close(); - if (!fs.exists(this.dir)) return; - FileStatus[] files = fs.listStatus(this.dir); + if (!fs.exists(this.fullPathLogDir)) return; + FileStatus[] files = fs.listStatus(this.fullPathLogDir); if (files != null) { for(FileStatus file : files) { - Path p = getHLogArchivePath(this.oldLogDir, file.getPath()); + Path p = getHLogArchivePath(this.fullPathOldLogDir, file.getPath()); // Tell our listeners that a log is going to be archived. if (!this.listeners.isEmpty()) { for (WALActionsListener i : this.listeners) { @@ -880,45 +1019,17 @@ class FSHLog implements HLog, Syncable { } } } - LOG.debug("Moved " + files.length + " WAL file(s) to " + FSUtils.getPath(this.oldLogDir)); + LOG.debug("Moved " + files.length + " WAL file(s) to " + + FSUtils.getPath(this.fullPathOldLogDir)); } - if (!fs.delete(dir, true)) { - LOG.info("Unable to delete " + dir); + if (!fs.delete(fullPathLogDir, true)) { + LOG.info("Unable to delete " + fullPathLogDir); } } @Override public void close() throws IOException { - if (this.closed) { - return; - } - - try { - asyncNotifier.interrupt(); - asyncNotifier.join(); - } catch (InterruptedException e) { - LOG.error("Exception while waiting for " + asyncNotifier.getName() + - " threads to die", e); - } - - for (int i = 0; i < asyncSyncers.length; ++i) { - try { - asyncSyncers[i].interrupt(); - asyncSyncers[i].join(); - } catch (InterruptedException e) { - LOG.error("Exception while waiting for " + asyncSyncers[i].getName() + - " threads to die", e); - } - } - - try { - asyncWriter.interrupt(); - asyncWriter.join(); - } catch (InterruptedException e) { - LOG.error("Exception while waiting for " + asyncWriter.getName() + - " thread to die", e); - } - + if (this.closed) return; try { // Prevent all further flushing and rolling. closeBarrier.stopAndDrainOps(); @@ -927,21 +1038,36 @@ class FSHLog implements HLog, Syncable { Thread.currentThread().interrupt(); } + // Shutdown the disruptor. Will stop after all entries have been processed. Make sure we have + // stopped incoming appends before calling this else it will not shutdown. We are + // conservative below waiting a long time and if not elapsed, then halting. + if (this.disruptor != null) { + long timeoutms = conf.getLong("hbase.wal.disruptor.shutdown.timeout.ms", 60000); + try { + this.disruptor.shutdown(timeoutms, TimeUnit.MILLISECONDS); + } catch (TimeoutException e) { + LOG.warn("Timed out bringing down disruptor after " + timeoutms + "ms; forcing halt " + + "(It is a problem if this is NOT an ABORT! -- DATALOSS!!!!)"); + this.disruptor.halt(); + this.disruptor.shutdown(); + } + } + // With disruptor down, this is safe to let go. + if (this.appendExecutor != null) this.appendExecutor.shutdown(); + // Tell our listeners that the log is closing if (!this.listeners.isEmpty()) { for (WALActionsListener i : this.listeners) { i.logCloseRequested(); } } - synchronized (updateLock) { - this.closed = true; - if (LOG.isDebugEnabled()) { - LOG.debug("Closing WAL writer in " + this.dir.toString()); - } - if (this.writer != null) { - this.writer.close(); - this.writer = null; - } + this.closed = true; + if (LOG.isDebugEnabled()) { + LOG.debug("Closing WAL writer in " + this.fullPathLogDir.toString()); + } + if (this.writer != null) { + this.writer.close(); + this.writer = null; } } @@ -961,418 +1087,255 @@ class FSHLog implements HLog, Syncable { @Override @VisibleForTesting public void append(HRegionInfo info, TableName tableName, WALEdit edits, - final long now, HTableDescriptor htd, AtomicLong sequenceId) throws IOException { + final long now, HTableDescriptor htd, AtomicLong sequenceId) + throws IOException { append(info, tableName, edits, new ArrayList(), now, htd, true, true, sequenceId, HConstants.NO_NONCE, HConstants.NO_NONCE); } + @Override + public long appendNoSync(HRegionInfo info, TableName tableName, WALEdit edits, + List clusterIds, final long now, HTableDescriptor htd, AtomicLong sequenceId, + boolean isInMemstore, long nonceGroup, long nonce) throws IOException { + return append(info, tableName, edits, clusterIds, now, htd, false, isInMemstore, sequenceId, + nonceGroup, nonce); + } + /** - * Append a set of edits to the log. Log edits are keyed by (encoded) - * regionName, rowname, and log-sequence-id. + * Append a set of edits to the log. Log edits are keyed by (encoded) regionName, rowname, and + * log-sequence-id. * - * Later, if we sort by these keys, we obtain all the relevant edits for a - * given key-range of the HRegion (TODO). Any edits that do not have a - * matching COMPLETE_CACHEFLUSH message can be discarded. + * Later, if we sort by these keys, we obtain all the relevant edits for a given key-range of the + * HRegion (TODO). Any edits that do not have a matching COMPLETE_CACHEFLUSH message can be + * discarded. * - *

- * Logs cannot be restarted once closed, or once the HLog process dies. Each - * time the HLog starts, it must create a new log. This means that other - * systems should process the log appropriately upon each startup (and prior - * to initializing HLog). + *

Logs cannot be restarted once closed, or once the HLog process dies. Each time the HLog + * starts, it must create a new log. This means that other systems should process the log + * appropriately upon each startup (and prior to initializing HLog). * - * synchronized prevents appends during the completion of a cache flush or for - * the duration of a log roll. + * Synchronized prevents appends during the completion of a cache flush or for the duration of a + * log roll. * * @param info * @param tableName * @param edits * @param clusterIds that have consumed the change (for replication) * @param now + * @param htd * @param doSync shall we sync? + * @param inMemstore * @param sequenceId of the region. - * @return txid of this transaction + * @param nonceGroup + * @param nonce + * @return txid of this transaction or if nothing to do, the last txid * @throws IOException */ - @SuppressWarnings("deprecation") private long append(HRegionInfo info, TableName tableName, WALEdit edits, List clusterIds, - final long now, HTableDescriptor htd, boolean doSync, boolean isInMemstore, - AtomicLong sequenceId, long nonceGroup, long nonce) throws IOException { - if (edits.isEmpty()) return this.unflushedEntries.get(); - if (this.closed) { - throw new IOException("Cannot append; log is closed"); - } - TraceScope traceScope = Trace.startSpan("FSHlog.append"); - try { - long txid = 0; - synchronized (this.updateLock) { - // get the sequence number from the passed Long. In normal flow, it is coming from the - // region. - long seqNum = sequenceId.incrementAndGet(); - // The 'lastSeqWritten' map holds the sequence number of the oldest - // write for each region (i.e. the first edit added to the particular - // memstore). . When the cache is flushed, the entry for the - // region being flushed is removed if the sequence number of the flush - // is greater than or equal to the value in lastSeqWritten. - // Use encoded name. Its shorter, guaranteed unique and a subset of - // actual name. - byte [] encodedRegionName = info.getEncodedNameAsBytes(); - if (isInMemstore) this.oldestUnflushedSeqNums.putIfAbsent(encodedRegionName, seqNum); - HLogKey logKey = makeKey( - encodedRegionName, tableName, seqNum, now, clusterIds, nonceGroup, nonce); - - synchronized (pendingWritesLock) { - doWrite(info, logKey, edits, htd); - txid = this.unflushedEntries.incrementAndGet(); - } - this.numEntries.incrementAndGet(); - this.asyncWriter.setPendingTxid(txid); - - if (htd.isAsyncLogFlush()) { - lastUnSyncedTxid = txid; - } - this.latestSequenceNums.put(encodedRegionName, seqNum); - } - // TODO: note that only tests currently call append w/sync. - // Therefore, this code here is not actually used by anything. - // Sync if catalog region, and if not then check if that table supports - // deferred log flushing - if (doSync && - (info.isMetaRegion() || - !htd.isAsyncLogFlush())) { - // sync txn to file system - this.sync(txid); - } - return txid; - } finally { - traceScope.close(); - } + final long now, HTableDescriptor htd, boolean doSync, boolean inMemstore, + AtomicLong sequenceId, long nonceGroup, long nonce) + throws IOException { + if (!this.enabled || edits.isEmpty()) return this.highestUnsyncedSequence; + if (this.closed) throw new IOException("Cannot append; log is closed"); + // TODO: trace model here does not work any more. It does not match how we append. + TraceScope traceScope = Trace.startSpan("FSHlog.append"); + // Make a key but do not set the WALEdit by region sequence id now -- set it to -1 for now -- + // and then later just before we write it out to the DFS stream, then set the sequence id; + // late-binding. + HLogKey logKey = + makeKey(info.getEncodedNameAsBytes(), tableName, -1, now, clusterIds, nonceGroup, nonce); + // This is crazy how much it takes to make an edit. Do we need all this stuff!!!!???? We need + // all the stuff to make a key and then below to append the edit, we need to carry htd, info, + // etc. all over the ring buffer. + long sequence = this.disruptor.getRingBuffer().next(); + try { + RingBufferTruck truck = this.disruptor.getRingBuffer().get(sequence); + FSWALEntry entry = + new FSWALEntry(sequence, logKey, edits, sequenceId, inMemstore, htd, info); + truck.loadPayload(entry, traceScope.detach()); + } finally { + this.disruptor.getRingBuffer().publish(sequence); } - - @Override - public long appendNoSync(HRegionInfo info, TableName tableName, WALEdit edits, - List clusterIds, final long now, HTableDescriptor htd, AtomicLong sequenceId, - boolean isInMemstore, long nonceGroup, long nonce) throws IOException { - return append(info, tableName, edits, clusterIds, - now, htd, false, isInMemstore, sequenceId, nonceGroup, nonce); + // Sync if we have been asked to -- only tests do this -- or if it is a meta table edit (these + // are precious). When we sync, we will sync to the current point, the txid of the last edit + // added. Since we are single writer, the next txid should be the just next one in sequence; + // do not explicitly specify it (sequence id/txid is an implementation internal detail. + if (doSync || info.isMetaRegion()) publishSyncThenBlockOnCompletion(); + return sequence; } - /* The work of current write process of HLog goes as below: - * 1). All write handler threads append edits to HLog's local pending buffer; - * (it notifies AsyncWriter thread that there is new edits in local buffer) - * 2). All write handler threads wait in HLog.syncer() function for underlying threads to - * finish the sync that contains its txid; - * 3). An AsyncWriter thread is responsible for retrieving all edits in HLog's - * local pending buffer and writing to the hdfs (hlog.writer.append); - * (it notifies AsyncSyncer threads that there is new writes to hdfs which needs a sync) - * 4). AsyncSyncer threads are responsible for issuing sync request to hdfs to persist the - * writes by AsyncWriter; (they notify the AsyncNotifier thread that sync is done) - * 5). An AsyncNotifier thread is responsible for notifying all pending write handler - * threads which are waiting in the HLog.syncer() function - * 6). No LogSyncer thread any more (since there is always AsyncWriter/AsyncFlusher threads - * do the same job it does) - * note: more than one AsyncSyncer threads are needed here to guarantee good enough performance - * when less concurrent write handler threads. since sync is the most time-consuming - * operation in the whole write process, multiple AsyncSyncer threads can provide better - * parallelism of sync to get better overall throughput + /** + * Thread to runs the hdfs sync call. This call takes a while to complete. This is the longest + * pole adding edits to the WAL and this must complete to be sure all edits persisted. We run + * multiple threads sync'ng rather than one that just syncs in series so we have better + * latencies; otherwise, an edit that arrived just after a sync started, might have to wait + * almost the length of two sync invocations before it is marked done. + *

When the sync completes, it marks all the passed in futures done. On the other end of the + * sync future is a blocked thread, usually a regionserver Handler. There may be more than one + * future passed in the case where a few threads arrive at about the same time and all invoke + * 'sync'. In this case we'll batch up the invocations and run one filesystem sync only for a + * batch of Handler sync invocations. Do not confuse these Handler SyncFutures with the futures + * an ExecutorService returns when you call submit. We have no use for these in this model. These + * SyncFutures are 'artificial', something to hold the Handler until the filesystem sync + * completes. */ - // thread to write locally buffered writes to HDFS - private class AsyncWriter extends HasThread { - private long pendingTxid = 0; - private long txidToWrite = 0; - private long lastWrittenTxid = 0; - private Object writeLock = new Object(); - - public AsyncWriter(String name) { + private class SyncRunner extends HasThread { + private volatile long sequence; + private final BlockingQueue syncFutures; + + /** + * UPDATE! + * @param syncs the batch of calls to sync that arrived as this thread was starting; when done, + * we will put the result of the actual hdfs sync call as the result. + * @param sequence The sequence number on the ring buffer when this thread was set running. + * If this actual writer sync completes then all appends up this point have been + * flushed/synced/pushed to datanodes. If we fail, then the passed in syncs + * futures will return the exception to their clients; some of the edits may have made it out + * to data nodes but we will report all that were part of this session as failed. + */ + SyncRunner(final String name, final int maxHandlersCount) { super(name); + // LinkedBlockingQueue because of + // http://www.javacodegeeks.com/2010/09/java-best-practices-queue-battle-and.html + // Could use other blockingqueues here or concurrent queues. + this.syncFutures = new LinkedBlockingQueue(maxHandlersCount); } - // wake up (called by (write) handler thread) AsyncWriter thread - // to write buffered writes to HDFS - public void setPendingTxid(long txid) { - synchronized (this.writeLock) { - if (txid <= this.pendingTxid) - return; + void offer(final long sequence, final SyncFuture [] syncFutures, final int syncFutureCount) { + // Set sequence first because the add to the queue will wake the thread if sleeping. + this.sequence = sequence; + for (int i = 0; i < syncFutureCount; i++) this.syncFutures.add(syncFutures[i]); + } - this.pendingTxid = txid; - this.writeLock.notify(); + /** + * Release the passed syncFuture + * @param syncFuture + * @param currentSequence + * @param t + * @return Returns 1. + */ + private int releaseSyncFuture(final SyncFuture syncFuture, final long currentSequence, + final Throwable t) { + if (!syncFuture.done(currentSequence, t)) throw new IllegalStateException(); + // This function releases one sync future only. + return 1; + } + + /** + * Release all SyncFutures whose sequence is <= currentSequence. + * @param currentSequence + * @param t May be non-null if we are processing SyncFutures because an exception was thrown. + * @return Count of SyncFutures we let go. + */ + private int releaseSyncFutures(final long currentSequence, final Throwable t) { + int syncCount = 0; + for (SyncFuture syncFuture; (syncFuture = this.syncFutures.peek()) != null;) { + if (syncFuture.getRingBufferSequence() > currentSequence) break; + releaseSyncFuture(syncFuture, currentSequence, t); + if (!this.syncFutures.remove(syncFuture)) { + throw new IllegalStateException(syncFuture.toString()); + } + syncCount++; } + return syncCount; + } + + /** + * @param sequence The sequence we ran the filesystem sync against. + * @return Current highest synced sequence. + */ + private long updateHighestSyncedSequence(long sequence) { + long currentHighestSyncedSequence; + // Set the highestSyncedSequence IFF our current sequence id is the 'highest'. + do { + currentHighestSyncedSequence = highestSyncedSequence.get(); + if (currentHighestSyncedSequence >= sequence) { + // Set the sync number to current highwater mark; might be able to let go more + // queued sync futures + sequence = currentHighestSyncedSequence; + break; + } + } while (!highestSyncedSequence.compareAndSet(currentHighestSyncedSequence, sequence)); + return sequence; } public void run() { - try { - while (!this.isInterrupted()) { - // 1. wait until there is new writes in local buffer - synchronized (this.writeLock) { - while (this.pendingTxid <= this.lastWrittenTxid) { - this.writeLock.wait(); - } - } - - // 2. get all buffered writes and update 'real' pendingTxid - // since maybe newer writes enter buffer as AsyncWriter wakes - // up and holds the lock - // NOTE! can't hold 'updateLock' here since rollWriter will pend - // on 'sync()' with 'updateLock', but 'sync()' will wait for - // AsyncWriter/AsyncSyncer/AsyncNotifier series. without updateLock - // can leads to pendWrites more than pendingTxid, but not problem - List pendWrites = null; - synchronized (pendingWritesLock) { - this.txidToWrite = unflushedEntries.get(); - pendWrites = pendingWrites; - pendingWrites = new LinkedList(); - } - - // 3. write all buffered writes to HDFS(append, without sync) - try { - for (Entry e : pendWrites) { - writer.append(e); - } - postAppend(pendWrites); - } catch(IOException e) { - LOG.error("Error while AsyncWriter write, request close of hlog ", e); - requestLogRoll(); - - asyncIOE = e; - failedTxid.set(this.txidToWrite); - } - - // 4. update 'lastWrittenTxid' and notify AsyncSyncer to do 'sync' - this.lastWrittenTxid = this.txidToWrite; - boolean hasIdleSyncer = false; - for (int i = 0; i < asyncSyncers.length; ++i) { - if (!asyncSyncers[i].isSyncing()) { - hasIdleSyncer = true; - asyncSyncers[i].setWrittenTxid(this.lastWrittenTxid); - break; - } - } - if (!hasIdleSyncer) { - int idx = (int)this.lastWrittenTxid % asyncSyncers.length; - asyncSyncers[idx].setWrittenTxid(this.lastWrittenTxid); - } - } - } catch (InterruptedException e) { - LOG.debug(getName() + " interrupted while waiting for " + - "newer writes added to local buffer"); - } catch (Exception e) { - LOG.error("UNEXPECTED", e); - } finally { - LOG.info(getName() + " exiting"); - } - } - } - - // thread to request HDFS to sync the WALEdits written by AsyncWriter - // to make those WALEdits durable on HDFS side - private class AsyncSyncer extends HasThread { - private long writtenTxid = 0; - private long txidToSync = 0; - private long lastSyncedTxid = 0; - private volatile boolean isSyncing = false; - private Object syncLock = new Object(); - - public AsyncSyncer(String name) { - super(name); - } - - public boolean isSyncing() { - return this.isSyncing; - } - - // wake up (called by AsyncWriter thread) AsyncSyncer thread - // to sync(flush) writes written by AsyncWriter in HDFS - public void setWrittenTxid(long txid) { - synchronized (this.syncLock) { - if (txid <= this.writtenTxid) - return; - - this.writtenTxid = txid; - this.syncLock.notify(); - } - } - - public void run() { - try { - while (!this.isInterrupted()) { - // 1. wait until AsyncWriter has written data to HDFS and - // called setWrittenTxid to wake up us - synchronized (this.syncLock) { - while (this.writtenTxid <= this.lastSyncedTxid) { - this.syncLock.wait(); - } - this.txidToSync = this.writtenTxid; - } - - // if this syncer's writes have been synced by other syncer: - // 1. just set lastSyncedTxid - // 2. don't do real sync, don't notify AsyncNotifier, don't logroll check - // regardless of whether the writer is null or not - if (this.txidToSync <= syncedTillHere.get()) { - this.lastSyncedTxid = this.txidToSync; - continue; - } - - // 2. do 'sync' to HDFS to provide durability - long now = EnvironmentEdgeManager.currentTimeMillis(); - try { - if (writer == null) { - // the only possible case where writer == null is as below: - // 1. t1: AsyncWriter append writes to hdfs, - // envokes AsyncSyncer 1 with writtenTxid==100 - // 2. t2: AsyncWriter append writes to hdfs, - // envokes AsyncSyncer 2 with writtenTxid==200 - // 3. t3: rollWriter starts, it grabs the updateLock which - // prevents further writes entering pendingWrites and - // wait for all items(200) in pendingWrites to append/sync - // to hdfs - // 4. t4: AsyncSyncer 2 finishes, now syncedTillHere==200 - // 5. t5: rollWriter close writer, set writer=null... - // 6. t6: AsyncSyncer 1 starts to use writer to do sync... before - // rollWriter set writer to the newly created Writer - // - // Now writer == null and txidToSync > syncedTillHere here: - // we need fail all the writes with txid <= txidToSync to avoid - // 'data loss' where user get successful write response but can't - // read the writes! - LOG.fatal("should never happen: has unsynced writes but writer is null!"); - asyncIOE = new IOException("has unsynced writes but writer is null!"); - failedTxid.set(this.txidToSync); - } else { - this.isSyncing = true; - writer.sync(); - this.isSyncing = false; - } - postSync(); - } catch (IOException e) { - LOG.fatal("Error while AsyncSyncer sync, request close of hlog ", e); - requestLogRoll(); - - asyncIOE = e; - failedTxid.set(this.txidToSync); - - this.isSyncing = false; - } - metrics.finishSync(EnvironmentEdgeManager.currentTimeMillis() - now); - - // 3. wake up AsyncNotifier to notify(wake-up) all pending 'put' - // handler threads on 'sync()' - this.lastSyncedTxid = this.txidToSync; - asyncNotifier.setFlushedTxid(this.lastSyncedTxid); - - // 4. check and do logRoll if needed - boolean logRollNeeded = false; - if (rollWriterLock.tryLock()) { - try { - logRollNeeded = checkLowReplication(); - } finally { - rollWriterLock.unlock(); - } - try { - if (logRollNeeded || writer != null && writer.getLength() > logrollsize) { - requestLogRoll(); - } - } catch (IOException e) { - LOG.warn("writer.getLength() failed,this failure won't block here"); - } - } - } - } catch (InterruptedException e) { - LOG.debug(getName() + " interrupted while waiting for " + - "notification from AsyncWriter thread"); - } catch (Exception e) { - LOG.error("UNEXPECTED", e); - } finally { - LOG.info(getName() + " exiting"); - } - } - } - - // thread to notify all write handler threads which are pending on - // their written WALEdits' durability(sync) - // why an extra 'notifier' thread is needed rather than letting - // AsyncSyncer thread itself notifies when sync is done is to let - // AsyncSyncer thread do next sync as soon as possible since 'notify' - // has heavy synchronization with all pending write handler threads - private class AsyncNotifier extends HasThread { - private long flushedTxid = 0; - private long lastNotifiedTxid = 0; - private Object notifyLock = new Object(); - - public AsyncNotifier(String name) { - super(name); - } - - public void setFlushedTxid(long txid) { - synchronized (this.notifyLock) { - if (txid <= this.flushedTxid) { - return; - } - - this.flushedTxid = txid; - this.notifyLock.notify(); - } - } - - public void run() { - try { - while (!this.isInterrupted()) { - synchronized (this.notifyLock) { - while (this.flushedTxid <= this.lastNotifiedTxid) { - this.notifyLock.wait(); - } - this.lastNotifiedTxid = this.flushedTxid; - } - - // notify(wake-up) all pending (write) handler thread - // (or logroller thread which also may pend on sync()) - synchronized (syncedTillHere) { - syncedTillHere.set(this.lastNotifiedTxid); - syncedTillHere.notifyAll(); - } - } - } catch (InterruptedException e) { - LOG.debug(getName() + " interrupted while waiting for " + - " notification from AsyncSyncer thread"); - } catch (Exception e) { - LOG.error("UNEXPECTED", e); - } finally { - LOG.info(getName() + " exiting"); - } - } - } - - // sync all known transactions - private void syncer() throws IOException { - syncer(this.unflushedEntries.get()); // sync all pending items - } - - // sync all transactions upto the specified txid - private void syncer(long txid) throws IOException { - synchronized (this.syncedTillHere) { - while (this.syncedTillHere.get() < txid) { + long currentSequence; + while (!isInterrupted()) { + int syncCount = 0; + SyncFuture takeSyncFuture; try { - this.syncedTillHere.wait(); - - if (txid <= this.failedTxid.get()) { - assert asyncIOE != null : - "current txid is among(under) failed txids, but asyncIOE is null!"; - throw asyncIOE; + while (true) { + // We have to process what we 'take' from the queue + takeSyncFuture = this.syncFutures.take(); + currentSequence = this.sequence; + long syncFutureSequence = takeSyncFuture.getRingBufferSequence(); + if (syncFutureSequence > currentSequence) { + throw new IllegalStateException("currentSequence=" + syncFutureSequence + + ", syncFutureSequence=" + syncFutureSequence); + } + // See if we can process any syncfutures BEFORE we go sync. + long currentHighestSyncedSequence = highestSyncedSequence.get(); + if (currentSequence < currentHighestSyncedSequence) { + syncCount += releaseSyncFuture(takeSyncFuture, currentHighestSyncedSequence, null); + // Done with the 'take'. Go around again and do a new 'take'. + continue; + } + break; } + // I got something. Lets run. Save off current sequence number in case it changes + // while we run. + long start = System.nanoTime(); + Throwable t = null; + try { + writer.sync(); + currentSequence = updateHighestSyncedSequence(currentSequence); + } catch (IOException e) { + LOG.error("Error syncing, request close of hlog ", e); + t = e; + } catch (Exception e) { + LOG.warn("UNEXPECTED", e); + t = e; + } finally { + // First release what we 'took' from the queue. + syncCount += releaseSyncFuture(takeSyncFuture, currentSequence, t); + // Can we release other syncs? + syncCount += releaseSyncFutures(currentSequence, t); + if (t != null) { + requestLogRoll(); + } else checkLogRoll(); + } + postSync(System.nanoTime() - start, syncCount); } catch (InterruptedException e) { - LOG.debug("interrupted while waiting for notification from AsyncNotifier"); + // Presume legit interrupt. + Thread.currentThread().interrupt(); + } catch (Throwable t) { + LOG.warn("UNEXPECTED, continuing", t); } } } } - @Override - public void postSync() {} - - @Override - public void postAppend(List entries) {} + /** + * Schedule a log roll if needed. + */ + void checkLogRoll() { + // Will return immediately if we are in the middle of a WAL log roll currently. + if (!rollWriterLock.tryLock()) return; + boolean lowReplication; + try { + lowReplication = checkLowReplication(); + } finally { + rollWriterLock.unlock(); + } + try { + if (lowReplication || writer != null && writer.getLength() > logrollsize) requestLogRoll(); + } catch (IOException e) { + LOG.warn("Writer.getLength() failed; continuing", e); + } + } /* - * @return whether log roll should be requested + * @return true if number of replicas for the WAL is lower than threshold */ private boolean checkLowReplication() { boolean logRollNeeded = false; @@ -1380,8 +1343,7 @@ class FSHLog implements HLog, Syncable { // value, then roll logs. try { int numCurrentReplicas = getLogReplication(); - if (numCurrentReplicas != 0 - && numCurrentReplicas < this.minTolerableReplication) { + if (numCurrentReplicas != 0 && numCurrentReplicas < this.minTolerableReplication) { if (this.lowReplicationRollEnabled) { if (this.consecutiveLogRolls.get() < this.lowReplicationRollLimit) { LOG.warn("HDFS pipeline error detected. " + "Found " @@ -1401,7 +1363,6 @@ class FSHLog implements HLog, Syncable { } } } else if (numCurrentReplicas >= this.minTolerableReplication) { - if (!this.lowReplicationRollEnabled) { // The new writer's log replicas is always the default value. // So we should not enable LowReplication-Roller. If numEntries @@ -1417,11 +1378,66 @@ class FSHLog implements HLog, Syncable { } } catch (Exception e) { LOG.warn("Unable to invoke DFSOutputStream.getNumCurrentReplicas" + e + - " still proceeding ahead..."); + " still proceeding ahead..."); } return logRollNeeded; } + private SyncFuture publishSyncOnRingBuffer() { + long sequence = this.disruptor.getRingBuffer().next(); + SyncFuture syncFuture = getSyncFuture(sequence); + try { + RingBufferTruck truck = this.disruptor.getRingBuffer().get(sequence); + truck.loadPayload(syncFuture); + } finally { + this.disruptor.getRingBuffer().publish(sequence); + } + return syncFuture; + } + + // Sync all known transactions + private void publishSyncThenBlockOnCompletion() throws IOException { + blockOnSync(publishSyncOnRingBuffer()); + } + + private void blockOnSync(final SyncFuture syncFuture) throws IOException { + // Now we have published the ringbuffer, halt the current thread until we get an answer back. + try { + syncFuture.get(); + } catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + IOException ioe = new InterruptedIOException(); + ioe.initCause(ie); + throw ioe; + } catch (ExecutionException e) { + throw ensureIOException(e.getCause()); + } + } + + private SyncFuture getSyncFuture(final long sequence) { + SyncFuture syncFuture = this.syncFuturesByHandler.get(Thread.currentThread()); + if (syncFuture == null) { + syncFuture = new SyncFuture(); + this.syncFuturesByHandler.put(Thread.currentThread(), syncFuture); + } + return syncFuture.reset(sequence); + } + + @Override + public void postSync(final long timeInNanos, final int handlerSyncs) { + // TODO: Add metric for handler syncs done at a time. + if (this.metrics != null) metrics.finishSync(timeInNanos/1000000); + } + + @Override + public long postAppend(final Entry e, final long elapsedTime) { + long len = 0; + if (this.metrics == null) return len; + for (KeyValue kv : e.getEdit().getKeyValues()) len += kv.getLength(); + metrics.finishAppend(elapsedTime, len); + return len; + } + /** * This method gets the datanode replication count for the current HLog. * @@ -1451,25 +1467,26 @@ class FSHLog implements HLog, Syncable { @Override public void hsync() throws IOException { - syncer(); + publishSyncThenBlockOnCompletion(); } @Override public void hflush() throws IOException { - syncer(); + publishSyncThenBlockOnCompletion(); } @Override public void sync() throws IOException { - syncer(); + publishSyncThenBlockOnCompletion(); } @Override + // txid is unused. txid is an implementation detail. It should not leak outside of WAL. public void sync(long txid) throws IOException { - syncer(txid); + publishSyncThenBlockOnCompletion(); } - private void requestLogRoll() { + void requestLogRoll() { if (!this.listeners.isEmpty()) { for (WALActionsListener i: this.listeners) { i.logRollRequested(); @@ -1477,44 +1494,6 @@ class FSHLog implements HLog, Syncable { } } - // TODO: Remove info. Unused. - protected void doWrite(HRegionInfo info, HLogKey logKey, WALEdit logEdit, - HTableDescriptor htd) - throws IOException { - if (!this.enabled) { - return; - } - if (!this.listeners.isEmpty()) { - for (WALActionsListener i: this.listeners) { - i.visitLogEntryBeforeWrite(htd, logKey, logEdit); - } - } - try { - long now = EnvironmentEdgeManager.currentTimeMillis(); - // coprocessor hook: - if (!coprocessorHost.preWALWrite(info, logKey, logEdit)) { - if (logEdit.isReplay()) { - // set replication scope null so that this won't be replicated - logKey.setScopes(null); - } - // write to our buffer for the Hlog file. - this.pendingWrites.add(new HLog.Entry(logKey, logEdit)); - } - long took = EnvironmentEdgeManager.currentTimeMillis() - now; - coprocessorHost.postWALWrite(info, logKey, logEdit); - long len = 0; - for (KeyValue kv : logEdit.getKeyValues()) { - len += kv.getLength(); - } - this.metrics.finishAppend(took, len); - } catch (IOException e) { - LOG.fatal("Could not append. Requesting close of hlog", e); - requestLogRoll(); - throw e; - } - } - - /** @return How many items have been added to the log */ int getNumEntries() { return numEntries.get(); @@ -1522,7 +1501,7 @@ class FSHLog implements HLog, Syncable { /** @return the number of rolled log files */ public int getNumRolledLogFiles() { - return hlogSequenceNums.size(); + return byWalRegionSequenceIds.size(); } /** @return the number of log files in use */ @@ -1535,7 +1514,7 @@ class FSHLog implements HLog, Syncable { /** @return the size of log files in use */ @Override public long getLogFileSize() { - return totalLogSize.get() + curLogSize; + return this.totalLogSize.get(); } @Override @@ -1546,12 +1525,13 @@ class FSHLog implements HLog, Syncable { " - because the server is closing."); return false; } - synchronized (oldestSeqNumsLock) { - oldRegionSeqNum = this.oldestUnflushedSeqNums.remove(encodedRegionName); + synchronized (regionSequenceIdLock) { + oldRegionSeqNum = this.oldestUnflushedRegionSequenceIds.remove(encodedRegionName); if (oldRegionSeqNum != null) { - Long oldValue = this.oldestFlushingSeqNums.put(encodedRegionName, oldRegionSeqNum); - assert oldValue == null : "Flushing map not cleaned up for " - + Bytes.toString(encodedRegionName); + Long oldValue = + this.lowestFlushingRegionSequenceIds.put(encodedRegionName, oldRegionSeqNum); + assert oldValue == + null : "Flushing map not cleaned up for " + Bytes.toString(encodedRegionName); } } if (oldRegionSeqNum == null) { @@ -1567,10 +1547,9 @@ class FSHLog implements HLog, Syncable { } @Override - public void completeCacheFlush(final byte [] encodedRegionName) - { - synchronized (oldestSeqNumsLock) { - this.oldestFlushingSeqNums.remove(encodedRegionName); + public void completeCacheFlush(final byte [] encodedRegionName) { + synchronized (regionSequenceIdLock) { + this.lowestFlushingRegionSequenceIds.remove(encodedRegionName); } closeBarrier.endOp(); } @@ -1578,11 +1557,11 @@ class FSHLog implements HLog, Syncable { @Override public void abortCacheFlush(byte[] encodedRegionName) { Long currentSeqNum = null, seqNumBeforeFlushStarts = null; - synchronized (oldestSeqNumsLock) { - seqNumBeforeFlushStarts = this.oldestFlushingSeqNums.remove(encodedRegionName); + synchronized (regionSequenceIdLock) { + seqNumBeforeFlushStarts = this.lowestFlushingRegionSequenceIds.remove(encodedRegionName); if (seqNumBeforeFlushStarts != null) { currentSeqNum = - this.oldestUnflushedSeqNums.put(encodedRegionName, seqNumBeforeFlushStarts); + this.oldestUnflushedRegionSequenceIds.put(encodedRegionName, seqNumBeforeFlushStarts); } } closeBarrier.endOp(); @@ -1608,7 +1587,7 @@ class FSHLog implements HLog, Syncable { * @return dir */ protected Path getDir() { - return dir; + return fullPathLogDir; } static Path getHLogArchivePath(Path oldLogDir, Path p) { @@ -1623,15 +1602,6 @@ class FSHLog implements HLog, Syncable { ClassSize.OBJECT + (5 * ClassSize.REFERENCE) + ClassSize.ATOMIC_INTEGER + Bytes.SIZEOF_INT + (3 * Bytes.SIZEOF_LONG)); - private static void usage() { - System.err.println("Usage: HLog "); - System.err.println("Arguments:"); - System.err.println(" --dump Dump textual representation of passed one or more files"); - System.err.println(" For example: HLog --dump hdfs://example.com:9000/hbase/.logs/MACHINE/LOGFILE"); - System.err.println(" --split Split the passed directory of WAL logs"); - System.err.println(" For example: HLog --split hdfs://example.com:9000/hbase/.logs/DIR"); - } - private static void split(final Configuration conf, final Path p) throws IOException { FileSystem fs = FileSystem.get(conf); @@ -1652,17 +1622,319 @@ class FSHLog implements HLog, Syncable { return coprocessorHost; } - /** Provide access to currently deferred sequence num for tests */ - boolean hasUnSyncedEntries() { - return this.lastUnSyncedTxid > this.syncedTillHere.get(); - } - @Override public long getEarliestMemstoreSeqNum(byte[] encodedRegionName) { - Long result = oldestUnflushedSeqNums.get(encodedRegionName); + Long result = oldestUnflushedRegionSequenceIds.get(encodedRegionName); return result == null ? HConstants.NO_SEQNUM : result.longValue(); } + /** + * This class is used coordinating two threads holding one thread at a + * 'safe point' while the orchestrating thread does some work that requires the first thread + * paused: e.g. holding the WAL writer while its WAL is swapped out from under it by another + * thread. + * + *

Thread A signals Thread B to hold when it gets to a 'safe point'. Thread A wait until + * Thread B gets there. When the 'safe point' has been attained, Thread B signals Thread A. + * Thread B then holds at the 'safe point'. Thread A on notification that Thread B is paused, + * goes ahead and does the work it needs to do while Thread B is holding. When Thread A is done, + * it flags B and then Thread A and Thread B continue along on their merry way. Pause and + * signalling 'zigzags' between the two participating threads. We use two latches -- one the + * inverse of the other -- pausing and signaling when states are achieved. + * + *

To start up the drama, Thread A creates an instance of this class each time it would do + * this zigzag dance and passes it to Thread B (these classes use Latches so it is one shot + * only). Thread B notices the new instance (via reading a volatile reference or how ever) and it + * starts to work toward the 'safe point'. Thread A calls {@link #waitSafePoint()} when it + * cannot proceed until the Thread B 'safe point' is attained. Thread A will be held inside in + * {@link #waitSafePoint()} until Thread B reaches the 'safe point'. Once there, Thread B + * frees Thread A by calling {@link #safePointAttained()}. Thread A now knows Thread B + * is at the 'safe point' and that it is holding there (When Thread B calls + * {@link #safePointAttained()} it blocks here until Thread A calls {@link #releaseSafePoint()}). + * Thread A proceeds to do what it needs to do while Thread B is paused. When finished, + * it lets Thread B lose by calling {@link #releaseSafePoint()} and away go both Threads again. + */ + static class SafePointZigZagLatch { + /** + * Count down this latch when safe point attained. + */ + private volatile CountDownLatch safePointAttainedLatch = new CountDownLatch(1); + /** + * Latch to wait on. Will be released when we can proceed. + */ + private volatile CountDownLatch safePointReleasedLatch = new CountDownLatch(1); + + /** + * For Thread A to call when it is ready to wait on the 'safe point' to be attained. + * Thread A will be held in here until Thread B calls {@link #safePointAttained()} + * @throws InterruptedException + * @throws ExecutionException + * @param syncFuture We need this as barometer on outstanding syncs. If it comes home with + * an exception, then something is up w/ our syncing. + * @return The passed syncFuture + * @throws FailedSyncBeforeLogCloseException + */ + SyncFuture waitSafePoint(final SyncFuture syncFuture) + throws InterruptedException, FailedSyncBeforeLogCloseException { + while (true) { + if (this.safePointAttainedLatch.await(1, TimeUnit.NANOSECONDS)) break; + if (syncFuture.isThrowable()) { + throw new FailedSyncBeforeLogCloseException(syncFuture.getThrowable()); + } + } + return syncFuture; + } + + /** + * Called by Thread B when it attains the 'safe point'. In this method, Thread B signals + * Thread A it can proceed. Thread B will be held in here until {@link #releaseSafePoint()} + * is called by Thread A. + * @throws InterruptedException + */ + void safePointAttained() throws InterruptedException { + this.safePointAttainedLatch.countDown(); + this.safePointReleasedLatch.await(); + } + + /** + * Called by Thread A when it is done with the work it needs to do while Thread B is + * halted. This will release the Thread B held in a call to {@link #safePointAttained()} + */ + void releaseSafePoint() { + this.safePointReleasedLatch.countDown(); + } + + /** + * @return True is this is a 'cocked', fresh instance, and not one that has already fired. + */ + boolean isCocked() { + return this.safePointAttainedLatch.getCount() > 0 && + this.safePointReleasedLatch.getCount() > 0; + } + } + + /** + * Handler that is run by the disruptor ringbuffer consumer. Consumer is a SINGLE + * 'writer/appender' thread. Appends edits and starts up sync runs. Tries its best to batch up + * syncs. There is no discernible benefit batching appends so we just append as they come in + * because it simplifies the below implementation. See metrics for batching effectiveness + * (In measurement, at 100 concurrent handlers writing 1k, we are batching > 10 appends and 10 + * handler sync invocations for every actual dfsclient sync call; at 10 concurrent handlers, + * YMMV). + *

Herein, we have an array into which we store the sync futures as they come in. When we + * have a 'batch', we'll then pass what we have collected to a SyncRunner thread to do the + * filesystem sync. When it completes, it will then call {@link SyncFuture#done(long)} on each + * of SyncFutures in the batch to release blocked Handler threads. + *

I've tried various effects to try and make latencies low while keeping throughput high. + * I've tried keeping a single Queue of SyncFutures in this class appending to its tail as the + * syncs coming and having sync runner threads poll off the head to 'finish' completed + * SyncFutures. I've tried linkedlist, and various from concurrent utils whether + * LinkedBlockingQueue or ArrayBlockingQueue, etc. The more points of synchronization, the + * more 'work' (according to 'perf stats') that has to be done; small increases in stall + * percentages seem to have a big impact on throughput/latencies. The below model where we have + * an array into which we stash the syncs and then hand them off to the sync thread seemed like + * a decent compromise. See HBASE-8755 for more detail. + */ + class RingBufferEventHandler implements EventHandler, LifecycleAware { + private final SyncRunner [] syncRunners; + private final SyncFuture [] syncFutures; + // Had 'interesting' issues when this was non-volatile. On occasion, we'd not pass all + // syncFutures to the next sync'ing thread. + private volatile int syncFuturesCount = 0; + private volatile SafePointZigZagLatch zigzagLatch; + /** + * Object to block on while waiting on safe point. + */ + private final Object safePointWaiter = new Object(); + private volatile boolean shutdown = false; + + /** + * Which syncrunner to use next. + */ + private int syncRunnerIndex; + + RingBufferEventHandler(final int syncRunnerCount, final int maxHandlersCount) { + this.syncFutures = new SyncFuture[maxHandlersCount]; + this.syncRunners = new SyncRunner[syncRunnerCount]; + for (int i = 0; i < syncRunnerCount; i++) { + this.syncRunners[i] = new SyncRunner("sync." + i, maxHandlersCount); + } + } + + private void cleanupOutstandingSyncsOnException(final long sequence, final Exception e) { + for (int i = 0; i < this.syncFuturesCount; i++) this.syncFutures[i].done(sequence, e); + this.syncFuturesCount = 0; + } + + @Override + // We can set endOfBatch in the below method if at end of our this.syncFutures array + public void onEvent(final RingBufferTruck truck, final long sequence, boolean endOfBatch) + throws Exception { + // Appends and syncs are coming in order off the ringbuffer. We depend on this fact. We'll + // add appends to dfsclient as they come in. Batching appends doesn't give any significant + // benefit on measurement. Handler sync calls we will batch up. + + // TODO: Trace only working for appends, not for syncs. + TraceScope scope = + truck.getSpanPayload() != null? Trace.continueSpan(truck.getSpanPayload()): null; + try { + if (truck.getSyncFuturePayload() != null) { + this.syncFutures[this.syncFuturesCount++] = truck.getSyncFuturePayload(); + // Force flush of syncs if we are carrying a full complement of syncFutures. + if (this.syncFuturesCount == this.syncFutures.length) endOfBatch = true; + } else if (truck.getFSWALEntryPayload() != null) { + try { + append(truck.getFSWALEntryPayload()); + } catch (Exception e) { + // If append fails, presume any pending syncs will fail too; let all waiting handlers + // know of the exception. + cleanupOutstandingSyncsOnException(sequence, e); + // Return to keep processing. + return; + } + } else { + // They can't both be null. Fail all up to this!!! + cleanupOutstandingSyncsOnException(sequence, + new IllegalStateException("Neither append nor sync")); + // Return to keep processing. + return; + } + + // TODO: Check size and if big go ahead and call a sync if we have enough data. + + // If not a batch, return to consume more events from the ring buffer before proceeding; + // we want to get up a batch of syncs and appends before we go do a filesystem sync. + if (!endOfBatch || this.syncFuturesCount <= 0) return; + + // Now we have a batch. + + if (LOG.isTraceEnabled()) { + LOG.trace("Sequence=" + sequence + ", syncCount=" + this.syncFuturesCount); + } + + // Below expects that the offer 'transfers' responsibility for the outstanding syncs to the + // syncRunner. + int index = Math.abs(this.syncRunnerIndex++) % this.syncRunners.length; + this.syncRunners[index].offer(sequence, this.syncFutures, this.syncFuturesCount); + attainSafePoint(sequence); + this.syncFuturesCount = 0; + } catch (Throwable t) { + LOG.error("UNEXPECTED!!!", t); + } finally { + // This scope only makes sense for the append. Syncs will be pulled-up short so tracing + // will not give a good representation. TODO: Fix. + if (scope != null) scope.close(); + } + } + + SafePointZigZagLatch attainSafePoint() { + this.zigzagLatch = new SafePointZigZagLatch(); + return this.zigzagLatch; + } + + /** + * Check if we should attain safe point. If so, go there and then wait till signalled before + * we proceeding. + */ + private void attainSafePoint(final long currentSequence) { + if (this.zigzagLatch == null || !this.zigzagLatch.isCocked()) return; + // If here, another thread is waiting on us to get to safe point. Don't leave it hanging. + try { + // Wait on outstanding syncers; wait for them to finish syncing (unless we've been + // shutdown or unless our latch has been thrown because we have been aborted). + while (!this.shutdown && this.zigzagLatch.isCocked() && + highestSyncedSequence.get() < currentSequence) { + synchronized (this.safePointWaiter) { + this.safePointWaiter.wait(0, 1); + } + } + // Tell waiting thread we've attained safe point + this.zigzagLatch.safePointAttained(); + } catch (InterruptedException e) { + LOG.warn("Interrupted ", e); + Thread.currentThread().interrupt(); + } + } + + /** + * Append to the WAL. Does all CP and WAL listener calls. + * @param entry + * @throws Exception + */ + void append(final FSWALEntry entry) throws Exception { + // TODO: WORK ON MAKING THIS APPEND FASTER. OING WAY TOO MUCH WORK WITH CPs, PBing, etc. + + long start = EnvironmentEdgeManager.currentTimeMillis(); + byte [] encodedRegionName = entry.getKey().getEncodedRegionName(); + try { + // We are about to append this edit; update the region-scoped sequence number. Do it + // here inside this single appending/writing thread. Events are ordered on the ringbuffer + // so region sequenceids will also be in order. + long regionSequenceId = entry.getRegionSequenceIdReference().incrementAndGet(); + // Set the region-scoped sequence number back up into the key ("late-binding" -- + // setting before append). + entry.getKey().setLogSeqNum(regionSequenceId); + // Coprocessor hook. + if (!coprocessorHost.preWALWrite(entry.getHRegionInfo(), entry.getKey(), + entry.getEdit())) { + if (entry.getEdit().isReplay()) { + // Set replication scope null so that this won't be replicated + entry.getKey().setScopes(null); + } + } + if (!listeners.isEmpty()) { + for (WALActionsListener i: listeners) { + // TODO: Why does listener take a table description and CPs take a regioninfo? Fix. + i.visitLogEntryBeforeWrite(entry.getHTableDescriptor(), entry.getKey(), + entry.getEdit()); + } + } + writer.append(entry); + assert highestUnsyncedSequence < entry.getSequence(); + highestUnsyncedSequence = entry.getSequence(); + Long lRegionSequenceId = Long.valueOf(regionSequenceId); + highestRegionSequenceIds.put(encodedRegionName, lRegionSequenceId); + if (entry.isInMemstore()) { + oldestUnflushedRegionSequenceIds.putIfAbsent(encodedRegionName, lRegionSequenceId); + } + coprocessorHost.postWALWrite(entry.getHRegionInfo(), entry.getKey(), entry.getEdit()); + // Update metrics. + postAppend(entry, EnvironmentEdgeManager.currentTimeMillis() - start); + } catch (Exception e) { + LOG.fatal("Could not append. Requesting close of hlog", e); + requestLogRoll(); + throw e; + } + numEntries.incrementAndGet(); + } + + @Override + public void onStart() { + for (SyncRunner syncRunner: this.syncRunners) syncRunner.start(); + } + + @Override + public void onShutdown() { + for (SyncRunner syncRunner: this.syncRunners) syncRunner.interrupt(); + } + } + + private static IOException ensureIOException(final Throwable t) { + return (t instanceof IOException)? (IOException)t: new IOException(t); + } + + private static void usage() { + System.err.println("Usage: HLog "); + System.err.println("Arguments:"); + System.err.println(" --dump Dump textual representation of passed one or more files"); + System.err.println(" For example: " + + "FSHLog --dump hdfs://example.com:9000/hbase/.logs/MACHINE/LOGFILE"); + System.err.println(" --split Split the passed directory of WAL logs"); + System.err.println(" For example: " + + "FSHLog --split hdfs://example.com:9000/hbase/.logs/DIR"); + System.err.println(" --perf Write the same key times to a WAL: e.g. FSHLog --perf 10"); + } + /** * Pass one or more log file names and it will either dump out a text version * on stdout or split the specified log files. @@ -1678,6 +1950,25 @@ class FSHLog implements HLog, Syncable { // either dump using the HLogPrettyPrinter or split, depending on args if (args[0].compareTo("--dump") == 0) { HLogPrettyPrinter.run(Arrays.copyOfRange(args, 1, args.length)); + } else if (args[0].compareTo("--perf") == 0) { + final int count = Integer.parseInt(args[1]); + // Put up a WAL and just keep adding same edit to it. Simple perf test. + Configuration conf = HBaseConfiguration.create(); + Path rootDir = FSUtils.getRootDir(conf); + FileSystem fs = rootDir.getFileSystem(conf); + FSHLog wal = + new FSHLog(fs, rootDir, "perflog", "oldPerflog", conf, null, false, "perf", false); + long start = System.nanoTime(); + WALEdit walEdit = new WALEdit(); + walEdit.add(new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("family"), + Bytes.toBytes("qualifier"), -1, new byte [1000])); + for (AtomicLong i = new AtomicLong(0); i.get() < count; i.incrementAndGet()) { + wal.append(HRegionInfo.FIRST_META_REGIONINFO, TableName.META_TABLE_NAME, walEdit, start, + HTableDescriptor.META_TABLEDESC, i); + wal.sync(); + } + wal.close(); + LOG.info("Write " + count + " 1k edits in " + (System.nanoTime() - start) + "nanos"); } else if (args[0].compareTo("--split") == 0) { Configuration conf = HBaseConfiguration.create(); for (int i = 1; i < args.length; i++) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java new file mode 100644 index 00000000000..0d65a545609 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java @@ -0,0 +1,80 @@ +/** + * 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.util.concurrent.atomic.AtomicLong; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HTableDescriptor; + +/** + * A WAL Entry for {@link FSHLog} implementation. Immutable. + * It is a subclass of {@link HLog.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. + */ +@InterfaceAudience.Private +class FSWALEntry extends HLog.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; + private final transient AtomicLong regionSequenceIdReference; + private final transient boolean inMemstore; + private final transient HTableDescriptor htd; + private final transient HRegionInfo hri; + + FSWALEntry(final long sequence, final HLogKey key, final WALEdit edit, + final AtomicLong referenceToRegionSequenceId, final boolean inMemstore, + final HTableDescriptor htd, final HRegionInfo hri) { + super(key, edit); + this.regionSequenceIdReference = referenceToRegionSequenceId; + this.inMemstore = inMemstore; + this.htd = htd; + this.hri = hri; + this.sequence = sequence; + } + + public String toString() { + return "sequence=" + this.sequence + ", " + super.toString(); + }; + + AtomicLong getRegionSequenceIdReference() { + return this.regionSequenceIdReference; + } + + boolean isInMemstore() { + return this.inMemstore; + } + + HTableDescriptor getHTableDescriptor() { + return this.htd; + } + + HRegionInfo getHRegionInfo() { + return this.hri; + } + + /** + * @return The sequence on the ring buffer when this edit was added. + */ + long getSequence() { + return this.sequence; + } +} \ No newline at end of file diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java index 4aa5be4ddb4..ce29783b69a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java @@ -42,7 +42,9 @@ 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 { @@ -52,7 +54,8 @@ public interface HLog { // 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 */ + /** The hbase:meta region's HLog filename extension.*/ + // TODO: Implementation detail. Does not belong in here. String META_HLOG_FILE_EXTN = ".meta"; /** @@ -63,12 +66,14 @@ public interface HLog { String WAL_TRAILER_WARN_SIZE = "hbase.regionserver.waltrailer.warn.size"; int DEFAULT_WAL_TRAILER_WARN_SIZE = 1024 * 1024; // 1MB - // TODO: Implemenation detail. Why in here? + // TODO: Implementation detail. Why in here? Pattern EDITFILES_NAME_PATTERN = Pattern.compile("-?[0-9]+"); String RECOVERED_LOG_TMPFILE_SUFFIX = ".temp"; + /** + * WAL Reader Interface + */ interface Reader { - /** * @param fs File system. * @param path Path. @@ -90,12 +95,16 @@ public interface HLog { /** * @return the WALTrailer of the current HLog. It may be null in case of legacy or corrupt WAL - * files. + * files. */ - // TODO: What we need a trailer on WAL for? + // 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; @@ -108,17 +117,19 @@ public interface HLog { long getLength() throws IOException; /** - * Sets HLog's WALTrailer. This trailer is appended at the end of WAL on closing. + * 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 with it's key. - * Only used when splitting logs. + * 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? class Entry implements Writable { private WALEdit edit; private HLogKey key; @@ -135,7 +146,6 @@ public interface HLog { * @param key log's key */ public Entry(HLogKey key, WALEdit edit) { - super(); this.key = key; this.edit = edit; } @@ -161,8 +171,7 @@ public interface HLog { /** * Set compression context for this entry. * - * @param compressionContext - * Compression context + * @param compressionContext Compression context */ public void setCompressionContext(CompressionContext compressionContext) { edit.setCompressionContext(compressionContext); @@ -189,14 +198,14 @@ public interface HLog { } /** - * registers WALActionsListener + * Registers WALActionsListener * * @param listener */ void registerWALActionsListener(final WALActionsListener listener); /** - * unregisters WALActionsListener + * Unregisters WALActionsListener * * @param listener */ @@ -217,7 +226,7 @@ public interface HLog { * @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. @@ -250,8 +259,7 @@ public interface HLog { * @throws org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException * @throws IOException */ - byte[][] rollWriter(boolean force) throws FailedLogCloseException, - IOException; + byte[][] rollWriter(boolean force) throws FailedLogCloseException, IOException; /** * Shut down the log. @@ -261,43 +269,68 @@ public interface HLog { void close() throws IOException; /** - * Shut down the log and delete the log directory + * 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 appendNoSync(HRegionInfo, TableName, WALEdit, List, long, HTableDescriptor), + * Same as {@link #appendNoSync(HRegionInfo, TableName, WALEdit, List, long, HTableDescriptor, + * AtomicLong, boolean, long, long)} * except it causes a sync on the log - * @param sequenceId of the region. + * @param info + * @param tableName + * @param edits + * @param now + * @param htd + * @param sequenceId + * @throws IOException */ @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. - * @param entries + * For notification post append to the writer. Used by metrics system at least. + * @param entry + * @param elapsedTime + * @return Size of this append. */ - void postAppend(final List entries); + long postAppend(final Entry entry, final long elapsedTime); /** - * For notification post writer sync. + * 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(); + void postSync(final long timeInMillis, final int handlerSyncs); /** - * Append a set of edits to the log. Log edits are keyed by (encoded) regionName, rowname, and - * log-sequence-id. The HLog is not flushed after this transaction is written to the log. + * 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. + * Call {@link #sync()} to flush/sync all outstanding edits/appends. * @param info * @param tableName * @param edits - * @param clusterIds The clusters that have consumed the change (for replication) + * @param clusterIds * @param now * @param htd - * @param sequenceId of the region - * @return txid of this transaction + * @param sequenceId A reference to the atomic long the info 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 */ long appendNoSync(HRegionInfo info, TableName tableName, WALEdit edits, @@ -311,6 +344,14 @@ public interface HLog { void sync() throws IOException; + /** + * @param txid Transaction id to sync to. + * @throws IOException + * @deprecated Since 0.96.2. Just call {@link #sync()}. txid should not be allowed + * outside the implementation. + */ + // TODO: Why is this exposed? txid is an internal detail. + @Deprecated void sync(long txid) throws IOException; /** @@ -318,7 +359,7 @@ public interface HLog { * in order to be able to do cleanup. This method tells WAL that some region is about * to flush memstore. * - * We stash the oldest seqNum for the region, and let the the next edit inserted in this + *

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, diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogFactory.java index 2b9130cde0f..85aaef8e2a1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogFactory.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogFactory.java @@ -59,8 +59,8 @@ public class HLogFactory { public static HLog createMetaHLog(final FileSystem fs, final Path root, final String logName, final Configuration conf, final List listeners, final String prefix) throws IOException { - return new FSHLog(fs, root, logName, HConstants.HREGION_OLDLOGDIR_NAME, - conf, listeners, false, prefix, true); + return new FSHLog(fs, root, logName, HConstants.HREGION_OLDLOGDIR_NAME, conf, listeners, + false, prefix, true); } /* diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java index 42a125f72f9..dcb819e05d9 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java @@ -189,6 +189,14 @@ public class HLogKey implements WritableComparable { return this.logSeqNum; } + /** + * Allow that the log sequence id to be set post-construction. + * @param sequence + */ + void setLogSeqNum(final long sequence) { + this.logSeqNum = sequence; + } + /** * @return the write time */ @@ -439,17 +447,17 @@ public class HLogKey implements WritableComparable { // 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 { + public WALKey.Builder getBuilder(WALCellCodec.ByteStringCompressor compressor) + throws IOException { WALKey.Builder builder = WALKey.newBuilder(); if (compressionContext == null) { builder.setEncodedRegionName(ZeroCopyLiteralByteString.wrap(this.encodedRegionName)); builder.setTableName(ZeroCopyLiteralByteString.wrap(this.tablename.getName())); } else { - builder.setEncodedRegionName( - compressor.compress(this.encodedRegionName, compressionContext.regionDict)); + builder.setEncodedRegionName(compressor.compress(this.encodedRegionName, + compressionContext.regionDict)); builder.setTableName(compressor.compress(this.tablename.getName(), - compressionContext.tableDict)); + compressionContext.tableDict)); } builder.setLogSequenceNumber(this.logSeqNum); builder.setWriteTime(writeTime); @@ -467,7 +475,8 @@ public class HLogKey implements WritableComparable { } if (scopes != null) { for (Map.Entry e : scopes.entrySet()) { - ByteString family = (compressionContext == null) ? ZeroCopyLiteralByteString.wrap(e.getKey()) + ByteString family = (compressionContext == null) ? + ZeroCopyLiteralByteString.wrap(e.getKey()) : compressor.compress(e.getKey(), compressionContext.familyDict); builder.addScopes(FamilyScope.newBuilder() .setFamily(family).setScopeType(ScopeType.valueOf(e.getValue()))); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogUtil.java index 684f78ce510..9825feef6c7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogUtil.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogUtil.java @@ -189,7 +189,7 @@ public class HLogUtil { serverName = ServerName.parseServerName(logDirName); } catch (IllegalArgumentException ex) { serverName = null; - LOG.warn("Invalid log file path=" + logFile, ex); + 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); @@ -266,9 +266,9 @@ public class HLogUtil { WALEdit e = WALEdit.createCompaction(c); long now = EnvironmentEdgeManager.currentTimeMillis(); TableName tn = TableName.valueOf(c.getTableName().toByteArray()); - long txid = log.appendNoSync(info, tn, e, new ArrayList(), now, htd, sequenceId, - false, HConstants.NO_NONCE, HConstants.NO_NONCE); - log.sync(txid); + log.appendNoSync(info, tn, e, new ArrayList(), now, htd, sequenceId, false, + HConstants.NO_NONCE, HConstants.NO_NONCE); + log.sync(); if (LOG.isTraceEnabled()) { LOG.trace("Appended compaction marker " + TextFormat.shortDebugString(c)); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogWriter.java index 6d10b26e50b..6f07c4e592e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogWriter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogWriter.java @@ -64,7 +64,8 @@ public class ProtobufLogWriter extends WriterBase { @Override @SuppressWarnings("deprecation") - public void init(FileSystem fs, Path path, Configuration conf, boolean overwritable) throws IOException { + public void init(FileSystem fs, Path path, Configuration conf, boolean overwritable) + throws IOException { super.init(fs, path, conf, overwritable); assert this.output == null; boolean doCompress = initializeCompressionContext(conf, path); @@ -99,8 +100,8 @@ public class ProtobufLogWriter extends WriterBase { @Override public void append(HLog.Entry entry) throws IOException { entry.setCompressionContext(compressionContext); - entry.getKey().getBuilder(compressor).setFollowingKvCount(entry.getEdit().size()) - .build().writeDelimitedTo(output); + entry.getKey().getBuilder(compressor). + setFollowingKvCount(entry.getEdit().size()).build().writeDelimitedTo(output); for (KeyValue kv : entry.getEdit().getKeyValues()) { // cellEncoder must assume little about the stream, since we write PB and cells in turn. cellEncoder.write(kv); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/RingBufferTruck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/RingBufferTruck.java new file mode 100644 index 00000000000..197cefb80b2 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/RingBufferTruck.java @@ -0,0 +1,77 @@ +/** + * + * 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.classification.InterfaceAudience; +import org.cloudera.htrace.Span; + +import com.lmax.disruptor.EventFactory; + +/** + * A 'truck' to carry a payload across the {@link FSHLog} ring buffer from Handler to WAL. + * Has EITHER a {@link FSWALEntry} for making an append OR it has a {@link SyncFuture} to + * represent a 'sync' invocation. Immutable but instances get recycled on the ringbuffer. + */ +@InterfaceAudience.Private +class RingBufferTruck { + /** + * Either this syncFuture is set or entry is set, but not both. + */ + private SyncFuture syncFuture; + private FSWALEntry entry; + + /** + * The tracing span for this entry. Can be null. + * TODO: Fix up tracing. + */ + private Span span; + + void loadPayload(final FSWALEntry entry, final Span span) { + this.entry = entry; + this.span = span; + this.syncFuture = null; + } + + void loadPayload(final SyncFuture syncFuture) { + this.syncFuture = syncFuture; + this.entry = null; + this.span = null; + } + + FSWALEntry getFSWALEntryPayload() { + return this.entry; + } + + SyncFuture getSyncFuturePayload() { + return this.syncFuture; + } + + Span getSpanPayload() { + return this.span; + } + + /** + * Factory for making a bunch of these. Needed by the ringbuffer/disruptor. + */ + final static EventFactory EVENT_FACTORY = new EventFactory() { + public RingBufferTruck newInstance() { + return new RingBufferTruck(); + } + }; +} \ No newline at end of file diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SyncFuture.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SyncFuture.java new file mode 100644 index 00000000000..f0c3fa02f1f --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SyncFuture.java @@ -0,0 +1,152 @@ +/** + * 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.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; + +import org.apache.hadoop.classification.InterfaceAudience; + +/** + * A Future on a filesystem sync call. It given to a client or 'Handler' for it to wait on till + * the sync completes. + * + *

Handlers coming in call append, append, append, and then do a flush/sync of + * the edits they have appended the WAL before returning. Since sync takes a while to + * complete, we give the Handlers back this sync future to wait on until the + * actual HDFS sync completes. Meantime this sync future goes across the ringbuffer and into a + * sync runner thread; when it completes, it finishes up the future, the handler get or failed + * check completes and the Handler can then progress. + *

+ * This is just a partial implementation of Future; we just implement get and + * failure. Unimplemented methods throw {@link UnsupportedOperationException}. + *

+ * There is not a one-to-one correlation between dfs sync invocations and + * instances of this class. A single dfs sync call may complete and mark many + * SyncFutures as done; i.e. we batch up sync calls rather than do a dfs sync + * call every time a Handler asks for it. + *

+ * SyncFutures are immutable but recycled. Call {@link #reset(long)} before use even if it + * the first time, start the sync, then park the 'hitched' thread on a call to + * {@link #get()} + */ +@InterfaceAudience.Private +class SyncFuture { + private static final long NOT_DONE = 0; + + /** + * The sequence at which we were added to the ring buffer. + */ + private long ringBufferSequence; + + /** + * The sequence that was set in here when we were marked done. Should be equal + * or > ringBufferSequence. Put this data member into the NOT_DONE state while this + * class is in use. But for the first position on construction, let it be -1 so we can + * immediately call {@link #reset(long)} below and it will work. + */ + private long doneSequence = -1; + + /** + * If error, the associated throwable. Set when the future is 'done'. + */ + private Throwable throwable = null; + + private Thread t; + + /** + * Call this method to clear old usage and get it ready for new deploy. Call + * this method even if it is being used for the first time. + * + * @param sequence + * @return this + */ + synchronized SyncFuture reset(final long sequence) { + if (t != null && t != Thread.currentThread()) throw new IllegalStateException(); + t = Thread.currentThread(); + if (!isDone()) throw new IllegalStateException("" + sequence + " " + Thread.currentThread()); + this.doneSequence = NOT_DONE; + this.ringBufferSequence = sequence; + return this; + } + + @Override + public synchronized String toString() { + return "done=" + isDone() + ", ringBufferSequence=" + this.ringBufferSequence; + } + + synchronized long getRingBufferSequence() { + return this.ringBufferSequence; + } + + /** + * @param sequence Sync sequence at which this future 'completed'. + * @param t Can be null. Set if we are 'completing' on error (and this 't' is the error). + * @return True if we successfully marked this outstanding future as completed/done. + * Returns false if this future is already 'done' when this method called. + */ + synchronized boolean done(final long sequence, final Throwable t) { + if (isDone()) return false; + this.throwable = t; + if (sequence < this.ringBufferSequence) { + // Something badly wrong. + if (throwable == null) { + this.throwable = new IllegalStateException("sequence=" + sequence + + ", ringBufferSequence=" + this.ringBufferSequence); + } + } + // Mark done. + this.doneSequence = sequence; + // Wake up waiting threads. + notify(); + return true; + } + + public boolean cancel(boolean mayInterruptIfRunning) { + throw new UnsupportedOperationException(); + } + + public synchronized long get() throws InterruptedException, ExecutionException { + while (!isDone()) { + wait(1000); + } + if (this.throwable != null) throw new ExecutionException(this.throwable); + return this.doneSequence; + } + + public Long get(long timeout, TimeUnit unit) + throws InterruptedException, ExecutionException { + throw new UnsupportedOperationException(); + } + + public boolean isCancelled() { + throw new UnsupportedOperationException(); + } + + synchronized boolean isDone() { + return this.doneSequence != NOT_DONE; + } + + synchronized boolean isThrowable() { + return isDone() && getThrowable() != null; + } + + synchronized Throwable getThrowable() { + return this.throwable; + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCoprocessorHost.java index c0675cbc9d1..c560ca37d73 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCoprocessorHost.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCoprocessorHost.java @@ -103,6 +103,7 @@ public class WALCoprocessorHost public boolean preWALWrite(final HRegionInfo info, final HLogKey logKey, final WALEdit logEdit) throws IOException { boolean bypass = false; + if (this.coprocessors == null || this.coprocessors.isEmpty()) return bypass; ObserverContext ctx = null; for (WALEnvironment env: coprocessors) { if (env.getInstance() instanceof @@ -136,6 +137,7 @@ public class WALCoprocessorHost */ public void postWALWrite(final HRegionInfo info, final HLogKey logKey, final WALEdit logEdit) throws IOException { + if (this.coprocessors == null || this.coprocessors.isEmpty()) return; ObserverContext ctx = null; for (WALEnvironment env: coprocessors) { if (env.getInstance() instanceof diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java index d4b36a603a3..7ae7a98601e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java @@ -22,17 +22,16 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; import java.util.ArrayList; -import java.util.List; import java.util.NavigableMap; import java.util.TreeMap; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.hbase.codec.Codec; -import org.apache.hadoop.hbase.io.HeapSize; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.codec.Codec; +import org.apache.hadoop.hbase.io.HeapSize; import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ClassSize; @@ -271,5 +270,4 @@ public class WALEdit implements Writable, HeapSize { } return null; } -} - +} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHLogRecordReader.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHLogRecordReader.java index ed8551ca742..c6d80687aa4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHLogRecordReader.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHLogRecordReader.java @@ -24,6 +24,8 @@ import static org.junit.Assert.assertTrue; import java.util.List; import java.util.concurrent.atomic.AtomicLong; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -54,6 +56,7 @@ import org.junit.experimental.categories.Category; */ @Category(MediumTests.class) public class TestHLogRecordReader { + private final Log LOG = LogFactory.getLog(getClass()); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static Configuration conf; private static FileSystem fs; @@ -113,6 +116,8 @@ public class TestHLogRecordReader { @Test public void testPartialRead() throws Exception { HLog log = HLogFactory.createHLog(fs, hbaseDir, logName, conf); + // This test depends on timestamp being millisecond based and the filename of the WAL also + // being millisecond based. long ts = System.currentTimeMillis(); WALEdit edit = new WALEdit(); final AtomicLong sequenceId = new AtomicLong(0); @@ -121,7 +126,9 @@ public class TestHLogRecordReader { edit = new WALEdit(); edit.add(new KeyValue(rowName, family, Bytes.toBytes("2"), ts+1, value)); log.append(info, tableName, edit, ts+1, htd, sequenceId); + LOG.info("Before 1st WAL roll " + log.getFilenum()); log.rollWriter(); + LOG.info("Past 1st WAL roll " + log.getFilenum()); Thread.sleep(1); long ts1 = System.currentTimeMillis(); @@ -133,7 +140,9 @@ public class TestHLogRecordReader { edit.add(new KeyValue(rowName, family, Bytes.toBytes("4"), ts1+2, value)); log.append(info, tableName, edit, ts1+2, htd, sequenceId); log.close(); + LOG.info("Closed WAL " + log.getFilenum()); + HLogInputFormat input = new HLogInputFormat(); Configuration jobConf = new Configuration(conf); jobConf.set("mapred.input.dir", logDir.toString()); @@ -141,6 +150,7 @@ public class TestHLogRecordReader { // only 1st file is considered, and only its 1st entry is used List splits = input.getSplits(MapreduceTestingShim.createJobContext(jobConf)); + assertEquals(1, splits.size()); testSplit(splits.get(0), Bytes.toBytes("1")); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java index 984fb2e1b93..fdba12578ff 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java @@ -1229,7 +1229,7 @@ public class TestDistributedLogSplitting { WALEdit e = new WALEdit(); value++; e.add(new KeyValue(row, family, qualifier, timeStamp, Bytes.toBytes(value))); - hrs.getWAL().append(curRegionInfo, TableName.valueOf(tableName), e, + hrs.getWAL().append(curRegionInfo, TableName.valueOf(tableName), e, System.currentTimeMillis(), htd, sequenceId); } hrs.getWAL().sync(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestParallelPut.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestParallelPut.java index 2f6e7c12bcd..e99ee01d0c7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestParallelPut.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestParallelPut.java @@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants.OperationStatusCode; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; @@ -42,6 +43,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper; import org.junit.After; import org.junit.Before; +import org.junit.BeforeClass; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -57,8 +59,9 @@ public class TestParallelPut { static final Log LOG = LogFactory.getLog(TestParallelPut.class); @Rule public TestName name = new TestName(); - private static HRegion region = null; - private static HBaseTestingUtility hbtu = new HBaseTestingUtility(); + private HRegion region = null; + private static HBaseTestingUtility HBTU = new HBaseTestingUtility(); + private static final int THREADS100 = 100; // Test names static byte[] tableName; @@ -70,6 +73,13 @@ public class TestParallelPut { static final byte [] row = Bytes.toBytes("rowA"); static final byte [] row2 = Bytes.toBytes("rowB"); + @BeforeClass + public static void beforeClass() { + // Make sure enough handlers. + HBTU.getConfiguration().setInt(HConstants.REGION_SERVER_HANDLER_COUNT, THREADS100); + } + + /** * @see org.apache.hadoop.hbase.HBaseTestCase#setUp() */ @@ -81,6 +91,7 @@ public class TestParallelPut { @After public void tearDown() throws Exception { EnvironmentEdgeManagerTestHelper.reset(); + if (region != null) region.close(true); } public String getName() { @@ -98,7 +109,7 @@ public class TestParallelPut { @Test public void testPut() throws IOException { LOG.info("Starting testPut"); - initHRegion(tableName, getName(), fam1); + this.region = initHRegion(tableName, getName(), fam1); long value = 1L; @@ -106,7 +117,7 @@ public class TestParallelPut { put.add(fam1, qual1, Bytes.toBytes(value)); region.put(put); - assertGet(row, fam1, qual1, Bytes.toBytes(value)); + assertGet(this.region, row, fam1, qual1, Bytes.toBytes(value)); } /** @@ -116,25 +127,25 @@ public class TestParallelPut { public void testParallelPuts() throws IOException { LOG.info("Starting testParallelPuts"); - initHRegion(tableName, getName(), fam1); + + this.region = initHRegion(tableName, getName(), fam1); int numOps = 1000; // these many operations per thread // create 100 threads, each will do its own puts - int numThreads = 100; - Putter[] all = new Putter[numThreads]; + Putter[] all = new Putter[THREADS100]; // create all threads - for (int i = 0; i < numThreads; i++) { + for (int i = 0; i < THREADS100; i++) { all[i] = new Putter(region, i, numOps); } // run all threads - for (int i = 0; i < numThreads; i++) { + for (int i = 0; i < THREADS100; i++) { all[i].start(); } // wait for all threads to finish - for (int i = 0; i < numThreads; i++) { + for (int i = 0; i < THREADS100; i++) { try { all[i].join(); } catch (InterruptedException e) { @@ -143,14 +154,12 @@ public class TestParallelPut { } } LOG.info("testParallelPuts successfully verified " + - (numOps * numThreads) + " put operations."); + (numOps * THREADS100) + " put operations."); } - static private void assertGet(byte [] row, - byte [] familiy, - byte[] qualifier, - byte[] value) throws IOException { + private static void assertGet(final HRegion region, byte [] row, byte [] familiy, + byte[] qualifier, byte[] value) throws IOException { // run a get and see if the value matches Get get = new Get(row); get.addColumn(familiy, qualifier); @@ -162,7 +171,7 @@ public class TestParallelPut { assertTrue(Bytes.compareTo(r, value) == 0); } - private void initHRegion(byte [] tableName, String callingMethod, + private HRegion initHRegion(byte [] tableName, String callingMethod, byte[] ... families) throws IOException { HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName)); @@ -170,7 +179,7 @@ public class TestParallelPut { htd.addFamily(new HColumnDescriptor(family)); } HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false); - region = hbtu.createLocalHRegion(info, htd); + return HBTU.createLocalHRegion(info, htd); } /** @@ -211,7 +220,7 @@ public class TestParallelPut { OperationStatus[] ret = region.batchMutate(in); assertEquals(1, ret.length); assertEquals(OperationStatusCode.SUCCESS, ret[0].getOperationStatusCode()); - assertGet(rowkey, fam1, qual1, value); + assertGet(this.region, rowkey, fam1, qual1, value); } catch (IOException e) { assertTrue("Thread id " + threadNumber + " operation " + i + " failed.", false); @@ -219,6 +228,4 @@ public class TestParallelPut { } } } - -} - +} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/HLogPerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/HLogPerformanceEvaluation.java index 4eec8d10930..82dca794957 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/HLogPerformanceEvaluation.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/HLogPerformanceEvaluation.java @@ -54,8 +54,8 @@ import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; -import com.yammer.metrics.core.Meter; import com.yammer.metrics.core.Histogram; +import com.yammer.metrics.core.Meter; import com.yammer.metrics.core.MetricsRegistry; import com.yammer.metrics.reporting.ConsoleReporter; @@ -71,9 +71,16 @@ public final class HLogPerformanceEvaluation extends Configured implements Tool private final Meter syncMeter = metrics.newMeter(HLogPerformanceEvaluation.class, "syncMeter", "syncs", TimeUnit.MILLISECONDS); private final Histogram syncHistogram = - metrics.newHistogram(HLogPerformanceEvaluation.class, "syncHistogram", "nanos-between-syncs", true); + metrics.newHistogram(HLogPerformanceEvaluation.class, "syncHistogram", "nanos-between-syncs", + true); + private final Histogram syncCountHistogram = + metrics.newHistogram(HLogPerformanceEvaluation.class, "syncCountHistogram", "countPerSync", + true); private final Meter appendMeter = - metrics.newMeter(HLogPerformanceEvaluation.class, "appendMeter", "bytes", TimeUnit.MILLISECONDS); + metrics.newMeter(HLogPerformanceEvaluation.class, "appendMeter", "bytes", + TimeUnit.MILLISECONDS); + private final Histogram latencyHistogram = + metrics.newHistogram(HLogPerformanceEvaluation.class, "latencyHistogram", "nanos", true); private HBaseTestingUtility TEST_UTIL; @@ -127,8 +134,8 @@ public final class HLogPerformanceEvaluation extends Configured implements Tool long startTime = System.currentTimeMillis(); int lastSync = 0; for (int i = 0; i < numIterations; ++i) { + long now = System.nanoTime(); Put put = setupPut(rand, key, value, numFamilies); - long now = System.currentTimeMillis(); WALEdit walEdit = new WALEdit(); addFamilyMapToWALEdit(put.getFamilyCellMap(), walEdit); HRegionInfo hri = region.getRegionInfo(); @@ -140,6 +147,7 @@ public final class HLogPerformanceEvaluation extends Configured implements Tool lastSync = 0; } } + latencyHistogram.update(System.nanoTime() - now); } long totalTime = (System.currentTimeMillis() - startTime); logBenchmarkResult(Thread.currentThread().getName(), numIterations, totalTime); @@ -231,6 +239,10 @@ public final class HLogPerformanceEvaluation extends Configured implements Tool conf.set(HConstants.CRYPTO_WAL_ALGORITHM_CONF_KEY, cipher); } + // Internal config. goes off number of threads; if more threads than handlers, stuff breaks. + // In regionserver, number of handlers == number of threads. + getConf().setInt(HConstants.REGION_SERVER_HANDLER_COUNT, numThreads); + // Run HLog Performance Evaluation // First set the fs from configs. In case we are on hadoop1 FSUtils.setFsDefault(getConf(), FSUtils.getRootDir(getConf())); @@ -245,47 +257,72 @@ public final class HLogPerformanceEvaluation extends Configured implements Tool // Initialize Table Descriptor HTableDescriptor htd = createHTableDescriptor(numFamilies); final long whenToRoll = roll; - HLog hlog = new FSHLog(fs, rootRegionDir, "wals", getConf()) { - int appends = 0; - long lastSync = 0; + final HLog hlog = new FSHLog(fs, rootRegionDir, "wals", getConf()) { @Override - protected void doWrite(HRegionInfo info, HLogKey logKey, WALEdit logEdit, - HTableDescriptor htd) - throws IOException { + public void postSync(final long timeInNanos, final int handlerSyncs) { + super.postSync(timeInNanos, handlerSyncs); + syncMeter.mark(); + syncHistogram.update(timeInNanos); + syncCountHistogram.update(handlerSyncs); + } + + @Override + public long postAppend(final HLog.Entry entry, final long elapsedTime) { + long size = super.postAppend(entry, elapsedTime); + appendMeter.mark(size); + return size; + } + }; + hlog.registerWALActionsListener(new WALActionsListener() { + private int appends = 0; + + @Override + public void visitLogEntryBeforeWrite(HTableDescriptor htd, HLogKey logKey, + WALEdit logEdit) { this.appends++; if (this.appends % whenToRoll == 0) { LOG.info("Rolling after " + appends + " edits"); - rollWriter(); + // We used to do explicit call to rollWriter but changed it to a request + // to avoid dead lock (there are less threads going on in this class than + // in the regionserver -- regionserver does not have the issue). + ((FSHLog)hlog).requestLogRoll(); } - super.doWrite(info, logKey, logEdit, htd); - }; - - @Override - public void postSync() { - super.postSync(); - syncMeter.mark(); - long now = System.nanoTime(); - if (lastSync > 0) { - long diff = now - lastSync; - syncHistogram.update(diff); - } - this.lastSync = now; } @Override - public void postAppend(List entries) { - super.postAppend(entries); - int size = 0; - for (Entry e: entries) size += e.getEdit().heapSize(); - appendMeter.mark(size); + public void visitLogEntryBeforeWrite(HRegionInfo info, HLogKey logKey, WALEdit logEdit) { } - }; + + @Override + public void preLogRoll(Path oldPath, Path newPath) throws IOException { + } + + @Override + public void preLogArchive(Path oldPath, Path newPath) throws IOException { + } + + @Override + public void postLogRoll(Path oldPath, Path newPath) throws IOException { + } + + @Override + public void postLogArchive(Path oldPath, Path newPath) throws IOException { + } + + @Override + public void logRollRequested() { + } + + @Override + public void logCloseRequested() { + } + }); hlog.rollWriter(); HRegion region = null; try { region = openRegion(fs, rootRegionDir, htd, hlog); - ConsoleReporter.enable(this.metrics, 60, TimeUnit.SECONDS); + ConsoleReporter.enable(this.metrics, 30, TimeUnit.SECONDS); long putTime = runBenchmark(new HLogPutBenchmark(region, htd, numIterations, noSync, syncInterval), numThreads); @@ -391,21 +428,27 @@ public final class HLogPerformanceEvaluation extends Configured implements Tool System.err.println(" -nocleanup Do NOT remove test data when done."); System.err.println(" -noclosefs Do NOT close the filesystem when done."); System.err.println(" -nosync Append without syncing"); - System.err.println(" -syncInterval Append N edits and then sync. Default=0, i.e. sync every edit."); + System.err.println(" -syncInterval Append N edits and then sync. " + + "Default=0, i.e. sync every edit."); System.err.println(" -verify Verify edits written in sequence"); - System.err.println(" -verbose Output extra info; e.g. all edit seq ids when verifying"); + System.err.println(" -verbose Output extra info; " + + "e.g. all edit seq ids when verifying"); System.err.println(" -roll Roll the way every N appends"); System.err.println(" -encryption Encrypt the WAL with algorithm A, e.g. AES"); System.err.println(""); System.err.println("Examples:"); System.err.println(""); - System.err.println(" To run 100 threads on hdfs with log rolling every 10k edits and verification afterward do:"); - System.err.println(" $ ./bin/hbase org.apache.hadoop.hbase.regionserver.wal.HLogPerformanceEvaluation \\"); - System.err.println(" -conf ./core-site.xml -path hdfs://example.org:7000/tmp -threads 100 -roll 10000 -verify"); + System.err.println(" To run 100 threads on hdfs with log rolling every 10k edits and " + + "verification afterward do:"); + System.err.println(" $ ./bin/hbase org.apache.hadoop.hbase.regionserver.wal." + + "HLogPerformanceEvaluation \\"); + System.err.println(" -conf ./core-site.xml -path hdfs://example.org:7000/tmp " + + "-threads 100 -roll 10000 -verify"); System.exit(1); } - private HRegion openRegion(final FileSystem fs, final Path dir, final HTableDescriptor htd, final HLog hlog) + private HRegion openRegion(final FileSystem fs, final Path dir, final HTableDescriptor htd, + final HLog hlog) throws IOException { // Initialize HRegion HRegionInfo regionInfo = new HRegionInfo(htd.getTableName()); @@ -474,4 +517,4 @@ public final class HLogPerformanceEvaluation extends Configured implements Tool public static void main(String[] args) throws Exception { System.exit(innerMain(HBaseConfiguration.create(), args)); } -} \ No newline at end of file +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java index fd3c7d6b7ea..f8b5847253e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java @@ -85,7 +85,7 @@ public class TestDurability { region.put(newPut(null)); verifyHLogCount(wal, 1); - // a put through the deferred table does not write to the wal immdiately, + // a put through the deferred table does not write to the wal immediately, // but maybe has been successfully sync-ed by the underlying AsyncWriter + // AsyncFlusher thread deferredRegion.put(newPut(null)); @@ -114,7 +114,7 @@ public class TestDurability { wal.sync(); verifyHLogCount(wal, 6); - // async overrides sync table default + // Async overrides sync table default region.put(newPut(Durability.ASYNC_WAL)); deferredRegion.put(newPut(Durability.ASYNC_WAL)); wal.sync(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java index 94cfe698f95..57dc333752e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java @@ -85,12 +85,10 @@ public class TestHLog { @Before public void setUp() throws Exception { - FileStatus[] entries = fs.listStatus(new Path("/")); for (FileStatus dir : entries) { fs.delete(dir.getPath(), true); } - } @After @@ -127,6 +125,7 @@ public class TestHLog { oldLogDir = new Path(hbaseDir, HConstants.HREGION_OLDLOGDIR_NAME); dir = new Path(hbaseDir, getName()); } + @AfterClass public static void tearDownAfterClass() throws Exception { TEST_UTIL.shutdownMiniCluster(); @@ -158,13 +157,11 @@ public class TestHLog { */ @Test public void testSplit() throws IOException { - final TableName tableName = TableName.valueOf(getName()); final byte [] rowName = tableName.getName(); Path logdir = new Path(hbaseDir, HConstants.HREGION_LOGDIR_NAME); - HLog log = HLogFactory.createHLog(fs, hbaseDir, - HConstants.HREGION_LOGDIR_NAME, conf); + HLog log = HLogFactory.createHLog(fs, hbaseDir, HConstants.HREGION_LOGDIR_NAME, conf); final int howmany = 3; HRegionInfo[] infos = new HRegionInfo[3]; Path tabledir = FSUtils.getTableDir(hbaseDir, tableName); @@ -199,8 +196,7 @@ public class TestHLog { log.rollWriter(); } log.close(); - List splits = HLogSplitter.split( - hbaseDir, logdir, oldLogDir, fs, conf); + List splits = HLogSplitter.split(hbaseDir, logdir, oldLogDir, fs, conf); verifySplits(splits, howmany); log = null; } finally { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java index 57a2549683b..7ecef823a87 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java @@ -1097,7 +1097,7 @@ public class TestHLogSplit { } // Send the data to HDFS datanodes and close the HDFS writer log.sync(); - ((FSHLog) log).cleanupCurrentWriter(log.getFilenum()); + ((FSHLog) log).replaceWriter(((FSHLog)log).getOldPath(), null, null, null); /* code taken from ProcessServerShutdown.process() * handles RS shutdowns (as observed by the Master) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java index 9381454fc1b..9afad872287 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java @@ -18,23 +18,23 @@ package org.apache.hadoop.hbase.regionserver.wal; import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.commons.logging.impl.Log4JLogger; -import org.apache.hadoop.hbase.*; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.MediumTests; +import org.apache.hadoop.hbase.MiniHBaseCluster; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; -import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hdfs.MiniDFSCluster; -import org.apache.hadoop.hdfs.server.datanode.DataNode; -import org.apache.hadoop.hdfs.server.namenode.LeaseManager; -import org.apache.log4j.Level; import org.junit.After; import org.junit.Before; import org.junit.BeforeClass; @@ -53,17 +53,6 @@ public class TestLogRollAbort { private static MiniHBaseCluster cluster; private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - // verbose logging on classes that are touched in these tests - { - ((Log4JLogger)DataNode.LOG).getLogger().setLevel(Level.ALL); - ((Log4JLogger)LeaseManager.LOG).getLogger().setLevel(Level.ALL); - ((Log4JLogger)LogFactory.getLog("org.apache.hadoop.hdfs.server.namenode.FSNamesystem")) - .getLogger().setLevel(Level.ALL); - ((Log4JLogger)HRegionServer.LOG).getLogger().setLevel(Level.ALL); - ((Log4JLogger)HRegion.LOG).getLogger().setLevel(Level.ALL); - ((Log4JLogger)HLog.LOG).getLogger().setLevel(Level.ALL); - } - // Need to override this setup so we can edit the config before it gets sent // to the HDFS & HBase cluster startup. @BeforeClass @@ -120,41 +109,45 @@ public class TestLogRollAbort { // Create the test table and open it String tableName = this.getClass().getSimpleName(); - HTableDescriptor desc = new HTableDescriptor(tableName); + HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName)); desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); - desc.setAsyncLogFlush(true); admin.createTable(desc); HTable table = new HTable(TEST_UTIL.getConfiguration(), tableName); + try { - HRegionServer server = TEST_UTIL.getRSForFirstRegionInTable(Bytes.toBytes(tableName)); - HLog log = server.getWAL(); + HRegionServer server = TEST_UTIL.getRSForFirstRegionInTable(Bytes.toBytes(tableName)); + HLog log = server.getWAL(); - assertTrue("Need HDFS-826 for this test", ((FSHLog) log).canGetCurReplicas()); - // don't run this test without append support (HDFS-200 & HDFS-142) - assertTrue("Need append support for this test", + assertTrue("Need HDFS-826 for this test", ((FSHLog) log).canGetCurReplicas()); + // don't run this test without append support (HDFS-200 & HDFS-142) + assertTrue("Need append support for this test", FSUtils.isAppendSupported(TEST_UTIL.getConfiguration())); - Put p = new Put(Bytes.toBytes("row2001")); - p.add(HConstants.CATALOG_FAMILY, Bytes.toBytes("col"), Bytes.toBytes(2001)); - table.put(p); + Put p = new Put(Bytes.toBytes("row2001")); + p.add(HConstants.CATALOG_FAMILY, Bytes.toBytes("col"), Bytes.toBytes(2001)); + table.put(p); - log.sync(); + log.sync(); - p = new Put(Bytes.toBytes("row2002")); - p.add(HConstants.CATALOG_FAMILY, Bytes.toBytes("col"), Bytes.toBytes(2002)); - table.put(p); + p = new Put(Bytes.toBytes("row2002")); + p.add(HConstants.CATALOG_FAMILY, Bytes.toBytes("col"), Bytes.toBytes(2002)); + table.put(p); - dfsCluster.restartDataNodes(); - LOG.info("Restarted datanodes"); + dfsCluster.restartDataNodes(); + LOG.info("Restarted datanodes"); - try { - log.rollWriter(true); - } catch (FailedLogCloseException flce) { - assertTrue("Should have deferred flush log edits outstanding", - ((FSHLog) log).hasUnSyncedEntries()); + try { + log.rollWriter(true); + } catch (FailedLogCloseException flce) { + // Expected exception. We used to expect that there would be unsynced appends but this + // not reliable now that sync plays a roll in wall rolling. The above puts also now call + // sync. + } catch (Throwable t) { + LOG.fatal("FAILED TEST: Got wrong exception", t); + } + } finally { + table.close(); } } - } - diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java index 557212e9ae8..912a5b84cad 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java @@ -91,18 +91,6 @@ public class TestLogRolling { private MiniHBaseCluster cluster; private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - // verbose logging on classes that are touched in these tests - { - ((Log4JLogger)DataNode.LOG).getLogger().setLevel(Level.ALL); - ((Log4JLogger)LeaseManager.LOG).getLogger().setLevel(Level.ALL); - ((Log4JLogger)LogFactory.getLog("org.apache.hadoop.hdfs.server.namenode.FSNamesystem")) - .getLogger().setLevel(Level.ALL); - ((Log4JLogger)DFSClient.LOG).getLogger().setLevel(Level.ALL); - ((Log4JLogger)HRegionServer.LOG).getLogger().setLevel(Level.ALL); - ((Log4JLogger)HRegion.LOG).getLogger().setLevel(Level.ALL); - ((Log4JLogger)HLog.LOG).getLogger().setLevel(Level.ALL); - } - /** * constructor * @throws Exception @@ -135,8 +123,7 @@ public class TestLogRolling { // We roll the log after every 32 writes TEST_UTIL.getConfiguration().setInt("hbase.regionserver.maxlogentries", 32); - TEST_UTIL.getConfiguration().setInt( - "hbase.regionserver.logroll.errors.tolerated", 2); + TEST_UTIL.getConfiguration().setInt("hbase.regionserver.logroll.errors.tolerated", 2); TEST_UTIL.getConfiguration().setInt("ipc.ping.interval", 10 * 1000); TEST_UTIL.getConfiguration().setInt("ipc.socket.timeout", 10 * 1000); TEST_UTIL.getConfiguration().setInt("hbase.rpc.timeout", 10 * 1000); @@ -162,13 +149,11 @@ public class TestLogRolling { // quickly detects datanode failures TEST_UTIL.getConfiguration().setInt("heartbeat.recheck.interval", 5000); TEST_UTIL.getConfiguration().setInt("dfs.heartbeat.interval", 1); - // the namenode might still try to choose the recently-dead datanode - // for a pipeline, so try to a new pipeline multiple times - TEST_UTIL.getConfiguration().setInt("dfs.client.block.write.retries", 30); - TEST_UTIL.getConfiguration().setInt( - "hbase.regionserver.hlog.tolerable.lowreplication", 2); - TEST_UTIL.getConfiguration().setInt( - "hbase.regionserver.hlog.lowreplication.rolllimit", 3); + // the namenode might still try to choose the recently-dead datanode + // for a pipeline, so try to a new pipeline multiple times + TEST_UTIL.getConfiguration().setInt("dfs.client.block.write.retries", 30); + TEST_UTIL.getConfiguration().setInt("hbase.regionserver.hlog.tolerable.lowreplication", 2); + TEST_UTIL.getConfiguration().setInt("hbase.regionserver.hlog.lowreplication.rolllimit", 3); } @Before @@ -220,6 +205,7 @@ public class TestLogRolling { @Test public void testLogRolling() throws Exception { this.tableName = getName(); + // TODO: Why does this write data take for ever? startAndWriteData(); LOG.info("after writing there are " + ((FSHLog) log).getNumRolledLogFiles() + " log files"); @@ -322,6 +308,7 @@ public class TestLogRolling { */ @Test public void testLogRollOnDatanodeDeath() throws Exception { + TEST_UTIL.ensureSomeRegionServersAvailable(2); assertTrue("This test requires HLog file replication set to 2.", fs.getDefaultReplication() == 2); LOG.info("Replication=" + fs.getDefaultReplication()); @@ -363,7 +350,7 @@ public class TestLogRolling { assertTrue("DataNodes " + dfsCluster.getDataNodes().size() + " default replication " + fs.getDefaultReplication(), - dfsCluster.getDataNodes().size() >= fs.getDefaultReplication() + 1); + dfsCluster.getDataNodes().size() >= fs.getDefaultReplication() + 1); writeData(table, 2); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java index 6831869a975..c2717417216 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java @@ -27,6 +27,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; @@ -54,8 +55,9 @@ public class TestLogRollingNoCluster { public void testContendedLogRolling() throws IOException, InterruptedException { FileSystem fs = FileSystem.get(TEST_UTIL.getConfiguration()); Path dir = TEST_UTIL.getDataTestDir(); - HLog wal = HLogFactory.createHLog(fs, dir, "logs", - TEST_UTIL.getConfiguration()); + // The implementation needs to know the 'handler' count. + TEST_UTIL.getConfiguration().setInt(HConstants.REGION_SERVER_HANDLER_COUNT, THREAD_COUNT); + HLog wal = HLogFactory.createHLog(fs, dir, "logs", TEST_UTIL.getConfiguration()); Appender [] appenders = null; @@ -122,7 +124,6 @@ public class TestLogRollingNoCluster { WALEdit edit = new WALEdit(); byte[] bytes = Bytes.toBytes(i); edit.add(new KeyValue(bytes, bytes, bytes, now, EMPTY_1K_ARRAY)); - this.wal.append(HRegionInfo.FIRST_META_REGIONINFO, HTableDescriptor.META_TABLEDESC.getTableName(), edit, now, HTableDescriptor.META_TABLEDESC, sequenceId); @@ -135,6 +136,13 @@ public class TestLogRollingNoCluster { } catch (Exception e) { this.e = e; log.info("Caught exception from Appender:" + getName(), e); + } finally { + // Call sync on our log.else threads just hang out. + try { + this.wal.sync(); + } catch (IOException e) { + throw new RuntimeException(e); + } } } } diff --git a/pom.xml b/pom.xml index e341ece717f..4587129f9ac 100644 --- a/pom.xml +++ b/pom.xml @@ -899,6 +899,7 @@ 2.6 1.1.3 2.2 + 3.2.0 3.2.1 3.1 2.1.2 @@ -1362,6 +1363,11 @@ htrace-core ${htrace.version} + + com.lmax + disruptor + ${disruptor.version} +