HBASE-15265 Implement an asynchronous FSHLog

This commit is contained in:
zhangduo 2016-03-26 15:40:45 +08:00
parent 1a9837ab74
commit c96b642f15
32 changed files with 3940 additions and 2321 deletions

View File

@ -167,6 +167,9 @@ public class LogRoller extends HasThread {
}
}
}
for (WAL wal : walNeedsRoll.keySet()) {
wal.logRollerExited();
}
LOG.info("LogRoller exiting.");
}

View File

@ -0,0 +1,910 @@
/**
* 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 static org.apache.hadoop.hbase.wal.AbstractFSWALProvider.WAL_FILE_NAME_DELIMITER;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.lang.management.ManagementFactory;
import java.lang.management.MemoryUsage;
import java.net.URLEncoder;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ConcurrentNavigableMap;
import java.util.concurrent.ConcurrentSkipListMap;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.locks.ReentrantLock;
import com.google.common.annotations.VisibleForTesting;
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;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.io.util.HeapMemorySizeUtil;
import org.apache.hadoop.hbase.util.Bytes;
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.wal.WAL;
import org.apache.hadoop.hbase.wal.WALFactory;
import org.apache.hadoop.hbase.wal.WALKey;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.util.StringUtils;
import org.apache.htrace.NullScope;
import org.apache.htrace.Span;
import org.apache.htrace.Trace;
import org.apache.htrace.TraceScope;
/**
* Implementation of {@link WAL} to go against {@link FileSystem}; i.e. keep WALs in HDFS. Only one
* WAL is ever being written at a time. When a WAL hits a configured maximum size, it is rolled.
* This is done internal to the implementation.
* <p>
* As data is flushed from the MemStore to other on-disk structures (files sorted by key, hfiles), a
* WAL becomes obsolete. We can let go of all the log edits/entries for a given HRegion-sequence id.
* A bunch of work in the below is done keeping account of these region sequence ids -- what is
* flushed out to hfiles, and what is yet in WAL and in memory only.
* <p>
* It is only practical to delete entire files. Thus, we delete an entire on-disk file
* <code>F</code> when all of the edits in <code>F</code> have a log-sequence-id that's older
* (smaller) than the most-recent flush.
* <p>
* To read an WAL, call
* {@link WALFactory#createReader(org.apache.hadoop.fs.FileSystem, org.apache.hadoop.fs.Path)}. *
* <h2>Failure Semantic</h2> If an exception on append or sync, roll the WAL because the current WAL
* is now a lame duck; any more appends or syncs will fail also with the same original exception. If
* we have made successful appends to the WAL and we then are unable to sync them, our current
* semantic is to return error to the client that the appends failed but also to abort the current
* context, usually the hosting server. We need to replay the WALs. <br>
* TODO: Change this semantic. A roll of WAL may be sufficient as long as we have flagged client
* that the append failed. <br>
* TODO: replication may pick up these last edits though they have been marked as failed append
* (Need to keep our own file lengths, not rely on HDFS).
*/
@InterfaceAudience.Private
public abstract class AbstractFSWAL<W> implements WAL {
private static final Log LOG = LogFactory.getLog(AbstractFSWAL.class);
protected static final int DEFAULT_SLOW_SYNC_TIME_MS = 100; // in ms
/**
* file system instance
*/
protected final FileSystem fs;
/**
* WAL directory, where all WAL files would be placed.
*/
protected final Path walDir;
/**
* dir path where old logs are kept.
*/
protected final Path walArchiveDir;
/**
* Matches just those wal files that belong to this wal instance.
*/
protected final PathFilter ourFiles;
/**
* Prefix of a WAL file, usually the region server name it is hosted on.
*/
protected final String walFilePrefix;
/**
* Suffix included on generated wal file names
*/
protected final String walFileSuffix;
/**
* Prefix used when checking for wal membership.
*/
protected final String prefixPathStr;
protected final WALCoprocessorHost coprocessorHost;
/**
* conf object
*/
protected final Configuration conf;
/** Listeners that are called on WAL events. */
protected final List<WALActionsListener> listeners =
new CopyOnWriteArrayList<WALActionsListener>();
/**
* Class that does accounting of sequenceids in WAL subsystem. Holds oldest outstanding sequence
* id as yet not flushed as well as the most recent edit sequence id appended to the WAL. Has
* facility for answering questions such as "Is it safe to GC a WAL?".
*/
protected final SequenceIdAccounting sequenceIdAccounting = new SequenceIdAccounting();
/** The barrier used to ensure that close() waits for all log rolls and flushes to finish. */
protected final DrainBarrier closeBarrier = new DrainBarrier();
protected final int slowSyncNs;
// If > than this size, roll the log.
protected final long logrollsize;
/*
* If more than this many logs, force flush of oldest region to oldest edit goes to disk. If too
* many and we crash, then will take forever replaying. Keep the number of logs tidy.
*/
protected final int maxLogs;
/**
* 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.
*/
protected final ReentrantLock rollWriterLock = new ReentrantLock(true);
// The timestamp (in ms) when the log file was created.
protected final AtomicLong filenum = new AtomicLong(-1);
// Number of transactions in the current Wal.
protected final AtomicInteger numEntries = new AtomicInteger(0);
/**
* The highest known outstanding unsync'd WALEdit transaction id. Usually, we use a queue to pass
* WALEdit to background consumer thread, and the transaction id is the sequence number of the
* corresponding entry in queue.
*/
protected volatile long highestUnsyncedTxid = -1;
/**
* Updated to the transaction id of the last successful sync call. This can be less than
* {@link #highestUnsyncedTxid} for case where we have an append where a sync has not yet come in
* for it.
*/
protected final AtomicLong highestSyncedTxid = new AtomicLong(0);
/**
* The total size of wal
*/
protected final AtomicLong totalLogSize = new AtomicLong(0);
/**
* Current log file.
*/
volatile W writer;
protected volatile boolean closed = false;
protected final AtomicBoolean shutdown = new AtomicBoolean(false);
/**
* WAL Comparator; it compares the timestamp (log filenum), present in the log file name. Throws
* an IllegalArgumentException if used to compare paths from different wals.
*/
final Comparator<Path> LOG_NAME_COMPARATOR = new Comparator<Path>() {
@Override
public int compare(Path o1, Path o2) {
return Long.compare(getFileNumFromFileName(o1), getFileNumFromFileName(o2));
}
};
/**
* Map of WAL log file to the latest sequence ids of all regions it has entries of. The map is
* sorted by the log file creation timestamp (contained in the log file name).
*/
protected ConcurrentNavigableMap<Path, Map<byte[], Long>> byWalRegionSequenceIds =
new ConcurrentSkipListMap<Path, Map<byte[], Long>>(LOG_NAME_COMPARATOR);
/**
* Map of {@link SyncFuture}s keyed by Handler objects. Used so we reuse SyncFutures.
* <p>
* TODO: Reus FSWALEntry's rather than create them anew each time as we do SyncFutures here.
* <p>
* TODO: Add a FSWalEntry and SyncFuture as thread locals on handlers rather than have them get
* them from this Map?
*/
private final ConcurrentMap<Thread, SyncFuture> syncFuturesByHandler;
/**
* A log file has a creation timestamp (in ms) in its file name ({@link #filenum}. This helper
* method returns the creation timestamp from a given log file. It extracts the timestamp assuming
* the filename is created with the {@link #computeFilename(long filenum)} method.
* @return timestamp, as in the log file name.
*/
protected long getFileNumFromFileName(Path fileName) {
if (fileName == null) {
throw new IllegalArgumentException("file name can't be null");
}
if (!ourFiles.accept(fileName)) {
throw new IllegalArgumentException("The log file " + fileName
+ " doesn't belong to this WAL. (" + toString() + ")");
}
final String fileNameString = fileName.toString();
String chompedPath =
fileNameString.substring(prefixPathStr.length(),
(fileNameString.length() - walFileSuffix.length()));
return Long.parseLong(chompedPath);
}
private int calculateMaxLogFiles(float memstoreSizeRatio, long logRollSize) {
MemoryUsage mu = ManagementFactory.getMemoryMXBean().getHeapMemoryUsage();
return Math.round(mu.getMax() * memstoreSizeRatio * 2 / logRollSize);
}
protected AbstractFSWAL(final FileSystem fs, final Path rootDir, final String logDir,
final String archiveDir, final Configuration conf, final List<WALActionsListener> listeners,
final boolean failIfWALExists, final String prefix, final String suffix)
throws FailedLogCloseException, IOException {
this.fs = fs;
this.walDir = new Path(rootDir, logDir);
this.walArchiveDir = new Path(rootDir, archiveDir);
this.conf = conf;
if (!fs.exists(walDir) && !fs.mkdirs(walDir)) {
throw new IOException("Unable to mkdir " + walDir);
}
if (!fs.exists(this.walArchiveDir)) {
if (!fs.mkdirs(this.walArchiveDir)) {
throw new IOException("Unable to mkdir " + this.walArchiveDir);
}
}
// If prefix is null||empty then just name it wal
this.walFilePrefix =
prefix == null || prefix.isEmpty() ? "wal" : URLEncoder.encode(prefix, "UTF8");
// we only correctly differentiate suffices when numeric ones start with '.'
if (suffix != null && !(suffix.isEmpty()) && !(suffix.startsWith(WAL_FILE_NAME_DELIMITER))) {
throw new IllegalArgumentException("WAL suffix must start with '" + WAL_FILE_NAME_DELIMITER
+ "' but instead was '" + suffix + "'");
}
// Now that it exists, set the storage policy for the entire directory of wal files related to
// this FSHLog instance
FSUtils.setStoragePolicy(fs, conf, this.walDir, HConstants.WAL_STORAGE_POLICY,
HConstants.DEFAULT_WAL_STORAGE_POLICY);
this.walFileSuffix = (suffix == null) ? "" : URLEncoder.encode(suffix, "UTF8");
this.prefixPathStr = new Path(walDir, walFilePrefix + WAL_FILE_NAME_DELIMITER).toString();
this.ourFiles = new PathFilter() {
@Override
public boolean accept(final Path fileName) {
// The path should start with dir/<prefix> and end with our suffix
final String fileNameString = fileName.toString();
if (!fileNameString.startsWith(prefixPathStr)) {
return false;
}
if (walFileSuffix.isEmpty()) {
// in the case of the null suffix, we need to ensure the filename ends with a timestamp.
return org.apache.commons.lang.StringUtils.isNumeric(fileNameString
.substring(prefixPathStr.length()));
} else if (!fileNameString.endsWith(walFileSuffix)) {
return false;
}
return true;
}
};
if (failIfWALExists) {
final FileStatus[] walFiles = FSUtils.listStatus(fs, walDir, ourFiles);
if (null != walFiles && 0 != walFiles.length) {
throw new IOException("Target WAL already exists within directory " + walDir);
}
}
// Register listeners. TODO: Should this exist anymore? We have CPs?
if (listeners != null) {
for (WALActionsListener i : listeners) {
registerWALActionsListener(i);
}
}
this.coprocessorHost = new WALCoprocessorHost(this, conf);
// Get size to roll log at. Roll at 95% of HDFS block size so we avoid crossing HDFS blocks
// (it costs a little x'ing bocks)
final long blocksize =
this.conf.getLong("hbase.regionserver.hlog.blocksize",
FSUtils.getDefaultBlockSize(this.fs, this.walDir));
this.logrollsize =
(long) (blocksize * conf.getFloat("hbase.regionserver.logroll.multiplier", 0.95f));
float memstoreRatio =
conf.getFloat(HeapMemorySizeUtil.MEMSTORE_SIZE_KEY, conf.getFloat(
HeapMemorySizeUtil.MEMSTORE_SIZE_OLD_KEY, HeapMemorySizeUtil.DEFAULT_MEMSTORE_SIZE));
boolean maxLogsDefined = conf.get("hbase.regionserver.maxlogs") != null;
if (maxLogsDefined) {
LOG.warn("'hbase.regionserver.maxlogs' was deprecated.");
}
this.maxLogs =
conf.getInt("hbase.regionserver.maxlogs",
Math.max(32, calculateMaxLogFiles(memstoreRatio, logrollsize)));
LOG.info("WAL configuration: blocksize=" + StringUtils.byteDesc(blocksize) + ", rollsize="
+ StringUtils.byteDesc(this.logrollsize) + ", prefix=" + this.walFilePrefix + ", suffix="
+ walFileSuffix + ", logDir=" + this.walDir + ", archiveDir=" + this.walArchiveDir);
this.slowSyncNs =
1000000 * conf.getInt("hbase.regionserver.hlog.slowsync.ms", DEFAULT_SLOW_SYNC_TIME_MS);
int maxHandlersCount = conf.getInt(HConstants.REGION_SERVER_HANDLER_COUNT, 200);
// Presize our map of SyncFutures by handler objects.
this.syncFuturesByHandler = new ConcurrentHashMap<Thread, SyncFuture>(maxHandlersCount);
}
@Override
public void registerWALActionsListener(WALActionsListener listener) {
this.listeners.add(listener);
}
@Override
public boolean unregisterWALActionsListener(WALActionsListener listener) {
return this.listeners.remove(listener);
}
@Override
public WALCoprocessorHost getCoprocessorHost() {
return coprocessorHost;
}
@Override
public Long startCacheFlush(byte[] encodedRegionName, Set<byte[]> families) {
if (!closeBarrier.beginOp()) {
LOG.info("Flush not started for " + Bytes.toString(encodedRegionName) + "; server closing.");
return null;
}
return this.sequenceIdAccounting.startCacheFlush(encodedRegionName, families);
}
@Override
public void completeCacheFlush(byte[] encodedRegionName) {
this.sequenceIdAccounting.completeCacheFlush(encodedRegionName);
closeBarrier.endOp();
}
@Override
public void abortCacheFlush(byte[] encodedRegionName) {
this.sequenceIdAccounting.abortCacheFlush(encodedRegionName);
closeBarrier.endOp();
}
@Override
public long getEarliestMemstoreSeqNum(byte[] encodedRegionName) {
// Used by tests. Deprecated as too subtle for general usage.
return this.sequenceIdAccounting.getLowestSequenceId(encodedRegionName);
}
@Override
public long getEarliestMemstoreSeqNum(byte[] encodedRegionName, byte[] familyName) {
// This method is used by tests and for figuring if we should flush or not because our
// sequenceids are too old. It is also used reporting the master our oldest sequenceid for use
// figuring what edits can be skipped during log recovery. getEarliestMemStoreSequenceId
// from this.sequenceIdAccounting is looking first in flushingOldestStoreSequenceIds, the
// currently flushing sequence ids, and if anything found there, it is returning these. This is
// the right thing to do for the reporting oldest sequenceids to master; we won't skip edits if
// we crash during the flush. For figuring what to flush, we might get requeued if our sequence
// id is old even though we are currently flushing. This may mean we do too much flushing.
return this.sequenceIdAccounting.getLowestSequenceId(encodedRegionName, familyName);
}
@Override
public byte[][] rollWriter() throws FailedLogCloseException, IOException {
return rollWriter(false);
}
/**
* This is a convenience method that computes a new filename with a given file-number.
* @param filenum to use
* @return Path
*/
protected Path computeFilename(final long filenum) {
if (filenum < 0) {
throw new RuntimeException("WAL file number can't be < 0");
}
String child = walFilePrefix + WAL_FILE_NAME_DELIMITER + filenum + walFileSuffix;
return new Path(walDir, child);
}
/**
* This is a convenience method that computes a new filename with a given using the current WAL
* file-number
* @return Path
*/
public Path getCurrentFileName() {
return computeFilename(this.filenum.get());
}
/**
* retrieve the next path to use for writing. Increments the internal filenum.
*/
private Path getNewPath() throws IOException {
this.filenum.set(System.currentTimeMillis());
Path newPath = getCurrentFileName();
while (fs.exists(newPath)) {
this.filenum.incrementAndGet();
newPath = getCurrentFileName();
}
return newPath;
}
@VisibleForTesting
Path getOldPath() {
long currentFilenum = this.filenum.get();
Path oldPath = null;
if (currentFilenum > 0) {
// ComputeFilename will take care of meta wal 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.
*/
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.
*/
private void tellListenersAboutPostLogRoll(final Path oldPath, final Path newPath)
throws IOException {
if (!this.listeners.isEmpty()) {
for (WALActionsListener i : this.listeners) {
i.postLogRoll(oldPath, newPath);
}
}
}
// public only until class moves to o.a.h.h.wal
/** @return the number of rolled log files */
public int getNumRolledLogFiles() {
return byWalRegionSequenceIds.size();
}
// public only until class moves to o.a.h.h.wal
/** @return the number of log files in use */
public int getNumLogFiles() {
// +1 for current use log
return getNumRolledLogFiles() + 1;
}
/**
* If the number of un-archived WAL files is greater than maximum allowed, check the first
* (oldest) WAL file, and returns those regions which should be flushed so that it can be
* archived.
* @return regions (encodedRegionNames) to flush in order to archive oldest WAL file.
*/
byte[][] findRegionsToForceFlush() throws IOException {
byte[][] regions = null;
int logCount = getNumRolledLogFiles();
if (logCount > this.maxLogs && logCount > 0) {
Map.Entry<Path, Map<byte[], Long>> firstWALEntry = this.byWalRegionSequenceIds.firstEntry();
regions = this.sequenceIdAccounting.findLower(firstWALEntry.getValue());
}
if (regions != null) {
StringBuilder sb = new StringBuilder();
for (int i = 0; i < regions.length; i++) {
if (i > 0) {
sb.append(", ");
}
sb.append(Bytes.toStringBinary(regions[i]));
}
LOG.info("Too many WALs; count=" + logCount + ", max=" + this.maxLogs + "; forcing flush of "
+ regions.length + " regions(s): " + sb.toString());
}
return regions;
}
/**
* Archive old logs. A WAL is eligible for archiving if all its WALEdits have been flushed.
*/
private void cleanOldLogs() throws IOException {
List<Path> logsToArchive = null;
// For each log file, look at its Map of regions to highest sequence id; if all sequence ids
// are older than what is currently in memory, the WAL can be GC'd.
for (Map.Entry<Path, Map<byte[], Long>> e : this.byWalRegionSequenceIds.entrySet()) {
Path log = e.getKey();
Map<byte[], Long> sequenceNums = e.getValue();
if (this.sequenceIdAccounting.areAllLower(sequenceNums)) {
if (logsToArchive == null) {
logsToArchive = new ArrayList<Path>();
}
logsToArchive.add(log);
if (LOG.isTraceEnabled()) {
LOG.trace("WAL file ready for archiving " + log);
}
}
}
if (logsToArchive != null) {
for (Path p : logsToArchive) {
this.totalLogSize.addAndGet(-this.fs.getFileStatus(p).getLen());
archiveLogFile(p);
this.byWalRegionSequenceIds.remove(p);
}
}
}
/*
* only public so WALSplitter can use.
* @return archived location of a WAL file with the given path p
*/
public static Path getWALArchivePath(Path archiveDir, Path p) {
return new Path(archiveDir, p.getName());
}
private void archiveLogFile(final Path p) throws IOException {
Path newPath = getWALArchivePath(this.walArchiveDir, p);
// Tell our listeners that a log is going to be archived.
if (!this.listeners.isEmpty()) {
for (WALActionsListener i : this.listeners) {
i.preLogArchive(p, newPath);
}
}
LOG.info("Archiving " + p + " to " + newPath);
if (!FSUtils.renameAndSetModifyTime(this.fs, p, newPath)) {
throw new IOException("Unable to rename " + p + " to " + newPath);
}
// Tell our listeners that a log has been archived.
if (!this.listeners.isEmpty()) {
for (WALActionsListener i : this.listeners) {
i.postLogArchive(p, newPath);
}
}
}
/**
* Cleans up current writer closing it and then puts in place the passed in
* <code>nextWriter</code>.
* <p>
* <ul>
* <li>In the case of creating a new WAL, oldPath will be null.</li>
* <li>In the case of rolling over from one file to the next, none of the parameters will be null.
* </li>
* <li>In the case of closing out this FSHLog with no further use newPath and nextWriter will be
* null.</li>
* </ul>
* @param oldPath may be null
* @param newPath may be null
* @param nextWriter may be null
* @return the passed in <code>newPath</code>
* @throws IOException if there is a problem flushing or closing the underlying FS
*/
Path replaceWriter(Path oldPath, Path newPath, W nextWriter) throws IOException {
TraceScope scope = Trace.startSpan("FSHFile.replaceWriter");
try {
long oldFileLen = 0L;
doReplaceWriter(oldPath, newPath, nextWriter);
int oldNumEntries = this.numEntries.get();
final String newPathString = (null == newPath ? null : FSUtils.getPath(newPath));
if (oldPath != null) {
this.byWalRegionSequenceIds.put(oldPath, this.sequenceIdAccounting.resetHighest());
this.totalLogSize.addAndGet(oldFileLen);
LOG.info("Rolled WAL " + FSUtils.getPath(oldPath) + " with entries=" + oldNumEntries
+ ", filesize=" + StringUtils.byteDesc(oldFileLen) + "; new WAL " + newPathString);
} else {
LOG.info("New WAL " + newPathString);
}
return newPath;
} finally {
scope.close();
}
}
protected Span 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();
return syncFuture.getSpan();
} catch (InterruptedException ie) {
LOG.warn("Interrupted", ie);
throw convertInterruptedExceptionToIOException(ie);
} catch (ExecutionException e) {
throw ensureIOException(e.getCause());
}
}
private static IOException ensureIOException(final Throwable t) {
return (t instanceof IOException) ? (IOException) t : new IOException(t);
}
private IOException convertInterruptedExceptionToIOException(final InterruptedException ie) {
Thread.currentThread().interrupt();
IOException ioe = new InterruptedIOException();
ioe.initCause(ie);
return ioe;
}
@Override
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;
}
byte[][] regionsToFlush = null;
if (this.closed) {
LOG.debug("WAL closed. Skipping rolling of writer");
return regionsToFlush;
}
if (!closeBarrier.beginOp()) {
LOG.debug("WAL closing. Skipping rolling of writer");
return regionsToFlush;
}
TraceScope scope = Trace.startSpan("FSHLog.rollWriter");
try {
Path oldPath = getOldPath();
Path newPath = getNewPath();
// Any exception from here on is catastrophic, non-recoverable so we currently abort.
W nextWriter = this.createWriterInstance(newPath);
tellListenersAboutPreLogRoll(oldPath, newPath);
// NewPath could be equal to oldPath if replaceWriter fails.
newPath = replaceWriter(oldPath, newPath, nextWriter);
tellListenersAboutPostLogRoll(oldPath, newPath);
// Can we delete any of the old log files?
if (getNumRolledLogFiles() > 0) {
cleanOldLogs();
regionsToFlush = findRegionsToForceFlush();
}
} finally {
closeBarrier.endOp();
assert scope == NullScope.INSTANCE || !scope.isDetached();
scope.close();
}
return regionsToFlush;
} finally {
rollWriterLock.unlock();
}
}
// public only until class moves to o.a.h.h.wal
/** @return the size of log files in use */
public long getLogFileSize() {
return this.totalLogSize.get();
}
// public only until class moves to o.a.h.h.wal
public void requestLogRoll() {
requestLogRoll(false);
}
/**
* Get the backing files associated with this WAL.
* @return may be null if there are no files.
*/
protected FileStatus[] getFiles() throws IOException {
return FSUtils.listStatus(fs, walDir, ourFiles);
}
@Override
public void shutdown() throws IOException {
if (!shutdown.compareAndSet(false, true)) {
return;
}
closed = true;
try {
// Prevent all further flushing and rolling.
closeBarrier.stopAndDrainOps();
} catch (InterruptedException e) {
LOG.error("Exception while waiting for cache flushes and log rolls", e);
Thread.currentThread().interrupt();
}
// Tell our listeners that the log is closing
if (!this.listeners.isEmpty()) {
for (WALActionsListener i : this.listeners) {
i.logCloseRequested();
}
}
doShutdown();
}
@Override
public void close() throws IOException {
shutdown();
final FileStatus[] files = getFiles();
if (null != files && 0 != files.length) {
for (FileStatus file : files) {
Path p = getWALArchivePath(this.walArchiveDir, file.getPath());
// Tell our listeners that a log is going to be archived.
if (!this.listeners.isEmpty()) {
for (WALActionsListener i : this.listeners) {
i.preLogArchive(file.getPath(), p);
}
}
if (!FSUtils.renameAndSetModifyTime(fs, file.getPath(), p)) {
throw new IOException("Unable to rename " + file.getPath() + " to " + p);
}
// Tell our listeners that a log was archived.
if (!this.listeners.isEmpty()) {
for (WALActionsListener i : this.listeners) {
i.postLogArchive(file.getPath(), p);
}
}
}
LOG.debug("Moved " + files.length + " WAL file(s) to " + FSUtils.getPath(this.walArchiveDir));
}
LOG.info("Closed WAL: " + toString());
}
protected SyncFuture getSyncFuture(final long sequence, Span span) {
SyncFuture syncFuture = this.syncFuturesByHandler.get(Thread.currentThread());
if (syncFuture == null) {
syncFuture = new SyncFuture();
this.syncFuturesByHandler.put(Thread.currentThread(), syncFuture);
}
return syncFuture.reset(sequence, span);
}
protected void requestLogRoll(boolean tooFewReplicas) {
if (!this.listeners.isEmpty()) {
for (WALActionsListener i : this.listeners) {
i.logRollRequested(tooFewReplicas);
}
}
}
long getUnflushedEntriesCount() {
long highestSynced = this.highestSyncedTxid.get();
long highestUnsynced = this.highestUnsyncedTxid;
return highestSynced >= highestUnsynced ? 0 : highestUnsynced - highestSynced;
}
boolean isUnflushedEntries() {
return getUnflushedEntriesCount() > 0;
}
/**
* Exposed for testing only. Use to tricks like halt the ring buffer appending.
*/
@VisibleForTesting
void atHeadOfRingBufferEventHandlerAppend() {
// Noop
}
protected boolean append(W writer, FSWALEntry entry) throws IOException {
// TODO: WORK ON MAKING THIS APPEND FASTER. DOING WAY TOO MUCH WORK WITH CPs, PBing, etc.
atHeadOfRingBufferEventHandlerAppend();
long start = EnvironmentEdgeManager.currentTime();
byte[] encodedRegionName = entry.getKey().getEncodedRegionName();
long regionSequenceId = WALKey.NO_SEQUENCE_ID;
// 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.
regionSequenceId = entry.stampRegionSequenceId();
// Edits are empty, there is nothing to append. Maybe empty when we are looking for a
// region sequence id only, a region edit/sequence id that is not associated with an actual
// edit. It has to go through all the rigmarole to be sure we have the right ordering.
if (entry.getEdit().isEmpty()) {
return false;
}
// 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().serializeReplicationScope(false);
}
}
if (!listeners.isEmpty()) {
for (WALActionsListener i : listeners) {
i.visitLogEntryBeforeWrite(entry.getKey(), entry.getEdit());
}
}
doAppend(writer, entry);
assert highestUnsyncedTxid < entry.getTxid();
highestUnsyncedTxid = entry.getTxid();
sequenceIdAccounting.update(encodedRegionName, entry.getFamilyNames(), regionSequenceId,
entry.isInMemstore());
coprocessorHost.postWALWrite(entry.getHRegionInfo(), entry.getKey(), entry.getEdit());
// Update metrics.
postAppend(entry, EnvironmentEdgeManager.currentTime() - start);
numEntries.incrementAndGet();
return true;
}
private long postAppend(final Entry e, final long elapsedTime) {
long len = 0;
if (!listeners.isEmpty()) {
for (Cell cell : e.getEdit().getCells()) {
len += CellUtil.estimatedSerializedSizeOf(cell);
}
for (WALActionsListener listener : listeners) {
listener.postAppend(len, elapsedTime);
}
}
return len;
}
protected void postSync(final long timeInNanos, final int handlerSyncs) {
if (timeInNanos > this.slowSyncNs) {
String msg =
new StringBuilder().append("Slow sync cost: ").append(timeInNanos / 1000000)
.append(" ms, current pipeline: ").append(Arrays.toString(getPipeline())).toString();
Trace.addTimelineAnnotation(msg);
LOG.info(msg);
}
if (!listeners.isEmpty()) {
for (WALActionsListener listener : listeners) {
listener.postSync(timeInNanos, handlerSyncs);
}
}
}
/**
* NOTE: This append, at a time that is usually after this call returns, starts an mvcc
* transaction by calling 'begin' wherein which we assign this update a sequenceid. At assignment
* time, we stamp all the passed in Cells inside WALEdit with their sequenceId. You must
* 'complete' the transaction this mvcc transaction by calling
* MultiVersionConcurrencyControl#complete(...) or a variant otherwise mvcc will get stuck. Do it
* in the finally of a try/finally block within which this append lives and any subsequent
* operations like sync or update of memstore, etc. Get the WriteEntry to pass mvcc out of the
* passed in WALKey <code>walKey</code> parameter. Be warned that the WriteEntry is not
* immediately available on return from this method. It WILL be available subsequent to a sync of
* this append; otherwise, you will just have to wait on the WriteEntry to get filled in.
*/
@Override
public abstract long append(HRegionInfo info, WALKey key, WALEdit edits, boolean inMemstore)
throws IOException;
protected abstract void doAppend(W writer, FSWALEntry entry) throws IOException;
protected abstract W createWriterInstance(Path path) throws IOException;
/**
* @return old wal file size
*/
protected abstract long doReplaceWriter(Path oldPath, Path newPath, W nextWriter)
throws IOException;
protected abstract void doShutdown() throws IOException;
/**
* This method gets the pipeline for the current WAL.
*/
@VisibleForTesting
abstract DatanodeInfo[] getPipeline();
/**
* This method gets the datanode replication count for the current WAL.
*/
@VisibleForTesting
abstract int getLogReplication();
}

