HBASE-15265 Implement an asynchronous FSHLog
This commit is contained in:
parent
1a9837ab74
commit
c96b642f15
|
@ -167,6 +167,9 @@ public class LogRoller extends HasThread {
|
|||
}
|
||||
}
|
||||
}
|
||||
for (WAL wal : walNeedsRoll.keySet()) {
|
||||
wal.logRollerExited();
|
||||
}
|
||||
LOG.info("LogRoller exiting.");
|
||||
}
|
||||
|
||||
|
|
|
@ -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();
|
||||
}
|
|
@ -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();
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -48,17 +48,20 @@ import org.apache.hadoop.hbase.wal.WALKey;
|
|||
class FSWALEntry extends Entry {
|
||||
// The below data members are denoted 'transient' just to highlight these are not persisted;
|
||||
// they are only in memory and held here while passing over the ring buffer.
|
||||
private final transient long sequence;
|
||||
private final transient long txid;
|
||||
private final transient boolean inMemstore;
|
||||
private final transient HRegionInfo hri;
|
||||
private final Set<byte[]> familyNames;
|
||||
private final transient Set<byte[]> familyNames;
|
||||
// In the new WAL logic, we will rewrite failed WAL entries to new WAL file, so we need to avoid
|
||||
// calling stampRegionSequenceId again.
|
||||
private transient boolean stamped = false;
|
||||
|
||||
FSWALEntry(final long sequence, final WALKey key, final WALEdit edit,
|
||||
FSWALEntry(final long txid, final WALKey key, final WALEdit edit,
|
||||
final HRegionInfo hri, final boolean inMemstore) {
|
||||
super(key, edit);
|
||||
this.inMemstore = inMemstore;
|
||||
this.hri = hri;
|
||||
this.sequence = sequence;
|
||||
this.txid = txid;
|
||||
if (inMemstore) {
|
||||
// construct familyNames here to reduce the work of log sinker.
|
||||
ArrayList<Cell> cells = this.getEdit().getCells();
|
||||
|
@ -80,7 +83,7 @@ class FSWALEntry extends Entry {
|
|||
}
|
||||
|
||||
public String toString() {
|
||||
return "sequence=" + this.sequence + ", " + super.toString();
|
||||
return "sequence=" + this.txid + ", " + super.toString();
|
||||
};
|
||||
|
||||
boolean isInMemstore() {
|
||||
|
@ -92,10 +95,10 @@ class FSWALEntry extends Entry {
|
|||
}
|
||||
|
||||
/**
|
||||
* @return The sequence on the ring buffer when this edit was added.
|
||||
* @return The transaction id of this edit.
|
||||
*/
|
||||
long getSequence() {
|
||||
return this.sequence;
|
||||
long getTxid() {
|
||||
return this.txid;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -103,9 +106,12 @@ class FSWALEntry extends Entry {
|
|||
* SIDE-EFFECT is our stamping the sequenceid into every Cell AND setting the sequenceid into the
|
||||
* MVCC WriteEntry!!!!
|
||||
* @return The sequenceid we stamped on this edit.
|
||||
* @throws IOException
|
||||
*/
|
||||
long stampRegionSequenceId() throws IOException {
|
||||
if (stamped) {
|
||||
return getKey().getSequenceId();
|
||||
}
|
||||
stamped = true;
|
||||
long regionSequenceId = WALKey.NO_SEQUENCE_ID;
|
||||
MultiVersionConcurrencyControl mvcc = getKey().getMvcc();
|
||||
MultiVersionConcurrencyControl.WriteEntry we = null;
|
||||
|
|
|
@ -19,108 +19,42 @@
|
|||
|
||||
package org.apache.hadoop.hbase.regionserver.wal;
|
||||
|
||||
import static org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader.DEFAULT_WAL_TRAILER_WARN_SIZE;
|
||||
import static org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader.WAL_TRAILER_WARN_SIZE;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.codec.Codec;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALHeader;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALTrailer;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.wal.DefaultWALProvider;
|
||||
import org.apache.hadoop.hbase.wal.WAL.Entry;
|
||||
|
||||
/**
|
||||
* Writer for protobuf-based WAL.
|
||||
*/
|
||||
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
|
||||
public class ProtobufLogWriter extends WriterBase {
|
||||
@InterfaceAudience.Private
|
||||
public class ProtobufLogWriter extends AbstractProtobufLogWriter
|
||||
implements DefaultWALProvider.Writer {
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(ProtobufLogWriter.class);
|
||||
|
||||
protected FSDataOutputStream output;
|
||||
protected Codec.Encoder cellEncoder;
|
||||
protected WALCellCodec.ByteStringCompressor compressor;
|
||||
private boolean trailerWritten;
|
||||
private WALTrailer trailer;
|
||||
// maximum size of the wal Trailer in bytes. If a user writes/reads a trailer with size larger
|
||||
// than this size, it is written/read respectively, with a WARN message in the log.
|
||||
private int trailerWarnSize;
|
||||
|
||||
public ProtobufLogWriter() {
|
||||
super();
|
||||
}
|
||||
|
||||
protected WALCellCodec getCodec(Configuration conf, CompressionContext compressionContext)
|
||||
throws IOException {
|
||||
return WALCellCodec.create(conf, null, compressionContext);
|
||||
}
|
||||
|
||||
protected WALHeader buildWALHeader(Configuration conf, WALHeader.Builder builder)
|
||||
throws IOException {
|
||||
if (!builder.hasWriterClsName()) {
|
||||
builder.setWriterClsName(ProtobufLogWriter.class.getSimpleName());
|
||||
}
|
||||
if (!builder.hasCellCodecClsName()) {
|
||||
builder.setCellCodecClsName(WALCellCodec.getWALCellCodecClass(conf));
|
||||
}
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
@SuppressWarnings("deprecation")
|
||||
public void init(FileSystem fs, Path path, Configuration conf, boolean overwritable)
|
||||
throws IOException {
|
||||
super.init(fs, path, conf, overwritable);
|
||||
assert this.output == null;
|
||||
boolean doCompress = initializeCompressionContext(conf, path);
|
||||
this.trailerWarnSize = conf.getInt(WAL_TRAILER_WARN_SIZE, DEFAULT_WAL_TRAILER_WARN_SIZE);
|
||||
int bufferSize = FSUtils.getDefaultBufferSize(fs);
|
||||
short replication = (short)conf.getInt(
|
||||
"hbase.regionserver.hlog.replication", FSUtils.getDefaultReplication(fs, path));
|
||||
long blockSize = conf.getLong("hbase.regionserver.hlog.blocksize",
|
||||
FSUtils.getDefaultBlockSize(fs, path));
|
||||
output = fs.createNonRecursive(path, overwritable, bufferSize, replication, blockSize, null);
|
||||
output.write(ProtobufLogReader.PB_WAL_MAGIC);
|
||||
boolean doTagCompress = doCompress
|
||||
&& conf.getBoolean(CompressionContext.ENABLE_WAL_TAGS_COMPRESSION, true);
|
||||
buildWALHeader(conf,
|
||||
WALHeader.newBuilder().setHasCompression(doCompress).setHasTagCompression(doTagCompress))
|
||||
.writeDelimitedTo(output);
|
||||
|
||||
initAfterHeader(doCompress);
|
||||
|
||||
// instantiate trailer to default value.
|
||||
trailer = WALTrailer.newBuilder().build();
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("Initialized protobuf WAL=" + path + ", compression=" + doCompress);
|
||||
}
|
||||
}
|
||||
|
||||
protected void initAfterHeader(boolean doCompress) throws IOException {
|
||||
WALCellCodec codec = getCodec(conf, this.compressionContext);
|
||||
this.cellEncoder = codec.getEncoder(this.output);
|
||||
if (doCompress) {
|
||||
this.compressor = codec.getByteStringCompressor();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void append(Entry entry) throws IOException {
|
||||
entry.setCompressionContext(compressionContext);
|
||||
entry.getKey().getBuilder(compressor).
|
||||
setFollowingKvCount(entry.getEdit().size()).build().writeDelimitedTo(output);
|
||||
entry.getKey().getBuilder(compressor).setFollowingKvCount(entry.getEdit().size()).build()
|
||||
.writeDelimitedTo(output);
|
||||
for (Cell cell : entry.getEdit().getCells()) {
|
||||
// cellEncoder must assume little about the stream, since we write PB and cells in turn.
|
||||
cellEncoder.write(cell);
|
||||
}
|
||||
length.set(output.getPos());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -137,32 +71,6 @@ public class ProtobufLogWriter extends WriterBase {
|
|||
}
|
||||
}
|
||||
|
||||
WALTrailer buildWALTrailer(WALTrailer.Builder builder) {
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
private void writeWALTrailer() {
|
||||
try {
|
||||
int trailerSize = 0;
|
||||
if (this.trailer == null) {
|
||||
// use default trailer.
|
||||
LOG.warn("WALTrailer is null. Continuing with default.");
|
||||
this.trailer = buildWALTrailer(WALTrailer.newBuilder());
|
||||
trailerSize = this.trailer.getSerializedSize();
|
||||
} else if ((trailerSize = this.trailer.getSerializedSize()) > this.trailerWarnSize) {
|
||||
// continue writing after warning the user.
|
||||
LOG.warn("Please investigate WALTrailer usage. Trailer size > maximum size : " +
|
||||
trailerSize + " > " + this.trailerWarnSize);
|
||||
}
|
||||
this.trailer.writeTo(output);
|
||||
output.writeInt(trailerSize);
|
||||
output.write(ProtobufLogReader.PB_WAL_COMPLETE_MAGIC);
|
||||
this.trailerWritten = true;
|
||||
} catch (IOException ioe) {
|
||||
LOG.warn("Failed to write trailer, non-fatal, continuing...", ioe);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void sync() throws IOException {
|
||||
FSDataOutputStream fsdos = this.output;
|
||||
|
@ -171,21 +79,35 @@ public class ProtobufLogWriter extends WriterBase {
|
|||
fsdos.hflush();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLength() throws IOException {
|
||||
try {
|
||||
return this.output.getPos();
|
||||
} catch (NullPointerException npe) {
|
||||
// Concurrent close...
|
||||
throw new IOException(npe);
|
||||
}
|
||||
}
|
||||
|
||||
public FSDataOutputStream getStream() {
|
||||
return this.output;
|
||||
}
|
||||
|
||||
void setWALTrailer(WALTrailer walTrailer) {
|
||||
this.trailer = walTrailer;
|
||||
@SuppressWarnings("deprecation")
|
||||
@Override
|
||||
protected void initOutput(FileSystem fs, Path path, boolean overwritable, int bufferSize,
|
||||
short replication, long blockSize) throws IOException {
|
||||
this.output = fs.createNonRecursive(path, overwritable, bufferSize, replication, blockSize,
|
||||
null);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected long writeMagicAndWALHeader(byte[] magic, WALHeader header) throws IOException {
|
||||
output.write(magic);
|
||||
header.writeDelimitedTo(output);
|
||||
return output.getPos();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected OutputStream getOutputStreamForCellEncoder() {
|
||||
return this.output;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected long writeWALTrailerAndMagic(WALTrailer trailer, byte[] magic) throws IOException {
|
||||
trailer.writeTo(output);
|
||||
output.writeInt(trailer.getSerializedSize());
|
||||
output.write(magic);
|
||||
return output.getPos();
|
||||
}
|
||||
}
|
|
@ -18,33 +18,28 @@
|
|||
package org.apache.hadoop.hbase.regionserver.wal;
|
||||
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.htrace.Span;
|
||||
|
||||
/**
|
||||
* A Future on a filesystem sync call. It given to a client or 'Handler' for it to wait on till
|
||||
* the sync completes.
|
||||
*
|
||||
* <p>Handlers coming in call append, append, append, and then do a flush/sync of
|
||||
* the edits they have appended the WAL before returning. Since sync takes a while to
|
||||
* complete, we give the Handlers back this sync future to wait on until the
|
||||
* actual HDFS sync completes. Meantime this sync future goes across the ringbuffer and into a
|
||||
* sync runner thread; when it completes, it finishes up the future, the handler get or failed
|
||||
* check completes and the Handler can then progress.
|
||||
* A Future on a filesystem sync call. It given to a client or 'Handler' for it to wait on till the
|
||||
* sync completes.
|
||||
* <p>
|
||||
* This is just a partial implementation of Future; we just implement get and
|
||||
* failure. Unimplemented methods throw {@link UnsupportedOperationException}.
|
||||
* Handlers coming in call append, append, append, and then do a flush/sync of the edits they have
|
||||
* appended the WAL before returning. Since sync takes a while to complete, we give the Handlers
|
||||
* back this sync future to wait on until the actual HDFS sync completes. Meantime this sync future
|
||||
* goes across a queue and is handled by a background thread; when it completes, it finishes up the
|
||||
* future, the handler get or failed check completes and the Handler can then progress.
|
||||
* <p>
|
||||
* There is not a one-to-one correlation between dfs sync invocations and
|
||||
* instances of this class. A single dfs sync call may complete and mark many
|
||||
* SyncFutures as done; i.e. we batch up sync calls rather than do a dfs sync
|
||||
* call every time a Handler asks for it.
|
||||
* This is just a partial implementation of Future; we just implement get and failure.
|
||||
* <p>
|
||||
* SyncFutures are immutable but recycled. Call #reset(long, Span) before use even
|
||||
* if it the first time, start the sync, then park the 'hitched' thread on a call to
|
||||
* #get().
|
||||
* There is not a one-to-one correlation between dfs sync invocations and instances of this class. A
|
||||
* single dfs sync call may complete and mark many SyncFutures as done; i.e. we batch up sync calls
|
||||
* rather than do a dfs sync call every time a Handler asks for it.
|
||||
* <p>
|
||||
* SyncFutures are immutable but recycled. Call #reset(long, Span) before use even if it the first
|
||||
* time, start the sync, then park the 'hitched' thread on a call to #get().
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
class SyncFuture {
|
||||
|
@ -54,17 +49,17 @@ class SyncFuture {
|
|||
private static final long NOT_DONE = 0;
|
||||
|
||||
/**
|
||||
* The sequence at which we were added to the ring buffer.
|
||||
* The transaction id of this operation, monotonically increases.
|
||||
*/
|
||||
private long ringBufferSequence;
|
||||
private long txid;
|
||||
|
||||
/**
|
||||
* The sequence that was set in here when we were marked done. Should be equal
|
||||
* or > ringBufferSequence. Put this data member into the NOT_DONE state while this
|
||||
* class is in use. But for the first position on construction, let it be -1 so we can
|
||||
* immediately call {@link #reset(long, Span)} below and it will work.
|
||||
* The transaction id that was set in here when we were marked done. Should be equal or > txnId.
|
||||
* Put this data member into the NOT_DONE state while this class is in use. But for the first
|
||||
* position on construction, let it be -1 so we can immediately call {@link #reset(long, Span)}
|
||||
* below and it will work.
|
||||
*/
|
||||
private long doneSequence = -1;
|
||||
private long doneTxid = -1;
|
||||
|
||||
/**
|
||||
* If error, the associated throwable. Set when the future is 'done'.
|
||||
|
@ -79,80 +74,83 @@ class SyncFuture {
|
|||
private Span span;
|
||||
|
||||
/**
|
||||
* Call this method to clear old usage and get it ready for new deploy. Call
|
||||
* this method even if it is being used for the first time.
|
||||
*
|
||||
* @param sequence sequenceId from this Future's position in the RingBuffer
|
||||
* Call this method to clear old usage and get it ready for new deploy. Call this method even if
|
||||
* it is being used for the first time.
|
||||
* @param txnId the new transaction id
|
||||
* @return this
|
||||
*/
|
||||
synchronized SyncFuture reset(final long sequence) {
|
||||
return reset(sequence, null);
|
||||
synchronized SyncFuture reset(final long txnId) {
|
||||
return reset(txnId, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Call this method to clear old usage and get it ready for new deploy. Call
|
||||
* this method even if it is being used for the first time.
|
||||
*
|
||||
* Call this method to clear old usage and get it ready for new deploy. Call this method even if
|
||||
* it is being used for the first time.
|
||||
* @param sequence sequenceId from this Future's position in the RingBuffer
|
||||
* @param span curren span, detached from caller. Don't forget to attach it when
|
||||
* resuming after a call to {@link #get()}.
|
||||
* @param span curren span, detached from caller. Don't forget to attach it when resuming after a
|
||||
* call to {@link #get()}.
|
||||
* @return this
|
||||
*/
|
||||
synchronized SyncFuture reset(final long sequence, Span span) {
|
||||
if (t != null && t != Thread.currentThread()) throw new IllegalStateException();
|
||||
synchronized SyncFuture reset(final long txnId, Span span) {
|
||||
if (t != null && t != Thread.currentThread()) {
|
||||
throw new IllegalStateException();
|
||||
}
|
||||
t = Thread.currentThread();
|
||||
if (!isDone()) throw new IllegalStateException("" + sequence + " " + Thread.currentThread());
|
||||
this.doneSequence = NOT_DONE;
|
||||
this.ringBufferSequence = sequence;
|
||||
if (!isDone()) {
|
||||
throw new IllegalStateException("" + txnId + " " + Thread.currentThread());
|
||||
}
|
||||
this.doneTxid = NOT_DONE;
|
||||
this.txid = txnId;
|
||||
this.span = span;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized String toString() {
|
||||
return "done=" + isDone() + ", ringBufferSequence=" + this.ringBufferSequence;
|
||||
return "done=" + isDone() + ", txid=" + this.txid;
|
||||
}
|
||||
|
||||
synchronized long getRingBufferSequence() {
|
||||
return this.ringBufferSequence;
|
||||
synchronized long getTxid() {
|
||||
return this.txid;
|
||||
}
|
||||
|
||||
/**
|
||||
* Retrieve the {@code span} instance from this Future. EventHandler calls
|
||||
* this method to continue the span. Thread waiting on this Future musn't call
|
||||
* this method until AFTER calling {@link #get()} and the future has been
|
||||
* released back to the originating thread.
|
||||
* Retrieve the {@code span} instance from this Future. EventHandler calls this method to continue
|
||||
* the span. Thread waiting on this Future musn't call this method until AFTER calling
|
||||
* {@link #get()} and the future has been released back to the originating thread.
|
||||
*/
|
||||
synchronized Span getSpan() {
|
||||
return this.span;
|
||||
}
|
||||
|
||||
/**
|
||||
* Used to re-attach a {@code span} to the Future. Called by the EventHandler
|
||||
* after a it has completed processing and detached the span from its scope.
|
||||
* Used to re-attach a {@code span} to the Future. Called by the EventHandler after a it has
|
||||
* completed processing and detached the span from its scope.
|
||||
*/
|
||||
synchronized void setSpan(Span span) {
|
||||
this.span = span;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param sequence Sync sequence at which this future 'completed'.
|
||||
* @param txid the transaction id at which this future 'completed'.
|
||||
* @param t Can be null. Set if we are 'completing' on error (and this 't' is the error).
|
||||
* @return True if we successfully marked this outstanding future as completed/done.
|
||||
* Returns false if this future is already 'done' when this method called.
|
||||
* @return True if we successfully marked this outstanding future as completed/done. Returns false
|
||||
* if this future is already 'done' when this method called.
|
||||
*/
|
||||
synchronized boolean done(final long sequence, final Throwable t) {
|
||||
if (isDone()) return false;
|
||||
synchronized boolean done(final long txid, final Throwable t) {
|
||||
if (isDone()) {
|
||||
return false;
|
||||
}
|
||||
this.throwable = t;
|
||||
if (sequence < this.ringBufferSequence) {
|
||||
if (txid < this.txid) {
|
||||
// Something badly wrong.
|
||||
if (throwable == null) {
|
||||
this.throwable = new IllegalStateException("sequence=" + sequence +
|
||||
", ringBufferSequence=" + this.ringBufferSequence);
|
||||
this.throwable =
|
||||
new IllegalStateException("done txid=" + txid + ", my txid=" + this.txid);
|
||||
}
|
||||
}
|
||||
// Mark done.
|
||||
this.doneSequence = sequence;
|
||||
this.doneTxid = txid;
|
||||
// Wake up waiting threads.
|
||||
notify();
|
||||
return true;
|
||||
|
@ -166,21 +164,14 @@ class SyncFuture {
|
|||
while (!isDone()) {
|
||||
wait(1000);
|
||||
}
|
||||
if (this.throwable != null) throw new ExecutionException(this.throwable);
|
||||
return this.doneSequence;
|
||||
if (this.throwable != null) {
|
||||
throw new ExecutionException(this.throwable);
|
||||
}
|
||||
|
||||
public Long get(long timeout, TimeUnit unit)
|
||||
throws InterruptedException, ExecutionException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
public boolean isCancelled() {
|
||||
throw new UnsupportedOperationException();
|
||||
return this.doneTxid;
|
||||
}
|
||||
|
||||
synchronized boolean isDone() {
|
||||
return this.doneSequence != NOT_DONE;
|
||||
return this.doneTxid != NOT_DONE;
|
||||
}
|
||||
|
||||
synchronized boolean isThrowable() {
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
|
@ -72,6 +72,7 @@ import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
|
|||
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
|
||||
import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
|
||||
import org.apache.hadoop.io.EnumSetWritable;
|
||||
import org.apache.hadoop.ipc.RemoteException;
|
||||
import org.apache.hadoop.net.NetUtils;
|
||||
import org.apache.hadoop.util.DataChecksum;
|
||||
|
||||
|
@ -417,7 +418,6 @@ public class FanOutOneBlockAsyncDFSOutputHelper {
|
|||
public boolean progress() {
|
||||
return DFS_CLIENT_ADAPTOR.isClientRunning(client);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
static {
|
||||
|
@ -579,6 +579,18 @@ public class FanOutOneBlockAsyncDFSOutputHelper {
|
|||
return futureList;
|
||||
}
|
||||
|
||||
/**
|
||||
* Exception other than RemoteException thrown when calling create on namenode
|
||||
*/
|
||||
public static class NameNodeException extends IOException {
|
||||
|
||||
private static final long serialVersionUID = 3143237406477095390L;
|
||||
|
||||
public NameNodeException(Throwable cause) {
|
||||
super(cause);
|
||||
}
|
||||
}
|
||||
|
||||
private static FanOutOneBlockAsyncDFSOutput createOutput(DistributedFileSystem dfs, String src,
|
||||
boolean overwrite, boolean createParent, short replication, long blockSize,
|
||||
EventLoop eventLoop) throws IOException {
|
||||
|
@ -587,11 +599,20 @@ public class FanOutOneBlockAsyncDFSOutputHelper {
|
|||
DFSClient client = dfs.getClient();
|
||||
String clientName = client.getClientName();
|
||||
ClientProtocol namenode = client.getNamenode();
|
||||
HdfsFileStatus stat = FILE_CREATER.create(namenode, src,
|
||||
HdfsFileStatus stat;
|
||||
try {
|
||||
stat = FILE_CREATER.create(namenode, src,
|
||||
FsPermission.getFileDefault().applyUMask(FsPermission.getUMask(conf)), clientName,
|
||||
new EnumSetWritable<CreateFlag>(
|
||||
overwrite ? EnumSet.of(CREATE, OVERWRITE) : EnumSet.of(CREATE)),
|
||||
createParent, replication, blockSize);
|
||||
} catch (Exception e) {
|
||||
if (e instanceof RemoteException) {
|
||||
throw (RemoteException) e;
|
||||
} else {
|
||||
throw new NameNodeException(e);
|
||||
}
|
||||
}
|
||||
beginFileLease(client, src, stat.getFileId());
|
||||
boolean succ = false;
|
||||
LocatedBlock locatedBlock = null;
|
||||
|
@ -656,6 +677,13 @@ public class FanOutOneBlockAsyncDFSOutputHelper {
|
|||
}.resolve(dfs, f);
|
||||
}
|
||||
|
||||
public static boolean shouldRetryCreate(RemoteException e) {
|
||||
// RetryStartFileException is introduced in HDFS 2.6+, so here we can only use the class name.
|
||||
// For exceptions other than this, we just throw it out. This is same with
|
||||
// DFSOutputStream.newStreamForCreate.
|
||||
return e.getClassName().endsWith("RetryStartFileException");
|
||||
}
|
||||
|
||||
static void completeFile(DFSClient client, ClientProtocol namenode, String src, String clientName,
|
||||
ExtendedBlock block, long fileId) {
|
||||
for (int retry = 0;; retry++) {
|
||||
|
|
|
@ -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: <wal-name>{@link #WAL_FILE_NAME_DELIMITER}
|
||||
* <file-creation-timestamp>[.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://<name node>/hbase/.logs/<server name>-splitting/...</li>
|
||||
* <li>hdfs://<name node>/hbase/.logs/<server name>/...</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);
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -19,354 +19,39 @@
|
|||
package org.apache.hadoop.hbase.wal;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
// imports for things that haven't moved from regionserver.wal yet.
|
||||
import org.apache.hadoop.hbase.regionserver.wal.FSHLog;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogWriter;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
|
||||
/**
|
||||
* A WAL Provider that returns a single thread safe WAL that writes to HDFS.
|
||||
* By default, this implementation picks a directory in HDFS based on a combination of
|
||||
* <ul>
|
||||
* <li>the HBase root directory
|
||||
* <li>HConstants.HREGION_LOGDIR_NAME
|
||||
* <li>the given factory's factoryId (usually identifying the regionserver by host:port)
|
||||
* </ul>
|
||||
* It also uses the providerId to diffentiate among files.
|
||||
*
|
||||
* A WAL provider that use {@link FSHLog}.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Evolving
|
||||
public class DefaultWALProvider implements WALProvider {
|
||||
private static final Log LOG = LogFactory.getLog(DefaultWALProvider.class);
|
||||
public class DefaultWALProvider extends AbstractFSWALProvider<FSHLog> {
|
||||
|
||||
// Only public so classes back in regionserver.wal can access
|
||||
public interface Reader extends WAL.Reader {
|
||||
/**
|
||||
* @param fs File system.
|
||||
* @param path Path.
|
||||
* @param c Configuration.
|
||||
* @param s Input stream that may have been pre-opened by the caller; may be null.
|
||||
*/
|
||||
void init(FileSystem fs, Path path, Configuration c, FSDataInputStream s) throws IOException;
|
||||
}
|
||||
private static final Log LOG = LogFactory.getLog(DefaultWALProvider.class);
|
||||
|
||||
// Only public so classes back in regionserver.wal can access
|
||||
public interface Writer extends WALProvider.Writer {
|
||||
void init(FileSystem fs, Path path, Configuration c, boolean overwritable) throws IOException;
|
||||
}
|
||||
|
||||
protected volatile FSHLog log = null;
|
||||
private WALFactory factory = null;
|
||||
private Configuration conf = null;
|
||||
private List<WALActionsListener> listeners = null;
|
||||
private String providerId = null;
|
||||
private AtomicBoolean initialized = new AtomicBoolean(false);
|
||||
// for default wal provider, logPrefix won't change
|
||||
private String logPrefix = null;
|
||||
|
||||
/**
|
||||
* we synchronized on walCreateLock to prevent wal recreation in different threads
|
||||
*/
|
||||
private final Object walCreateLock = new Object();
|
||||
|
||||
/**
|
||||
* @param factory factory that made us, identity used for FS layout. may not be null
|
||||
* @param conf may not be null
|
||||
* @param listeners may be null
|
||||
* @param providerId differentiate between providers from one facotry, used for FS layout. may be
|
||||
* null
|
||||
*/
|
||||
@Override
|
||||
public void init(final WALFactory factory, final Configuration conf,
|
||||
final List<WALActionsListener> listeners, String providerId) throws IOException {
|
||||
if (!initialized.compareAndSet(false, true)) {
|
||||
throw new IllegalStateException("WALProvider.init should only be called once.");
|
||||
}
|
||||
this.factory = factory;
|
||||
this.conf = conf;
|
||||
this.listeners = listeners;
|
||||
this.providerId = providerId;
|
||||
// get log prefix
|
||||
StringBuilder sb = new StringBuilder().append(factory.factoryId);
|
||||
if (providerId != null) {
|
||||
if (providerId.startsWith(WAL_FILE_NAME_DELIMITER)) {
|
||||
sb.append(providerId);
|
||||
} else {
|
||||
sb.append(WAL_FILE_NAME_DELIMITER).append(providerId);
|
||||
}
|
||||
}
|
||||
logPrefix = sb.toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public WAL getWAL(final byte[] identifier, byte[] namespace) throws IOException {
|
||||
if (log == null) {
|
||||
// only lock when need to create wal, and need to lock since
|
||||
// creating hlog on fs is time consuming
|
||||
synchronized (walCreateLock) {
|
||||
if (log == null) {
|
||||
log = new FSHLog(FileSystem.get(conf), FSUtils.getRootDir(conf),
|
||||
getWALDirectoryName(factory.factoryId), HConstants.HREGION_OLDLOGDIR_NAME, conf,
|
||||
listeners, true, logPrefix,
|
||||
META_WAL_PROVIDER_ID.equals(providerId) ? META_WAL_PROVIDER_ID : null);
|
||||
}
|
||||
}
|
||||
}
|
||||
return log;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (log != null) log.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void shutdown() throws IOException {
|
||||
if (log != null) log.shutdown();
|
||||
}
|
||||
|
||||
// should be package private; more visible for use in FSHLog
|
||||
public static final String WAL_FILE_NAME_DELIMITER = ".";
|
||||
/** The hbase:meta region's WAL filename extension */
|
||||
@VisibleForTesting
|
||||
public static final String META_WAL_PROVIDER_ID = ".meta";
|
||||
static final String DEFAULT_PROVIDER_ID = "default";
|
||||
|
||||
// Implementation details that currently leak in tests or elsewhere follow
|
||||
/** File Extension used while splitting an WAL into regions (HBASE-2312) */
|
||||
public static final String SPLITTING_EXT = "-splitting";
|
||||
|
||||
/**
|
||||
* iff the given WALFactory is using the DefaultWALProvider for meta and/or non-meta,
|
||||
* count the number of files (rolled and active). if either of them aren't, count 0
|
||||
* for that provider.
|
||||
*/
|
||||
@Override
|
||||
public long getNumLogFiles() {
|
||||
return log == null ? 0 : this.log.getNumLogFiles();
|
||||
}
|
||||
|
||||
/**
|
||||
* iff the given WALFactory is using the DefaultWALProvider for meta and/or non-meta,
|
||||
* count the size of files (rolled and active). if either of them aren't, count 0
|
||||
* for that provider.
|
||||
*/
|
||||
@Override
|
||||
public long getLogFileSize() {
|
||||
return log == null ? 0 : this.log.getLogFileSize();
|
||||
}
|
||||
|
||||
/**
|
||||
* returns the number of rolled WAL files.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public static int getNumRolledLogFiles(WAL wal) {
|
||||
return ((FSHLog)wal).getNumRolledLogFiles();
|
||||
}
|
||||
|
||||
/**
|
||||
* return the current filename from the current wal.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public static Path getCurrentFileName(final WAL wal) {
|
||||
return ((FSHLog)wal).getCurrentFileName();
|
||||
}
|
||||
|
||||
/**
|
||||
* request a log roll, but don't actually do it.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
static void requestLogRoll(final WAL wal) {
|
||||
((FSHLog)wal).requestLogRoll();
|
||||
}
|
||||
|
||||
/**
|
||||
* It returns the file create timestamp from the file name.
|
||||
* For name format see {@link #validateWALFilename(String)}
|
||||
* public until remaining tests move to o.a.h.h.wal
|
||||
* @param wal must not be null
|
||||
* @return the file number that is part of the WAL file name
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public static long extractFileNumFromWAL(final WAL wal) {
|
||||
final Path walName = ((FSHLog)wal).getCurrentFileName();
|
||||
if (walName == null) {
|
||||
throw new IllegalArgumentException("The WAL path couldn't be null");
|
||||
}
|
||||
final String[] walPathStrs = walName.toString().split("\\" + WAL_FILE_NAME_DELIMITER);
|
||||
return Long.parseLong(walPathStrs[walPathStrs.length - (isMetaFile(walName) ? 2:1)]);
|
||||
}
|
||||
|
||||
/**
|
||||
* Pattern used to validate a WAL file name
|
||||
* see {@link #validateWALFilename(String)} for description.
|
||||
*/
|
||||
private static final Pattern pattern = Pattern.compile(".*\\.\\d*("+META_WAL_PROVIDER_ID+")*");
|
||||
|
||||
/**
|
||||
* A WAL file name is of the format:
|
||||
* <wal-name>{@link #WAL_FILE_NAME_DELIMITER}<file-creation-timestamp>[.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://<name node>/hbase/.logs/<server name>-splitting/...</li>
|
||||
* <li>hdfs://<name node>/hbase/.logs/<server name>/...</li>
|
||||
* </ul>
|
||||
* @param logFile
|
||||
* @return null if the passed in logFile isn't a valid WAL file path
|
||||
*/
|
||||
public static ServerName getServerNameFromWALDirectoryName(Path logFile) {
|
||||
String logDirName = logFile.getParent().getName();
|
||||
// We were passed the directory and not a file in it.
|
||||
if (logDirName.equals(HConstants.HREGION_LOGDIR_NAME)) {
|
||||
logDirName = logFile.getName();
|
||||
}
|
||||
ServerName serverName = null;
|
||||
if (logDirName.endsWith(SPLITTING_EXT)) {
|
||||
logDirName = logDirName.substring(0, logDirName.length() - SPLITTING_EXT.length());
|
||||
}
|
||||
try {
|
||||
serverName = ServerName.parseServerName(logDirName);
|
||||
} catch (IllegalArgumentException ex) {
|
||||
serverName = null;
|
||||
LOG.warn("Cannot parse a server name from path=" + logFile + "; " + ex.getMessage());
|
||||
}
|
||||
if (serverName != null && serverName.getStartcode() < 0) {
|
||||
LOG.warn("Invalid log file path=" + logFile);
|
||||
serverName = null;
|
||||
}
|
||||
return serverName;
|
||||
}
|
||||
|
||||
public static boolean isMetaFile(Path p) {
|
||||
return isMetaFile(p.getName());
|
||||
}
|
||||
|
||||
public static boolean isMetaFile(String p) {
|
||||
if (p != null && p.endsWith(META_WAL_PROVIDER_ID)) {
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* public because of FSHLog. Should be package-private
|
||||
*/
|
||||
public static Writer createWriter(final Configuration conf, final FileSystem fs, final Path path,
|
||||
final boolean overwritable)
|
||||
throws IOException {
|
||||
final boolean overwritable) throws IOException {
|
||||
// Configuration already does caching for the Class lookup.
|
||||
Class<? extends Writer> logWriterClass = conf.getClass("hbase.regionserver.hlog.writer.impl",
|
||||
ProtobufLogWriter.class, Writer.class);
|
||||
|
@ -380,15 +65,14 @@ public class DefaultWALProvider implements WALProvider {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get prefix of the log from its name, assuming WAL name in format of
|
||||
* log_prefix.filenumber.log_suffix @see {@link FSHLog#getCurrentFileName()}
|
||||
* @param name Name of the WAL to parse
|
||||
* @return prefix of the log
|
||||
*/
|
||||
public static String getWALPrefixFromWALName(String name) {
|
||||
int endIndex = name.replaceAll(META_WAL_PROVIDER_ID, "").lastIndexOf(".");
|
||||
return name.substring(0, endIndex);
|
||||
@Override
|
||||
protected FSHLog createWAL() throws IOException {
|
||||
return new FSHLog(FileSystem.get(conf), FSUtils.getRootDir(conf),
|
||||
getWALDirectoryName(factory.factoryId), HConstants.HREGION_OLDLOGDIR_NAME, conf, listeners,
|
||||
true, logPrefix, META_WAL_PROVIDER_ID.equals(providerId) ? META_WAL_PROVIDER_ID : null);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doInit(Configuration conf) throws IOException {
|
||||
}
|
||||
}
|
||||
|
|
|
@ -215,6 +215,10 @@ class DisabledWALProvider implements WALProvider {
|
|||
public String toString() {
|
||||
return "WAL disabled.";
|
||||
}
|
||||
|
||||
@Override
|
||||
public void logRollerExited() {
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -198,6 +198,13 @@ public interface WAL {
|
|||
@Override
|
||||
String toString();
|
||||
|
||||
/**
|
||||
* In some WAL implementation, we will write WAL entries to new file if sync failed, which means,
|
||||
* the fail recovery is depended on log roller. So here we tell the WAL that log roller has
|
||||
* already been exited so the WAL cloud give up recovery.
|
||||
*/
|
||||
void logRollerExited();
|
||||
|
||||
/**
|
||||
* When outside clients need to consume persisted WALs, they rely on a provided
|
||||
* Reader.
|
||||
|
@ -268,7 +275,5 @@ public interface WAL {
|
|||
public String toString() {
|
||||
return this.key + "=" + this.edit;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -76,7 +76,8 @@ public class WALFactory {
|
|||
static enum Providers {
|
||||
defaultProvider(DefaultWALProvider.class),
|
||||
filesystem(DefaultWALProvider.class),
|
||||
multiwal(RegionGroupingProvider.class);
|
||||
multiwal(RegionGroupingProvider.class),
|
||||
asyncfs(AsyncFSWALProvider.class);
|
||||
|
||||
Class<? extends WALProvider> clazz;
|
||||
Providers(Class<? extends WALProvider> clazz) {
|
||||
|
@ -350,6 +351,7 @@ public class WALFactory {
|
|||
|
||||
/**
|
||||
* Create a writer for the WAL.
|
||||
* <p>
|
||||
* should be package-private. public only for tests and
|
||||
* {@link org.apache.hadoop.hbase.regionserver.wal.Compressor}
|
||||
* @return A WAL writer. Close when done with it.
|
||||
|
|
|
@ -20,11 +20,11 @@ package org.apache.hadoop.hbase.wal;
|
|||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.nio.channels.CompletionHandler;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
// imports for things that haven't moved from regionserver.wal yet.
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
|
||||
|
||||
|
@ -80,6 +80,12 @@ public interface WALProvider {
|
|||
long getLength() throws IOException;
|
||||
}
|
||||
|
||||
interface AsyncWriter extends Closeable {
|
||||
<A> void sync(CompletionHandler<Long, A> handler, A attachment);
|
||||
void append(WAL.Entry entry);
|
||||
long getLength();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get number of the log files this provider is managing
|
||||
*/
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
|
@ -27,12 +27,15 @@ import java.util.TreeMap;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.io.util.LRUDictionary;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.wal.DefaultWALProvider;
|
||||
import org.apache.hadoop.hbase.wal.WAL;
|
||||
import org.apache.hadoop.io.SequenceFile;
|
||||
import org.apache.hadoop.io.SequenceFile.CompressionType;
|
||||
|
@ -51,7 +54,7 @@ import org.apache.hadoop.io.compress.DefaultCodec;
|
|||
* Hadoop serialization).
|
||||
*/
|
||||
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
|
||||
public class SequenceFileLogWriter extends WriterBase {
|
||||
public class SequenceFileLogWriter implements DefaultWALProvider.Writer {
|
||||
private static final Log LOG = LogFactory.getLog(SequenceFileLogWriter.class);
|
||||
// The sequence file we delegate to.
|
||||
private SequenceFile.Writer writer;
|
||||
|
@ -59,6 +62,8 @@ public class SequenceFileLogWriter extends WriterBase {
|
|||
// in the SequenceFile.Writer 'writer' instance above.
|
||||
private FSDataOutputStream writer_out;
|
||||
|
||||
private CompressionContext compressionContext;
|
||||
|
||||
// Legacy stuff from pre-PB WAL metadata.
|
||||
private static final Text WAL_VERSION_KEY = new Text("version");
|
||||
private static final Text WAL_COMPRESSION_TYPE_KEY = new Text("compression.type");
|
||||
|
@ -88,10 +93,23 @@ public class SequenceFileLogWriter extends WriterBase {
|
|||
return new Metadata(metaMap);
|
||||
}
|
||||
|
||||
private boolean initializeCompressionContext(Configuration conf, Path path) throws IOException {
|
||||
boolean doCompress = conf.getBoolean(HConstants.ENABLE_WAL_COMPRESSION, false);
|
||||
if (doCompress) {
|
||||
try {
|
||||
this.compressionContext = new CompressionContext(LRUDictionary.class,
|
||||
FSUtils.isRecoveredEdits(path), conf.getBoolean(
|
||||
CompressionContext.ENABLE_WAL_TAGS_COMPRESSION, true));
|
||||
} catch (Exception e) {
|
||||
throw new IOException("Failed to initiate CompressionContext", e);
|
||||
}
|
||||
}
|
||||
return doCompress;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void init(FileSystem fs, Path path, Configuration conf, boolean overwritable)
|
||||
throws IOException {
|
||||
super.init(fs, path, conf, overwritable);
|
||||
boolean compress = initializeCompressionContext(conf, path);
|
||||
// Create a SF.Writer instance.
|
||||
try {
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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));
|
||||
}
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -22,6 +22,7 @@ import static org.junit.Assert.assertEquals;
|
|||
import static org.junit.Assert.assertNull;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
|
@ -30,28 +31,35 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
|
|||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Durability;
|
||||
import org.apache.hadoop.hbase.client.Increment;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.wal.DefaultWALProvider;
|
||||
import org.apache.hadoop.hbase.wal.WAL;
|
||||
import org.apache.hadoop.hbase.wal.WALFactory;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
import org.junit.runners.Parameterized.Parameter;
|
||||
import org.junit.runners.Parameterized.Parameters;
|
||||
|
||||
/**
|
||||
* Tests for WAL write durability
|
||||
*/
|
||||
@RunWith(Parameterized.class)
|
||||
@Category({RegionServerTests.class, MediumTests.class})
|
||||
public class TestDurability {
|
||||
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||
|
@ -64,6 +72,13 @@ public class TestDurability {
|
|||
private static byte[] ROW = Bytes.toBytes("row");
|
||||
private static byte[] COL = Bytes.toBytes("col");
|
||||
|
||||
@Parameter
|
||||
public String walProvider;
|
||||
|
||||
@Parameters(name = "{index}: provider={0}")
|
||||
public static Iterable<Object[]> data() {
|
||||
return Arrays.asList(new Object[] { "defaultProvider" }, new Object[] { "asyncfs" });
|
||||
}
|
||||
|
||||
@BeforeClass
|
||||
public static void setUpBeforeClass() throws Exception {
|
||||
|
@ -81,6 +96,16 @@ public class TestDurability {
|
|||
TEST_UTIL.shutdownMiniCluster();
|
||||
}
|
||||
|
||||
@Before
|
||||
public void setUp() {
|
||||
CONF.set(WALFactory.WAL_PROVIDER, walProvider);
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws IOException {
|
||||
FS.delete(DIR, true);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDurability() throws Exception {
|
||||
final WALFactory wals = new WALFactory(CONF, null, "TestDurability");
|
||||
|
|
|
@ -65,11 +65,11 @@ import org.junit.experimental.categories.Category;
|
|||
*/
|
||||
@Category({RegionServerTests.class, MediumTests.class})
|
||||
public class TestLogRollAbort {
|
||||
private static final Log LOG = LogFactory.getLog(TestLogRolling.class);
|
||||
private static final Log LOG = LogFactory.getLog(AbstractTestLogRolling.class);
|
||||
private static MiniDFSCluster dfsCluster;
|
||||
private static Admin admin;
|
||||
private static MiniHBaseCluster cluster;
|
||||
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||
protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||
|
||||
/* For the split-then-roll test */
|
||||
private static final Path HBASEDIR = new Path("/hbase");
|
||||
|
@ -212,7 +212,7 @@ public class TestLogRollAbort {
|
|||
}
|
||||
// Send the data to HDFS datanodes and close the HDFS writer
|
||||
log.sync();
|
||||
((FSHLog) log).replaceWriter(((FSHLog)log).getOldPath(), null, null, null);
|
||||
((AbstractFSWAL<?>) log).replaceWriter(((FSHLog)log).getOldPath(), null, null);
|
||||
|
||||
/* code taken from MasterFileSystem.getLogDirs(), which is called from MasterFileSystem.splitLog()
|
||||
* handles RS shutdowns (as observed by the splitting process)
|
||||
|
|
|
@ -19,21 +19,21 @@ package org.apache.hadoop.hbase.regionserver.wal;
|
|||
|
||||
import static org.junit.Assert.assertFalse;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.wal.WAL;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.wal.WAL;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
@ -44,9 +44,9 @@ import org.junit.experimental.categories.Category;
|
|||
*/
|
||||
@Category({RegionServerTests.class, MediumTests.class})
|
||||
public class TestLogRollPeriod {
|
||||
private static final Log LOG = LogFactory.getLog(TestLogRolling.class);
|
||||
private static final Log LOG = LogFactory.getLog(AbstractTestLogRolling.class);
|
||||
|
||||
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||
protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||
|
||||
private final static long LOG_ROLL_PERIOD = 4000;
|
||||
|
||||
|
|
|
@ -1,5 +1,4 @@
|
|||
/**
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
|
@ -24,7 +23,6 @@ import static org.junit.Assert.assertNotNull;
|
|||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.EOFException;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
|
@ -33,111 +31,45 @@ import java.util.concurrent.atomic.AtomicBoolean;
|
|||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.MiniHBaseCluster;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
import org.apache.hadoop.hbase.client.HTable;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.ResultScanner;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.fs.HFileSystem;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.regionserver.Region;
|
||||
import org.apache.hadoop.hbase.regionserver.Store;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.testclassification.VerySlowRegionServerTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.JVMClusterUtil;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
import org.apache.hadoop.hbase.wal.DefaultWALProvider;
|
||||
import org.apache.hadoop.hbase.wal.WAL;
|
||||
import org.apache.hadoop.hbase.wal.WALFactory;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.rules.TestName;
|
||||
|
||||
/**
|
||||
* Test log deletion as logs are rolled.
|
||||
*/
|
||||
@Category({ VerySlowRegionServerTests.class, LargeTests.class })
|
||||
public class TestLogRolling {
|
||||
public class TestLogRolling extends AbstractTestLogRolling {
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(TestLogRolling.class);
|
||||
private HRegionServer server;
|
||||
private String tableName;
|
||||
private byte[] value;
|
||||
private FileSystem fs;
|
||||
private MiniDFSCluster dfsCluster;
|
||||
private Admin admin;
|
||||
private MiniHBaseCluster cluster;
|
||||
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||
@Rule public final TestName name = new TestName();
|
||||
|
||||
public TestLogRolling() {
|
||||
this.server = null;
|
||||
this.tableName = null;
|
||||
|
||||
String className = this.getClass().getName();
|
||||
StringBuilder v = new StringBuilder(className);
|
||||
while (v.length() < 1000) {
|
||||
v.append(className);
|
||||
}
|
||||
this.value = Bytes.toBytes(v.toString());
|
||||
}
|
||||
|
||||
// Need to override this setup so we can edit the config before it gets sent
|
||||
// to the HDFS & HBase cluster startup.
|
||||
@BeforeClass
|
||||
public static void setUpBeforeClass() throws Exception {
|
||||
// TODO: testLogRollOnDatanodeDeath fails if short circuit reads are on under the hadoop2
|
||||
// profile. See HBASE-9337 for related issues.
|
||||
System.setProperty("hbase.tests.use.shortcircuit.reads", "false");
|
||||
|
||||
/**** configuration for testLogRolling ****/
|
||||
// Force a region split after every 768KB
|
||||
TEST_UTIL.getConfiguration().setLong(HConstants.HREGION_MAX_FILESIZE, 768L * 1024L);
|
||||
|
||||
// We roll the log after every 32 writes
|
||||
TEST_UTIL.getConfiguration().setInt("hbase.regionserver.maxlogentries", 32);
|
||||
|
||||
TEST_UTIL.getConfiguration().setInt("hbase.regionserver.logroll.errors.tolerated", 2);
|
||||
TEST_UTIL.getConfiguration().setInt("hbase.rpc.timeout", 10 * 1000);
|
||||
|
||||
// For less frequently updated regions flush after every 2 flushes
|
||||
TEST_UTIL.getConfiguration().setInt("hbase.hregion.memstore.optionalflushcount", 2);
|
||||
|
||||
// We flush the cache after every 8192 bytes
|
||||
TEST_UTIL.getConfiguration().setInt(
|
||||
HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 8192);
|
||||
|
||||
// Increase the amount of time between client retries
|
||||
TEST_UTIL.getConfiguration().setLong("hbase.client.pause", 10 * 1000);
|
||||
|
||||
// Reduce thread wake frequency so that other threads can get
|
||||
// a chance to run.
|
||||
TEST_UTIL.getConfiguration().setInt(HConstants.THREAD_WAKE_FREQUENCY, 2 * 1000);
|
||||
|
||||
/**** configuration for testLogRollOnDatanodeDeath ****/
|
||||
// make sure log.hflush() calls syncFs() to open a pipeline
|
||||
TEST_UTIL.getConfiguration().setBoolean("dfs.support.append", true);
|
||||
|
@ -150,158 +82,19 @@ public class TestLogRolling {
|
|||
TEST_UTIL.getConfiguration().setInt("dfs.client.block.write.retries", 30);
|
||||
TEST_UTIL.getConfiguration().setInt("hbase.regionserver.hlog.tolerable.lowreplication", 2);
|
||||
TEST_UTIL.getConfiguration().setInt("hbase.regionserver.hlog.lowreplication.rolllimit", 3);
|
||||
}
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
TEST_UTIL.startMiniCluster(1, 1, 2);
|
||||
|
||||
cluster = TEST_UTIL.getHBaseCluster();
|
||||
dfsCluster = TEST_UTIL.getDFSCluster();
|
||||
fs = TEST_UTIL.getTestFileSystem();
|
||||
admin = TEST_UTIL.getHBaseAdmin();
|
||||
|
||||
// disable region rebalancing (interferes with log watching)
|
||||
cluster.getMaster().balanceSwitch(false);
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
TEST_UTIL.shutdownMiniCluster();
|
||||
}
|
||||
|
||||
private void startAndWriteData() throws IOException, InterruptedException {
|
||||
// When the hbase:meta table can be opened, the region servers are running
|
||||
TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME);
|
||||
this.server = cluster.getRegionServerThreads().get(0).getRegionServer();
|
||||
|
||||
Table table = createTestTable(this.tableName);
|
||||
|
||||
server = TEST_UTIL.getRSForFirstRegionInTable(table.getName());
|
||||
for (int i = 1; i <= 256; i++) { // 256 writes should cause 8 log rolls
|
||||
doPut(table, i);
|
||||
if (i % 32 == 0) {
|
||||
// After every 32 writes sleep to let the log roller run
|
||||
try {
|
||||
Thread.sleep(2000);
|
||||
} catch (InterruptedException e) {
|
||||
// continue
|
||||
}
|
||||
}
|
||||
}
|
||||
AbstractTestLogRolling.setUpBeforeClass();
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests that log rolling doesn't hang when no data is written.
|
||||
*/
|
||||
@Test(timeout=120000)
|
||||
public void testLogRollOnNothingWritten() throws Exception {
|
||||
final Configuration conf = TEST_UTIL.getConfiguration();
|
||||
final WALFactory wals = new WALFactory(conf, null,
|
||||
ServerName.valueOf("test.com",8080, 1).toString());
|
||||
final WAL newLog = wals.getWAL(new byte[]{}, null);
|
||||
try {
|
||||
// Now roll the log before we write anything.
|
||||
newLog.rollWriter(true);
|
||||
} finally {
|
||||
wals.close();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests that logs are deleted
|
||||
* @throws IOException
|
||||
* @throws org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException
|
||||
*/
|
||||
@Test
|
||||
public void testLogRolling() throws Exception {
|
||||
this.tableName = getName();
|
||||
// TODO: Why does this write data take for ever?
|
||||
startAndWriteData();
|
||||
HRegionInfo region =
|
||||
server.getOnlineRegions(TableName.valueOf(tableName)).get(0).getRegionInfo();
|
||||
final WAL log = server.getWAL(region);
|
||||
LOG.info("after writing there are " + DefaultWALProvider.getNumRolledLogFiles(log) +
|
||||
" log files");
|
||||
|
||||
// flush all regions
|
||||
for (Region r: server.getOnlineRegionsLocalContext()) {
|
||||
r.flush(true);
|
||||
}
|
||||
|
||||
// Now roll the log
|
||||
log.rollWriter();
|
||||
|
||||
int count = DefaultWALProvider.getNumRolledLogFiles(log);
|
||||
LOG.info("after flushing all regions and rolling logs there are " + count + " log files");
|
||||
assertTrue(("actual count: " + count), count <= 2);
|
||||
}
|
||||
|
||||
private String getName() {
|
||||
return "TestLogRolling-" + name.getMethodName();
|
||||
}
|
||||
|
||||
void writeData(Table table, int rownum) throws IOException {
|
||||
doPut(table, rownum);
|
||||
|
||||
// sleep to let the log roller run (if it needs to)
|
||||
try {
|
||||
Thread.sleep(2000);
|
||||
} catch (InterruptedException e) {
|
||||
// continue
|
||||
}
|
||||
}
|
||||
|
||||
void validateData(Table table, int rownum) throws IOException {
|
||||
String row = "row" + String.format("%1$04d", rownum);
|
||||
Get get = new Get(Bytes.toBytes(row));
|
||||
get.addFamily(HConstants.CATALOG_FAMILY);
|
||||
Result result = table.get(get);
|
||||
assertTrue(result.size() == 1);
|
||||
assertTrue(Bytes.equals(value,
|
||||
result.getValue(HConstants.CATALOG_FAMILY, null)));
|
||||
LOG.info("Validated row " + row);
|
||||
}
|
||||
|
||||
void batchWriteAndWait(Table table, final FSHLog log, int start, boolean expect, int timeout)
|
||||
throws IOException {
|
||||
for (int i = 0; i < 10; i++) {
|
||||
Put put = new Put(Bytes.toBytes("row"
|
||||
+ String.format("%1$04d", (start + i))));
|
||||
put.addColumn(HConstants.CATALOG_FAMILY, null, value);
|
||||
table.put(put);
|
||||
}
|
||||
Put tmpPut = new Put(Bytes.toBytes("tmprow"));
|
||||
tmpPut.addColumn(HConstants.CATALOG_FAMILY, null, value);
|
||||
long startTime = System.currentTimeMillis();
|
||||
long remaining = timeout;
|
||||
while (remaining > 0) {
|
||||
if (log.isLowReplicationRollEnabled() == expect) {
|
||||
break;
|
||||
} else {
|
||||
// Trigger calling FSHlog#checkLowReplication()
|
||||
table.put(tmpPut);
|
||||
try {
|
||||
Thread.sleep(200);
|
||||
} catch (InterruptedException e) {
|
||||
// continue
|
||||
}
|
||||
remaining = timeout - (System.currentTimeMillis() - startTime);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests that logs are rolled upon detecting datanode death
|
||||
* Requires an HDFS jar with HDFS-826 & syncFs() support (HDFS-200)
|
||||
* Tests that logs are rolled upon detecting datanode death Requires an HDFS jar with HDFS-826 &
|
||||
* syncFs() support (HDFS-200)
|
||||
*/
|
||||
@Test
|
||||
public void testLogRollOnDatanodeDeath() throws Exception {
|
||||
TEST_UTIL.ensureSomeRegionServersAvailable(2);
|
||||
assertTrue("This test requires WAL file replication set to 2.",
|
||||
fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()) == 2);
|
||||
LOG.info("Replication=" +
|
||||
fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()));
|
||||
LOG.info("Replication=" + fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()));
|
||||
|
||||
this.server = cluster.getRegionServer(0);
|
||||
|
||||
|
@ -328,8 +121,8 @@ public class TestLogRolling {
|
|||
});
|
||||
|
||||
// don't run this test without append support (HDFS-200 & HDFS-142)
|
||||
assertTrue("Need append support for this test", FSUtils
|
||||
.isAppendSupported(TEST_UTIL.getConfiguration()));
|
||||
assertTrue("Need append support for this test",
|
||||
FSUtils.isAppendSupported(TEST_UTIL.getConfiguration()));
|
||||
|
||||
// add up the datanode count, to ensure proper replication when we kill 1
|
||||
// This function is synchronous; when it returns, the dfs cluster is active
|
||||
|
@ -337,8 +130,7 @@ public class TestLogRolling {
|
|||
// when we stop/start a namenode later, as mentioned in HBASE-5163
|
||||
List<DataNode> existingNodes = dfsCluster.getDataNodes();
|
||||
int numDataNodes = 3;
|
||||
dfsCluster.startDataNodes(TEST_UTIL.getConfiguration(), numDataNodes, true,
|
||||
null, null);
|
||||
dfsCluster.startDataNodes(TEST_UTIL.getConfiguration(), numDataNodes, true, null, null);
|
||||
List<DataNode> allNodes = dfsCluster.getDataNodes();
|
||||
for (int i = allNodes.size() - 1; i >= 0; i--) {
|
||||
if (existingNodes.contains(allNodes.get(i))) {
|
||||
|
@ -346,11 +138,11 @@ public class TestLogRolling {
|
|||
}
|
||||
}
|
||||
|
||||
assertTrue("DataNodes " + dfsCluster.getDataNodes().size() +
|
||||
" default replication " +
|
||||
fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()),
|
||||
dfsCluster.getDataNodes().size() >=
|
||||
fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()) + 1);
|
||||
assertTrue(
|
||||
"DataNodes " + dfsCluster.getDataNodes().size() + " default replication "
|
||||
+ fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()),
|
||||
dfsCluster.getDataNodes()
|
||||
.size() >= fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()) + 1);
|
||||
|
||||
writeData(table, 2);
|
||||
|
||||
|
@ -362,9 +154,8 @@ public class TestLogRolling {
|
|||
|
||||
assertTrue("The log shouldn't have rolled yet",
|
||||
oldFilenum == DefaultWALProvider.extractFileNumFromWAL(log));
|
||||
final DatanodeInfo[] pipeline = log.getPipeLine();
|
||||
assertTrue(pipeline.length ==
|
||||
fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()));
|
||||
final DatanodeInfo[] pipeline = log.getPipeline();
|
||||
assertTrue(pipeline.length == fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()));
|
||||
|
||||
// kill a datanode in the pipeline to force a log roll on the next sync()
|
||||
// This function is synchronous, when it returns the node is killed.
|
||||
|
@ -390,26 +181,23 @@ public class TestLogRolling {
|
|||
|
||||
batchWriteAndWait(table, log, 3, false, 14000);
|
||||
int replication = log.getLogReplication();
|
||||
assertTrue("LowReplication Roller should've been disabled, current replication="
|
||||
+ replication, !log.isLowReplicationRollEnabled());
|
||||
assertTrue("LowReplication Roller should've been disabled, current replication=" + replication,
|
||||
!log.isLowReplicationRollEnabled());
|
||||
|
||||
dfsCluster
|
||||
.startDataNodes(TEST_UTIL.getConfiguration(), 1, true, null, null);
|
||||
dfsCluster.startDataNodes(TEST_UTIL.getConfiguration(), 1, true, null, null);
|
||||
|
||||
// Force roll writer. The new log file will have the default replications,
|
||||
// and the LowReplication Roller will be enabled.
|
||||
log.rollWriter(true);
|
||||
batchWriteAndWait(table, log, 13, true, 10000);
|
||||
replication = log.getLogReplication();
|
||||
assertTrue("New log file should have the default replication instead of " +
|
||||
replication,
|
||||
assertTrue("New log file should have the default replication instead of " + replication,
|
||||
replication == fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()));
|
||||
assertTrue("LowReplication Roller should've been enabled", log.isLowReplicationRollEnabled());
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that WAL is rolled when all data nodes in the pipeline have been
|
||||
* restarted.
|
||||
* Test that WAL is rolled when all data nodes in the pipeline have been restarted.
|
||||
* @throws Exception
|
||||
*/
|
||||
@Test
|
||||
|
@ -417,8 +205,7 @@ public class TestLogRolling {
|
|||
LOG.info("Starting testLogRollOnPipelineRestart");
|
||||
assertTrue("This test requires WAL file replication.",
|
||||
fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()) > 1);
|
||||
LOG.info("Replication=" +
|
||||
fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()));
|
||||
LOG.info("Replication=" + fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()));
|
||||
// When the hbase:meta table can be opened, the region servers are running
|
||||
Table t = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME);
|
||||
try {
|
||||
|
@ -445,6 +232,7 @@ public class TestLogRolling {
|
|||
LOG.debug("preLogRoll: oldFile=" + oldFile + " newFile=" + newFile);
|
||||
preLogRolledCalled.add(new Integer(1));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postLogRoll(Path oldFile, Path newFile) {
|
||||
paths.add(newFile);
|
||||
|
@ -452,8 +240,8 @@ public class TestLogRolling {
|
|||
});
|
||||
|
||||
// don't run this test without append support (HDFS-200 & HDFS-142)
|
||||
assertTrue("Need append support for this test", FSUtils
|
||||
.isAppendSupported(TEST_UTIL.getConfiguration()));
|
||||
assertTrue("Need append support for this test",
|
||||
FSUtils.isAppendSupported(TEST_UTIL.getConfiguration()));
|
||||
|
||||
writeData(table, 1002);
|
||||
|
||||
|
@ -463,8 +251,8 @@ public class TestLogRolling {
|
|||
assertTrue("Log should have a timestamp older than now",
|
||||
curTime > oldFilenum && oldFilenum != -1);
|
||||
|
||||
assertTrue("The log shouldn't have rolled yet", oldFilenum ==
|
||||
DefaultWALProvider.extractFileNumFromWAL(log));
|
||||
assertTrue("The log shouldn't have rolled yet",
|
||||
oldFilenum == DefaultWALProvider.extractFileNumFromWAL(log));
|
||||
|
||||
// roll all datanodes in the pipeline
|
||||
dfsCluster.restartDataNodes();
|
||||
|
@ -503,8 +291,8 @@ public class TestLogRolling {
|
|||
FSUtils fsUtils = FSUtils.getInstance(fs, TEST_UTIL.getConfiguration());
|
||||
for (Path p : paths) {
|
||||
LOG.debug("recovering lease for " + p);
|
||||
fsUtils.recoverFileLease(((HFileSystem)fs).getBackingFs(), p,
|
||||
TEST_UTIL.getConfiguration(), null);
|
||||
fsUtils.recoverFileLease(((HFileSystem) fs).getBackingFs(), p, TEST_UTIL.getConfiguration(),
|
||||
null);
|
||||
|
||||
LOG.debug("Reading WAL " + FSUtils.getPath(p));
|
||||
WAL.Reader reader = null;
|
||||
|
@ -512,10 +300,10 @@ public class TestLogRolling {
|
|||
reader = WALFactory.createReader(fs, p, TEST_UTIL.getConfiguration());
|
||||
WAL.Entry entry;
|
||||
while ((entry = reader.next()) != null) {
|
||||
LOG.debug("#"+entry.getKey().getLogSeqNum()+": "+entry.getEdit().getCells());
|
||||
LOG.debug("#" + entry.getKey().getSequenceId() + ": " + entry.getEdit().getCells());
|
||||
for (Cell cell : entry.getEdit().getCells()) {
|
||||
loggedRows.add(Bytes.toStringBinary(cell.getRowArray(), cell.getRowOffset(),
|
||||
cell.getRowLength()));
|
||||
loggedRows.add(
|
||||
Bytes.toStringBinary(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()));
|
||||
}
|
||||
}
|
||||
} catch (EOFException e) {
|
||||
|
@ -558,8 +346,8 @@ public class TestLogRolling {
|
|||
}
|
||||
|
||||
// verify that no region servers aborted
|
||||
for (JVMClusterUtil.RegionServerThread rsThread:
|
||||
TEST_UTIL.getHBaseCluster().getRegionServerThreads()) {
|
||||
for (JVMClusterUtil.RegionServerThread rsThread : TEST_UTIL.getHBaseCluster()
|
||||
.getRegionServerThreads()) {
|
||||
assertFalse(rsThread.getRegionServer().isAborted());
|
||||
}
|
||||
} finally {
|
||||
|
@ -567,80 +355,4 @@ public class TestLogRolling {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests that logs are deleted when some region has a compaction
|
||||
* record in WAL and no other records. See HBASE-8597.
|
||||
*/
|
||||
@Test
|
||||
public void testCompactionRecordDoesntBlockRolling() throws Exception {
|
||||
Table table = null;
|
||||
|
||||
// When the hbase:meta table can be opened, the region servers are running
|
||||
Table t = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME);
|
||||
try {
|
||||
table = createTestTable(getName());
|
||||
|
||||
server = TEST_UTIL.getRSForFirstRegionInTable(table.getName());
|
||||
Region region = server.getOnlineRegions(table.getName()).get(0);
|
||||
final WAL log = server.getWAL(region.getRegionInfo());
|
||||
Store s = region.getStore(HConstants.CATALOG_FAMILY);
|
||||
|
||||
//have to flush namespace to ensure it doesn't affect wall tests
|
||||
admin.flush(TableName.NAMESPACE_TABLE_NAME);
|
||||
|
||||
// Put some stuff into table, to make sure we have some files to compact.
|
||||
for (int i = 1; i <= 2; ++i) {
|
||||
doPut(table, i);
|
||||
admin.flush(table.getName());
|
||||
}
|
||||
doPut(table, 3); // don't flush yet, or compaction might trigger before we roll WAL
|
||||
assertEquals("Should have no WAL after initial writes", 0,
|
||||
DefaultWALProvider.getNumRolledLogFiles(log));
|
||||
assertEquals(2, s.getStorefilesCount());
|
||||
|
||||
// Roll the log and compact table, to have compaction record in the 2nd WAL.
|
||||
log.rollWriter();
|
||||
assertEquals("Should have WAL; one table is not flushed", 1,
|
||||
DefaultWALProvider.getNumRolledLogFiles(log));
|
||||
admin.flush(table.getName());
|
||||
region.compact(false);
|
||||
// Wait for compaction in case if flush triggered it before us.
|
||||
Assert.assertNotNull(s);
|
||||
for (int waitTime = 3000; s.getStorefilesCount() > 1 && waitTime > 0; waitTime -= 200) {
|
||||
Threads.sleepWithoutInterrupt(200);
|
||||
}
|
||||
assertEquals("Compaction didn't happen", 1, s.getStorefilesCount());
|
||||
|
||||
// Write some value to the table so the WAL cannot be deleted until table is flushed.
|
||||
doPut(table, 0); // Now 2nd WAL will have both compaction and put record for table.
|
||||
log.rollWriter(); // 1st WAL deleted, 2nd not deleted yet.
|
||||
assertEquals("Should have WAL; one table is not flushed", 1,
|
||||
DefaultWALProvider.getNumRolledLogFiles(log));
|
||||
|
||||
// Flush table to make latest WAL obsolete; write another record, and roll again.
|
||||
admin.flush(table.getName());
|
||||
doPut(table, 1);
|
||||
log.rollWriter(); // Now 2nd WAL is deleted and 3rd is added.
|
||||
assertEquals("Should have 1 WALs at the end", 1,
|
||||
DefaultWALProvider.getNumRolledLogFiles(log));
|
||||
} finally {
|
||||
if (t != null) t.close();
|
||||
if (table != null) table.close();
|
||||
}
|
||||
}
|
||||
|
||||
private void doPut(Table table, int i) throws IOException {
|
||||
Put put = new Put(Bytes.toBytes("row" + String.format("%1$04d", i)));
|
||||
put.addColumn(HConstants.CATALOG_FAMILY, null, value);
|
||||
table.put(put);
|
||||
}
|
||||
|
||||
private Table createTestTable(String tableName) throws IOException {
|
||||
// Create the test table and open it
|
||||
HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
|
||||
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
|
||||
admin.createTable(desc);
|
||||
return TEST_UTIL.getConnection().getTable(desc.getTableName());
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -1,5 +1,4 @@
|
|||
/**
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
|
@ -18,190 +17,32 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.regionserver.wal;
|
||||
|
||||
import static org.junit.Assert.assertArrayEquals;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
|
||||
import org.apache.hadoop.hbase.coprocessor.SampleRegionWALObserver;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.wal.WAL;
|
||||
import org.apache.hadoop.hbase.wal.WALFactory;
|
||||
import org.apache.hadoop.hbase.wal.WALKey;
|
||||
import org.apache.hadoop.hbase.wal.WAL.Entry;
|
||||
import org.apache.hadoop.hbase.wal.DefaultWALProvider;
|
||||
import org.apache.hadoop.hbase.wal.WALProvider;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.apache.hadoop.hbase.wal.WALProvider.Writer;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.rules.TestName;
|
||||
|
||||
/**
|
||||
* WAL tests that can be reused across providers.
|
||||
*/
|
||||
@Category({ RegionServerTests.class, MediumTests.class })
|
||||
public class TestProtobufLog {
|
||||
protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||
public class TestProtobufLog extends AbstractTestProtobufLog<WALProvider.Writer> {
|
||||
|
||||
protected FileSystem fs;
|
||||
protected Path dir;
|
||||
protected WALFactory wals;
|
||||
|
||||
@Rule
|
||||
public final TestName currentTest = new TestName();
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
fs = TEST_UTIL.getDFSCluster().getFileSystem();
|
||||
dir = new Path(TEST_UTIL.createRootDir(), currentTest.getMethodName());
|
||||
wals = new WALFactory(TEST_UTIL.getConfiguration(), null, currentTest.getMethodName());
|
||||
@Override
|
||||
protected Writer createWriter(Path path) throws IOException {
|
||||
return DefaultWALProvider.createWriter(TEST_UTIL.getConfiguration(), fs, path, false);
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
wals.close();
|
||||
FileStatus[] entries = fs.listStatus(new Path("/"));
|
||||
for (FileStatus dir : entries) {
|
||||
fs.delete(dir.getPath(), true);
|
||||
}
|
||||
@Override
|
||||
protected void append(Writer writer, Entry entry) throws IOException {
|
||||
writer.append(entry);
|
||||
}
|
||||
|
||||
@BeforeClass
|
||||
public static void setUpBeforeClass() throws Exception {
|
||||
// Make block sizes small.
|
||||
TEST_UTIL.getConfiguration().setInt("dfs.blocksize", 1024 * 1024);
|
||||
// needed for testAppendClose()
|
||||
TEST_UTIL.getConfiguration().setBoolean("dfs.support.broken.append", true);
|
||||
TEST_UTIL.getConfiguration().setBoolean("dfs.support.append", true);
|
||||
// quicker heartbeat interval for faster DN death notification
|
||||
TEST_UTIL.getConfiguration().setInt("dfs.namenode.heartbeat.recheck-interval", 5000);
|
||||
TEST_UTIL.getConfiguration().setInt("dfs.heartbeat.interval", 1);
|
||||
TEST_UTIL.getConfiguration().setInt("dfs.client.socket-timeout", 5000);
|
||||
|
||||
// faster failover with cluster.shutdown();fs.close() idiom
|
||||
TEST_UTIL.getConfiguration()
|
||||
.setInt("hbase.ipc.client.connect.max.retries", 1);
|
||||
TEST_UTIL.getConfiguration().setInt(
|
||||
"dfs.client.block.recovery.retries", 1);
|
||||
TEST_UTIL.getConfiguration().setInt(
|
||||
"hbase.ipc.client.connection.maxidletime", 500);
|
||||
TEST_UTIL.getConfiguration().set(CoprocessorHost.WAL_COPROCESSOR_CONF_KEY,
|
||||
SampleRegionWALObserver.class.getName());
|
||||
TEST_UTIL.startMiniDFSCluster(3);
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDownAfterClass() throws Exception {
|
||||
TEST_UTIL.shutdownMiniCluster();
|
||||
}
|
||||
|
||||
/**
|
||||
* Reads the WAL with and without WALTrailer.
|
||||
* @throws IOException
|
||||
*/
|
||||
@Test
|
||||
public void testWALTrailer() throws IOException {
|
||||
// read With trailer.
|
||||
doRead(true);
|
||||
// read without trailer
|
||||
doRead(false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Appends entries in the WAL and reads it.
|
||||
* @param withTrailer If 'withTrailer' is true, it calls a close on the WALwriter before reading
|
||||
* so that a trailer is appended to the WAL. Otherwise, it starts reading after the sync
|
||||
* call. This means that reader is not aware of the trailer. In this scenario, if the
|
||||
* reader tries to read the trailer in its next() call, it returns false from
|
||||
* ProtoBufLogReader.
|
||||
* @throws IOException
|
||||
*/
|
||||
private void doRead(boolean withTrailer) throws IOException {
|
||||
final int columnCount = 5;
|
||||
final int recordCount = 5;
|
||||
final TableName tableName =
|
||||
TableName.valueOf("tablename");
|
||||
final byte[] row = Bytes.toBytes("row");
|
||||
long timestamp = System.currentTimeMillis();
|
||||
Path path = new Path(dir, "tempwal");
|
||||
// delete the log if already exists, for test only
|
||||
fs.delete(path, true);
|
||||
WALProvider.Writer writer = null;
|
||||
ProtobufLogReader reader = null;
|
||||
try {
|
||||
HRegionInfo hri = new HRegionInfo(tableName,
|
||||
HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
|
||||
HTableDescriptor htd = new HTableDescriptor(tableName);
|
||||
fs.mkdirs(dir);
|
||||
// Write log in pb format.
|
||||
writer = wals.createWALWriter(fs, path);
|
||||
for (int i = 0; i < recordCount; ++i) {
|
||||
WALKey key = new WALKey(
|
||||
hri.getEncodedNameAsBytes(), tableName, i, timestamp, HConstants.DEFAULT_CLUSTER_ID);
|
||||
WALEdit edit = new WALEdit();
|
||||
for (int j = 0; j < columnCount; ++j) {
|
||||
if (i == 0) {
|
||||
htd.addFamily(new HColumnDescriptor("column" + j));
|
||||
}
|
||||
String value = i + "" + j;
|
||||
edit.add(new KeyValue(row, row, row, timestamp, Bytes.toBytes(value)));
|
||||
}
|
||||
writer.append(new WAL.Entry(key, edit));
|
||||
}
|
||||
@Override
|
||||
protected void sync(Writer writer) throws IOException {
|
||||
writer.sync();
|
||||
if (withTrailer) writer.close();
|
||||
|
||||
// Now read the log using standard means.
|
||||
reader = (ProtobufLogReader) wals.createReader(fs, path);
|
||||
if (withTrailer) {
|
||||
assertNotNull(reader.trailer);
|
||||
} else {
|
||||
assertNull(reader.trailer);
|
||||
}
|
||||
for (int i = 0; i < recordCount; ++i) {
|
||||
WAL.Entry entry = reader.next();
|
||||
assertNotNull(entry);
|
||||
assertEquals(columnCount, entry.getEdit().size());
|
||||
assertArrayEquals(hri.getEncodedNameAsBytes(), entry.getKey().getEncodedRegionName());
|
||||
assertEquals(tableName, entry.getKey().getTablename());
|
||||
int idx = 0;
|
||||
for (Cell val : entry.getEdit().getCells()) {
|
||||
assertTrue(Bytes.equals(row, 0, row.length, val.getRowArray(), val.getRowOffset(),
|
||||
val.getRowLength()));
|
||||
String value = i + "" + idx;
|
||||
assertArrayEquals(Bytes.toBytes(value), CellUtil.cloneValue(val));
|
||||
idx++;
|
||||
}
|
||||
}
|
||||
WAL.Entry entry = reader.next();
|
||||
assertNull(entry);
|
||||
} finally {
|
||||
if (writer != null) {
|
||||
writer.close();
|
||||
}
|
||||
if (reader != null) {
|
||||
reader.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -36,5 +36,4 @@ public class TestWALReplayCompressed extends TestWALReplay {
|
|||
Configuration conf = TestWALReplay.TEST_UTIL.getConfiguration();
|
||||
conf.setBoolean(HConstants.ENABLE_WAL_COMPRESSION, true);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue