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."); 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 { class FSWALEntry extends Entry {
// The below data members are denoted 'transient' just to highlight these are not persisted; // 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. // 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 boolean inMemstore;
private final transient HRegionInfo hri; 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) { final HRegionInfo hri, final boolean inMemstore) {
super(key, edit); super(key, edit);
this.inMemstore = inMemstore; this.inMemstore = inMemstore;
this.hri = hri; this.hri = hri;
this.sequence = sequence; this.txid = txid;
if (inMemstore) { if (inMemstore) {
// construct familyNames here to reduce the work of log sinker. // construct familyNames here to reduce the work of log sinker.
ArrayList<Cell> cells = this.getEdit().getCells(); ArrayList<Cell> cells = this.getEdit().getCells();
@ -80,7 +83,7 @@ class FSWALEntry extends Entry {
} }
public String toString() { public String toString() {
return "sequence=" + this.sequence + ", " + super.toString(); return "sequence=" + this.txid + ", " + super.toString();
}; };
boolean isInMemstore() { 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() { long getTxid() {
return this.sequence; 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 * SIDE-EFFECT is our stamping the sequenceid into every Cell AND setting the sequenceid into the
* MVCC WriteEntry!!!! * MVCC WriteEntry!!!!
* @return The sequenceid we stamped on this edit. * @return The sequenceid we stamped on this edit.
* @throws IOException
*/ */
long stampRegionSequenceId() throws IOException { long stampRegionSequenceId() throws IOException {
if (stamped) {
return getKey().getSequenceId();
}
stamped = true;
long regionSequenceId = WALKey.NO_SEQUENCE_ID; long regionSequenceId = WALKey.NO_SEQUENCE_ID;
MultiVersionConcurrencyControl mvcc = getKey().getMvcc(); MultiVersionConcurrencyControl mvcc = getKey().getMvcc();
MultiVersionConcurrencyControl.WriteEntry we = null; MultiVersionConcurrencyControl.WriteEntry we = null;

View File

@ -19,108 +19,42 @@
package org.apache.hadoop.hbase.regionserver.wal; 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.IOException;
import java.io.OutputStream;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceAudience; 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.WALHeader;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALTrailer; 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; import org.apache.hadoop.hbase.wal.WAL.Entry;
/** /**
* Writer for protobuf-based WAL. * Writer for protobuf-based WAL.
*/ */
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG) @InterfaceAudience.Private
public class ProtobufLogWriter extends WriterBase { public class ProtobufLogWriter extends AbstractProtobufLogWriter
implements DefaultWALProvider.Writer {
private static final Log LOG = LogFactory.getLog(ProtobufLogWriter.class); private static final Log LOG = LogFactory.getLog(ProtobufLogWriter.class);
protected FSDataOutputStream output; 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 @Override
public void append(Entry entry) throws IOException { public void append(Entry entry) throws IOException {
entry.setCompressionContext(compressionContext); entry.setCompressionContext(compressionContext);
entry.getKey().getBuilder(compressor). entry.getKey().getBuilder(compressor).setFollowingKvCount(entry.getEdit().size()).build()
setFollowingKvCount(entry.getEdit().size()).build().writeDelimitedTo(output); .writeDelimitedTo(output);
for (Cell cell : entry.getEdit().getCells()) { for (Cell cell : entry.getEdit().getCells()) {
// cellEncoder must assume little about the stream, since we write PB and cells in turn. // cellEncoder must assume little about the stream, since we write PB and cells in turn.
cellEncoder.write(cell); cellEncoder.write(cell);
} }
length.set(output.getPos());
} }
@Override @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 @Override
public void sync() throws IOException { public void sync() throws IOException {
FSDataOutputStream fsdos = this.output; FSDataOutputStream fsdos = this.output;
@ -171,21 +79,35 @@ public class ProtobufLogWriter extends WriterBase {
fsdos.hflush(); 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() { public FSDataOutputStream getStream() {
return this.output; return this.output;
} }
void setWALTrailer(WALTrailer walTrailer) { @SuppressWarnings("deprecation")
this.trailer = walTrailer; @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; package org.apache.hadoop.hbase.regionserver.wal;
import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.htrace.Span; 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 * A Future on a filesystem sync call. It given to a client or 'Handler' for it to wait on till the
* the sync completes. * 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.
* <p> * <p>
* This is just a partial implementation of Future; we just implement get and * Handlers coming in call append, append, append, and then do a flush/sync of the edits they have
* failure. Unimplemented methods throw {@link UnsupportedOperationException}. * 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> * <p>
* There is not a one-to-one correlation between dfs sync invocations and * This is just a partial implementation of Future; we just implement get and failure.
* 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> * <p>
* SyncFutures are immutable but recycled. Call #reset(long, Span) before use even * There is not a one-to-one correlation between dfs sync invocations and instances of this class. A
* if it the first time, start the sync, then park the 'hitched' thread on a call to * single dfs sync call may complete and mark many SyncFutures as done; i.e. we batch up sync calls
* #get(). * 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 @InterfaceAudience.Private
class SyncFuture { class SyncFuture {
@ -54,17 +49,17 @@ class SyncFuture {
private static final long NOT_DONE = 0; 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 * The transaction id that was set in here when we were marked done. Should be equal or > txnId.
* or > ringBufferSequence. Put this data member into the NOT_DONE state while this * Put this data member into the NOT_DONE state while this class is in use. But for the first
* class is in use. But for the first position on construction, let it be -1 so we can * position on construction, let it be -1 so we can immediately call {@link #reset(long, Span)}
* immediately call {@link #reset(long, Span)} below and it will work. * below and it will work.
*/ */
private long doneSequence = -1; private long doneTxid = -1;
/** /**
* If error, the associated throwable. Set when the future is 'done'. * If error, the associated throwable. Set when the future is 'done'.
@ -79,80 +74,83 @@ class SyncFuture {
private Span span; private Span span;
/** /**
* Call this method to clear old usage and get it ready for new deploy. Call * Call this method to clear old usage and get it ready for new deploy. Call this method even if
* this method even if it is being used for the first time. * it is being used for the first time.
* * @param txnId the new transaction id
* @param sequence sequenceId from this Future's position in the RingBuffer
* @return this * @return this
*/ */
synchronized SyncFuture reset(final long sequence) { synchronized SyncFuture reset(final long txnId) {
return reset(sequence, null); return reset(txnId, null);
} }
/** /**
* Call this method to clear old usage and get it ready for new deploy. Call * Call this method to clear old usage and get it ready for new deploy. Call this method even if
* this method even if it is being used for the first time. * it is being used for the first time.
*
* @param sequence sequenceId from this Future's position in the RingBuffer * @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 * @param span curren span, detached from caller. Don't forget to attach it when resuming after a
* resuming after a call to {@link #get()}. * call to {@link #get()}.
* @return this * @return this
*/ */
synchronized SyncFuture reset(final long sequence, Span span) { synchronized SyncFuture reset(final long txnId, Span span) {
if (t != null && t != Thread.currentThread()) throw new IllegalStateException(); if (t != null && t != Thread.currentThread()) {
throw new IllegalStateException();
}
t = Thread.currentThread(); t = Thread.currentThread();
if (!isDone()) throw new IllegalStateException("" + sequence + " " + Thread.currentThread()); if (!isDone()) {
this.doneSequence = NOT_DONE; throw new IllegalStateException("" + txnId + " " + Thread.currentThread());
this.ringBufferSequence = sequence; }
this.doneTxid = NOT_DONE;
this.txid = txnId;
this.span = span; this.span = span;
return this; return this;
} }
@Override @Override
public synchronized String toString() { public synchronized String toString() {
return "done=" + isDone() + ", ringBufferSequence=" + this.ringBufferSequence; return "done=" + isDone() + ", txid=" + this.txid;
} }
synchronized long getRingBufferSequence() { synchronized long getTxid() {
return this.ringBufferSequence; return this.txid;
} }
/** /**
* Retrieve the {@code span} instance from this Future. EventHandler calls * Retrieve the {@code span} instance from this Future. EventHandler calls this method to continue
* this method to continue the span. Thread waiting on this Future musn't call * the span. Thread waiting on this Future musn't call this method until AFTER calling
* this method until AFTER calling {@link #get()} and the future has been * {@link #get()} and the future has been released back to the originating thread.
* released back to the originating thread.
*/ */
synchronized Span getSpan() { synchronized Span getSpan() {
return this.span; return this.span;
} }
/** /**
* Used to re-attach a {@code span} to the Future. Called by the EventHandler * Used to re-attach a {@code span} to the Future. Called by the EventHandler after a it has
* after a it has completed processing and detached the span from its scope. * completed processing and detached the span from its scope.
*/ */
synchronized void setSpan(Span span) { synchronized void setSpan(Span span) {
this.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). * @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. * @return True if we successfully marked this outstanding future as completed/done. Returns false
* Returns false if this future is already 'done' when this method called. * if this future is already 'done' when this method called.
*/ */
synchronized boolean done(final long sequence, final Throwable t) { synchronized boolean done(final long txid, final Throwable t) {
if (isDone()) return false; if (isDone()) {
return false;
}
this.throwable = t; this.throwable = t;
if (sequence < this.ringBufferSequence) { if (txid < this.txid) {
// Something badly wrong. // Something badly wrong.
if (throwable == null) { if (throwable == null) {
this.throwable = new IllegalStateException("sequence=" + sequence + this.throwable =
", ringBufferSequence=" + this.ringBufferSequence); new IllegalStateException("done txid=" + txid + ", my txid=" + this.txid);
} }
} }
// Mark done. // Mark done.
this.doneSequence = sequence; this.doneTxid = txid;
// Wake up waiting threads. // Wake up waiting threads.
notify(); notify();
return true; return true;
@ -166,21 +164,14 @@ class SyncFuture {
while (!isDone()) { while (!isDone()) {
wait(1000); wait(1000);
} }
if (this.throwable != null) throw new ExecutionException(this.throwable); if (this.throwable != null) {
return this.doneSequence; throw new ExecutionException(this.throwable);
} }
return this.doneTxid;
public Long get(long timeout, TimeUnit unit)
throws InterruptedException, ExecutionException {
throw new UnsupportedOperationException();
}
public boolean isCancelled() {
throw new UnsupportedOperationException();
} }
synchronized boolean isDone() { synchronized boolean isDone() {
return this.doneSequence != NOT_DONE; return this.doneTxid != NOT_DONE;
} }
synchronized boolean isThrowable() { 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.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException; import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
import org.apache.hadoop.io.EnumSetWritable; import org.apache.hadoop.io.EnumSetWritable;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.util.DataChecksum; import org.apache.hadoop.util.DataChecksum;
@ -417,7 +418,6 @@ public class FanOutOneBlockAsyncDFSOutputHelper {
public boolean progress() { public boolean progress() {
return DFS_CLIENT_ADAPTOR.isClientRunning(client); return DFS_CLIENT_ADAPTOR.isClientRunning(client);
} }
} }
static { static {
@ -579,6 +579,18 @@ public class FanOutOneBlockAsyncDFSOutputHelper {
return futureList; 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, private static FanOutOneBlockAsyncDFSOutput createOutput(DistributedFileSystem dfs, String src,
boolean overwrite, boolean createParent, short replication, long blockSize, boolean overwrite, boolean createParent, short replication, long blockSize,
EventLoop eventLoop) throws IOException { EventLoop eventLoop) throws IOException {
@ -587,11 +599,20 @@ public class FanOutOneBlockAsyncDFSOutputHelper {
DFSClient client = dfs.getClient(); DFSClient client = dfs.getClient();
String clientName = client.getClientName(); String clientName = client.getClientName();
ClientProtocol namenode = client.getNamenode(); 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, FsPermission.getFileDefault().applyUMask(FsPermission.getUMask(conf)), clientName,
new EnumSetWritable<CreateFlag>( new EnumSetWritable<CreateFlag>(
overwrite ? EnumSet.of(CREATE, OVERWRITE) : EnumSet.of(CREATE)), overwrite ? EnumSet.of(CREATE, OVERWRITE) : EnumSet.of(CREATE)),
createParent, replication, blockSize); createParent, replication, blockSize);
} catch (Exception e) {
if (e instanceof RemoteException) {
throw (RemoteException) e;
} else {
throw new NameNodeException(e);
}
}
beginFileLease(client, src, stat.getFileId()); beginFileLease(client, src, stat.getFileId());
boolean succ = false; boolean succ = false;
LocatedBlock locatedBlock = null; LocatedBlock locatedBlock = null;
@ -656,6 +677,13 @@ public class FanOutOneBlockAsyncDFSOutputHelper {
}.resolve(dfs, f); }.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, static void completeFile(DFSClient client, ClientProtocol namenode, String src, String clientName,
ExtendedBlock block, long fileId) { ExtendedBlock block, long fileId) {
for (int retry = 0;; retry++) { 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; package org.apache.hadoop.hbase.wal;
import java.io.IOException; 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.Log;
import org.apache.commons.logging.LogFactory; 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.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.classification.InterfaceStability;
// imports for things that haven't moved from regionserver.wal yet. // 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.FSHLog;
import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogWriter; 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. * A WAL provider that use {@link FSHLog}.
* 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.
*
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
@InterfaceStability.Evolving @InterfaceStability.Evolving
public class DefaultWALProvider implements WALProvider { public class DefaultWALProvider extends AbstractFSWALProvider<FSHLog> {
private static final Log LOG = LogFactory.getLog(DefaultWALProvider.class);
// Only public so classes back in regionserver.wal can access private static final Log LOG = LogFactory.getLog(DefaultWALProvider.class);
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;
}
// Only public so classes back in regionserver.wal can access // Only public so classes back in regionserver.wal can access
public interface Writer extends WALProvider.Writer { public interface Writer extends WALProvider.Writer {
void init(FileSystem fs, Path path, Configuration c, boolean overwritable) throws IOException; 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 because of FSHLog. Should be package-private
*/ */
public static Writer createWriter(final Configuration conf, final FileSystem fs, final Path path, public static Writer createWriter(final Configuration conf, final FileSystem fs, final Path path,
final boolean overwritable) final boolean overwritable) throws IOException {
throws IOException {
// Configuration already does caching for the Class lookup. // Configuration already does caching for the Class lookup.
Class<? extends Writer> logWriterClass = conf.getClass("hbase.regionserver.hlog.writer.impl", Class<? extends Writer> logWriterClass = conf.getClass("hbase.regionserver.hlog.writer.impl",
ProtobufLogWriter.class, Writer.class); ProtobufLogWriter.class, Writer.class);
@ -380,15 +65,14 @@ public class DefaultWALProvider implements WALProvider {
} }
} }
/** @Override
* Get prefix of the log from its name, assuming WAL name in format of protected FSHLog createWAL() throws IOException {
* log_prefix.filenumber.log_suffix @see {@link FSHLog#getCurrentFileName()} return new FSHLog(FileSystem.get(conf), FSUtils.getRootDir(conf),
* @param name Name of the WAL to parse getWALDirectoryName(factory.factoryId), HConstants.HREGION_OLDLOGDIR_NAME, conf, listeners,
* @return prefix of the log true, logPrefix, META_WAL_PROVIDER_ID.equals(providerId) ? META_WAL_PROVIDER_ID : null);
*/
public static String getWALPrefixFromWALName(String name) {
int endIndex = name.replaceAll(META_WAL_PROVIDER_ID, "").lastIndexOf(".");
return name.substring(0, endIndex);
} }
@Override
protected void doInit(Configuration conf) throws IOException {
}
} }

View File

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

View File

@ -198,6 +198,13 @@ public interface WAL {
@Override @Override
String toString(); 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 * When outside clients need to consume persisted WALs, they rely on a provided
* Reader. * Reader.
@ -268,7 +275,5 @@ public interface WAL {
public String toString() { public String toString() {
return this.key + "=" + this.edit; return this.key + "=" + this.edit;
} }
} }
} }

View File

@ -76,7 +76,8 @@ public class WALFactory {
static enum Providers { static enum Providers {
defaultProvider(DefaultWALProvider.class), defaultProvider(DefaultWALProvider.class),
filesystem(DefaultWALProvider.class), filesystem(DefaultWALProvider.class),
multiwal(RegionGroupingProvider.class); multiwal(RegionGroupingProvider.class),
asyncfs(AsyncFSWALProvider.class);
Class<? extends WALProvider> clazz; Class<? extends WALProvider> clazz;
Providers(Class<? extends WALProvider> clazz) { Providers(Class<? extends WALProvider> clazz) {
@ -350,6 +351,7 @@ public class WALFactory {
/** /**
* Create a writer for the WAL. * Create a writer for the WAL.
* <p>
* should be package-private. public only for tests and * should be package-private. public only for tests and
* {@link org.apache.hadoop.hbase.regionserver.wal.Compressor} * {@link org.apache.hadoop.hbase.regionserver.wal.Compressor}
* @return A WAL writer. Close when done with it. * @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.Closeable;
import java.io.IOException; import java.io.IOException;
import java.nio.channels.CompletionHandler;
import java.util.List; import java.util.List;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
// imports for things that haven't moved from regionserver.wal yet. // imports for things that haven't moved from regionserver.wal yet.
import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener; import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
@ -80,6 +80,12 @@ public interface WALProvider {
long getLength() throws IOException; 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 * 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.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience; 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.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.util.FSUtils; 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.WAL;
import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.SequenceFile.CompressionType; import org.apache.hadoop.io.SequenceFile.CompressionType;
@ -51,7 +54,7 @@ import org.apache.hadoop.io.compress.DefaultCodec;
* Hadoop serialization). * Hadoop serialization).
*/ */
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG) @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
public class SequenceFileLogWriter extends WriterBase { public class SequenceFileLogWriter implements DefaultWALProvider.Writer {
private static final Log LOG = LogFactory.getLog(SequenceFileLogWriter.class); private static final Log LOG = LogFactory.getLog(SequenceFileLogWriter.class);
// The sequence file we delegate to. // The sequence file we delegate to.
private SequenceFile.Writer writer; private SequenceFile.Writer writer;
@ -59,6 +62,8 @@ public class SequenceFileLogWriter extends WriterBase {
// in the SequenceFile.Writer 'writer' instance above. // in the SequenceFile.Writer 'writer' instance above.
private FSDataOutputStream writer_out; private FSDataOutputStream writer_out;
private CompressionContext compressionContext;
// Legacy stuff from pre-PB WAL metadata. // Legacy stuff from pre-PB WAL metadata.
private static final Text WAL_VERSION_KEY = new Text("version"); private static final Text WAL_VERSION_KEY = new Text("version");
private static final Text WAL_COMPRESSION_TYPE_KEY = new Text("compression.type"); 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); 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 @Override
public void init(FileSystem fs, Path path, Configuration conf, boolean overwritable) public void init(FileSystem fs, Path path, Configuration conf, boolean overwritable)
throws IOException { throws IOException {
super.init(fs, path, conf, overwritable);
boolean compress = initializeCompressionContext(conf, path); boolean compress = initializeCompressionContext(conf, path);
// Create a SF.Writer instance. // Create a SF.Writer instance.
try { 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 static org.junit.Assert.assertNull;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem; 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.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor; 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.TableName;
import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Increment; import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.regionserver.HRegion; 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.Bytes;
import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.wal.DefaultWALProvider; import org.apache.hadoop.hbase.wal.DefaultWALProvider;
import org.apache.hadoop.hbase.wal.WAL; import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.wal.WALFactory; import org.apache.hadoop.hbase.wal.WALFactory;
import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.junit.After;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass; import org.junit.BeforeClass;
import org.junit.Test; import org.junit.Test;
import org.junit.experimental.categories.Category; 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 * Tests for WAL write durability
*/ */
@RunWith(Parameterized.class)
@Category({RegionServerTests.class, MediumTests.class}) @Category({RegionServerTests.class, MediumTests.class})
public class TestDurability { public class TestDurability {
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); 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[] ROW = Bytes.toBytes("row");
private static byte[] COL = Bytes.toBytes("col"); 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 @BeforeClass
public static void setUpBeforeClass() throws Exception { public static void setUpBeforeClass() throws Exception {
@ -81,6 +96,16 @@ public class TestDurability {
TEST_UTIL.shutdownMiniCluster(); TEST_UTIL.shutdownMiniCluster();
} }
@Before
public void setUp() {
CONF.set(WALFactory.WAL_PROVIDER, walProvider);
}
@After
public void tearDown() throws IOException {
FS.delete(DIR, true);
}
@Test @Test
public void testDurability() throws Exception { public void testDurability() throws Exception {
final WALFactory wals = new WALFactory(CONF, null, "TestDurability"); 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}) @Category({RegionServerTests.class, MediumTests.class})
public class TestLogRollAbort { 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 MiniDFSCluster dfsCluster;
private static Admin admin; private static Admin admin;
private static MiniHBaseCluster cluster; 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 */ /* For the split-then-roll test */
private static final Path HBASEDIR = new Path("/hbase"); 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 // Send the data to HDFS datanodes and close the HDFS writer
log.sync(); 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() /* code taken from MasterFileSystem.getLogDirs(), which is called from MasterFileSystem.splitLog()
* handles RS shutdowns (as observed by the splitting process) * 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 static org.junit.Assert.assertFalse;
import java.util.List;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.TableName; 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.Put;
import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.regionserver.HRegionServer; 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.util.Bytes;
import org.apache.hadoop.hbase.wal.WAL;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.BeforeClass; import org.junit.BeforeClass;
import org.junit.Test; import org.junit.Test;
@ -44,9 +44,9 @@ import org.junit.experimental.categories.Category;
*/ */
@Category({RegionServerTests.class, MediumTests.class}) @Category({RegionServerTests.class, MediumTests.class})
public class TestLogRollPeriod { 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; private final static long LOG_ROLL_PERIOD = 4000;

View File

@ -1,5 +1,4 @@
/** /**
*
* Licensed to the Apache Software Foundation (ASF) under one * Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file * or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information * distributed with this work for additional information
@ -24,7 +23,6 @@ import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import java.io.EOFException; import java.io.EOFException;
import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.HashSet; import java.util.HashSet;
import java.util.List; 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.Log;
import org.apache.commons.logging.LogFactory; 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.fs.Path;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor; 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.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.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.fs.HFileSystem; 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.Region;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.VerySlowRegionServerTests; import org.apache.hadoop.hbase.testclassification.VerySlowRegionServerTests;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.JVMClusterUtil; 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.DefaultWALProvider;
import org.apache.hadoop.hbase.wal.WAL; import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.wal.WALFactory; 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.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.server.datanode.DataNode; 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.BeforeClass;
import org.junit.Rule;
import org.junit.Test; import org.junit.Test;
import org.junit.experimental.categories.Category; import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
/** @Category({ VerySlowRegionServerTests.class, LargeTests.class })
* Test log deletion as logs are rolled. public class TestLogRolling extends AbstractTestLogRolling {
*/
@Category({VerySlowRegionServerTests.class, LargeTests.class})
public class TestLogRolling {
private static final Log LOG = LogFactory.getLog(TestLogRolling.class); 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 @BeforeClass
public static void setUpBeforeClass() throws Exception { public static void setUpBeforeClass() throws Exception {
// TODO: testLogRollOnDatanodeDeath fails if short circuit reads are on under the hadoop2 // TODO: testLogRollOnDatanodeDeath fails if short circuit reads are on under the hadoop2
// profile. See HBASE-9337 for related issues. // profile. See HBASE-9337 for related issues.
System.setProperty("hbase.tests.use.shortcircuit.reads", "false"); 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 ****/ /**** configuration for testLogRollOnDatanodeDeath ****/
// make sure log.hflush() calls syncFs() to open a pipeline // make sure log.hflush() calls syncFs() to open a pipeline
TEST_UTIL.getConfiguration().setBoolean("dfs.support.append", true); 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("dfs.client.block.write.retries", 30);
TEST_UTIL.getConfiguration().setInt("hbase.regionserver.hlog.tolerable.lowreplication", 2); TEST_UTIL.getConfiguration().setInt("hbase.regionserver.hlog.tolerable.lowreplication", 2);
TEST_UTIL.getConfiguration().setInt("hbase.regionserver.hlog.lowreplication.rolllimit", 3); TEST_UTIL.getConfiguration().setInt("hbase.regionserver.hlog.lowreplication.rolllimit", 3);
} AbstractTestLogRolling.setUpBeforeClass();
@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
}
}
}
} }
/** /**
* Tests that log rolling doesn't hang when no data is written. * Tests that logs are rolled upon detecting datanode death Requires an HDFS jar with HDFS-826 &
*/ * syncFs() support (HDFS-200)
@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)
*/ */
@Test @Test
public void testLogRollOnDatanodeDeath() throws Exception { public void testLogRollOnDatanodeDeath() throws Exception {
TEST_UTIL.ensureSomeRegionServersAvailable(2); TEST_UTIL.ensureSomeRegionServersAvailable(2);
assertTrue("This test requires WAL file replication set to 2.", assertTrue("This test requires WAL file replication set to 2.",
fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()) == 2); fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()) == 2);
LOG.info("Replication=" + LOG.info("Replication=" + fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()));
fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()));
this.server = cluster.getRegionServer(0); this.server = cluster.getRegionServer(0);
@ -328,8 +121,8 @@ public class TestLogRolling {
}); });
// don't run this test without append support (HDFS-200 & HDFS-142) // don't run this test without append support (HDFS-200 & HDFS-142)
assertTrue("Need append support for this test", FSUtils assertTrue("Need append support for this test",
.isAppendSupported(TEST_UTIL.getConfiguration())); FSUtils.isAppendSupported(TEST_UTIL.getConfiguration()));
// add up the datanode count, to ensure proper replication when we kill 1 // 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 // 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 // when we stop/start a namenode later, as mentioned in HBASE-5163
List<DataNode> existingNodes = dfsCluster.getDataNodes(); List<DataNode> existingNodes = dfsCluster.getDataNodes();
int numDataNodes = 3; int numDataNodes = 3;
dfsCluster.startDataNodes(TEST_UTIL.getConfiguration(), numDataNodes, true, dfsCluster.startDataNodes(TEST_UTIL.getConfiguration(), numDataNodes, true, null, null);
null, null);
List<DataNode> allNodes = dfsCluster.getDataNodes(); 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))) { if (existingNodes.contains(allNodes.get(i))) {
dfsCluster.stopDataNode( i ); dfsCluster.stopDataNode(i);
} }
} }
assertTrue("DataNodes " + dfsCluster.getDataNodes().size() + assertTrue(
" default replication " + "DataNodes " + dfsCluster.getDataNodes().size() + " default replication "
fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()), + fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()),
dfsCluster.getDataNodes().size() >= dfsCluster.getDataNodes()
fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()) + 1); .size() >= fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()) + 1);
writeData(table, 2); writeData(table, 2);
@ -362,9 +154,8 @@ public class TestLogRolling {
assertTrue("The log shouldn't have rolled yet", assertTrue("The log shouldn't have rolled yet",
oldFilenum == DefaultWALProvider.extractFileNumFromWAL(log)); oldFilenum == DefaultWALProvider.extractFileNumFromWAL(log));
final DatanodeInfo[] pipeline = log.getPipeLine(); final DatanodeInfo[] pipeline = log.getPipeline();
assertTrue(pipeline.length == assertTrue(pipeline.length == fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()));
fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()));
// kill a datanode in the pipeline to force a log roll on the next sync() // 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. // This function is synchronous, when it returns the node is killed.
@ -390,26 +181,23 @@ public class TestLogRolling {
batchWriteAndWait(table, log, 3, false, 14000); batchWriteAndWait(table, log, 3, false, 14000);
int replication = log.getLogReplication(); int replication = log.getLogReplication();
assertTrue("LowReplication Roller should've been disabled, current replication=" assertTrue("LowReplication Roller should've been disabled, current replication=" + replication,
+ replication, !log.isLowReplicationRollEnabled()); !log.isLowReplicationRollEnabled());
dfsCluster dfsCluster.startDataNodes(TEST_UTIL.getConfiguration(), 1, true, null, null);
.startDataNodes(TEST_UTIL.getConfiguration(), 1, true, null, null);
// Force roll writer. The new log file will have the default replications, // Force roll writer. The new log file will have the default replications,
// and the LowReplication Roller will be enabled. // and the LowReplication Roller will be enabled.
log.rollWriter(true); log.rollWriter(true);
batchWriteAndWait(table, log, 13, true, 10000); batchWriteAndWait(table, log, 13, true, 10000);
replication = log.getLogReplication(); replication = log.getLogReplication();
assertTrue("New log file should have the default replication instead of " + assertTrue("New log file should have the default replication instead of " + replication,
replication,
replication == fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS())); replication == fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()));
assertTrue("LowReplication Roller should've been enabled", log.isLowReplicationRollEnabled()); assertTrue("LowReplication Roller should've been enabled", log.isLowReplicationRollEnabled());
} }
/** /**
* Test that WAL is rolled when all data nodes in the pipeline have been * Test that WAL is rolled when all data nodes in the pipeline have been restarted.
* restarted.
* @throws Exception * @throws Exception
*/ */
@Test @Test
@ -417,8 +205,7 @@ public class TestLogRolling {
LOG.info("Starting testLogRollOnPipelineRestart"); LOG.info("Starting testLogRollOnPipelineRestart");
assertTrue("This test requires WAL file replication.", assertTrue("This test requires WAL file replication.",
fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()) > 1); fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()) > 1);
LOG.info("Replication=" + LOG.info("Replication=" + fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()));
fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()));
// When the hbase:meta table can be opened, the region servers are running // When the hbase:meta table can be opened, the region servers are running
Table t = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME); Table t = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME);
try { try {
@ -442,9 +229,10 @@ public class TestLogRolling {
@Override @Override
public void preLogRoll(Path oldFile, Path newFile) { 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)); preLogRolledCalled.add(new Integer(1));
} }
@Override @Override
public void postLogRoll(Path oldFile, Path newFile) { public void postLogRoll(Path oldFile, Path newFile) {
paths.add(newFile); paths.add(newFile);
@ -452,8 +240,8 @@ public class TestLogRolling {
}); });
// don't run this test without append support (HDFS-200 & HDFS-142) // don't run this test without append support (HDFS-200 & HDFS-142)
assertTrue("Need append support for this test", FSUtils assertTrue("Need append support for this test",
.isAppendSupported(TEST_UTIL.getConfiguration())); FSUtils.isAppendSupported(TEST_UTIL.getConfiguration()));
writeData(table, 1002); writeData(table, 1002);
@ -463,8 +251,8 @@ public class TestLogRolling {
assertTrue("Log should have a timestamp older than now", assertTrue("Log should have a timestamp older than now",
curTime > oldFilenum && oldFilenum != -1); curTime > oldFilenum && oldFilenum != -1);
assertTrue("The log shouldn't have rolled yet", oldFilenum == assertTrue("The log shouldn't have rolled yet",
DefaultWALProvider.extractFileNumFromWAL(log)); oldFilenum == DefaultWALProvider.extractFileNumFromWAL(log));
// roll all datanodes in the pipeline // roll all datanodes in the pipeline
dfsCluster.restartDataNodes(); dfsCluster.restartDataNodes();
@ -503,23 +291,23 @@ public class TestLogRolling {
FSUtils fsUtils = FSUtils.getInstance(fs, TEST_UTIL.getConfiguration()); FSUtils fsUtils = FSUtils.getInstance(fs, TEST_UTIL.getConfiguration());
for (Path p : paths) { for (Path p : paths) {
LOG.debug("recovering lease for " + p); LOG.debug("recovering lease for " + p);
fsUtils.recoverFileLease(((HFileSystem)fs).getBackingFs(), p, fsUtils.recoverFileLease(((HFileSystem) fs).getBackingFs(), p, TEST_UTIL.getConfiguration(),
TEST_UTIL.getConfiguration(), null); null);
LOG.debug("Reading WAL "+FSUtils.getPath(p)); LOG.debug("Reading WAL " + FSUtils.getPath(p));
WAL.Reader reader = null; WAL.Reader reader = null;
try { try {
reader = WALFactory.createReader(fs, p, TEST_UTIL.getConfiguration()); reader = WALFactory.createReader(fs, p, TEST_UTIL.getConfiguration());
WAL.Entry entry; WAL.Entry entry;
while ((entry = reader.next()) != null) { 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()) { for (Cell cell : entry.getEdit().getCells()) {
loggedRows.add(Bytes.toStringBinary(cell.getRowArray(), cell.getRowOffset(), loggedRows.add(
cell.getRowLength())); Bytes.toStringBinary(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()));
} }
} }
} catch (EOFException e) { } catch (EOFException e) {
LOG.debug("EOF reading file "+FSUtils.getPath(p)); LOG.debug("EOF reading file " + FSUtils.getPath(p));
} finally { } finally {
if (reader != null) reader.close(); if (reader != null) reader.close();
} }
@ -532,7 +320,7 @@ public class TestLogRolling {
assertTrue(loggedRows.contains("row1005")); assertTrue(loggedRows.contains("row1005"));
// flush all regions // flush all regions
for (Region r: server.getOnlineRegionsLocalContext()) { for (Region r : server.getOnlineRegionsLocalContext()) {
try { try {
r.flush(true); r.flush(true);
} catch (Exception e) { } catch (Exception e) {
@ -547,19 +335,19 @@ public class TestLogRolling {
ResultScanner scanner = table.getScanner(new Scan()); ResultScanner scanner = table.getScanner(new Scan());
try { try {
for (int i=2; i<=5; i++) { for (int i = 2; i <= 5; i++) {
Result r = scanner.next(); Result r = scanner.next();
assertNotNull(r); assertNotNull(r);
assertFalse(r.isEmpty()); assertFalse(r.isEmpty());
assertEquals("row100"+i, Bytes.toString(r.getRow())); assertEquals("row100" + i, Bytes.toString(r.getRow()));
} }
} finally { } finally {
scanner.close(); scanner.close();
} }
// verify that no region servers aborted // verify that no region servers aborted
for (JVMClusterUtil.RegionServerThread rsThread: for (JVMClusterUtil.RegionServerThread rsThread : TEST_UTIL.getHBaseCluster()
TEST_UTIL.getHBaseCluster().getRegionServerThreads()) { .getRegionServerThreads()) {
assertFalse(rsThread.getRegionServer().isAborted()); assertFalse(rsThread.getRegionServer().isAborted());
} }
} finally { } 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 * Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file * or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information * distributed with this work for additional information
@ -18,190 +17,32 @@
*/ */
package org.apache.hadoop.hbase.regionserver.wal; 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 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.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.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.wal.WAL.Entry;
import org.apache.hadoop.hbase.wal.WAL; import org.apache.hadoop.hbase.wal.DefaultWALProvider;
import org.apache.hadoop.hbase.wal.WALFactory;
import org.apache.hadoop.hbase.wal.WALKey;
import org.apache.hadoop.hbase.wal.WALProvider; import org.apache.hadoop.hbase.wal.WALProvider;
import org.junit.After; import org.apache.hadoop.hbase.wal.WALProvider.Writer;
import org.junit.AfterClass;
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.experimental.categories.Category;
import org.junit.rules.TestName;
/** @Category({ RegionServerTests.class, MediumTests.class })
* WAL tests that can be reused across providers. public class TestProtobufLog extends AbstractTestProtobufLog<WALProvider.Writer> {
*/
@Category({RegionServerTests.class, MediumTests.class})
public class TestProtobufLog {
protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
protected FileSystem fs; @Override
protected Path dir; protected Writer createWriter(Path path) throws IOException {
protected WALFactory wals; return DefaultWALProvider.createWriter(TEST_UTIL.getConfiguration(), fs, path, false);
@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 @Override
public void tearDown() throws Exception { protected void append(Writer writer, Entry entry) throws IOException {
wals.close(); writer.append(entry);
FileStatus[] entries = fs.listStatus(new Path("/"));
for (FileStatus dir : entries) {
fs.delete(dir.getPath(), true);
}
} }
@BeforeClass @Override
public static void setUpBeforeClass() throws Exception { protected void sync(Writer writer) throws IOException {
// 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));
}
writer.sync(); 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(); Configuration conf = TestWALReplay.TEST_UTIL.getConfiguration();
conf.setBoolean(HConstants.ENABLE_WAL_COMPRESSION, true); conf.setBoolean(HConstants.ENABLE_WAL_COMPRESSION, true);
} }
} }