View File

@ -0,0 +1,170 @@
/**
* 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 static org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader.DEFAULT_WAL_TRAILER_WARN_SIZE;
import static org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader.WAL_TRAILER_WARN_SIZE;
import java.io.IOException;
import java.io.OutputStream;
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.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.codec.Codec;
import org.apache.hadoop.hbase.io.util.LRUDictionary;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALHeader;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALTrailer;
import org.apache.hadoop.hbase.util.FSUtils;
/**
* Base class for Protobuf log writer.
*/
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
public abstract class AbstractProtobufLogWriter {
private static final Log LOG = LogFactory.getLog(AbstractProtobufLogWriter.class);
protected CompressionContext compressionContext;
protected Configuration conf;
protected Codec.Encoder cellEncoder;
protected WALCellCodec.ByteStringCompressor compressor;
protected boolean trailerWritten;
protected WALTrailer trailer;
// maximum size of the wal Trailer in bytes. If a user writes/reads a trailer with size larger
// than this size, it is written/read respectively, with a WARN message in the log.
protected int trailerWarnSize;
protected AtomicLong length = new AtomicLong();
private WALCellCodec getCodec(Configuration conf, CompressionContext compressionContext)
throws IOException {
return WALCellCodec.create(conf, null, compressionContext);
}
protected WALHeader buildWALHeader(Configuration conf, WALHeader.Builder builder)
throws IOException {
if (!builder.hasWriterClsName()) {
builder.setWriterClsName(ProtobufLogWriter.class.getSimpleName());
}
if (!builder.hasCellCodecClsName()) {
builder.setCellCodecClsName(WALCellCodec.getWALCellCodecClass(conf));
}
return builder.build();
}
private boolean initializeCompressionContext(Configuration conf, Path path) throws IOException {
boolean doCompress = conf.getBoolean(HConstants.ENABLE_WAL_COMPRESSION, false);
if (doCompress) {
try {
this.compressionContext = new CompressionContext(LRUDictionary.class,
FSUtils.isRecoveredEdits(path),
conf.getBoolean(CompressionContext.ENABLE_WAL_TAGS_COMPRESSION, true));
} catch (Exception e) {
throw new IOException("Failed to initiate CompressionContext", e);
}
}
return doCompress;
}
public void init(FileSystem fs, Path path, Configuration conf, boolean overwritable)
throws IOException {
this.conf = conf;
boolean doCompress = initializeCompressionContext(conf, path);
this.trailerWarnSize = conf.getInt(WAL_TRAILER_WARN_SIZE, DEFAULT_WAL_TRAILER_WARN_SIZE);
int bufferSize = FSUtils.getDefaultBufferSize(fs);
short replication = (short) conf.getInt("hbase.regionserver.hlog.replication",
FSUtils.getDefaultReplication(fs, path));
long blockSize = conf.getLong("hbase.regionserver.hlog.blocksize",
FSUtils.getDefaultBlockSize(fs, path));
initOutput(fs, path, overwritable, bufferSize, replication, blockSize);
boolean doTagCompress = doCompress
&& conf.getBoolean(CompressionContext.ENABLE_WAL_TAGS_COMPRESSION, true);
length.set(writeMagicAndWALHeader(ProtobufLogReader.PB_WAL_MAGIC, buildWALHeader(conf,
WALHeader.newBuilder().setHasCompression(doCompress).setHasTagCompression(doTagCompress))));
initAfterHeader(doCompress);
// instantiate trailer to default value.
trailer = WALTrailer.newBuilder().build();
if (LOG.isTraceEnabled()) {
LOG.trace("Initialized protobuf WAL=" + path + ", compression=" + doCompress);
}
}
protected void initAfterHeader(boolean doCompress) throws IOException {
WALCellCodec codec = getCodec(conf, this.compressionContext);
this.cellEncoder = codec.getEncoder(getOutputStreamForCellEncoder());
if (doCompress) {
this.compressor = codec.getByteStringCompressor();
}
}
void setWALTrailer(WALTrailer walTrailer) {
this.trailer = walTrailer;
}
public long getLength() {
return length.get();
}
private WALTrailer buildWALTrailer(WALTrailer.Builder builder) {
return builder.build();
}
protected void writeWALTrailer() {
try {
int trailerSize = 0;
if (this.trailer == null) {
// use default trailer.
LOG.warn("WALTrailer is null. Continuing with default.");
this.trailer = buildWALTrailer(WALTrailer.newBuilder());
trailerSize = this.trailer.getSerializedSize();
} else if ((trailerSize = this.trailer.getSerializedSize()) > this.trailerWarnSize) {
// continue writing after warning the user.
LOG.warn("Please investigate WALTrailer usage. Trailer size > maximum size : " + trailerSize
+ " > " + this.trailerWarnSize);
}
length.set(writeWALTrailerAndMagic(trailer, ProtobufLogReader.PB_WAL_COMPLETE_MAGIC));
this.trailerWritten = true;
} catch (IOException ioe) {
LOG.warn("Failed to write trailer, non-fatal, continuing...", ioe);
}
}
protected abstract void initOutput(FileSystem fs, Path path, boolean overwritable, int bufferSize,
short replication, long blockSize) throws IOException;
/**
* return the file length after written.
*/
protected abstract long writeMagicAndWALHeader(byte[] magic, WALHeader header) throws IOException;
protected abstract long writeWALTrailerAndMagic(WALTrailer trailer, byte[] magic)
throws IOException;
protected abstract OutputStream getOutputStreamForCellEncoder();
}

View File

@ -0,0 +1,732 @@
/**
* 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 static org.apache.hadoop.hbase.HConstants.REGION_SERVER_HANDLER_COUNT;
import static org.apache.hadoop.hbase.util.FanOutOneBlockAsyncDFSOutputHelper.shouldRetryCreate;
import io.netty.channel.EventLoop;
import io.netty.util.concurrent.Future;
import io.netty.util.concurrent.Promise;
import io.netty.util.concurrent.ScheduledFuture;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.nio.channels.CompletionHandler;
import java.util.ArrayDeque;
import java.util.Comparator;
import java.util.Deque;
import java.util.Iterator;
import java.util.List;
import java.util.PriorityQueue;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.ConnectionUtils;
import org.apache.hadoop.hbase.util.FanOutOneBlockAsyncDFSOutput;
import org.apache.hadoop.hbase.util.FanOutOneBlockAsyncDFSOutputHelper.NameNodeException;
import org.apache.hadoop.hbase.wal.AsyncFSWALProvider;
import org.apache.hadoop.hbase.wal.WALKey;
import org.apache.hadoop.hbase.wal.WALProvider.AsyncWriter;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.htrace.NullScope;
import org.apache.htrace.Trace;
import org.apache.htrace.TraceScope;
/**
* An asynchronous implementation of FSWAL.
* <p>
* Here 'waitingConsumePayloads' acts as the RingBuffer in FSHLog. We do not use RingBuffer here
* because RingBuffer need an exclusive thread to consume the entries in it, and here we want to run
* the append and sync operation inside EventLoop. We can not use EventLoop as the RingBuffer's
* executor otherwise the EventLoop can not process any other events such as socket read and write.
* <p>
* For append, we process it as follow:
* <ol>
* <li>In the caller thread(typically, in the rpc handler thread):
* <ol>
* <li>Lock 'waitingConsumePayloads', bump nextTxid, and insert the entry to
* 'waitingConsumePayloads'.</li>
* <li>Schedule the consumer task if needed. See {@link #shouldScheduleConsumer()} for more details.
* </li>
* </ol>
* </li>
* <li>In the consumer task(in the EventLoop thread)
* <ol>
* <li>Poll the entry from 'waitingConsumePayloads' and insert it into 'waitingAppendEntries'</li>
* <li>Poll the entry from 'waitingAppendEntries', append it to the AsyncWriter, and insert it into
* 'unackedEntries'</li>
* <li>If the buffered size reaches {@link #batchSize}, or there is a sync request, then we call
* sync on the AsyncWriter.</li>
* <li>In the callback methods(CompletionHandler):
* <ul>
* <li>If succeeded, poll the entry from 'unackedEntries' and drop it.</li>
* <li>If failed, add all the entries in 'unackedEntries' back to 'waitingAppendEntries' and wait
* for writing them again.</li>
* </ul>
* </li>
* </ol>
* </li>
* </ol>
* For sync, the processing stages are almost same except that if it is not assigned with a new
* 'txid', we just assign the previous 'txid' to it without bumping the 'nextTxid'. And different
* from FSHLog, we will open a new writer and rewrite unacked entries to the new writer and sync
* again if we hit a sync error.
* <p>
* Here we only describe the logic of doReplaceWriter. The main logic of rollWriter is same with
* FSHLog.<br>
* For a normal roll request(for example, we have reached the log roll size):
* <ol>
* <li>In the log roller thread, we add a roll payload to 'waitingConsumePayloads', and then wait on
* the rollPromise(see {@link #waitForSafePoint()}).</li>
* <li>In the consumer thread, we will stop polling entries from 'waitingConsumePayloads' if we hit
* a Payload which contains a roll request.</li>
* <li>Append all entries to current writer, issue a sync request if possible.</li>
* <li>If sync succeeded, check if we could finish a roll request. There 3 conditions:
* <ul>
* <li>'rollPromise' is not null which means we have a pending roll request.</li>
* <li>'waitingAppendEntries' is empty.</li>
* <li>'unackedEntries' is empty.</li>
* </ul>
* </li>
* <li>Back to the log roller thread, now we can confirm that there are no out-going entries, i.e.,
* we reach a safe point. So it is safe to replace old writer with new writer now.</li>
* <li>Acquire 'waitingConsumePayloads' lock, set 'writerBroken' and 'waitingRoll' to false, cancel
* log roller exit checker if any(see the comments in the 'failed' method of the sync
* CompletionHandler to see why we need a checker here).</li>
* <li>Schedule the consumer task if needed.</li>
* <li>Schedule a background task to close the old writer.</li>
* </ol>
* For a broken writer roll request, the only difference is that we can bypass the wait for safe
* point stage. See the comments in the 'failed' method of the sync CompletionHandler for more
* details.
*/
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
private static final Log LOG = LogFactory.getLog(AsyncFSWAL.class);
public static final String WAL_BATCH_SIZE = "hbase.wal.batch.size";
public static final long DEFAULT_WAL_BATCH_SIZE = 64L * 1024;
public static final String ASYNC_WAL_CREATE_MAX_RETRIES = "hbase.wal.async.create.retries";
public static final int DEFAULT_ASYNC_WAL_CREATE_MAX_RETRIES = 10;
public static final String ASYNC_WAL_LOG_ROLLER_EXITED_CHECK_INTERVAL_MS =
"hbase.wal.async.logroller.exited.check.interval.ms";
public static final long DEFAULT_ASYNC_WAL_LOG_ROLLER_EXITED_CHECK_INTERVAL_MS = 1000;
/**
* Carry things that we want to pass to the consume task in event loop. Only one field can be
* non-null.
* <p>
* TODO: need to unify this and {@link RingBufferTruck}. There are mostly the same thing.
*/
private static final class Payload {
// a wal entry which need to be appended
public final FSWALEntry entry;
// indicate that we need to sync our wal writer.
public final SyncFuture sync;
// incidate that we want to roll the writer.
public final Promise<Void> roll;
public Payload(FSWALEntry entry) {
this.entry = entry;
this.sync = null;
this.roll = null;
}
public Payload(SyncFuture sync) {
this.entry = null;
this.sync = sync;
this.roll = null;
}
public Payload(Promise<Void> roll) {
this.entry = null;
this.sync = null;
this.roll = roll;
}
@Override
public String toString() {
return "Payload [entry=" + entry + ", sync=" + sync + ", roll=" + roll + "]";
}
}
private final EventLoop eventLoop;
private final Deque<Payload> waitingConsumePayloads;
// like the ringbuffer sequence. Every FSWALEntry and SyncFuture will be assigned a txid and
// then added to waitingConsumePayloads.
private long nextTxid = 1L;
private boolean consumerScheduled;
// new writer is created and we are waiting for old writer to be closed.
private boolean waitingRoll;
// writer is broken and rollWriter is needed.
private boolean writerBroken;
private final long batchSize;
private final int createMaxRetries;
private final long logRollerExitedCheckIntervalMs;
private final ExecutorService closeExecutor = Executors
.newCachedThreadPool(new ThreadFactoryBuilder().setDaemon(true)
.setNameFormat("Close-WAL-Writer-%d").build());
private volatile FanOutOneBlockAsyncDFSOutput hdfsOut;
private final Deque<FSWALEntry> waitingAppendEntries = new ArrayDeque<FSWALEntry>();
private final Deque<FSWALEntry> unackedEntries = new ArrayDeque<FSWALEntry>();
private final PriorityQueue<SyncFuture> syncFutures = new PriorityQueue<SyncFuture>(11,
SEQ_COMPARATOR);
private Promise<Void> rollPromise;
// the highest txid of WAL entries being processed
private long highestProcessedTxid;
// file length when we issue last sync request on the writer
private long fileLengthAtLastSync;
private volatile boolean logRollerExited;
private final class LogRollerExitedChecker implements Runnable {
private boolean cancelled;
private ScheduledFuture<?> future;
public synchronized void setFuture(ScheduledFuture<?> future) {
this.future = future;
}
@Override
public void run() {
if (!logRollerExited) {
return;
}
// rollWriter is called in the log roller thread, and logRollerExited will be set just before
// the log rolled exit. So here we can confirm that no one could cancel us if the 'canceled'
// check passed. So it is safe to release the lock after checking 'canceled' flag.
synchronized (this) {
if (cancelled) {
return;
}
}
unackedEntries.clear();
waitingAppendEntries.clear();
IOException error = new IOException("sync failed but log roller exited");
for (SyncFuture future; (future = syncFutures.peek()) != null;) {
future.done(highestProcessedTxid, error);
syncFutures.remove();
}
synchronized (waitingConsumePayloads) {
for (Payload p : waitingConsumePayloads) {
if (p.entry != null) {
try {
p.entry.stampRegionSequenceId();
} catch (IOException e) {
throw new AssertionError("should not happen", e);
}
} else if (p.sync != null) {
p.sync.done(nextTxid, error);
}
}
waitingConsumePayloads.clear();
}
}
public synchronized void cancel() {
future.cancel(false);
cancelled = true;
}
}
private LogRollerExitedChecker logRollerExitedChecker;
public AsyncFSWAL(FileSystem fs, Path rootDir, String logDir, String archiveDir,
Configuration conf, List<WALActionsListener> listeners, boolean failIfWALExists,
String prefix, String suffix, EventLoop eventLoop) throws FailedLogCloseException,
IOException {
super(fs, rootDir, logDir, archiveDir, conf, listeners, failIfWALExists, prefix, suffix);
this.eventLoop = eventLoop;
int maxHandlersCount = conf.getInt(REGION_SERVER_HANDLER_COUNT, 200);
waitingConsumePayloads = new ArrayDeque<Payload>(maxHandlersCount * 3);
batchSize = conf.getLong(WAL_BATCH_SIZE, DEFAULT_WAL_BATCH_SIZE);
createMaxRetries =
conf.getInt(ASYNC_WAL_CREATE_MAX_RETRIES, DEFAULT_ASYNC_WAL_CREATE_MAX_RETRIES);
logRollerExitedCheckIntervalMs =
conf.getLong(ASYNC_WAL_LOG_ROLLER_EXITED_CHECK_INTERVAL_MS,
DEFAULT_ASYNC_WAL_LOG_ROLLER_EXITED_CHECK_INTERVAL_MS);
rollWriter();
}
private void tryFinishRoll() {
// 1. a roll is requested
// 2. we have written out all entries before the roll point.
// 3. all entries have been acked.
if (rollPromise != null && waitingAppendEntries.isEmpty() && unackedEntries.isEmpty()) {
rollPromise.trySuccess(null);
rollPromise = null;
}
}
private void sync(final AsyncWriter writer, final long processedTxid) {
fileLengthAtLastSync = writer.getLength();
final long startTimeNs = System.nanoTime();
writer.sync(new CompletionHandler<Long, Void>() {
@Override
public void completed(Long result, Void attachment) {
highestSyncedTxid.set(processedTxid);
int syncCount = finishSync(true);
for (Iterator<FSWALEntry> iter = unackedEntries.iterator(); iter.hasNext();) {
if (iter.next().getTxid() <= processedTxid) {
iter.remove();
} else {
break;
}
}
postSync(System.nanoTime() - startTimeNs, syncCount);
tryFinishRoll();
if (!rollWriterLock.tryLock()) {
return;
}
try {
if (writer.getLength() >= logrollsize) {
requestLogRoll();
}
} finally {
rollWriterLock.unlock();
}
}
@Override
public void failed(Throwable exc, Void attachment) {
LOG.warn("sync failed", exc);
// Here we depends on the implementation of FanOutOneBlockAsyncDFSOutput and netty.
// When error occur, FanOutOneBlockAsyncDFSOutput will fail all pending flush requests. It
// is execute inside EventLoop. And in DefaultPromise in netty, it will notifyListener
// directly if it is already in the EventLoop thread. And in the listener method, it will
// call us. So here we know that all failed flush request will call us continuously, and
// before the last one finish, no other task can be executed in EventLoop. So here we are
// safe to use writerBroken as a guard.
// Do not forget to revisit this if we change the implementation of
// FanOutOneBlockAsyncDFSOutput!
synchronized (waitingConsumePayloads) {
if (writerBroken) {
return;
}
// schedule a periodical task to check if log roller is exited. Otherwise the the sync
// request maybe blocked forever since we are still waiting for a new writer to write the
// pending data and sync it...
logRollerExitedChecker = new LogRollerExitedChecker();
// we are currently in the EventLoop thread, so it is safe to set the future after
// schedule it since the task can not be executed before we release the thread.
logRollerExitedChecker.setFuture(eventLoop.scheduleAtFixedRate(logRollerExitedChecker,
logRollerExitedCheckIntervalMs, logRollerExitedCheckIntervalMs, TimeUnit.MILLISECONDS));
writerBroken = true;
}
for (Iterator<FSWALEntry> iter = unackedEntries.descendingIterator(); iter.hasNext();) {
waitingAppendEntries.addFirst(iter.next());
}
highestUnsyncedTxid = highestSyncedTxid.get();
if (rollPromise != null) {
rollPromise.trySuccess(null);
rollPromise = null;
return;
}
// request a roll.
if (!rollWriterLock.tryLock()) {
return;
}
try {
requestLogRoll();
} finally {
rollWriterLock.unlock();
}
}
}, null);
}
private void addTimeAnnotation(SyncFuture future, String annotation) {
TraceScope scope = Trace.continueSpan(future.getSpan());
Trace.addTimelineAnnotation(annotation);
future.setSpan(scope.detach());
}
private int finishSync(boolean addSyncTrace) {
long doneTxid = highestSyncedTxid.get();
int finished = 0;
for (SyncFuture future; (future = syncFutures.peek()) != null;) {
if (future.getTxid() <= doneTxid) {
future.done(doneTxid, null);
syncFutures.remove();
finished++;
addTimeAnnotation(future, "writer synced");
} else {
break;
}
}
return finished;
}
private void consume() {
final AsyncWriter writer = this.writer;
// maybe a sync request is not queued when we issue a sync, so check here to see if we could
// finish some.
finishSync(false);
long newHighestProcessedTxid = -1L;
for (Iterator<FSWALEntry> iter = waitingAppendEntries.iterator(); iter.hasNext();) {
FSWALEntry entry = iter.next();
boolean appended;
try {
appended = append(writer, entry);
} catch (IOException e) {
throw new AssertionError("should not happen", e);
}
newHighestProcessedTxid = entry.getTxid();
iter.remove();
if (appended) {
unackedEntries.addLast(entry);
if (writer.getLength() - fileLengthAtLastSync >= batchSize) {
break;
}
}
}
// if we have a newer transaction id, update it.
// otherwise, use the previous transaction id.
if (newHighestProcessedTxid > 0) {
highestProcessedTxid = newHighestProcessedTxid;
} else {
newHighestProcessedTxid = highestProcessedTxid;
}
if (writer.getLength() - fileLengthAtLastSync >= batchSize) {
// sync because buffer size limit.
sync(writer, newHighestProcessedTxid);
} else if ((!syncFutures.isEmpty() || rollPromise != null)
&& writer.getLength() > fileLengthAtLastSync) {
// first we should have at least one sync request or a roll request
// second we should have some unsynced data.
sync(writer, newHighestProcessedTxid);
} else if (writer.getLength() == fileLengthAtLastSync) {
// we haven't written anything out, just advance the highestSyncedSequence since we may only
// stamped some region sequence id.
highestSyncedTxid.set(newHighestProcessedTxid);
finishSync(false);
tryFinishRoll();
}
}
private static final Comparator<SyncFuture> SEQ_COMPARATOR = new Comparator<SyncFuture>() {
@Override
public int compare(SyncFuture o1, SyncFuture o2) {
int c = Long.compare(o1.getTxid(), o2.getTxid());
return c != 0 ? c : Integer.compare(System.identityHashCode(o1), System.identityHashCode(o2));
}
};
private final Runnable consumer = new Runnable() {
@Override
public void run() {
synchronized (waitingConsumePayloads) {
assert consumerScheduled;
if (writerBroken) {
// waiting for reschedule after rollWriter.
consumerScheduled = false;
return;
}
if (waitingRoll) {
// we may have toWriteEntries if the consume method does not write all pending entries
// out, this is usually happen if we have too many toWriteEntries that exceeded the
// batchSize limit.
if (waitingAppendEntries.isEmpty()) {
consumerScheduled = false;
return;
}
} else {
for (Payload p; (p = waitingConsumePayloads.pollFirst()) != null;) {
if (p.entry != null) {
waitingAppendEntries.addLast(p.entry);
} else if (p.sync != null) {
syncFutures.add(p.sync);
} else {
rollPromise = p.roll;
waitingRoll = true;
break;
}
}
}
}
consume();
synchronized (waitingConsumePayloads) {
if (waitingRoll) {
if (waitingAppendEntries.isEmpty()) {
consumerScheduled = false;
return;
}
} else {
if (waitingConsumePayloads.isEmpty() && waitingAppendEntries.isEmpty()) {
consumerScheduled = false;
return;
}
}
}
// reschedule if we still have something to write.
eventLoop.execute(this);
}
};
private boolean shouldScheduleConsumer() {
if (writerBroken || waitingRoll) {
return false;
}
if (consumerScheduled) {
return false;
}
consumerScheduled = true;
return true;
}
@Override
public long append(HRegionInfo hri, WALKey key, WALEdit edits, boolean inMemstore)
throws IOException {
boolean scheduleTask;
long txid;
synchronized (waitingConsumePayloads) {
if (this.closed) {
throw new IOException("Cannot append; log is closed");
}
txid = nextTxid++;
FSWALEntry entry = new FSWALEntry(txid, key, edits, hri, inMemstore);
scheduleTask = shouldScheduleConsumer();
waitingConsumePayloads.add(new Payload(entry));
}
if (scheduleTask) {
eventLoop.execute(consumer);
}
return txid;
}
@Override
public void sync() throws IOException {
TraceScope scope = Trace.startSpan("AsyncFSWAL.sync");
try {
SyncFuture future;
boolean scheduleTask;
synchronized (waitingConsumePayloads) {
scheduleTask = shouldScheduleConsumer();
future = getSyncFuture(nextTxid - 1, scope.detach());
waitingConsumePayloads.addLast(new Payload(future));
}
if (scheduleTask) {
eventLoop.execute(consumer);
}
scope = Trace.continueSpan(blockOnSync(future));
} finally {
assert scope == NullScope.INSTANCE || !scope.isDetached();
scope.close();
}
}
@Override
public void sync(long txid) throws IOException {
if (highestSyncedTxid.get() >= txid) {
return;
}
TraceScope scope = Trace.startSpan("AsyncFSWAL.sync");
try {
SyncFuture future = getSyncFuture(txid, scope.detach());
boolean scheduleTask;
synchronized (waitingConsumePayloads) {
scheduleTask = shouldScheduleConsumer();
waitingConsumePayloads.addLast(new Payload(future));
}
if (scheduleTask) {
eventLoop.execute(consumer);
}
scope = Trace.continueSpan(blockOnSync(future));
} finally {
assert scope == NullScope.INSTANCE || !scope.isDetached();
scope.close();
}
}
@Override
public void logRollerExited() {
logRollerExited = true;
}
@Override
protected AsyncWriter createWriterInstance(Path path) throws IOException {
boolean overwrite = false;
for (int retry = 0;; retry++) {
try {
return AsyncFSWALProvider.createAsyncWriter(conf, fs, path, overwrite, eventLoop);
} catch (RemoteException e) {
LOG.warn("create wal log writer " + path + " failed, retry = " + retry, e);
if (shouldRetryCreate(e)) {
if (retry >= createMaxRetries) {
break;
}
} else {
throw e.unwrapRemoteException();
}
} catch (NameNodeException e) {
throw e;
} catch (IOException e) {
LOG.warn("create wal log writer " + path + " failed, retry = " + retry, e);
if (retry >= createMaxRetries) {
break;
}
// overwrite the old broken file.
overwrite = true;
try {
Thread.sleep(ConnectionUtils.getPauseTime(100, retry));
} catch (InterruptedException ie) {
throw new InterruptedIOException();
}
}
}
throw new IOException("Failed to create wal log writer " + path + " after retrying "
+ createMaxRetries + " time(s)");
}
private void waitForSafePoint() {
Future<Void> roll;
boolean scheduleTask;
synchronized (waitingConsumePayloads) {
if (!writerBroken && this.writer != null) {
Promise<Void> promise = eventLoop.newPromise();
if (consumerScheduled) {
scheduleTask = false;
} else {
scheduleTask = consumerScheduled = true;
}
waitingConsumePayloads.addLast(new Payload(promise));
roll = promise;
} else {
roll = eventLoop.newSucceededFuture(null);
scheduleTask = false;
}
}
if (scheduleTask) {
eventLoop.execute(consumer);
}
roll.awaitUninterruptibly();
}
@Override
protected long doReplaceWriter(Path oldPath, Path newPath, AsyncWriter nextWriter)
throws IOException {
waitForSafePoint();
final AsyncWriter oldWriter = this.writer;
this.writer = nextWriter;
if (nextWriter != null && nextWriter instanceof AsyncProtobufLogWriter) {
this.hdfsOut = ((AsyncProtobufLogWriter) nextWriter).getOutput();
}
this.fileLengthAtLastSync = 0L;
boolean scheduleTask;
synchronized (waitingConsumePayloads) {
writerBroken = waitingRoll = false;
if (logRollerExitedChecker != null) {
logRollerExitedChecker.cancel();
logRollerExitedChecker = null;
}
if (consumerScheduled) {
scheduleTask = false;
} else {
if (waitingConsumePayloads.isEmpty() && waitingAppendEntries.isEmpty()) {
scheduleTask = false;
} else {
scheduleTask = consumerScheduled = true;
}
}
}
if (scheduleTask) {
eventLoop.execute(consumer);
}
long oldFileLen;
if (oldWriter != null) {
oldFileLen = oldWriter.getLength();
closeExecutor.execute(new Runnable() {
@Override
public void run() {
try {
oldWriter.close();
} catch (IOException e) {
LOG.warn("close old writer failed", e);
}
}
});
} else {
oldFileLen = 0L;
}
return oldFileLen;
}
@Override
protected void doShutdown() throws IOException {
waitForSafePoint();
this.writer.close();
this.writer = null;
closeExecutor.shutdown();
}
@Override
protected void doAppend(AsyncWriter writer, FSWALEntry entry) {
writer.append(entry);
}
@Override
DatanodeInfo[] getPipeline() {
FanOutOneBlockAsyncDFSOutput output = this.hdfsOut;
return output != null ? output.getPipeline() : new DatanodeInfo[0];
}
@Override
int getLogReplication() {
return getPipeline().length;
}
}

View File

@ -0,0 +1,203 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver.wal;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.io.OutputStream;
import java.nio.channels.CompletionHandler;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.io.ByteArrayOutputStream;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALHeader;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALTrailer;
import org.apache.hadoop.hbase.util.FanOutOneBlockAsyncDFSOutput;
import org.apache.hadoop.hbase.util.FanOutOneBlockAsyncDFSOutputHelper;
import org.apache.hadoop.hbase.wal.AsyncFSWALProvider;
import org.apache.hadoop.hbase.wal.WAL.Entry;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import com.google.common.base.Throwables;
import com.google.common.primitives.Ints;
import io.netty.channel.EventLoop;
/**
* AsyncWriter for protobuf-based WAL.
*/
@InterfaceAudience.Private
public class AsyncProtobufLogWriter extends AbstractProtobufLogWriter implements
AsyncFSWALProvider.AsyncWriter {
private static final Log LOG = LogFactory.getLog(AsyncProtobufLogWriter.class);
private static final class BlockingCompletionHandler implements CompletionHandler<Long, Void> {
private long size;
private Throwable error;
private boolean finished;
@Override
public void completed(Long result, Void attachment) {
synchronized (this) {
size = result.longValue();
finished = true;
notifyAll();
}
}
@Override
public void failed(Throwable exc, Void attachment) {
synchronized (this) {
error = exc;
finished = true;
notifyAll();
}
}
public long get() throws IOException {
synchronized (this) {
while (!finished) {
try {
wait();
} catch (InterruptedException e) {
throw new InterruptedIOException();
}
}
if (error != null) {
Throwables.propagateIfPossible(error, IOException.class);
throw new RuntimeException(error);
}
return size;
}
}
}
private final EventLoop eventLoop;
private FanOutOneBlockAsyncDFSOutput output;
private ByteArrayOutputStream buf;
public AsyncProtobufLogWriter(EventLoop eventLoop) {
this.eventLoop = eventLoop;
}
@Override
public void append(Entry entry) {
buf.reset();
entry.setCompressionContext(compressionContext);
try {
entry.getKey().getBuilder(compressor).setFollowingKvCount(entry.getEdit().size()).build()
.writeDelimitedTo(buf);
} catch (IOException e) {
throw new AssertionError("should not happen", e);
}
length.addAndGet(buf.size());
output.write(buf.getBuffer(), 0, buf.size());
try {
for (Cell cell : entry.getEdit().getCells()) {
buf.reset();
cellEncoder.write(cell);
length.addAndGet(buf.size());
output.write(buf.getBuffer(), 0, buf.size());
}
} catch (IOException e) {
throw new AssertionError("should not happen", e);
}
}
@Override
public <A> void sync(CompletionHandler<Long, A> handler, A attachment) {
output.flush(attachment, handler, false);
}
@Override
public synchronized void close() throws IOException {
if (this.output == null) {
return;
}
try {
writeWALTrailer();
output.close();
} catch (Exception e) {
LOG.warn("normal close failed, try recover", e);
output.recoverAndClose(null);
}
this.output = null;
}
public FanOutOneBlockAsyncDFSOutput getOutput() {
return this.output;
}
@Override
protected void initOutput(FileSystem fs, Path path, boolean overwritable, int bufferSize,
short replication, long blockSize) throws IOException {
this.output =
FanOutOneBlockAsyncDFSOutputHelper.createOutput((DistributedFileSystem) fs, path,
overwritable, false, replication, blockSize, eventLoop);
this.buf = new ByteArrayOutputStream();
}
@Override
protected long writeMagicAndWALHeader(byte[] magic, WALHeader header) throws IOException {
buf.reset();
header.writeDelimitedTo(buf);
final BlockingCompletionHandler handler = new BlockingCompletionHandler();
eventLoop.execute(new Runnable() {
@Override
public void run() {
output.write(ProtobufLogReader.PB_WAL_MAGIC);
output.write(buf.getBuffer(), 0, buf.size());
output.flush(null, handler, false);
}
});
return handler.get();
}
@Override
protected long writeWALTrailerAndMagic(WALTrailer trailer, final byte[] magic)
throws IOException {
buf.reset();
trailer.writeTo(buf);
final BlockingCompletionHandler handler = new BlockingCompletionHandler();
eventLoop.execute(new Runnable() {
public void run() {
output.write(buf.getBuffer(), 0, buf.size());
output.write(Ints.toByteArray(buf.size()));
output.write(magic);
output.flush(null, handler, false);
}
});
return handler.get();
}
@Override
protected OutputStream getOutputStreamForCellEncoder() {
return buf;
}
}

View File

@ -48,17 +48,20 @@ import org.apache.hadoop.hbase.wal.WALKey;
class FSWALEntry extends Entry {
// The below data members are denoted 'transient' just to highlight these are not persisted;
// they are only in memory and held here while passing over the ring buffer.
private final transient long sequence;
private final transient long txid;
private final transient boolean inMemstore;
private final transient HRegionInfo hri;
private final Set<byte[]> familyNames;
private final transient Set<byte[]> familyNames;
// In the new WAL logic, we will rewrite failed WAL entries to new WAL file, so we need to avoid
// calling stampRegionSequenceId again.
private transient boolean stamped = false;
FSWALEntry(final long sequence, final WALKey key, final WALEdit edit,
FSWALEntry(final long txid, final WALKey key, final WALEdit edit,
final HRegionInfo hri, final boolean inMemstore) {
super(key, edit);
this.inMemstore = inMemstore;
this.hri = hri;
this.sequence = sequence;
this.txid = txid;
if (inMemstore) {
// construct familyNames here to reduce the work of log sinker.
ArrayList<Cell> cells = this.getEdit().getCells();
@ -80,7 +83,7 @@ class FSWALEntry extends Entry {
}
public String toString() {
return "sequence=" + this.sequence + ", " + super.toString();
return "sequence=" + this.txid + ", " + super.toString();
};
boolean isInMemstore() {
@ -92,10 +95,10 @@ class FSWALEntry extends Entry {
}
/**
* @return The sequence on the ring buffer when this edit was added.
* @return The transaction id of this edit.
*/
long getSequence() {
return this.sequence;
long getTxid() {
return this.txid;
}
/**
@ -103,9 +106,12 @@ class FSWALEntry extends Entry {
* SIDE-EFFECT is our stamping the sequenceid into every Cell AND setting the sequenceid into the
* MVCC WriteEntry!!!!
* @return The sequenceid we stamped on this edit.
* @throws IOException
*/
long stampRegionSequenceId() throws IOException {
if (stamped) {
return getKey().getSequenceId();
}
stamped = true;
long regionSequenceId = WALKey.NO_SEQUENCE_ID;
MultiVersionConcurrencyControl mvcc = getKey().getMvcc();
MultiVersionConcurrencyControl.WriteEntry we = null;

View File

@ -19,108 +19,42 @@
package org.apache.hadoop.hbase.regionserver.wal;
import static org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader.DEFAULT_WAL_TRAILER_WARN_SIZE;
import static org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader.WAL_TRAILER_WARN_SIZE;
import java.io.IOException;
import java.io.OutputStream;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.codec.Codec;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALHeader;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALTrailer;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.wal.DefaultWALProvider;
import org.apache.hadoop.hbase.wal.WAL.Entry;
/**
* Writer for protobuf-based WAL.
*/
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
public class ProtobufLogWriter extends WriterBase {
@InterfaceAudience.Private
public class ProtobufLogWriter extends AbstractProtobufLogWriter
implements DefaultWALProvider.Writer {
private static final Log LOG = LogFactory.getLog(ProtobufLogWriter.class);
protected FSDataOutputStream output;
protected Codec.Encoder cellEncoder;
protected WALCellCodec.ByteStringCompressor compressor;
private boolean trailerWritten;
private WALTrailer trailer;
// maximum size of the wal Trailer in bytes. If a user writes/reads a trailer with size larger
// than this size, it is written/read respectively, with a WARN message in the log.
private int trailerWarnSize;
public ProtobufLogWriter() {
super();
}
protected WALCellCodec getCodec(Configuration conf, CompressionContext compressionContext)
throws IOException {
return WALCellCodec.create(conf, null, compressionContext);
}
protected WALHeader buildWALHeader(Configuration conf, WALHeader.Builder builder)
throws IOException {
if (!builder.hasWriterClsName()) {
builder.setWriterClsName(ProtobufLogWriter.class.getSimpleName());
}
if (!builder.hasCellCodecClsName()) {
builder.setCellCodecClsName(WALCellCodec.getWALCellCodecClass(conf));
}
return builder.build();
}
@Override
@SuppressWarnings("deprecation")
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);
this.trailerWarnSize = conf.getInt(WAL_TRAILER_WARN_SIZE, DEFAULT_WAL_TRAILER_WARN_SIZE);
int bufferSize = FSUtils.getDefaultBufferSize(fs);
short replication = (short)conf.getInt(
"hbase.regionserver.hlog.replication", FSUtils.getDefaultReplication(fs, path));
long blockSize = conf.getLong("hbase.regionserver.hlog.blocksize",
FSUtils.getDefaultBlockSize(fs, path));
output = fs.createNonRecursive(path, overwritable, bufferSize, replication, blockSize, null);
output.write(ProtobufLogReader.PB_WAL_MAGIC);
boolean doTagCompress = doCompress
&& conf.getBoolean(CompressionContext.ENABLE_WAL_TAGS_COMPRESSION, true);
buildWALHeader(conf,
WALHeader.newBuilder().setHasCompression(doCompress).setHasTagCompression(doTagCompress))
.writeDelimitedTo(output);
initAfterHeader(doCompress);
// instantiate trailer to default value.
trailer = WALTrailer.newBuilder().build();
if (LOG.isTraceEnabled()) {
LOG.trace("Initialized protobuf WAL=" + path + ", compression=" + doCompress);
}
}
protected void initAfterHeader(boolean doCompress) throws IOException {
WALCellCodec codec = getCodec(conf, this.compressionContext);
this.cellEncoder = codec.getEncoder(this.output);
if (doCompress) {
this.compressor = codec.getByteStringCompressor();
}
}
@Override
public void append(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 (Cell cell : entry.getEdit().getCells()) {
// cellEncoder must assume little about the stream, since we write PB and cells in turn.
cellEncoder.write(cell);
}
length.set(output.getPos());
}
@Override
@ -137,32 +71,6 @@ public class ProtobufLogWriter extends WriterBase {
}
}
WALTrailer buildWALTrailer(WALTrailer.Builder builder) {
return builder.build();
}
private void writeWALTrailer() {
try {
int trailerSize = 0;
if (this.trailer == null) {
// use default trailer.
LOG.warn("WALTrailer is null. Continuing with default.");
this.trailer = buildWALTrailer(WALTrailer.newBuilder());
trailerSize = this.trailer.getSerializedSize();
} else if ((trailerSize = this.trailer.getSerializedSize()) > this.trailerWarnSize) {
// continue writing after warning the user.
LOG.warn("Please investigate WALTrailer usage. Trailer size > maximum size : " +
trailerSize + " > " + this.trailerWarnSize);
}
this.trailer.writeTo(output);
output.writeInt(trailerSize);
output.write(ProtobufLogReader.PB_WAL_COMPLETE_MAGIC);
this.trailerWritten = true;
} catch (IOException ioe) {
LOG.warn("Failed to write trailer, non-fatal, continuing...", ioe);
}
}
@Override
public void sync() throws IOException {
FSDataOutputStream fsdos = this.output;
@ -171,21 +79,35 @@ public class ProtobufLogWriter extends WriterBase {
fsdos.hflush();
}
@Override
public long getLength() throws IOException {
try {
return this.output.getPos();
} catch (NullPointerException npe) {
// Concurrent close...
throw new IOException(npe);
}
}
public FSDataOutputStream getStream() {
return this.output;
}
void setWALTrailer(WALTrailer walTrailer) {
this.trailer = walTrailer;
@SuppressWarnings("deprecation")
@Override
protected void initOutput(FileSystem fs, Path path, boolean overwritable, int bufferSize,
short replication, long blockSize) throws IOException {
this.output = fs.createNonRecursive(path, overwritable, bufferSize, replication, blockSize,
null);
}
@Override
protected long writeMagicAndWALHeader(byte[] magic, WALHeader header) throws IOException {
output.write(magic);
header.writeDelimitedTo(output);
return output.getPos();
}
@Override
protected OutputStream getOutputStreamForCellEncoder() {
return this.output;
}
@Override
protected long writeWALTrailerAndMagic(WALTrailer trailer, byte[] magic) throws IOException {
trailer.writeTo(output);
output.writeInt(trailer.getSerializedSize());
output.write(magic);
return output.getPos();
}
}

View File

@ -18,33 +18,28 @@
package org.apache.hadoop.hbase.regionserver.wal;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.htrace.Span;
/**
* A Future on a filesystem sync call. It given to a client or 'Handler' for it to wait on till
* the sync completes.
*
* <p>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.
* A Future on a filesystem sync call. It given to a client or 'Handler' for it to wait on till the
* sync completes.
* <p>
* This is just a partial implementation of Future; we just implement get and
* failure. Unimplemented methods throw {@link UnsupportedOperationException}.
* 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 a queue and is handled by a background thread; when it completes, it finishes up the
* future, the handler get or failed check completes and the Handler can then progress.
* <p>
* 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.
* This is just a partial implementation of Future; we just implement get and failure.
* <p>
* SyncFutures are immutable but recycled. Call #reset(long, Span) before use even
* if it the first time, start the sync, then park the 'hitched' thread on a call to
* #get().
* 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.
* <p>
* SyncFutures are immutable but recycled. Call #reset(long, Span) before use even if it the first
* time, start the sync, then park the 'hitched' thread on a call to #get().
*/
@InterfaceAudience.Private
class SyncFuture {
@ -54,17 +49,17 @@ class SyncFuture {
private static final long NOT_DONE = 0;
/**
* The sequence at which we were added to the ring buffer.
* The transaction id of this operation, monotonically increases.
*/
private long ringBufferSequence;
private long txid;
/**
* 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, Span)} below and it will work.
* The transaction id that was set in here when we were marked done. Should be equal or > txnId.
* 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, Span)}
* below and it will work.
*/
private long doneSequence = -1;
private long doneTxid = -1;
/**
* If error, the associated throwable. Set when the future is 'done'.
@ -79,80 +74,83 @@ class SyncFuture {
private Span span;
/**
* 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 sequenceId from this Future's position in the RingBuffer
* 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 txnId the new transaction id
* @return this
*/
synchronized SyncFuture reset(final long sequence) {
return reset(sequence, null);
synchronized SyncFuture reset(final long txnId) {
return reset(txnId, null);
}
/**
* 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.
*
* 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 sequenceId from this Future's position in the RingBuffer
* @param span curren span, detached from caller. Don't forget to attach it when
* resuming after a call to {@link #get()}.
* @param span curren span, detached from caller. Don't forget to attach it when resuming after a
* call to {@link #get()}.
* @return this
*/
synchronized SyncFuture reset(final long sequence, Span span) {
if (t != null && t != Thread.currentThread()) throw new IllegalStateException();
synchronized SyncFuture reset(final long txnId, Span span) {
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;
if (!isDone()) {
throw new IllegalStateException("" + txnId + " " + Thread.currentThread());
}
this.doneTxid = NOT_DONE;
this.txid = txnId;
this.span = span;
return this;
}
@Override
public synchronized String toString() {
return "done=" + isDone() + ", ringBufferSequence=" + this.ringBufferSequence;
return "done=" + isDone() + ", txid=" + this.txid;
}
synchronized long getRingBufferSequence() {
return this.ringBufferSequence;
synchronized long getTxid() {
return this.txid;
}
/**
* Retrieve the {@code span} instance from this Future. EventHandler calls
* this method to continue the span. Thread waiting on this Future musn't call
* this method until AFTER calling {@link #get()} and the future has been
* released back to the originating thread.
* Retrieve the {@code span} instance from this Future. EventHandler calls this method to continue
* the span. Thread waiting on this Future musn't call this method until AFTER calling
* {@link #get()} and the future has been released back to the originating thread.
*/
synchronized Span getSpan() {
return this.span;
}
/**
* Used to re-attach a {@code span} to the Future. Called by the EventHandler
* after a it has completed processing and detached the span from its scope.
* Used to re-attach a {@code span} to the Future. Called by the EventHandler after a it has
* completed processing and detached the span from its scope.
*/
synchronized void setSpan(Span span) {
this.span = span;
}
/**
* @param sequence Sync sequence at which this future 'completed'.
* @param txid the transaction id 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.
* @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;
synchronized boolean done(final long txid, final Throwable t) {
if (isDone()) {
return false;
}
this.throwable = t;
if (sequence < this.ringBufferSequence) {
if (txid < this.txid) {
// Something badly wrong.
if (throwable == null) {
this.throwable = new IllegalStateException("sequence=" + sequence +
", ringBufferSequence=" + this.ringBufferSequence);
this.throwable =
new IllegalStateException("done txid=" + txid + ", my txid=" + this.txid);
}
}
// Mark done.
this.doneSequence = sequence;
this.doneTxid = txid;
// Wake up waiting threads.
notify();
return true;
@ -166,21 +164,14 @@ class SyncFuture {
while (!isDone()) {
wait(1000);
}
if (this.throwable != null) throw new ExecutionException(this.throwable);
return this.doneSequence;
if (this.throwable != null) {
throw new ExecutionException(this.throwable);
}
public Long get(long timeout, TimeUnit unit)
throws InterruptedException, ExecutionException {
throw new UnsupportedOperationException();
}
public boolean isCancelled() {
throw new UnsupportedOperationException();
return this.doneTxid;
}
synchronized boolean isDone() {
return this.doneSequence != NOT_DONE;
return this.doneTxid != NOT_DONE;
}
synchronized boolean isThrowable() {

View File

@ -1,62 +0,0 @@
/**
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver.wal;
import java.io.IOException;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.io.util.LRUDictionary;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.wal.DefaultWALProvider;
/**
* Context used by our wal dictionary compressor. Null if we're not to do our
* custom dictionary compression.
*/
@InterfaceAudience.Private
public abstract class WriterBase implements DefaultWALProvider.Writer {
protected CompressionContext compressionContext;
protected Configuration conf;
@Override
public void init(FileSystem fs, Path path, Configuration conf, boolean overwritable) throws IOException {
this.conf = conf;
}
public boolean initializeCompressionContext(Configuration conf, Path path) throws IOException {
boolean doCompress = conf.getBoolean(HConstants.ENABLE_WAL_COMPRESSION, false);
if (doCompress) {
try {
this.compressionContext = new CompressionContext(LRUDictionary.class,
FSUtils.isRecoveredEdits(path), conf.getBoolean(
CompressionContext.ENABLE_WAL_TAGS_COMPRESSION, true));
} catch (Exception e) {
throw new IOException("Failed to initiate CompressionContext", e);
}
}
return doCompress;
}
}

View File

@ -72,6 +72,7 @@ import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
import org.apache.hadoop.io.EnumSetWritable;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.util.DataChecksum;
@ -417,7 +418,6 @@ public class FanOutOneBlockAsyncDFSOutputHelper {
public boolean progress() {
return DFS_CLIENT_ADAPTOR.isClientRunning(client);
}
}
static {
@ -579,6 +579,18 @@ public class FanOutOneBlockAsyncDFSOutputHelper {
return futureList;
}
/**
* Exception other than RemoteException thrown when calling create on namenode
*/
public static class NameNodeException extends IOException {
private static final long serialVersionUID = 3143237406477095390L;
public NameNodeException(Throwable cause) {
super(cause);
}
}
private static FanOutOneBlockAsyncDFSOutput createOutput(DistributedFileSystem dfs, String src,
boolean overwrite, boolean createParent, short replication, long blockSize,
EventLoop eventLoop) throws IOException {
@ -587,11 +599,20 @@ public class FanOutOneBlockAsyncDFSOutputHelper {
DFSClient client = dfs.getClient();
String clientName = client.getClientName();
ClientProtocol namenode = client.getNamenode();
HdfsFileStatus stat = FILE_CREATER.create(namenode, src,
HdfsFileStatus stat;
try {
stat = FILE_CREATER.create(namenode, src,
FsPermission.getFileDefault().applyUMask(FsPermission.getUMask(conf)), clientName,
new EnumSetWritable<CreateFlag>(
overwrite ? EnumSet.of(CREATE, OVERWRITE) : EnumSet.of(CREATE)),
createParent, replication, blockSize);
} catch (Exception e) {
if (e instanceof RemoteException) {
throw (RemoteException) e;
} else {
throw new NameNodeException(e);
}
}
beginFileLease(client, src, stat.getFileId());
boolean succ = false;
LocatedBlock locatedBlock = null;
@ -656,6 +677,13 @@ public class FanOutOneBlockAsyncDFSOutputHelper {
}.resolve(dfs, f);
}
public static boolean shouldRetryCreate(RemoteException e) {
// RetryStartFileException is introduced in HDFS 2.6+, so here we can only use the class name.
// For exceptions other than this, we just throw it out. This is same with
// DFSOutputStream.newStreamForCreate.
return e.getClassName().endsWith("RetryStartFileException");
}
static void completeFile(DFSClient client, ClientProtocol namenode, String src, String clientName,
ExtendedBlock block, long fileId) {
for (int retry = 0;; retry++) {

View File

@ -0,0 +1,361 @@
/**
* 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.wal;
import java.io.IOException;
import java.util.List;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.regex.Pattern;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL;
import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
import com.google.common.annotations.VisibleForTesting;
/**
* Base class of a WAL Provider that returns a single thread safe WAL that writes to HDFS. By
* default, this implementation picks a directory in HDFS based on a combination of
* <ul>
* <li>the HBase root directory
* <li>HConstants.HREGION_LOGDIR_NAME
* <li>the given factory's factoryId (usually identifying the regionserver by host:port)
* </ul>
* It also uses the providerId to differentiate among files.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public abstract class AbstractFSWALProvider<T extends AbstractFSWAL<?>> implements WALProvider {
private static final Log LOG = LogFactory.getLog(AbstractFSWALProvider.class);
// Only public so classes back in regionserver.wal can access
public interface Reader extends WAL.Reader {
/**
* @param fs File system.
* @param path Path.
* @param c Configuration.
* @param s Input stream that may have been pre-opened by the caller; may be null.
*/
void init(FileSystem fs, Path path, Configuration c, FSDataInputStream s) throws IOException;
}
protected volatile T wal;
protected WALFactory factory = null;
protected Configuration conf = null;
protected List<WALActionsListener> listeners = null;
protected String providerId = null;
protected AtomicBoolean initialized = new AtomicBoolean(false);
// for default wal provider, logPrefix won't change
protected String logPrefix = null;
/**
* we synchronized on walCreateLock to prevent wal recreation in different threads
*/
private final Object walCreateLock = new Object();
/**
* @param factory factory that made us, identity used for FS layout. may not be null
* @param conf may not be null
* @param listeners may be null
* @param providerId differentiate between providers from one facotry, used for FS layout. may be
* null
*/
@Override
public void init(WALFactory factory, Configuration conf, List<WALActionsListener> listeners,
String providerId) throws IOException {
if (!initialized.compareAndSet(false, true)) {
throw new IllegalStateException("WALProvider.init should only be called once.");
}
this.factory = factory;
this.conf = conf;
this.listeners = listeners;
this.providerId = providerId;
// get log prefix
StringBuilder sb = new StringBuilder().append(factory.factoryId);
if (providerId != null) {
if (providerId.startsWith(WAL_FILE_NAME_DELIMITER)) {
sb.append(providerId);
} else {
sb.append(WAL_FILE_NAME_DELIMITER).append(providerId);
}
}
logPrefix = sb.toString();
doInit(conf);
}
@Override
public WAL getWAL(byte[] identifier, byte[] namespace) throws IOException {
T walCopy = wal;
if (walCopy == null) {
// only lock when need to create wal, and need to lock since
// creating hlog on fs is time consuming
synchronized (walCreateLock) {
walCopy = wal;
if (walCopy == null) {
walCopy = createWAL();
wal = walCopy;
}
}
}
return walCopy;
}
protected abstract T createWAL() throws IOException;
protected abstract void doInit(Configuration conf) throws IOException;
@Override
public void shutdown() throws IOException {
T log = this.wal;
if (log != null) {
log.shutdown();
}
}
@Override
public void close() throws IOException {
T log = this.wal;
if (log != null) {
log.close();
}
}
/**
* iff the given WALFactory is using the DefaultWALProvider for meta and/or non-meta, count the
* number of files (rolled and active). if either of them aren't, count 0 for that provider.
*/
@Override
public long getNumLogFiles() {
T log = this.wal;
return log == null ? 0 : log.getNumLogFiles();
}
/**
* iff the given WALFactory is using the DefaultWALProvider for meta and/or non-meta, count the
* size of files (rolled and active). if either of them aren't, count 0 for that provider.
*/
@Override
public long getLogFileSize() {
T log = this.wal;
return log == null ? 0 : log.getLogFileSize();
}
/**
* returns the number of rolled WAL files.
*/
@VisibleForTesting
public static int getNumRolledLogFiles(WAL wal) {
return ((AbstractFSWAL<?>) wal).getNumRolledLogFiles();
}
/**
* return the current filename from the current wal.
*/
@VisibleForTesting
public static Path getCurrentFileName(final WAL wal) {
return ((AbstractFSWAL<?>) wal).getCurrentFileName();
}
/**
* request a log roll, but don't actually do it.
*/
@VisibleForTesting
static void requestLogRoll(final WAL wal) {
((AbstractFSWAL<?>) wal).requestLogRoll();
}
// should be package private; more visible for use in AbstractFSWAL
public static final String WAL_FILE_NAME_DELIMITER = ".";
/** The hbase:meta region's WAL filename extension */
@VisibleForTesting
public static final String META_WAL_PROVIDER_ID = ".meta";
static final String DEFAULT_PROVIDER_ID = "default";
// Implementation details that currently leak in tests or elsewhere follow
/** File Extension used while splitting an WAL into regions (HBASE-2312) */
public static final String SPLITTING_EXT = "-splitting";
/**
* It returns the file create timestamp from the file name. For name format see
* {@link #validateWALFilename(String)} public until remaining tests move to o.a.h.h.wal
* @param wal must not be null
* @return the file number that is part of the WAL file name
*/
@VisibleForTesting
public static long extractFileNumFromWAL(final WAL wal) {
final Path walName = ((AbstractFSWAL<?>) wal).getCurrentFileName();
if (walName == null) {
throw new IllegalArgumentException("The WAL path couldn't be null");
}
final String[] walPathStrs = walName.toString().split("\\" + WAL_FILE_NAME_DELIMITER);
return Long.parseLong(walPathStrs[walPathStrs.length - (isMetaFile(walName) ? 2 : 1)]);
}
/**
* Pattern used to validate a WAL file name see {@link #validateWALFilename(String)} for
* description.
*/
private static final Pattern pattern = Pattern
.compile(".*\\.\\d*(" + META_WAL_PROVIDER_ID + ")*");
/**
* A WAL file name is of the format: &lt;wal-name&gt;{@link #WAL_FILE_NAME_DELIMITER}
* &lt;file-creation-timestamp&gt;[.meta]. provider-name is usually made up of a server-name and a
* provider-id
* @param filename name of the file to validate
* @return <tt>true</tt> if the filename matches an WAL, <tt>false</tt> otherwise
*/
public static boolean validateWALFilename(String filename) {
return pattern.matcher(filename).matches();
}
/**
* Construct the directory name for all WALs on a given server.
* @param serverName Server name formatted as described in {@link ServerName}
* @return the relative WAL directory name, e.g. <code>.logs/1.example.org,60030,12345</code> if
* <code>serverName</code> passed is <code>1.example.org,60030,12345</code>
*/
public static String getWALDirectoryName(final String serverName) {
StringBuilder dirName = new StringBuilder(HConstants.HREGION_LOGDIR_NAME);
dirName.append("/");
dirName.append(serverName);
return dirName.toString();
}
/**
* Pulls a ServerName out of a Path generated according to our layout rules. In the below layouts,
* this method ignores the format of the logfile component. Current format: [base directory for
* hbase]/hbase/.logs/ServerName/logfile or [base directory for
* hbase]/hbase/.logs/ServerName-splitting/logfile Expected to work for individual log files and
* server-specific directories.
* @return null if it's not a log file. Returns the ServerName of the region server that created
* this log file otherwise.
*/
public static ServerName getServerNameFromWALDirectoryName(Configuration conf, String path)
throws IOException {
if (path == null || path.length() <= HConstants.HREGION_LOGDIR_NAME.length()) {
return null;
}
if (conf == null) {
throw new IllegalArgumentException("parameter conf must be set");
}
final String rootDir = conf.get(HConstants.HBASE_DIR);
if (rootDir == null || rootDir.isEmpty()) {
throw new IllegalArgumentException(HConstants.HBASE_DIR + " key not found in conf.");
}
final StringBuilder startPathSB = new StringBuilder(rootDir);
if (!rootDir.endsWith("/")) {
startPathSB.append('/');
}
startPathSB.append(HConstants.HREGION_LOGDIR_NAME);
if (!HConstants.HREGION_LOGDIR_NAME.endsWith("/")) {
startPathSB.append('/');
}
final String startPath = startPathSB.toString();
String fullPath;
try {
fullPath = FileSystem.get(conf).makeQualified(new Path(path)).toString();
} catch (IllegalArgumentException e) {
LOG.info("Call to makeQualified failed on " + path + " " + e.getMessage());
return null;
}
if (!fullPath.startsWith(startPath)) {
return null;
}
final String serverNameAndFile = fullPath.substring(startPath.length());
if (serverNameAndFile.indexOf('/') < "a,0,0".length()) {
// Either it's a file (not a directory) or it's not a ServerName format
return null;
}
Path p = new Path(path);
return getServerNameFromWALDirectoryName(p);
}
/**
* This function returns region server name from a log file name which is in one of the following
* formats:
* <ul>
* <li>hdfs://&lt;name node&gt;/hbase/.logs/&lt;server name&gt;-splitting/...</li>
* <li>hdfs://&lt;name node&gt;/hbase/.logs/&lt;server name&gt;/...</li>
* </ul>
* @return null if the passed in logFile isn't a valid WAL file path
*/
public static ServerName getServerNameFromWALDirectoryName(Path logFile) {
String logDirName = logFile.getParent().getName();
// We were passed the directory and not a file in it.
if (logDirName.equals(HConstants.HREGION_LOGDIR_NAME)) {
logDirName = logFile.getName();
}
ServerName serverName = null;
if (logDirName.endsWith(SPLITTING_EXT)) {
logDirName = logDirName.substring(0, logDirName.length() - SPLITTING_EXT.length());
}
try {
serverName = ServerName.parseServerName(logDirName);
} catch (IllegalArgumentException ex) {
serverName = null;
LOG.warn("Cannot parse a server name from path=" + logFile + "; " + ex.getMessage());
}
if (serverName != null && serverName.getStartcode() < 0) {
LOG.warn("Invalid log file path=" + logFile);
serverName = null;
}
return serverName;
}
public static boolean isMetaFile(Path p) {
return isMetaFile(p.getName());
}
public static boolean isMetaFile(String p) {
if (p != null && p.endsWith(META_WAL_PROVIDER_ID)) {
return true;
}
return false;
}
/**
* Get prefix of the log from its name, assuming WAL name in format of
* log_prefix.filenumber.log_suffix
* @param name Name of the WAL to parse
* @return prefix of the log
* @see AbstractFSWAL#getCurrentFileName()
*/
public static String getWALPrefixFromWALName(String name) {
int endIndex = name.replaceAll(META_WAL_PROVIDER_ID, "").lastIndexOf(".");
return name.substring(0, endIndex);
}
}

View File

@ -0,0 +1,73 @@
/**
* 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.wal;
import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL;
import org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Threads;
import io.netty.channel.EventLoop;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
/**
* A WAL provider that use {@link AsyncFSWAL}.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class AsyncFSWALProvider extends AbstractFSWALProvider<AsyncFSWAL> {
// Only public so classes back in regionserver.wal can access
public interface AsyncWriter extends WALProvider.AsyncWriter {
void init(FileSystem fs, Path path, Configuration c, boolean overwritable) throws IOException;
}
private EventLoopGroup eventLoopGroup = null;
@Override
protected AsyncFSWAL createWAL() throws IOException {
return new AsyncFSWAL(FileSystem.get(conf), FSUtils.getRootDir(conf),
getWALDirectoryName(factory.factoryId), HConstants.HREGION_OLDLOGDIR_NAME, conf, listeners,
true, logPrefix, META_WAL_PROVIDER_ID.equals(providerId) ? META_WAL_PROVIDER_ID : null,
eventLoopGroup.next());
}
@Override
protected void doInit(Configuration conf) throws IOException {
eventLoopGroup = new NioEventLoopGroup(1, Threads.newDaemonThreadFactory("AsyncFSWAL"));
}
/**
* public because of AsyncFSWAL. Should be package-private
*/
public static AsyncWriter createAsyncWriter(Configuration conf, FileSystem fs, Path path,
boolean overwritable, EventLoop eventLoop) throws IOException {
AsyncWriter writer = new AsyncProtobufLogWriter(eventLoop);
writer.init(fs, path, conf, overwritable);
return writer;
}
}

View File

@ -19,354 +19,39 @@
package org.apache.hadoop.hbase.wal;
import java.io.IOException;
import java.util.List;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.regex.Pattern;
import com.google.common.annotations.VisibleForTesting;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
// imports for things that haven't moved from regionserver.wal yet.
import org.apache.hadoop.hbase.regionserver.wal.FSHLog;
import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogWriter;
import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
import org.apache.hadoop.hbase.util.FSUtils;
/**
* A WAL Provider that returns a single thread safe WAL that writes to HDFS.
* By default, this implementation picks a directory in HDFS based on a combination of
* <ul>
* <li>the HBase root directory
* <li>HConstants.HREGION_LOGDIR_NAME
* <li>the given factory's factoryId (usually identifying the regionserver by host:port)
* </ul>
* It also uses the providerId to diffentiate among files.
*
* A WAL provider that use {@link FSHLog}.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class DefaultWALProvider implements WALProvider {
private static final Log LOG = LogFactory.getLog(DefaultWALProvider.class);
public class DefaultWALProvider extends AbstractFSWALProvider<FSHLog> {
// Only public so classes back in regionserver.wal can access
public interface Reader extends WAL.Reader {
/**
* @param fs File system.
* @param path Path.
* @param c Configuration.
* @param s Input stream that may have been pre-opened by the caller; may be null.
*/
void init(FileSystem fs, Path path, Configuration c, FSDataInputStream s) throws IOException;
}
private static final Log LOG = LogFactory.getLog(DefaultWALProvider.class);
// Only public so classes back in regionserver.wal can access
public interface Writer extends WALProvider.Writer {
void init(FileSystem fs, Path path, Configuration c, boolean overwritable) throws IOException;
}
protected volatile FSHLog log = null;
private WALFactory factory = null;
private Configuration conf = null;
private List<WALActionsListener> listeners = null;
private String providerId = null;
private AtomicBoolean initialized = new AtomicBoolean(false);
// for default wal provider, logPrefix won't change
private String logPrefix = null;
/**
* we synchronized on walCreateLock to prevent wal recreation in different threads
*/
private final Object walCreateLock = new Object();
/**
* @param factory factory that made us, identity used for FS layout. may not be null
* @param conf may not be null
* @param listeners may be null
* @param providerId differentiate between providers from one facotry, used for FS layout. may be
* null
*/
@Override
public void init(final WALFactory factory, final Configuration conf,
final List<WALActionsListener> listeners, String providerId) throws IOException {
if (!initialized.compareAndSet(false, true)) {
throw new IllegalStateException("WALProvider.init should only be called once.");
}
this.factory = factory;
this.conf = conf;
this.listeners = listeners;
this.providerId = providerId;
// get log prefix
StringBuilder sb = new StringBuilder().append(factory.factoryId);
if (providerId != null) {
if (providerId.startsWith(WAL_FILE_NAME_DELIMITER)) {
sb.append(providerId);
} else {
sb.append(WAL_FILE_NAME_DELIMITER).append(providerId);
}
}
logPrefix = sb.toString();
}
@Override
public WAL getWAL(final byte[] identifier, byte[] namespace) throws IOException {
if (log == null) {
// only lock when need to create wal, and need to lock since
// creating hlog on fs is time consuming
synchronized (walCreateLock) {
if (log == null) {
log = new FSHLog(FileSystem.get(conf), FSUtils.getRootDir(conf),
getWALDirectoryName(factory.factoryId), HConstants.HREGION_OLDLOGDIR_NAME, conf,
listeners, true, logPrefix,
META_WAL_PROVIDER_ID.equals(providerId) ? META_WAL_PROVIDER_ID : null);
}
}
}
return log;
}
@Override
public void close() throws IOException {
if (log != null) log.close();
}
@Override
public void shutdown() throws IOException {
if (log != null) log.shutdown();
}
// should be package private; more visible for use in FSHLog
public static final String WAL_FILE_NAME_DELIMITER = ".";
/** The hbase:meta region's WAL filename extension */
@VisibleForTesting
public static final String META_WAL_PROVIDER_ID = ".meta";
static final String DEFAULT_PROVIDER_ID = "default";
// Implementation details that currently leak in tests or elsewhere follow
/** File Extension used while splitting an WAL into regions (HBASE-2312) */
public static final String SPLITTING_EXT = "-splitting";
/**
* iff the given WALFactory is using the DefaultWALProvider for meta and/or non-meta,
* count the number of files (rolled and active). if either of them aren't, count 0
* for that provider.
*/
@Override
public long getNumLogFiles() {
return log == null ? 0 : this.log.getNumLogFiles();
}
/**
* iff the given WALFactory is using the DefaultWALProvider for meta and/or non-meta,
* count the size of files (rolled and active). if either of them aren't, count 0
* for that provider.
*/
@Override
public long getLogFileSize() {
return log == null ? 0 : this.log.getLogFileSize();
}
/**
* returns the number of rolled WAL files.
*/
@VisibleForTesting
public static int getNumRolledLogFiles(WAL wal) {
return ((FSHLog)wal).getNumRolledLogFiles();
}
/**
* return the current filename from the current wal.
*/
@VisibleForTesting
public static Path getCurrentFileName(final WAL wal) {
return ((FSHLog)wal).getCurrentFileName();
}
/**
* request a log roll, but don't actually do it.
*/
@VisibleForTesting
static void requestLogRoll(final WAL wal) {
((FSHLog)wal).requestLogRoll();
}
/**
* It returns the file create timestamp from the file name.
* For name format see {@link #validateWALFilename(String)}
* public until remaining tests move to o.a.h.h.wal
* @param wal must not be null
* @return the file number that is part of the WAL file name
*/
@VisibleForTesting
public static long extractFileNumFromWAL(final WAL wal) {
final Path walName = ((FSHLog)wal).getCurrentFileName();
if (walName == null) {
throw new IllegalArgumentException("The WAL path couldn't be null");
}
final String[] walPathStrs = walName.toString().split("\\" + WAL_FILE_NAME_DELIMITER);
return Long.parseLong(walPathStrs[walPathStrs.length - (isMetaFile(walName) ? 2:1)]);
}
/**
* Pattern used to validate a WAL file name
* see {@link #validateWALFilename(String)} for description.
*/
private static final Pattern pattern = Pattern.compile(".*\\.\\d*("+META_WAL_PROVIDER_ID+")*");
/**
* A WAL file name is of the format:
* &lt;wal-name&gt;{@link #WAL_FILE_NAME_DELIMITER}&lt;file-creation-timestamp&gt;[.meta].
*
* provider-name is usually made up of a server-name and a provider-id
*
* @param filename name of the file to validate
* @return <tt>true</tt> if the filename matches an WAL, <tt>false</tt>
* otherwise
*/
public static boolean validateWALFilename(String filename) {
return pattern.matcher(filename).matches();
}
/**
* Construct the directory name for all WALs on a given server.
*
* @param serverName
* Server name formatted as described in {@link ServerName}
* @return the relative WAL directory name, e.g.
* <code>.logs/1.example.org,60030,12345</code> if
* <code>serverName</code> passed is
* <code>1.example.org,60030,12345</code>
*/
public static String getWALDirectoryName(final String serverName) {
StringBuilder dirName = new StringBuilder(HConstants.HREGION_LOGDIR_NAME);
dirName.append("/");
dirName.append(serverName);
return dirName.toString();
}
/**
* Pulls a ServerName out of a Path generated according to our layout rules.
*
* In the below layouts, this method ignores the format of the logfile component.
*
* Current format:
*
* [base directory for hbase]/hbase/.logs/ServerName/logfile
* or
* [base directory for hbase]/hbase/.logs/ServerName-splitting/logfile
*
* Expected to work for individual log files and server-specific directories.
*
* @return null if it's not a log file. Returns the ServerName of the region
* server that created this log file otherwise.
*/
public static ServerName getServerNameFromWALDirectoryName(Configuration conf, String path)
throws IOException {
if (path == null
|| path.length() <= HConstants.HREGION_LOGDIR_NAME.length()) {
return null;
}
if (conf == null) {
throw new IllegalArgumentException("parameter conf must be set");
}
final String rootDir = conf.get(HConstants.HBASE_DIR);
if (rootDir == null || rootDir.isEmpty()) {
throw new IllegalArgumentException(HConstants.HBASE_DIR
+ " key not found in conf.");
}
final StringBuilder startPathSB = new StringBuilder(rootDir);
if (!rootDir.endsWith("/"))
startPathSB.append('/');
startPathSB.append(HConstants.HREGION_LOGDIR_NAME);
if (!HConstants.HREGION_LOGDIR_NAME.endsWith("/"))
startPathSB.append('/');
final String startPath = startPathSB.toString();
String fullPath;
try {
fullPath = FileSystem.get(conf).makeQualified(new Path(path)).toString();
} catch (IllegalArgumentException e) {
LOG.info("Call to makeQualified failed on " + path + " " + e.getMessage());
return null;
}
if (!fullPath.startsWith(startPath)) {
return null;
}
final String serverNameAndFile = fullPath.substring(startPath.length());
if (serverNameAndFile.indexOf('/') < "a,0,0".length()) {
// Either it's a file (not a directory) or it's not a ServerName format
return null;
}
Path p = new Path(path);
return getServerNameFromWALDirectoryName(p);
}
/**
* This function returns region server name from a log file name which is in one of the following
* formats:
* <ul>
* <li>hdfs://&lt;name node&gt;/hbase/.logs/&lt;server name&gt;-splitting/...</li>
* <li>hdfs://&lt;name node&gt;/hbase/.logs/&lt;server name&gt;/...</li>
* </ul>
* @param logFile
* @return null if the passed in logFile isn't a valid WAL file path
*/
public static ServerName getServerNameFromWALDirectoryName(Path logFile) {
String logDirName = logFile.getParent().getName();
// We were passed the directory and not a file in it.
if (logDirName.equals(HConstants.HREGION_LOGDIR_NAME)) {
logDirName = logFile.getName();
}
ServerName serverName = null;
if (logDirName.endsWith(SPLITTING_EXT)) {
logDirName = logDirName.substring(0, logDirName.length() - SPLITTING_EXT.length());
}
try {
serverName = ServerName.parseServerName(logDirName);
} catch (IllegalArgumentException ex) {
serverName = null;
LOG.warn("Cannot parse a server name from path=" + logFile + "; " + ex.getMessage());
}
if (serverName != null && serverName.getStartcode() < 0) {
LOG.warn("Invalid log file path=" + logFile);
serverName = null;
}
return serverName;
}
public static boolean isMetaFile(Path p) {
return isMetaFile(p.getName());
}
public static boolean isMetaFile(String p) {
if (p != null && p.endsWith(META_WAL_PROVIDER_ID)) {
return true;
}
return false;
}
/**
* public because of FSHLog. Should be package-private
*/
public static Writer createWriter(final Configuration conf, final FileSystem fs, final Path path,
final boolean overwritable)
throws IOException {
final boolean overwritable) throws IOException {
// Configuration already does caching for the Class lookup.
Class<? extends Writer> logWriterClass = conf.getClass("hbase.regionserver.hlog.writer.impl",
ProtobufLogWriter.class, Writer.class);
@ -380,15 +65,14 @@ public class DefaultWALProvider implements WALProvider {
}
}
/**
* Get prefix of the log from its name, assuming WAL name in format of
* log_prefix.filenumber.log_suffix @see {@link FSHLog#getCurrentFileName()}
* @param name Name of the WAL to parse
* @return prefix of the log
*/
public static String getWALPrefixFromWALName(String name) {
int endIndex = name.replaceAll(META_WAL_PROVIDER_ID, "").lastIndexOf(".");
return name.substring(0, endIndex);
@Override
protected FSHLog createWAL() throws IOException {
return new FSHLog(FileSystem.get(conf), FSUtils.getRootDir(conf),
getWALDirectoryName(factory.factoryId), HConstants.HREGION_OLDLOGDIR_NAME, conf, listeners,
true, logPrefix, META_WAL_PROVIDER_ID.equals(providerId) ? META_WAL_PROVIDER_ID : null);
}
@Override
protected void doInit(Configuration conf) throws IOException {
}
}

View File

@ -215,6 +215,10 @@ class DisabledWALProvider implements WALProvider {
public String toString() {
return "WAL disabled.";
}
@Override
public void logRollerExited() {
}
}
@Override

View File

@ -198,6 +198,13 @@ public interface WAL {
@Override
String toString();
/**
* In some WAL implementation, we will write WAL entries to new file if sync failed, which means,
* the fail recovery is depended on log roller. So here we tell the WAL that log roller has
* already been exited so the WAL cloud give up recovery.
*/
void logRollerExited();
/**
* When outside clients need to consume persisted WALs, they rely on a provided
* Reader.
@ -268,7 +275,5 @@ public interface WAL {
public String toString() {
return this.key + "=" + this.edit;
}
}
}

View File

@ -76,7 +76,8 @@ public class WALFactory {
static enum Providers {
defaultProvider(DefaultWALProvider.class),
filesystem(DefaultWALProvider.class),
multiwal(RegionGroupingProvider.class);
multiwal(RegionGroupingProvider.class),
asyncfs(AsyncFSWALProvider.class);
Class<? extends WALProvider> clazz;
Providers(Class<? extends WALProvider> clazz) {
@ -350,6 +351,7 @@ public class WALFactory {
/**
* Create a writer for the WAL.
* <p>
* should be package-private. public only for tests and
* {@link org.apache.hadoop.hbase.regionserver.wal.Compressor}
* @return A WAL writer. Close when done with it.

View File

@ -20,11 +20,11 @@ package org.apache.hadoop.hbase.wal;
import java.io.Closeable;
import java.io.IOException;
import java.nio.channels.CompletionHandler;
import java.util.List;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
// imports for things that haven't moved from regionserver.wal yet.
import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
@ -80,6 +80,12 @@ public interface WALProvider {
long getLength() throws IOException;
}
interface AsyncWriter extends Closeable {
<A> void sync(CompletionHandler<Long, A> handler, A attachment);
void append(WAL.Entry entry);
long getLength();
}
/**
* Get number of the log files this provider is managing
*/

View File

@ -0,0 +1,332 @@
/**
*
* 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 static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.wal.DefaultWALProvider;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.wal.WALFactory;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TestName;
/**
* Test log deletion as logs are rolled.
*/
public abstract class AbstractTestLogRolling {
private static final Log LOG = LogFactory.getLog(AbstractTestLogRolling.class);
protected HRegionServer server;
protected String tableName;
protected byte[] value;
protected FileSystem fs;
protected MiniDFSCluster dfsCluster;
protected Admin admin;
protected MiniHBaseCluster cluster;
protected static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
@Rule public final TestName name = new TestName();
public AbstractTestLogRolling() {
this.server = null;
this.tableName = null;
String className = this.getClass().getName();
StringBuilder v = new StringBuilder(className);
while (v.length() < 1000) {
v.append(className);
}
this.value = Bytes.toBytes(v.toString());
}
// Need to override this setup so we can edit the config before it gets sent
// to the HDFS & HBase cluster startup.
@BeforeClass
public static void setUpBeforeClass() throws Exception {
/**** configuration for testLogRolling ****/
// Force a region split after every 768KB
TEST_UTIL.getConfiguration().setLong(HConstants.HREGION_MAX_FILESIZE, 768L * 1024L);
// 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.rpc.timeout", 10 * 1000);
// For less frequently updated regions flush after every 2 flushes
TEST_UTIL.getConfiguration().setInt("hbase.hregion.memstore.optionalflushcount", 2);
// We flush the cache after every 8192 bytes
TEST_UTIL.getConfiguration().setInt(
HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 8192);
// Increase the amount of time between client retries
TEST_UTIL.getConfiguration().setLong("hbase.client.pause", 10 * 1000);
// Reduce thread wake frequency so that other threads can get
// a chance to run.
TEST_UTIL.getConfiguration().setInt(HConstants.THREAD_WAKE_FREQUENCY, 2 * 1000);
}
@Before
public void setUp() throws Exception {
TEST_UTIL.startMiniCluster(1, 1, 2);
cluster = TEST_UTIL.getHBaseCluster();
dfsCluster = TEST_UTIL.getDFSCluster();
fs = TEST_UTIL.getTestFileSystem();
admin = TEST_UTIL.getHBaseAdmin();
// disable region rebalancing (interferes with log watching)
cluster.getMaster().balanceSwitch(false);
}
@After
public void tearDown() throws Exception {
TEST_UTIL.shutdownMiniCluster();
}
protected void startAndWriteData() throws IOException, InterruptedException {
// When the hbase:meta table can be opened, the region servers are running
TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME);
this.server = cluster.getRegionServerThreads().get(0).getRegionServer();
Table table = createTestTable(this.tableName);
server = TEST_UTIL.getRSForFirstRegionInTable(table.getName());
for (int i = 1; i <= 256; i++) { // 256 writes should cause 8 log rolls
doPut(table, i);
if (i % 32 == 0) {
// After every 32 writes sleep to let the log roller run
try {
Thread.sleep(2000);
} catch (InterruptedException e) {
// continue
}
}
}
}
/**
* Tests that log rolling doesn't hang when no data is written.
*/
@Test(timeout=120000)
public void testLogRollOnNothingWritten() throws Exception {
final Configuration conf = TEST_UTIL.getConfiguration();
final WALFactory wals = new WALFactory(conf, null,
ServerName.valueOf("test.com",8080, 1).toString());
final WAL newLog = wals.getWAL(new byte[]{}, null);
try {
// Now roll the log before we write anything.
newLog.rollWriter(true);
} finally {
wals.close();
}
}
/**
* Tests that logs are deleted
* @throws IOException
* @throws org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException
*/
@Test
public void testLogRolling() throws Exception {
this.tableName = getName();
// TODO: Why does this write data take for ever?
startAndWriteData();
HRegionInfo region =
server.getOnlineRegions(TableName.valueOf(tableName)).get(0).getRegionInfo();
final WAL log = server.getWAL(region);
LOG.info("after writing there are " + DefaultWALProvider.getNumRolledLogFiles(log) +
" log files");
// flush all regions
for (Region r: server.getOnlineRegionsLocalContext()) {
r.flush(true);
}
// Now roll the log
log.rollWriter();
int count = DefaultWALProvider.getNumRolledLogFiles(log);
LOG.info("after flushing all regions and rolling logs there are " + count + " log files");
assertTrue(("actual count: " + count), count <= 2);
}
protected String getName() {
return "TestLogRolling-" + name.getMethodName();
}
void writeData(Table table, int rownum) throws IOException {
doPut(table, rownum);
// sleep to let the log roller run (if it needs to)
try {
Thread.sleep(2000);
} catch (InterruptedException e) {
// continue
}
}
void validateData(Table table, int rownum) throws IOException {
String row = "row" + String.format("%1$04d", rownum);
Get get = new Get(Bytes.toBytes(row));
get.addFamily(HConstants.CATALOG_FAMILY);
Result result = table.get(get);
assertTrue(result.size() == 1);
assertTrue(Bytes.equals(value,
result.getValue(HConstants.CATALOG_FAMILY, null)));
LOG.info("Validated row " + row);
}
void batchWriteAndWait(Table table, final FSHLog log, int start, boolean expect, int timeout)
throws IOException {
for (int i = 0; i < 10; i++) {
Put put = new Put(Bytes.toBytes("row"
+ String.format("%1$04d", (start + i))));
put.addColumn(HConstants.CATALOG_FAMILY, null, value);
table.put(put);
}
Put tmpPut = new Put(Bytes.toBytes("tmprow"));
tmpPut.addColumn(HConstants.CATALOG_FAMILY, null, value);
long startTime = System.currentTimeMillis();
long remaining = timeout;
while (remaining > 0) {
if (log.isLowReplicationRollEnabled() == expect) {
break;
} else {
// Trigger calling FSHlog#checkLowReplication()
table.put(tmpPut);
try {
Thread.sleep(200);
} catch (InterruptedException e) {
// continue
}
remaining = timeout - (System.currentTimeMillis() - startTime);
}
}
}
/**
* Tests that logs are deleted when some region has a compaction
* record in WAL and no other records. See HBASE-8597.
*/
@Test
public void testCompactionRecordDoesntBlockRolling() throws Exception {
Table table = null;
// When the hbase:meta table can be opened, the region servers are running
Table t = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME);
try {
table = createTestTable(getName());
server = TEST_UTIL.getRSForFirstRegionInTable(table.getName());
Region region = server.getOnlineRegions(table.getName()).get(0);
final WAL log = server.getWAL(region.getRegionInfo());
Store s = region.getStore(HConstants.CATALOG_FAMILY);
//have to flush namespace to ensure it doesn't affect wall tests
admin.flush(TableName.NAMESPACE_TABLE_NAME);
// Put some stuff into table, to make sure we have some files to compact.
for (int i = 1; i <= 2; ++i) {
doPut(table, i);
admin.flush(table.getName());
}
doPut(table, 3); // don't flush yet, or compaction might trigger before we roll WAL
assertEquals("Should have no WAL after initial writes", 0,
DefaultWALProvider.getNumRolledLogFiles(log));
assertEquals(2, s.getStorefilesCount());
// Roll the log and compact table, to have compaction record in the 2nd WAL.
log.rollWriter();
assertEquals("Should have WAL; one table is not flushed", 1,
DefaultWALProvider.getNumRolledLogFiles(log));
admin.flush(table.getName());
region.compact(false);
// Wait for compaction in case if flush triggered it before us.
Assert.assertNotNull(s);
for (int waitTime = 3000; s.getStorefilesCount() > 1 && waitTime > 0; waitTime -= 200) {
Threads.sleepWithoutInterrupt(200);
}
assertEquals("Compaction didn't happen", 1, s.getStorefilesCount());
// Write some value to the table so the WAL cannot be deleted until table is flushed.
doPut(table, 0); // Now 2nd WAL will have both compaction and put record for table.
log.rollWriter(); // 1st WAL deleted, 2nd not deleted yet.
assertEquals("Should have WAL; one table is not flushed", 1,
DefaultWALProvider.getNumRolledLogFiles(log));
// Flush table to make latest WAL obsolete; write another record, and roll again.
admin.flush(table.getName());
doPut(table, 1);
log.rollWriter(); // Now 2nd WAL is deleted and 3rd is added.
assertEquals("Should have 1 WALs at the end", 1,
DefaultWALProvider.getNumRolledLogFiles(log));
} finally {
if (t != null) t.close();
if (table != null) table.close();
}
}
protected void doPut(Table table, int i) throws IOException {
Put put = new Put(Bytes.toBytes("row" + String.format("%1$04d", i)));
put.addColumn(HConstants.CATALOG_FAMILY, null, value);
table.put(put);
}
protected Table createTestTable(String tableName) throws IOException {
// Create the test table and open it
HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
admin.createTable(desc);
return TEST_UTIL.getConnection().getTable(desc.getTableName());
}
}

View File

@ -0,0 +1,209 @@
/**
*
* 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 static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import java.io.Closeable;
import java.io.IOException;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.coprocessor.SampleRegionWALObserver;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.wal.WALFactory;
import org.apache.hadoop.hbase.wal.WALKey;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TestName;
/**
* WAL tests that can be reused across providers.
*/
public abstract class AbstractTestProtobufLog<W extends Closeable> {
protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
protected FileSystem fs;
protected Path dir;
protected WALFactory wals;
@Rule
public final TestName currentTest = new TestName();
@Before
public void setUp() throws Exception {
fs = TEST_UTIL.getDFSCluster().getFileSystem();
dir = new Path(TEST_UTIL.createRootDir(), currentTest.getMethodName());
wals = new WALFactory(TEST_UTIL.getConfiguration(), null, currentTest.getMethodName());
}
@After
public void tearDown() throws Exception {
wals.close();
FileStatus[] entries = fs.listStatus(new Path("/"));
for (FileStatus dir : entries) {
fs.delete(dir.getPath(), true);
}
}
@BeforeClass
public static void setUpBeforeClass() throws Exception {
// Make block sizes small.
TEST_UTIL.getConfiguration().setInt("dfs.blocksize", 1024 * 1024);
// needed for testAppendClose()
TEST_UTIL.getConfiguration().setBoolean("dfs.support.broken.append", true);
TEST_UTIL.getConfiguration().setBoolean("dfs.support.append", true);
// quicker heartbeat interval for faster DN death notification
TEST_UTIL.getConfiguration().setInt("dfs.namenode.heartbeat.recheck-interval", 5000);
TEST_UTIL.getConfiguration().setInt("dfs.heartbeat.interval", 1);
TEST_UTIL.getConfiguration().setInt("dfs.client.socket-timeout", 5000);
// faster failover with cluster.shutdown();fs.close() idiom
TEST_UTIL.getConfiguration()
.setInt("hbase.ipc.client.connect.max.retries", 1);
TEST_UTIL.getConfiguration().setInt(
"dfs.client.block.recovery.retries", 1);
TEST_UTIL.getConfiguration().setInt(
"hbase.ipc.client.connection.maxidletime", 500);
TEST_UTIL.getConfiguration().set(CoprocessorHost.WAL_COPROCESSOR_CONF_KEY,
SampleRegionWALObserver.class.getName());
TEST_UTIL.startMiniDFSCluster(3);
}
@AfterClass
public static void tearDownAfterClass() throws Exception {
TEST_UTIL.shutdownMiniCluster();
}
/**
* Reads the WAL with and without WALTrailer.
* @throws IOException
*/
@Test
public void testWALTrailer() throws IOException {
// read With trailer.
doRead(true);
// read without trailer
doRead(false);
}
/**
* Appends entries in the WAL and reads it.
* @param withTrailer If 'withTrailer' is true, it calls a close on the WALwriter before reading
* so that a trailer is appended to the WAL. Otherwise, it starts reading after the sync
* call. This means that reader is not aware of the trailer. In this scenario, if the
* reader tries to read the trailer in its next() call, it returns false from
* ProtoBufLogReader.
* @throws IOException
*/
private void doRead(boolean withTrailer) throws IOException {
final int columnCount = 5;
final int recordCount = 5;
final TableName tableName =
TableName.valueOf("tablename");
final byte[] row = Bytes.toBytes("row");
long timestamp = System.currentTimeMillis();
Path path = new Path(dir, "tempwal");
// delete the log if already exists, for test only
fs.delete(path, true);
W writer = null;
ProtobufLogReader reader = null;
try {
HRegionInfo hri = new HRegionInfo(tableName,
HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
HTableDescriptor htd = new HTableDescriptor(tableName);
fs.mkdirs(dir);
// Write log in pb format.
writer = createWriter(path);
for (int i = 0; i < recordCount; ++i) {
WALKey key = new WALKey(
hri.getEncodedNameAsBytes(), tableName, i, timestamp, HConstants.DEFAULT_CLUSTER_ID);
WALEdit edit = new WALEdit();
for (int j = 0; j < columnCount; ++j) {
if (i == 0) {
htd.addFamily(new HColumnDescriptor("column" + j));
}
String value = i + "" + j;
edit.add(new KeyValue(row, row, row, timestamp, Bytes.toBytes(value)));
}
append(writer, new WAL.Entry(key, edit));
}
sync(writer);
if (withTrailer) writer.close();
// Now read the log using standard means.
reader = (ProtobufLogReader) wals.createReader(fs, path);
if (withTrailer) {
assertNotNull(reader.trailer);
} else {
assertNull(reader.trailer);
}
for (int i = 0; i < recordCount; ++i) {
WAL.Entry entry = reader.next();
assertNotNull(entry);
assertEquals(columnCount, entry.getEdit().size());
assertArrayEquals(hri.getEncodedNameAsBytes(), entry.getKey().getEncodedRegionName());
assertEquals(tableName, entry.getKey().getTablename());
int idx = 0;
for (Cell val : entry.getEdit().getCells()) {
assertTrue(Bytes.equals(row, 0, row.length, val.getRowArray(), val.getRowOffset(),
val.getRowLength()));
String value = i + "" + idx;
assertArrayEquals(Bytes.toBytes(value), CellUtil.cloneValue(val));
idx++;
}
}
WAL.Entry entry = reader.next();
assertNull(entry);
} finally {
if (writer != null) {
writer.close();
}
if (reader != null) {
reader.close();
}
}
}
protected abstract W createWriter(Path path) throws IOException;
protected abstract void append(W writer, WAL.Entry entry) throws IOException;
protected abstract void sync(W writer) throws IOException;
}

View File

@ -27,12 +27,15 @@ import java.util.TreeMap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.io.util.LRUDictionary;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.wal.DefaultWALProvider;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.SequenceFile.CompressionType;
@ -51,7 +54,7 @@ import org.apache.hadoop.io.compress.DefaultCodec;
* Hadoop serialization).
*/
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
public class SequenceFileLogWriter extends WriterBase {
public class SequenceFileLogWriter implements DefaultWALProvider.Writer {
private static final Log LOG = LogFactory.getLog(SequenceFileLogWriter.class);
// The sequence file we delegate to.
private SequenceFile.Writer writer;
@ -59,6 +62,8 @@ public class SequenceFileLogWriter extends WriterBase {
// in the SequenceFile.Writer 'writer' instance above.
private FSDataOutputStream writer_out;
private CompressionContext compressionContext;
// Legacy stuff from pre-PB WAL metadata.
private static final Text WAL_VERSION_KEY = new Text("version");
private static final Text WAL_COMPRESSION_TYPE_KEY = new Text("compression.type");
@ -88,10 +93,23 @@ public class SequenceFileLogWriter extends WriterBase {
return new Metadata(metaMap);
}
private boolean initializeCompressionContext(Configuration conf, Path path) throws IOException {
boolean doCompress = conf.getBoolean(HConstants.ENABLE_WAL_COMPRESSION, false);
if (doCompress) {
try {
this.compressionContext = new CompressionContext(LRUDictionary.class,
FSUtils.isRecoveredEdits(path), conf.getBoolean(
CompressionContext.ENABLE_WAL_TAGS_COMPRESSION, true));
} catch (Exception e) {
throw new IOException("Failed to initiate CompressionContext", e);
}
}
return doCompress;
}
@Override
public void init(FileSystem fs, Path path, Configuration conf, boolean overwritable)
throws IOException {
super.init(fs, path, conf, overwritable);
boolean compress = initializeCompressionContext(conf, path);
// Create a SF.Writer instance.
try {

View File

@ -0,0 +1,36 @@
/**
* 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.conf.Configuration;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.wal.WALFactory;
import org.junit.BeforeClass;
import org.junit.experimental.categories.Category;
@Category({ RegionServerTests.class, MediumTests.class })
public class TestAsyncLogRollPeriod extends TestLogRollPeriod {
@BeforeClass
public static void setUpBeforeClass() throws Exception {
Configuration conf = TestLogRollPeriod.TEST_UTIL.getConfiguration();
conf.set(WALFactory.WAL_PROVIDER, "asyncfs");
TestLogRollPeriod.setUpBeforeClass();
}
}

View File

@ -0,0 +1,65 @@
/**
* 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 static org.junit.Assert.assertEquals;
import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.VerySlowRegionServerTests;
import org.apache.hadoop.hbase.wal.AsyncFSWALProvider;
import org.apache.hadoop.hbase.wal.WALFactory;
import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category({ VerySlowRegionServerTests.class, LargeTests.class })
public class TestAsyncLogRolling extends AbstractTestLogRolling {
@BeforeClass
public static void setUpBeforeClass() throws Exception {
Configuration conf = TestAsyncLogRolling.TEST_UTIL.getConfiguration();
conf.setInt(AsyncFSWAL.ASYNC_WAL_CREATE_MAX_RETRIES, 100);
conf.set(WALFactory.WAL_PROVIDER, "asyncfs");
AbstractTestLogRolling.setUpBeforeClass();
}
@Test(timeout = 180000)
public void testLogRollOnDatanodeDeath() throws IOException, InterruptedException {
dfsCluster.startDataNodes(TEST_UTIL.getConfiguration(), 3, true, null, null);
tableName = getName();
Table table = createTestTable(tableName);
TEST_UTIL.waitUntilAllRegionsAssigned(table.getName());
doPut(table, 1);
server = TEST_UTIL.getRSForFirstRegionInTable(table.getName());
HRegionInfo hri = server.getOnlineRegions(table.getName()).get(0).getRegionInfo();
AsyncFSWAL wal = (AsyncFSWAL) server.getWAL(hri);
int numRolledLogFiles = AsyncFSWALProvider.getNumRolledLogFiles(wal);
DatanodeInfo[] dnInfos = wal.getPipeline();
DataNodeProperties dnProp = TEST_UTIL.getDFSCluster().stopDataNode(dnInfos[0].getName());
TEST_UTIL.getDFSCluster().restartDataNode(dnProp);
doPut(table, 2);
assertEquals(numRolledLogFiles + 1, AsyncFSWALProvider.getNumRolledLogFiles(wal));
}
}

View File

@ -0,0 +1,82 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver.wal;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.concurrent.ExecutionException;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.wal.AsyncFSWALProvider;
import org.apache.hadoop.hbase.wal.WAL.Entry;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.util.FanOutOneBlockAsyncDFSOutputFlushHandler;
import org.apache.hadoop.hbase.wal.WALProvider;
import org.apache.hadoop.hbase.wal.WALProvider.AsyncWriter;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.experimental.categories.Category;
import com.google.common.base.Throwables;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
@Category({ RegionServerTests.class, MediumTests.class })
public class TestAsyncProtobufLog extends AbstractTestProtobufLog<WALProvider.AsyncWriter> {
private static EventLoopGroup EVENT_LOOP_GROUP;
@BeforeClass
public static void setUpBeforeClass() throws Exception {
EVENT_LOOP_GROUP = new NioEventLoopGroup();
AbstractTestProtobufLog.setUpBeforeClass();
}
@AfterClass
public static void tearDownAfterClass() throws Exception {
AbstractTestProtobufLog.tearDownAfterClass();
EVENT_LOOP_GROUP.shutdownGracefully().syncUninterruptibly();
}
@Override
protected AsyncWriter createWriter(Path path) throws IOException {
return AsyncFSWALProvider.createAsyncWriter(TEST_UTIL.getConfiguration(), fs, path, false,
EVENT_LOOP_GROUP.next());
}
@Override
protected void append(AsyncWriter writer, Entry entry) throws IOException {
writer.append(entry);
}
@Override
protected void sync(AsyncWriter writer) throws IOException {
FanOutOneBlockAsyncDFSOutputFlushHandler handler = new FanOutOneBlockAsyncDFSOutputFlushHandler();
writer.sync(handler, null);
try {
handler.get();
} catch (InterruptedException e) {
throw new InterruptedIOException();
} catch (ExecutionException e) {
Throwables.propagateIfPossible(e.getCause(), IOException.class);
throw new IOException(e.getCause());
}
}
}

View File

@ -0,0 +1,36 @@
/**
* 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.conf.Configuration;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.wal.WALFactory;
import org.junit.BeforeClass;
import org.junit.experimental.categories.Category;
@Category({ RegionServerTests.class, MediumTests.class })
public class TestAsyncWALReplay extends TestWALReplay {
@BeforeClass
public static void setUpBeforeClass() throws Exception {
Configuration conf = TestWALReplay.TEST_UTIL.getConfiguration();
conf.set(WALFactory.WAL_PROVIDER, "asyncfs");
TestWALReplay.setUpBeforeClass();
}
}

View File

@ -0,0 +1,38 @@
/**
* 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.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.wal.WALFactory;
import org.junit.BeforeClass;
import org.junit.experimental.categories.Category;
@Category({ RegionServerTests.class, MediumTests.class })
public class TestAsyncWALReplayCompressed extends TestWALReplay {
@BeforeClass
public static void setUpBeforeClass() throws Exception {
Configuration conf = TestWALReplay.TEST_UTIL.getConfiguration();
conf.set(WALFactory.WAL_PROVIDER, "asyncfs");
conf.setBoolean(HConstants.ENABLE_WAL_COMPRESSION, true);
TestWALReplay.setUpBeforeClass();
}
}

View File

@ -22,6 +22,7 @@ import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
import java.io.IOException;
import java.util.Arrays;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
@ -30,28 +31,35 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.wal.DefaultWALProvider;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.wal.WALFactory;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import org.junit.runners.Parameterized.Parameter;
import org.junit.runners.Parameterized.Parameters;
/**
* Tests for WAL write durability
*/
@RunWith(Parameterized.class)
@Category({RegionServerTests.class, MediumTests.class})
public class TestDurability {
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
@ -64,6 +72,13 @@ public class TestDurability {
private static byte[] ROW = Bytes.toBytes("row");
private static byte[] COL = Bytes.toBytes("col");
@Parameter
public String walProvider;
@Parameters(name = "{index}: provider={0}")
public static Iterable<Object[]> data() {
return Arrays.asList(new Object[] { "defaultProvider" }, new Object[] { "asyncfs" });
}
@BeforeClass
public static void setUpBeforeClass() throws Exception {
@ -81,6 +96,16 @@ public class TestDurability {
TEST_UTIL.shutdownMiniCluster();
}
@Before
public void setUp() {
CONF.set(WALFactory.WAL_PROVIDER, walProvider);
}
@After
public void tearDown() throws IOException {
FS.delete(DIR, true);
}
@Test
public void testDurability() throws Exception {
final WALFactory wals = new WALFactory(CONF, null, "TestDurability");

View File

@ -65,11 +65,11 @@ import org.junit.experimental.categories.Category;
*/
@Category({RegionServerTests.class, MediumTests.class})
public class TestLogRollAbort {
private static final Log LOG = LogFactory.getLog(TestLogRolling.class);
private static final Log LOG = LogFactory.getLog(AbstractTestLogRolling.class);
private static MiniDFSCluster dfsCluster;
private static Admin admin;
private static MiniHBaseCluster cluster;
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
/* For the split-then-roll test */
private static final Path HBASEDIR = new Path("/hbase");
@ -212,7 +212,7 @@ public class TestLogRollAbort {
}
// Send the data to HDFS datanodes and close the HDFS writer
log.sync();
((FSHLog) log).replaceWriter(((FSHLog)log).getOldPath(), null, null, null);
((AbstractFSWAL<?>) log).replaceWriter(((FSHLog)log).getOldPath(), null, null);
/* code taken from MasterFileSystem.getLogDirs(), which is called from MasterFileSystem.splitLog()
* handles RS shutdowns (as observed by the splitting process)

View File

@ -19,21 +19,21 @@ package org.apache.hadoop.hbase.regionserver.wal;
import static org.junit.Assert.assertFalse;
import java.util.List;
import java.util.ArrayList;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.wal.WAL;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
@ -44,9 +44,9 @@ import org.junit.experimental.categories.Category;
*/
@Category({RegionServerTests.class, MediumTests.class})
public class TestLogRollPeriod {
private static final Log LOG = LogFactory.getLog(TestLogRolling.class);
private static final Log LOG = LogFactory.getLog(AbstractTestLogRolling.class);
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
private final static long LOG_ROLL_PERIOD = 4000;

View File

@ -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
@ -24,7 +23,6 @@ import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import java.io.EOFException;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
@ -33,111 +31,45 @@ import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.fs.HFileSystem;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.VerySlowRegionServerTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.JVMClusterUtil;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.wal.DefaultWALProvider;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.wal.WALFactory;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
/**
* Test log deletion as logs are rolled.
*/
@Category({VerySlowRegionServerTests.class, LargeTests.class})
public class TestLogRolling {
@Category({ VerySlowRegionServerTests.class, LargeTests.class })
public class TestLogRolling extends AbstractTestLogRolling {
private static final Log LOG = LogFactory.getLog(TestLogRolling.class);
private HRegionServer server;
private String tableName;
private byte[] value;
private FileSystem fs;
private MiniDFSCluster dfsCluster;
private Admin admin;
private MiniHBaseCluster cluster;
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
@Rule public final TestName name = new TestName();
public TestLogRolling() {
this.server = null;
this.tableName = null;
String className = this.getClass().getName();
StringBuilder v = new StringBuilder(className);
while (v.length() < 1000) {
v.append(className);
}
this.value = Bytes.toBytes(v.toString());
}
// Need to override this setup so we can edit the config before it gets sent
// to the HDFS & HBase cluster startup.
@BeforeClass
public static void setUpBeforeClass() throws Exception {
// TODO: testLogRollOnDatanodeDeath fails if short circuit reads are on under the hadoop2
// profile. See HBASE-9337 for related issues.
System.setProperty("hbase.tests.use.shortcircuit.reads", "false");
/**** configuration for testLogRolling ****/
// Force a region split after every 768KB
TEST_UTIL.getConfiguration().setLong(HConstants.HREGION_MAX_FILESIZE, 768L * 1024L);
// 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.rpc.timeout", 10 * 1000);
// For less frequently updated regions flush after every 2 flushes
TEST_UTIL.getConfiguration().setInt("hbase.hregion.memstore.optionalflushcount", 2);
// We flush the cache after every 8192 bytes
TEST_UTIL.getConfiguration().setInt(
HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 8192);
// Increase the amount of time between client retries
TEST_UTIL.getConfiguration().setLong("hbase.client.pause", 10 * 1000);
// Reduce thread wake frequency so that other threads can get
// a chance to run.
TEST_UTIL.getConfiguration().setInt(HConstants.THREAD_WAKE_FREQUENCY, 2 * 1000);
/**** configuration for testLogRollOnDatanodeDeath ****/
// make sure log.hflush() calls syncFs() to open a pipeline
TEST_UTIL.getConfiguration().setBoolean("dfs.support.append", true);
@ -150,158 +82,19 @@ public class TestLogRolling {
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
public void setUp() throws Exception {
TEST_UTIL.startMiniCluster(1, 1, 2);
cluster = TEST_UTIL.getHBaseCluster();
dfsCluster = TEST_UTIL.getDFSCluster();
fs = TEST_UTIL.getTestFileSystem();
admin = TEST_UTIL.getHBaseAdmin();
// disable region rebalancing (interferes with log watching)
cluster.getMaster().balanceSwitch(false);
}
@After
public void tearDown() throws Exception {
TEST_UTIL.shutdownMiniCluster();
}
private void startAndWriteData() throws IOException, InterruptedException {
// When the hbase:meta table can be opened, the region servers are running
TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME);
this.server = cluster.getRegionServerThreads().get(0).getRegionServer();
Table table = createTestTable(this.tableName);
server = TEST_UTIL.getRSForFirstRegionInTable(table.getName());
for (int i = 1; i <= 256; i++) { // 256 writes should cause 8 log rolls
doPut(table, i);
if (i % 32 == 0) {
// After every 32 writes sleep to let the log roller run
try {
Thread.sleep(2000);
} catch (InterruptedException e) {
// continue
}
}
}
AbstractTestLogRolling.setUpBeforeClass();
}
/**
* Tests that log rolling doesn't hang when no data is written.
*/
@Test(timeout=120000)
public void testLogRollOnNothingWritten() throws Exception {
final Configuration conf = TEST_UTIL.getConfiguration();
final WALFactory wals = new WALFactory(conf, null,
ServerName.valueOf("test.com",8080, 1).toString());
final WAL newLog = wals.getWAL(new byte[]{}, null);
try {
// Now roll the log before we write anything.
newLog.rollWriter(true);
} finally {
wals.close();
}
}
/**
* Tests that logs are deleted
* @throws IOException
* @throws org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException
*/
@Test
public void testLogRolling() throws Exception {
this.tableName = getName();
// TODO: Why does this write data take for ever?
startAndWriteData();
HRegionInfo region =
server.getOnlineRegions(TableName.valueOf(tableName)).get(0).getRegionInfo();
final WAL log = server.getWAL(region);
LOG.info("after writing there are " + DefaultWALProvider.getNumRolledLogFiles(log) +
" log files");
// flush all regions
for (Region r: server.getOnlineRegionsLocalContext()) {
r.flush(true);
}
// Now roll the log
log.rollWriter();
int count = DefaultWALProvider.getNumRolledLogFiles(log);
LOG.info("after flushing all regions and rolling logs there are " + count + " log files");
assertTrue(("actual count: " + count), count <= 2);
}
private String getName() {
return "TestLogRolling-" + name.getMethodName();
}
void writeData(Table table, int rownum) throws IOException {
doPut(table, rownum);
// sleep to let the log roller run (if it needs to)
try {
Thread.sleep(2000);
} catch (InterruptedException e) {
// continue
}
}
void validateData(Table table, int rownum) throws IOException {
String row = "row" + String.format("%1$04d", rownum);
Get get = new Get(Bytes.toBytes(row));
get.addFamily(HConstants.CATALOG_FAMILY);
Result result = table.get(get);
assertTrue(result.size() == 1);
assertTrue(Bytes.equals(value,
result.getValue(HConstants.CATALOG_FAMILY, null)));
LOG.info("Validated row " + row);
}
void batchWriteAndWait(Table table, final FSHLog log, int start, boolean expect, int timeout)
throws IOException {
for (int i = 0; i < 10; i++) {
Put put = new Put(Bytes.toBytes("row"
+ String.format("%1$04d", (start + i))));
put.addColumn(HConstants.CATALOG_FAMILY, null, value);
table.put(put);
}
Put tmpPut = new Put(Bytes.toBytes("tmprow"));
tmpPut.addColumn(HConstants.CATALOG_FAMILY, null, value);
long startTime = System.currentTimeMillis();
long remaining = timeout;
while (remaining > 0) {
if (log.isLowReplicationRollEnabled() == expect) {
break;
} else {
// Trigger calling FSHlog#checkLowReplication()
table.put(tmpPut);
try {
Thread.sleep(200);
} catch (InterruptedException e) {
// continue
}
remaining = timeout - (System.currentTimeMillis() - startTime);
}
}
}
/**
* Tests that logs are rolled upon detecting datanode death
* Requires an HDFS jar with HDFS-826 & syncFs() support (HDFS-200)
* Tests that logs are rolled upon detecting datanode death Requires an HDFS jar with HDFS-826 &
* syncFs() support (HDFS-200)
*/
@Test
public void testLogRollOnDatanodeDeath() throws Exception {
TEST_UTIL.ensureSomeRegionServersAvailable(2);
assertTrue("This test requires WAL file replication set to 2.",
fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()) == 2);
LOG.info("Replication=" +
fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()));
LOG.info("Replication=" + fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()));
this.server = cluster.getRegionServer(0);
@ -328,8 +121,8 @@ public class TestLogRolling {
});
// don't run this test without append support (HDFS-200 & HDFS-142)
assertTrue("Need append support for this test", FSUtils
.isAppendSupported(TEST_UTIL.getConfiguration()));
assertTrue("Need append support for this test",
FSUtils.isAppendSupported(TEST_UTIL.getConfiguration()));
// add up the datanode count, to ensure proper replication when we kill 1
// This function is synchronous; when it returns, the dfs cluster is active
@ -337,20 +130,19 @@ public class TestLogRolling {
// when we stop/start a namenode later, as mentioned in HBASE-5163
List<DataNode> existingNodes = dfsCluster.getDataNodes();
int numDataNodes = 3;
dfsCluster.startDataNodes(TEST_UTIL.getConfiguration(), numDataNodes, true,
null, null);
dfsCluster.startDataNodes(TEST_UTIL.getConfiguration(), numDataNodes, true, null, null);
List<DataNode> allNodes = dfsCluster.getDataNodes();
for (int i = allNodes.size()-1; i >= 0; i--) {
for (int i = allNodes.size() - 1; i >= 0; i--) {
if (existingNodes.contains(allNodes.get(i))) {
dfsCluster.stopDataNode( i );
dfsCluster.stopDataNode(i);
}
}
assertTrue("DataNodes " + dfsCluster.getDataNodes().size() +
" default replication " +
fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()),
dfsCluster.getDataNodes().size() >=
fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()) + 1);
assertTrue(
"DataNodes " + dfsCluster.getDataNodes().size() + " default replication "
+ fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()),
dfsCluster.getDataNodes()
.size() >= fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()) + 1);
writeData(table, 2);
@ -362,9 +154,8 @@ public class TestLogRolling {
assertTrue("The log shouldn't have rolled yet",
oldFilenum == DefaultWALProvider.extractFileNumFromWAL(log));
final DatanodeInfo[] pipeline = log.getPipeLine();
assertTrue(pipeline.length ==
fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()));
final DatanodeInfo[] pipeline = log.getPipeline();
assertTrue(pipeline.length == fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()));
// kill a datanode in the pipeline to force a log roll on the next sync()
// This function is synchronous, when it returns the node is killed.
@ -390,26 +181,23 @@ public class TestLogRolling {
batchWriteAndWait(table, log, 3, false, 14000);
int replication = log.getLogReplication();
assertTrue("LowReplication Roller should've been disabled, current replication="
+ replication, !log.isLowReplicationRollEnabled());
assertTrue("LowReplication Roller should've been disabled, current replication=" + replication,
!log.isLowReplicationRollEnabled());
dfsCluster
.startDataNodes(TEST_UTIL.getConfiguration(), 1, true, null, null);
dfsCluster.startDataNodes(TEST_UTIL.getConfiguration(), 1, true, null, null);
// Force roll writer. The new log file will have the default replications,
// and the LowReplication Roller will be enabled.
log.rollWriter(true);
batchWriteAndWait(table, log, 13, true, 10000);
replication = log.getLogReplication();
assertTrue("New log file should have the default replication instead of " +
replication,
assertTrue("New log file should have the default replication instead of " + replication,
replication == fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()));
assertTrue("LowReplication Roller should've been enabled", log.isLowReplicationRollEnabled());
}
/**
* Test that WAL is rolled when all data nodes in the pipeline have been
* restarted.
* Test that WAL is rolled when all data nodes in the pipeline have been restarted.
* @throws Exception
*/
@Test
@ -417,8 +205,7 @@ public class TestLogRolling {
LOG.info("Starting testLogRollOnPipelineRestart");
assertTrue("This test requires WAL file replication.",
fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()) > 1);
LOG.info("Replication=" +
fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()));
LOG.info("Replication=" + fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()));
// When the hbase:meta table can be opened, the region servers are running
Table t = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME);
try {
@ -442,9 +229,10 @@ public class TestLogRolling {
@Override
public void preLogRoll(Path oldFile, Path newFile) {
LOG.debug("preLogRoll: oldFile="+oldFile+" newFile="+newFile);
LOG.debug("preLogRoll: oldFile=" + oldFile + " newFile=" + newFile);
preLogRolledCalled.add(new Integer(1));
}
@Override
public void postLogRoll(Path oldFile, Path newFile) {
paths.add(newFile);
@ -452,8 +240,8 @@ public class TestLogRolling {
});
// don't run this test without append support (HDFS-200 & HDFS-142)
assertTrue("Need append support for this test", FSUtils
.isAppendSupported(TEST_UTIL.getConfiguration()));
assertTrue("Need append support for this test",
FSUtils.isAppendSupported(TEST_UTIL.getConfiguration()));
writeData(table, 1002);
@ -463,8 +251,8 @@ public class TestLogRolling {
assertTrue("Log should have a timestamp older than now",
curTime > oldFilenum && oldFilenum != -1);
assertTrue("The log shouldn't have rolled yet", oldFilenum ==
DefaultWALProvider.extractFileNumFromWAL(log));
assertTrue("The log shouldn't have rolled yet",
oldFilenum == DefaultWALProvider.extractFileNumFromWAL(log));
// roll all datanodes in the pipeline
dfsCluster.restartDataNodes();
@ -503,23 +291,23 @@ public class TestLogRolling {
FSUtils fsUtils = FSUtils.getInstance(fs, TEST_UTIL.getConfiguration());
for (Path p : paths) {
LOG.debug("recovering lease for " + p);
fsUtils.recoverFileLease(((HFileSystem)fs).getBackingFs(), p,
TEST_UTIL.getConfiguration(), null);
fsUtils.recoverFileLease(((HFileSystem) fs).getBackingFs(), p, TEST_UTIL.getConfiguration(),
null);
LOG.debug("Reading WAL "+FSUtils.getPath(p));
LOG.debug("Reading WAL " + FSUtils.getPath(p));
WAL.Reader reader = null;
try {
reader = WALFactory.createReader(fs, p, TEST_UTIL.getConfiguration());
WAL.Entry entry;
while ((entry = reader.next()) != null) {
LOG.debug("#"+entry.getKey().getLogSeqNum()+": "+entry.getEdit().getCells());
LOG.debug("#" + entry.getKey().getSequenceId() + ": " + entry.getEdit().getCells());
for (Cell cell : entry.getEdit().getCells()) {
loggedRows.add(Bytes.toStringBinary(cell.getRowArray(), cell.getRowOffset(),
cell.getRowLength()));
loggedRows.add(
Bytes.toStringBinary(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()));
}
}
} catch (EOFException e) {
LOG.debug("EOF reading file "+FSUtils.getPath(p));
LOG.debug("EOF reading file " + FSUtils.getPath(p));
} finally {
if (reader != null) reader.close();
}
@ -532,7 +320,7 @@ public class TestLogRolling {
assertTrue(loggedRows.contains("row1005"));
// flush all regions
for (Region r: server.getOnlineRegionsLocalContext()) {
for (Region r : server.getOnlineRegionsLocalContext()) {
try {
r.flush(true);
} catch (Exception e) {
@ -547,19 +335,19 @@ public class TestLogRolling {
ResultScanner scanner = table.getScanner(new Scan());
try {
for (int i=2; i<=5; i++) {
for (int i = 2; i <= 5; i++) {
Result r = scanner.next();
assertNotNull(r);
assertFalse(r.isEmpty());
assertEquals("row100"+i, Bytes.toString(r.getRow()));
assertEquals("row100" + i, Bytes.toString(r.getRow()));
}
} finally {
scanner.close();
}
// verify that no region servers aborted
for (JVMClusterUtil.RegionServerThread rsThread:
TEST_UTIL.getHBaseCluster().getRegionServerThreads()) {
for (JVMClusterUtil.RegionServerThread rsThread : TEST_UTIL.getHBaseCluster()
.getRegionServerThreads()) {
assertFalse(rsThread.getRegionServer().isAborted());
}
} finally {
@ -567,80 +355,4 @@ public class TestLogRolling {
}
}
/**
* Tests that logs are deleted when some region has a compaction
* record in WAL and no other records. See HBASE-8597.
*/
@Test
public void testCompactionRecordDoesntBlockRolling() throws Exception {
Table table = null;
// When the hbase:meta table can be opened, the region servers are running
Table t = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME);
try {
table = createTestTable(getName());
server = TEST_UTIL.getRSForFirstRegionInTable(table.getName());
Region region = server.getOnlineRegions(table.getName()).get(0);
final WAL log = server.getWAL(region.getRegionInfo());
Store s = region.getStore(HConstants.CATALOG_FAMILY);
//have to flush namespace to ensure it doesn't affect wall tests
admin.flush(TableName.NAMESPACE_TABLE_NAME);
// Put some stuff into table, to make sure we have some files to compact.
for (int i = 1; i <= 2; ++i) {
doPut(table, i);
admin.flush(table.getName());
}
doPut(table, 3); // don't flush yet, or compaction might trigger before we roll WAL
assertEquals("Should have no WAL after initial writes", 0,
DefaultWALProvider.getNumRolledLogFiles(log));
assertEquals(2, s.getStorefilesCount());
// Roll the log and compact table, to have compaction record in the 2nd WAL.
log.rollWriter();
assertEquals("Should have WAL; one table is not flushed", 1,
DefaultWALProvider.getNumRolledLogFiles(log));
admin.flush(table.getName());
region.compact(false);
// Wait for compaction in case if flush triggered it before us.
Assert.assertNotNull(s);
for (int waitTime = 3000; s.getStorefilesCount() > 1 && waitTime > 0; waitTime -= 200) {
Threads.sleepWithoutInterrupt(200);
}
assertEquals("Compaction didn't happen", 1, s.getStorefilesCount());
// Write some value to the table so the WAL cannot be deleted until table is flushed.
doPut(table, 0); // Now 2nd WAL will have both compaction and put record for table.
log.rollWriter(); // 1st WAL deleted, 2nd not deleted yet.
assertEquals("Should have WAL; one table is not flushed", 1,
DefaultWALProvider.getNumRolledLogFiles(log));
// Flush table to make latest WAL obsolete; write another record, and roll again.
admin.flush(table.getName());
doPut(table, 1);
log.rollWriter(); // Now 2nd WAL is deleted and 3rd is added.
assertEquals("Should have 1 WALs at the end", 1,
DefaultWALProvider.getNumRolledLogFiles(log));
} finally {
if (t != null) t.close();
if (table != null) table.close();
}
}
private void doPut(Table table, int i) throws IOException {
Put put = new Put(Bytes.toBytes("row" + String.format("%1$04d", i)));
put.addColumn(HConstants.CATALOG_FAMILY, null, value);
table.put(put);
}
private Table createTestTable(String tableName) throws IOException {
// Create the test table and open it
HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
admin.createTable(desc);
return TEST_UTIL.getConnection().getTable(desc.getTableName());
}
}

View File

@ -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
@ -18,190 +17,32 @@
*/
package org.apache.hadoop.hbase.regionserver.wal;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.coprocessor.SampleRegionWALObserver;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.wal.WALFactory;
import org.apache.hadoop.hbase.wal.WALKey;
import org.apache.hadoop.hbase.wal.WAL.Entry;
import org.apache.hadoop.hbase.wal.DefaultWALProvider;
import org.apache.hadoop.hbase.wal.WALProvider;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Rule;
import org.junit.Test;
import org.apache.hadoop.hbase.wal.WALProvider.Writer;
import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
/**
* WAL tests that can be reused across providers.
*/
@Category({RegionServerTests.class, MediumTests.class})
public class TestProtobufLog {
protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
@Category({ RegionServerTests.class, MediumTests.class })
public class TestProtobufLog extends AbstractTestProtobufLog<WALProvider.Writer> {
protected FileSystem fs;
protected Path dir;
protected WALFactory wals;
@Rule
public final TestName currentTest = new TestName();
@Before
public void setUp() throws Exception {
fs = TEST_UTIL.getDFSCluster().getFileSystem();
dir = new Path(TEST_UTIL.createRootDir(), currentTest.getMethodName());
wals = new WALFactory(TEST_UTIL.getConfiguration(), null, currentTest.getMethodName());
@Override
protected Writer createWriter(Path path) throws IOException {
return DefaultWALProvider.createWriter(TEST_UTIL.getConfiguration(), fs, path, false);
}
@After
public void tearDown() throws Exception {
wals.close();
FileStatus[] entries = fs.listStatus(new Path("/"));
for (FileStatus dir : entries) {
fs.delete(dir.getPath(), true);
}
@Override
protected void append(Writer writer, Entry entry) throws IOException {
writer.append(entry);
}
@BeforeClass
public static void setUpBeforeClass() throws Exception {
// Make block sizes small.
TEST_UTIL.getConfiguration().setInt("dfs.blocksize", 1024 * 1024);
// needed for testAppendClose()
TEST_UTIL.getConfiguration().setBoolean("dfs.support.broken.append", true);
TEST_UTIL.getConfiguration().setBoolean("dfs.support.append", true);
// quicker heartbeat interval for faster DN death notification
TEST_UTIL.getConfiguration().setInt("dfs.namenode.heartbeat.recheck-interval", 5000);
TEST_UTIL.getConfiguration().setInt("dfs.heartbeat.interval", 1);
TEST_UTIL.getConfiguration().setInt("dfs.client.socket-timeout", 5000);
// faster failover with cluster.shutdown();fs.close() idiom
TEST_UTIL.getConfiguration()
.setInt("hbase.ipc.client.connect.max.retries", 1);
TEST_UTIL.getConfiguration().setInt(
"dfs.client.block.recovery.retries", 1);
TEST_UTIL.getConfiguration().setInt(
"hbase.ipc.client.connection.maxidletime", 500);
TEST_UTIL.getConfiguration().set(CoprocessorHost.WAL_COPROCESSOR_CONF_KEY,
SampleRegionWALObserver.class.getName());
TEST_UTIL.startMiniDFSCluster(3);
}
@AfterClass
public static void tearDownAfterClass() throws Exception {
TEST_UTIL.shutdownMiniCluster();
}
/**
* Reads the WAL with and without WALTrailer.
* @throws IOException
*/
@Test
public void testWALTrailer() throws IOException {
// read With trailer.
doRead(true);
// read without trailer
doRead(false);
}
/**
* Appends entries in the WAL and reads it.
* @param withTrailer If 'withTrailer' is true, it calls a close on the WALwriter before reading
* so that a trailer is appended to the WAL. Otherwise, it starts reading after the sync
* call. This means that reader is not aware of the trailer. In this scenario, if the
* reader tries to read the trailer in its next() call, it returns false from
* ProtoBufLogReader.
* @throws IOException
*/
private void doRead(boolean withTrailer) throws IOException {
final int columnCount = 5;
final int recordCount = 5;
final TableName tableName =
TableName.valueOf("tablename");
final byte[] row = Bytes.toBytes("row");
long timestamp = System.currentTimeMillis();
Path path = new Path(dir, "tempwal");
// delete the log if already exists, for test only
fs.delete(path, true);
WALProvider.Writer writer = null;
ProtobufLogReader reader = null;
try {
HRegionInfo hri = new HRegionInfo(tableName,
HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
HTableDescriptor htd = new HTableDescriptor(tableName);
fs.mkdirs(dir);
// Write log in pb format.
writer = wals.createWALWriter(fs, path);
for (int i = 0; i < recordCount; ++i) {
WALKey key = new WALKey(
hri.getEncodedNameAsBytes(), tableName, i, timestamp, HConstants.DEFAULT_CLUSTER_ID);
WALEdit edit = new WALEdit();
for (int j = 0; j < columnCount; ++j) {
if (i == 0) {
htd.addFamily(new HColumnDescriptor("column" + j));
}
String value = i + "" + j;
edit.add(new KeyValue(row, row, row, timestamp, Bytes.toBytes(value)));
}
writer.append(new WAL.Entry(key, edit));
}
@Override
protected void sync(Writer writer) throws IOException {
writer.sync();
if (withTrailer) writer.close();
// Now read the log using standard means.
reader = (ProtobufLogReader) wals.createReader(fs, path);
if (withTrailer) {
assertNotNull(reader.trailer);
} else {
assertNull(reader.trailer);
}
for (int i = 0; i < recordCount; ++i) {
WAL.Entry entry = reader.next();
assertNotNull(entry);
assertEquals(columnCount, entry.getEdit().size());
assertArrayEquals(hri.getEncodedNameAsBytes(), entry.getKey().getEncodedRegionName());
assertEquals(tableName, entry.getKey().getTablename());
int idx = 0;
for (Cell val : entry.getEdit().getCells()) {
assertTrue(Bytes.equals(row, 0, row.length, val.getRowArray(), val.getRowOffset(),
val.getRowLength()));
String value = i + "" + idx;
assertArrayEquals(Bytes.toBytes(value), CellUtil.cloneValue(val));
idx++;
}
}
WAL.Entry entry = reader.next();
assertNull(entry);
} finally {
if (writer != null) {
writer.close();
}
if (reader != null) {
reader.close();
}
}
}
}

View File

@ -36,5 +36,4 @@ public class TestWALReplayCompressed extends TestWALReplay {
Configuration conf = TestWALReplay.TEST_UTIL.getConfiguration();
conf.setBoolean(HConstants.ENABLE_WAL_COMPRESSION, true);
}
}