HBASE-2059 Break out WAL reader and writer impl from HLog
git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@892451 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
7b8a829cec
commit
23cbf39343
|
@ -242,6 +242,7 @@ Release 0.21.0 - Unreleased
|
||||||
HBASE-2049 Cleanup HLog binary log output (Dave Latham via Stack)
|
HBASE-2049 Cleanup HLog binary log output (Dave Latham via Stack)
|
||||||
HBASE-2052 Make hbase more 'live' when comes to noticing table creation,
|
HBASE-2052 Make hbase more 'live' when comes to noticing table creation,
|
||||||
splits, etc., for 0.20.3
|
splits, etc., for 0.20.3
|
||||||
|
HBASE-2059 Break out WAL reader and writer impl from HLog
|
||||||
|
|
||||||
NEW FEATURES
|
NEW FEATURES
|
||||||
HBASE-1901 "General" partitioner for "hbase-48" bulk (behind the api, write
|
HBASE-1901 "General" partitioner for "hbase-48" bulk (behind the api, write
|
||||||
|
|
|
@ -190,6 +190,16 @@
|
||||||
<value>3600000</value>
|
<value>3600000</value>
|
||||||
<description>Period at which we will roll the commit log.</description>
|
<description>Period at which we will roll the commit log.</description>
|
||||||
</property>
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>hbase.regionserver.hlog.reader.impl</name>
|
||||||
|
<value>org.apache.hadoop.hbase.regionserver.wal.SequenceFileLogReader</value>
|
||||||
|
<description>The HLog file reader implementation.</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>hbase.regionserver.hlog.writer.impl</name>
|
||||||
|
<value>org.apache.hadoop.hbase.regionserver.wal.SequenceFileLogWriter</value>
|
||||||
|
<description>The HLog file writer implementation.</description>
|
||||||
|
</property>
|
||||||
<property>
|
<property>
|
||||||
<name>hbase.regionserver.thread.splitcompactcheckfrequency</name>
|
<name>hbase.regionserver.thread.splitcompactcheckfrequency</name>
|
||||||
<value>20000</value>
|
<value>20000</value>
|
||||||
|
|
|
@ -33,7 +33,6 @@ import org.apache.hadoop.hbase.client.Put;
|
||||||
import org.apache.hadoop.hbase.regionserver.wal.HLog;
|
import org.apache.hadoop.hbase.regionserver.wal.HLog;
|
||||||
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
|
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
|
||||||
import org.apache.hadoop.hbase.regionserver.wal.LogRollListener;
|
import org.apache.hadoop.hbase.regionserver.wal.LogRollListener;
|
||||||
import org.apache.hadoop.io.SequenceFile;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Add support for transactional operations to the regionserver's
|
* Add support for transactional operations to the regionserver's
|
||||||
|
@ -47,11 +46,6 @@ class THLog extends HLog {
|
||||||
super(fs, dir, conf, listener);
|
super(fs, dir, conf, listener);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
protected SequenceFile.Writer createWriter(Path path) throws IOException {
|
|
||||||
return super.createWriter(path, THLogKey.class, KeyValue.class);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected HLogKey makeKey(byte[] regionName, byte[] tableName, long seqNum,
|
protected HLogKey makeKey(byte[] regionName, byte[] tableName, long seqNum,
|
||||||
long now) {
|
long now) {
|
||||||
|
|
|
@ -42,7 +42,6 @@ import org.apache.hadoop.hbase.regionserver.wal.HLog;
|
||||||
import org.apache.hadoop.hbase.client.transactional.HBaseBackedTransactionLogger;
|
import org.apache.hadoop.hbase.client.transactional.HBaseBackedTransactionLogger;
|
||||||
import org.apache.hadoop.hbase.client.transactional.TransactionLogger;
|
import org.apache.hadoop.hbase.client.transactional.TransactionLogger;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
import org.apache.hadoop.io.SequenceFile;
|
|
||||||
import org.apache.hadoop.util.Progressable;
|
import org.apache.hadoop.util.Progressable;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -107,12 +106,8 @@ class THLogRecoveryManager {
|
||||||
Set<Long> commitedTransactions = new HashSet<Long>();
|
Set<Long> commitedTransactions = new HashSet<Long>();
|
||||||
Set<Long> abortedTransactions = new HashSet<Long>();
|
Set<Long> abortedTransactions = new HashSet<Long>();
|
||||||
|
|
||||||
SequenceFile.Reader logReader = HLog.getReader(fileSystem,
|
HLog.Reader reader = HLog.getReader(fileSystem, reconstructionLog, conf);
|
||||||
reconstructionLog, conf);
|
try {
|
||||||
|
|
||||||
try {
|
|
||||||
THLogKey key = new THLogKey();
|
|
||||||
KeyValue val = new KeyValue();
|
|
||||||
long skippedEdits = 0;
|
long skippedEdits = 0;
|
||||||
long totalEdits = 0;
|
long totalEdits = 0;
|
||||||
long startCount = 0;
|
long startCount = 0;
|
||||||
|
@ -123,7 +118,10 @@ class THLogRecoveryManager {
|
||||||
int reportInterval = conf.getInt("hbase.hstore.report.interval.edits",
|
int reportInterval = conf.getInt("hbase.hstore.report.interval.edits",
|
||||||
2000);
|
2000);
|
||||||
|
|
||||||
while (logReader.next(key, val)) {
|
HLog.Entry entry;
|
||||||
|
while ((entry = reader.next()) != null) {
|
||||||
|
THLogKey key = (THLogKey)entry.getKey();
|
||||||
|
KeyValue val = entry.getEdit();
|
||||||
if (LOG.isTraceEnabled()) {
|
if (LOG.isTraceEnabled()) {
|
||||||
LOG.trace("Processing edit: key: " + key.toString() + " val: "
|
LOG.trace("Processing edit: key: " + key.toString() + " val: "
|
||||||
+ val.toString());
|
+ val.toString());
|
||||||
|
@ -200,7 +198,7 @@ class THLogRecoveryManager {
|
||||||
+ " aborts, and " + commitCount + " commits.");
|
+ " aborts, and " + commitCount + " commits.");
|
||||||
}
|
}
|
||||||
} finally {
|
} finally {
|
||||||
logReader.close();
|
reader.close();
|
||||||
}
|
}
|
||||||
|
|
||||||
if (pendingTransactionsById.size() > 0) {
|
if (pendingTransactionsById.size() > 0) {
|
||||||
|
|
|
@ -108,6 +108,8 @@ public class TransactionalRegionServer extends HRegionServer implements
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected HLog instantiateHLog(Path logdir) throws IOException {
|
protected HLog instantiateHLog(Path logdir) throws IOException {
|
||||||
|
conf.set("hbase.regionserver.hlog.keyclass",
|
||||||
|
THLogKey.class.getCanonicalName());
|
||||||
HLog newlog = new THLog(super.getFileSystem(), logdir, conf, super.getLogRoller());
|
HLog newlog = new THLog(super.getFileSystem(), logdir, conf, super.getLogRoller());
|
||||||
return newlog;
|
return newlog;
|
||||||
}
|
}
|
||||||
|
|
|
@ -58,6 +58,8 @@ public class TestTHLog extends HBaseTestCase implements
|
||||||
// Set the hbase.rootdir to be the home directory in mini dfs.
|
// Set the hbase.rootdir to be the home directory in mini dfs.
|
||||||
this.conf.set(HConstants.HBASE_DIR, this.cluster.getFileSystem()
|
this.conf.set(HConstants.HBASE_DIR, this.cluster.getFileSystem()
|
||||||
.getHomeDirectory().toString());
|
.getHomeDirectory().toString());
|
||||||
|
this.conf.set("hbase.regionserver.hlog.keyclass",
|
||||||
|
THLogKey.class.getCanonicalName());
|
||||||
super.setUp();
|
super.setUp();
|
||||||
this.dir = new Path("/hbase", getName());
|
this.dir = new Path("/hbase", getName());
|
||||||
if (fs.exists(dir)) {
|
if (fs.exists(dir)) {
|
||||||
|
|
|
@ -59,7 +59,6 @@ import org.apache.hadoop.hbase.io.hfile.HFile.Reader;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
import org.apache.hadoop.hbase.util.ClassSize;
|
import org.apache.hadoop.hbase.util.ClassSize;
|
||||||
import org.apache.hadoop.hbase.util.FSUtils;
|
import org.apache.hadoop.hbase.util.FSUtils;
|
||||||
import org.apache.hadoop.io.SequenceFile;
|
|
||||||
import org.apache.hadoop.util.Progressable;
|
import org.apache.hadoop.util.Progressable;
|
||||||
import org.apache.hadoop.util.StringUtils;
|
import org.apache.hadoop.util.StringUtils;
|
||||||
|
|
||||||
|
@ -317,17 +316,17 @@ public class Store implements HConstants, HeapSize {
|
||||||
// general memory usage accounting.
|
// general memory usage accounting.
|
||||||
long maxSeqIdInLog = -1;
|
long maxSeqIdInLog = -1;
|
||||||
long firstSeqIdInLog = -1;
|
long firstSeqIdInLog = -1;
|
||||||
SequenceFile.Reader logReader = HLog.getReader(this.fs, reconstructionLog,
|
HLog.Reader logReader = HLog.getReader(this.fs, reconstructionLog, conf);
|
||||||
this.conf);
|
|
||||||
try {
|
try {
|
||||||
HLogKey key = HLog.newKey(conf);
|
|
||||||
KeyValue val = new KeyValue();
|
|
||||||
long skippedEdits = 0;
|
long skippedEdits = 0;
|
||||||
long editsCount = 0;
|
long editsCount = 0;
|
||||||
// How many edits to apply before we send a progress report.
|
// How many edits to apply before we send a progress report.
|
||||||
int reportInterval =
|
int reportInterval =
|
||||||
this.conf.getInt("hbase.hstore.report.interval.edits", 2000);
|
this.conf.getInt("hbase.hstore.report.interval.edits", 2000);
|
||||||
while (logReader.next(key, val)) {
|
HLog.Entry entry;
|
||||||
|
while ((entry = logReader.next()) != null) {
|
||||||
|
HLogKey key = entry.getKey();
|
||||||
|
KeyValue val = entry.getEdit();
|
||||||
if (firstSeqIdInLog == -1) {
|
if (firstSeqIdInLog == -1) {
|
||||||
firstSeqIdInLog = key.getLogSeqNum();
|
firstSeqIdInLog = key.getLogSeqNum();
|
||||||
}
|
}
|
||||||
|
|
|
@ -23,7 +23,6 @@ import java.io.EOFException;
|
||||||
import java.io.FileNotFoundException;
|
import java.io.FileNotFoundException;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.UnsupportedEncodingException;
|
import java.io.UnsupportedEncodingException;
|
||||||
import java.lang.reflect.Field;
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.LinkedList;
|
import java.util.LinkedList;
|
||||||
|
@ -46,8 +45,6 @@ import java.util.concurrent.Executors;
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.FSDataInputStream;
|
|
||||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
|
@ -64,12 +61,6 @@ import org.apache.hadoop.hbase.util.Bytes;
|
||||||
import org.apache.hadoop.hbase.util.ClassSize;
|
import org.apache.hadoop.hbase.util.ClassSize;
|
||||||
import org.apache.hadoop.hbase.util.FSUtils;
|
import org.apache.hadoop.hbase.util.FSUtils;
|
||||||
import org.apache.hadoop.hbase.util.Threads;
|
import org.apache.hadoop.hbase.util.Threads;
|
||||||
import org.apache.hadoop.io.SequenceFile;
|
|
||||||
import org.apache.hadoop.io.SequenceFile.CompressionType;
|
|
||||||
import org.apache.hadoop.io.SequenceFile.Metadata;
|
|
||||||
import org.apache.hadoop.io.SequenceFile.Reader;
|
|
||||||
import org.apache.hadoop.io.compress.DefaultCodec;
|
|
||||||
import org.apache.hadoop.util.Progressable;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* HLog stores all the edits to the HStore. Its the hbase write-ahead-log
|
* HLog stores all the edits to the HStore. Its the hbase write-ahead-log
|
||||||
|
@ -123,7 +114,30 @@ public class HLog implements HConstants, Syncable {
|
||||||
private final long blocksize;
|
private final long blocksize;
|
||||||
private final int flushlogentries;
|
private final int flushlogentries;
|
||||||
private final AtomicInteger unflushedEntries = new AtomicInteger(0);
|
private final AtomicInteger unflushedEntries = new AtomicInteger(0);
|
||||||
private final short replicationLevel;
|
|
||||||
|
public interface Reader {
|
||||||
|
|
||||||
|
void init(FileSystem fs, Path path, Configuration c) throws IOException;
|
||||||
|
|
||||||
|
void close() throws IOException;
|
||||||
|
|
||||||
|
Entry next() throws IOException;
|
||||||
|
|
||||||
|
Entry next(Entry reuse) throws IOException;
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
public interface Writer {
|
||||||
|
|
||||||
|
void init(FileSystem fs, Path path, Configuration c) throws IOException;
|
||||||
|
|
||||||
|
void close() throws IOException;
|
||||||
|
|
||||||
|
void sync() throws IOException;
|
||||||
|
|
||||||
|
void append(Entry entry) throws IOException;
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
// used to indirectly tell syncFs to force the sync
|
// used to indirectly tell syncFs to force the sync
|
||||||
private boolean forceSync = false;
|
private boolean forceSync = false;
|
||||||
|
@ -131,10 +145,7 @@ public class HLog implements HConstants, Syncable {
|
||||||
/*
|
/*
|
||||||
* Current log file.
|
* Current log file.
|
||||||
*/
|
*/
|
||||||
SequenceFile.Writer writer;
|
Writer writer;
|
||||||
// This is the above writer's output stream. Its private but we use reflection
|
|
||||||
// to expose it so we can call sync on it.
|
|
||||||
FSDataOutputStream writer_out;
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* Map of all log files but the current one.
|
* Map of all log files but the current one.
|
||||||
|
@ -218,8 +229,6 @@ public class HLog implements HConstants, Syncable {
|
||||||
conf.getInt("hbase.regionserver.flushlogentries", 1);
|
conf.getInt("hbase.regionserver.flushlogentries", 1);
|
||||||
this.blocksize = conf.getLong("hbase.regionserver.hlog.blocksize",
|
this.blocksize = conf.getLong("hbase.regionserver.hlog.blocksize",
|
||||||
this.fs.getDefaultBlockSize());
|
this.fs.getDefaultBlockSize());
|
||||||
this.replicationLevel = (short) conf.getInt("hbase.regionserver.hlog.replication",
|
|
||||||
this.fs.getDefaultReplication());
|
|
||||||
// Roll at 95% of block size.
|
// Roll at 95% of block size.
|
||||||
float multi = conf.getFloat("hbase.regionserver.logroll.multiplier", 0.95f);
|
float multi = conf.getFloat("hbase.regionserver.logroll.multiplier", 0.95f);
|
||||||
this.logrollsize = (long)(this.blocksize * multi);
|
this.logrollsize = (long)(this.blocksize * multi);
|
||||||
|
@ -249,16 +258,6 @@ public class HLog implements HConstants, Syncable {
|
||||||
return this.filenum;
|
return this.filenum;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Get the compression type for the hlog files
|
|
||||||
* @param c Configuration to use.
|
|
||||||
* @return the kind of compression to use
|
|
||||||
*/
|
|
||||||
static CompressionType getCompressionType(final Configuration c) {
|
|
||||||
// Compression makes no sense for commit log. Always return NONE.
|
|
||||||
return CompressionType.NONE;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Called by HRegionServer when it opens a new region to ensure that log
|
* Called by HRegionServer when it opens a new region to ensure that log
|
||||||
* sequence numbers are always greater than the latest sequence number of the
|
* sequence numbers are always greater than the latest sequence number of the
|
||||||
|
@ -318,7 +317,7 @@ public class HLog implements HConstants, Syncable {
|
||||||
Path oldFile = cleanupCurrentWriter(this.filenum);
|
Path oldFile = cleanupCurrentWriter(this.filenum);
|
||||||
this.filenum = System.currentTimeMillis();
|
this.filenum = System.currentTimeMillis();
|
||||||
Path newPath = computeFilename(this.filenum);
|
Path newPath = computeFilename(this.filenum);
|
||||||
this.writer = createWriter(newPath);
|
this.writer = createWriter(fs, newPath, new HBaseConfiguration(conf));
|
||||||
LOG.info((oldFile != null?
|
LOG.info((oldFile != null?
|
||||||
"Roll " + FSUtils.getPath(oldFile) + ", entries=" +
|
"Roll " + FSUtils.getPath(oldFile) + ", entries=" +
|
||||||
this.numEntries.get() +
|
this.numEntries.get() +
|
||||||
|
@ -349,113 +348,54 @@ public class HLog implements HConstants, Syncable {
|
||||||
return regionToFlush;
|
return regionToFlush;
|
||||||
}
|
}
|
||||||
|
|
||||||
protected SequenceFile.Writer createWriter(Path path) throws IOException {
|
|
||||||
return createWriter(path, HLogKey.class, KeyValue.class);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Hack just to set the correct file length up in SequenceFile.Reader.
|
* Get a reader for the WAL.
|
||||||
* See HADOOP-6307. The below is all about setting the right length on the
|
* @param fs
|
||||||
* file we are reading. fs.getFileStatus(file).getLen() is passed down to
|
* @param path
|
||||||
* a private SequenceFile.Reader constructor. This won't work. Need to do
|
* @param keyClass
|
||||||
* the available on the stream. The below is ugly. It makes getPos, the
|
* @param valueClass
|
||||||
* first time its called, return length of the file -- i.e. tell a lie -- just
|
* @return A WAL reader. Close when done with it.
|
||||||
* so this line up in SF.Reader's constructor ends up with right answer:
|
|
||||||
*
|
|
||||||
* this.end = in.getPos() + length;
|
|
||||||
*/
|
|
||||||
private static class WALReader extends SequenceFile.Reader {
|
|
||||||
|
|
||||||
WALReader(final FileSystem fs, final Path p, final Configuration c)
|
|
||||||
throws IOException {
|
|
||||||
super(fs, p, c);
|
|
||||||
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected FSDataInputStream openFile(FileSystem fs, Path file,
|
|
||||||
int bufferSize, long length)
|
|
||||||
throws IOException {
|
|
||||||
return new WALReaderFSDataInputStream(super.openFile(fs, file, bufferSize,
|
|
||||||
length), length);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Override just so can intercept first call to getPos.
|
|
||||||
*/
|
|
||||||
static class WALReaderFSDataInputStream extends FSDataInputStream {
|
|
||||||
private boolean firstGetPosInvocation = true;
|
|
||||||
private long length;
|
|
||||||
|
|
||||||
WALReaderFSDataInputStream(final FSDataInputStream is, final long l)
|
|
||||||
throws IOException {
|
|
||||||
super(is);
|
|
||||||
this.length = l;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public long getPos() throws IOException {
|
|
||||||
if (this.firstGetPosInvocation) {
|
|
||||||
this.firstGetPosInvocation = false;
|
|
||||||
// Tell a lie. We're doing this just so that this line up in
|
|
||||||
// SequenceFile.Reader constructor comes out with the correct length
|
|
||||||
// on the file:
|
|
||||||
// this.end = in.getPos() + length;
|
|
||||||
//
|
|
||||||
long available = this.in.available();
|
|
||||||
// Length gets added up in the SF.Reader constructor so subtract the
|
|
||||||
// difference. If available < this.length, then return this.length.
|
|
||||||
// I ain't sure what else to do.
|
|
||||||
return available >= this.length? available - this.length: this.length;
|
|
||||||
}
|
|
||||||
return super.getPos();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Get a Reader for WAL.
|
|
||||||
* Reader is a subclass of SequenceFile.Reader. The subclass has amendments
|
|
||||||
* to make it so we see edits up to the last sync (HDFS-265). Of note, we
|
|
||||||
* can only see up to the sync that happened before this file was opened.
|
|
||||||
* Will require us doing up our own WAL Reader if we want to keep up with
|
|
||||||
* a syncing Writer.
|
|
||||||
* @param p
|
|
||||||
* @return A WAL Reader. Close when done with it.
|
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public static SequenceFile.Reader getReader(final FileSystem fs,
|
@SuppressWarnings("unchecked")
|
||||||
final Path p, final Configuration c)
|
public static Reader getReader(final FileSystem fs,
|
||||||
|
final Path path, HBaseConfiguration conf)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
return new WALReader(fs, p, c);
|
try {
|
||||||
|
Class c = Class.forName(conf.get("hbase.regionserver.hlog.reader.impl",
|
||||||
|
SequenceFileLogReader.class.getCanonicalName()));
|
||||||
|
HLog.Reader reader = (HLog.Reader) c.newInstance();
|
||||||
|
reader.init(fs, path, conf);
|
||||||
|
return reader;
|
||||||
|
} catch (Exception e) {
|
||||||
|
IOException ie = new IOException("cannot get log reader");
|
||||||
|
ie.initCause(e);
|
||||||
|
throw ie;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
protected SequenceFile.Writer createWriter(Path path,
|
/**
|
||||||
Class<? extends HLogKey> keyClass, Class<? extends KeyValue> valueClass)
|
* Get a writer for the WAL.
|
||||||
throws IOException {
|
* @param path
|
||||||
SequenceFile.Writer writer =
|
* @param keyClass
|
||||||
SequenceFile.createWriter(this.fs, this.conf, path, keyClass,
|
* @param valueClass
|
||||||
valueClass, fs.getConf().getInt("io.file.buffer.size", 4096),
|
* @return A WAL writer. Close when done with it.
|
||||||
this.replicationLevel, this.blocksize,
|
* @throws IOException
|
||||||
SequenceFile.CompressionType.NONE, new DefaultCodec(), null,
|
*/
|
||||||
new Metadata());
|
@SuppressWarnings("unchecked")
|
||||||
// Get at the private FSDataOutputStream inside in SequenceFile so we can
|
public static Writer createWriter(final FileSystem fs,
|
||||||
// call sync on it. Make it accessible. Stash it aside for call up in
|
final Path path, HBaseConfiguration conf) throws IOException {
|
||||||
// the sync method above.
|
try {
|
||||||
final Field fields[] = writer.getClass().getDeclaredFields();
|
Class c = Class.forName(conf.get("hbase.regionserver.hlog.writer.impl",
|
||||||
final String fieldName = "out";
|
SequenceFileLogWriter.class.getCanonicalName()));
|
||||||
for (int i = 0; i < fields.length; ++i) {
|
HLog.Writer writer = (HLog.Writer) c.newInstance();
|
||||||
if (fieldName.equals(fields[i].getName())) {
|
writer.init(fs, path, conf);
|
||||||
try {
|
return writer;
|
||||||
fields[i].setAccessible(true);
|
} catch (Exception e) {
|
||||||
this.writer_out = (FSDataOutputStream)fields[i].get(writer);
|
IOException ie = new IOException("cannot get log writer");
|
||||||
break;
|
ie.initCause(e);
|
||||||
} catch (IllegalAccessException ex) {
|
throw ie;
|
||||||
throw new IOException("Accessing " + fieldName, ex);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
return writer;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/*
|
/*
|
||||||
|
@ -820,9 +760,6 @@ public class HLog implements HConstants, Syncable {
|
||||||
this.unflushedEntries.get() >= this.flushlogentries) {
|
this.unflushedEntries.get() >= this.flushlogentries) {
|
||||||
try {
|
try {
|
||||||
this.writer.sync();
|
this.writer.sync();
|
||||||
if (this.writer_out != null) {
|
|
||||||
this.writer_out.sync();
|
|
||||||
}
|
|
||||||
this.forceSync = false;
|
this.forceSync = false;
|
||||||
this.unflushedEntries.set(0);
|
this.unflushedEntries.set(0);
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
|
@ -857,7 +794,7 @@ public class HLog implements HConstants, Syncable {
|
||||||
LOG.debug("edit=" + this.numEntries.get() + ", write=" +
|
LOG.debug("edit=" + this.numEntries.get() + ", write=" +
|
||||||
logKey.toString());
|
logKey.toString());
|
||||||
}
|
}
|
||||||
this.writer.append(logKey, logEdit);
|
this.writer.append(new HLog.Entry(logKey, logEdit));
|
||||||
long took = System.currentTimeMillis() - now;
|
long took = System.currentTimeMillis() - now;
|
||||||
if (took > 1000) {
|
if (took > 1000) {
|
||||||
LOG.warn(Thread.currentThread().getName() + " took " + took +
|
LOG.warn(Thread.currentThread().getName() + " took " + took +
|
||||||
|
@ -936,8 +873,9 @@ public class HLog implements HConstants, Syncable {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
synchronized (updateLock) {
|
synchronized (updateLock) {
|
||||||
this.writer.append(makeKey(regionName, tableName, logSeqId, System.currentTimeMillis()),
|
this.writer.append(new HLog.Entry(
|
||||||
completeCacheFlushLogEdit());
|
makeKey(regionName, tableName, logSeqId, System.currentTimeMillis()),
|
||||||
|
completeCacheFlushLogEdit()));
|
||||||
this.numEntries.incrementAndGet();
|
this.numEntries.incrementAndGet();
|
||||||
Long seq = this.lastSeqWritten.get(regionName);
|
Long seq = this.lastSeqWritten.get(regionName);
|
||||||
if (seq != null && logSeqId >= seq.longValue()) {
|
if (seq != null && logSeqId >= seq.longValue()) {
|
||||||
|
@ -1018,20 +956,20 @@ public class HLog implements HConstants, Syncable {
|
||||||
// Private immutable datastructure to hold Writer and its Path.
|
// Private immutable datastructure to hold Writer and its Path.
|
||||||
private final static class WriterAndPath {
|
private final static class WriterAndPath {
|
||||||
final Path p;
|
final Path p;
|
||||||
final SequenceFile.Writer w;
|
final Writer w;
|
||||||
WriterAndPath(final Path p, final SequenceFile.Writer w) {
|
WriterAndPath(final Path p, final Writer w) {
|
||||||
this.p = p;
|
this.p = p;
|
||||||
this.w = w;
|
this.w = w;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
static Class<? extends HLogKey> getKeyClass(HBaseConfiguration conf) {
|
public static Class<? extends HLogKey> getKeyClass(Configuration conf) {
|
||||||
return (Class<? extends HLogKey>)
|
return (Class<? extends HLogKey>)
|
||||||
conf.getClass("hbase.regionserver.hlog.keyclass", HLogKey.class);
|
conf.getClass("hbase.regionserver.hlog.keyclass", HLogKey.class);
|
||||||
}
|
}
|
||||||
|
|
||||||
public static HLogKey newKey(HBaseConfiguration conf) throws IOException {
|
public static HLogKey newKey(Configuration conf) throws IOException {
|
||||||
Class<? extends HLogKey> keyClass = getKeyClass(conf);
|
Class<? extends HLogKey> keyClass = getKeyClass(conf);
|
||||||
try {
|
try {
|
||||||
return keyClass.newInstance();
|
return keyClass.newInstance();
|
||||||
|
@ -1072,8 +1010,8 @@ public class HLog implements HConstants, Syncable {
|
||||||
int maxSteps = Double.valueOf(Math.ceil((logfiles.length * 1.0) /
|
int maxSteps = Double.valueOf(Math.ceil((logfiles.length * 1.0) /
|
||||||
concurrentLogReads)).intValue();
|
concurrentLogReads)).intValue();
|
||||||
for (int step = 0; step < maxSteps; step++) {
|
for (int step = 0; step < maxSteps; step++) {
|
||||||
final Map<byte[], LinkedList<HLogEntry>> logEntries =
|
final Map<byte[], LinkedList<HLog.Entry>> logEntries =
|
||||||
new TreeMap<byte[], LinkedList<HLogEntry>>(Bytes.BYTES_COMPARATOR);
|
new TreeMap<byte[], LinkedList<HLog.Entry>>(Bytes.BYTES_COMPARATOR);
|
||||||
// Stop at logfiles.length when it's the last step
|
// Stop at logfiles.length when it's the last step
|
||||||
int endIndex = step == maxSteps - 1? logfiles.length:
|
int endIndex = step == maxSteps - 1? logfiles.length:
|
||||||
step * concurrentLogReads + concurrentLogReads;
|
step * concurrentLogReads + concurrentLogReads;
|
||||||
|
@ -1086,28 +1024,22 @@ public class HLog implements HConstants, Syncable {
|
||||||
LOG.debug("Splitting hlog " + (i + 1) + " of " + logfiles.length +
|
LOG.debug("Splitting hlog " + (i + 1) + " of " + logfiles.length +
|
||||||
": " + logfiles[i].getPath() + ", length=" + logfiles[i].getLen());
|
": " + logfiles[i].getPath() + ", length=" + logfiles[i].getLen());
|
||||||
}
|
}
|
||||||
SequenceFile.Reader in = null;
|
Reader in = null;
|
||||||
int count = 0;
|
int count = 0;
|
||||||
try {
|
try {
|
||||||
in = HLog.getReader(fs, logfiles[i].getPath(), conf);
|
in = HLog.getReader(fs, logfiles[i].getPath(), conf);
|
||||||
try {
|
try {
|
||||||
HLogKey key = newKey(conf);
|
HLog.Entry entry;
|
||||||
KeyValue val = new KeyValue();
|
while ((entry = in.next()) != null) {
|
||||||
while (in.next(key, val)) {
|
byte [] regionName = entry.getKey().getRegionName();
|
||||||
byte [] regionName = key.getRegionName();
|
LinkedList<HLog.Entry> queue = logEntries.get(regionName);
|
||||||
LinkedList<HLogEntry> queue = logEntries.get(regionName);
|
|
||||||
if (queue == null) {
|
if (queue == null) {
|
||||||
queue = new LinkedList<HLogEntry>();
|
queue = new LinkedList<HLog.Entry>();
|
||||||
LOG.debug("Adding queue for " + Bytes.toStringBinary(regionName));
|
LOG.debug("Adding queue for " + Bytes.toStringBinary(regionName));
|
||||||
logEntries.put(regionName, queue);
|
logEntries.put(regionName, queue);
|
||||||
}
|
}
|
||||||
HLogEntry hle = new HLogEntry(val, key);
|
queue.push(entry);
|
||||||
queue.push(hle);
|
|
||||||
count++;
|
count++;
|
||||||
// Make the key and value new each time; otherwise same instance
|
|
||||||
// is used over and over.
|
|
||||||
key = newKey(conf);
|
|
||||||
val = new KeyValue();
|
|
||||||
}
|
}
|
||||||
LOG.debug("Pushed=" + count + " entries from " +
|
LOG.debug("Pushed=" + count + " entries from " +
|
||||||
logfiles[i].getPath());
|
logfiles[i].getPath());
|
||||||
|
@ -1148,17 +1080,17 @@ public class HLog implements HConstants, Syncable {
|
||||||
Thread thread = new Thread(Bytes.toStringBinary(key)) {
|
Thread thread = new Thread(Bytes.toStringBinary(key)) {
|
||||||
@Override
|
@Override
|
||||||
public void run() {
|
public void run() {
|
||||||
LinkedList<HLogEntry> entries = logEntries.get(key);
|
LinkedList<HLog.Entry> entries = logEntries.get(key);
|
||||||
LOG.debug("Thread got " + entries.size() + " to process");
|
LOG.debug("Thread got " + entries.size() + " to process");
|
||||||
long threadTime = System.currentTimeMillis();
|
long threadTime = System.currentTimeMillis();
|
||||||
try {
|
try {
|
||||||
int count = 0;
|
int count = 0;
|
||||||
// Items were added to the linkedlist oldest first. Pull them
|
// Items were added to the linkedlist oldest first. Pull them
|
||||||
// out in that order.
|
// out in that order.
|
||||||
for (ListIterator<HLogEntry> i =
|
for (ListIterator<HLog.Entry> i =
|
||||||
entries.listIterator(entries.size());
|
entries.listIterator(entries.size());
|
||||||
i.hasPrevious();) {
|
i.hasPrevious();) {
|
||||||
HLogEntry logEntry = i.previous();
|
HLog.Entry logEntry = i.previous();
|
||||||
WriterAndPath wap = logWriters.get(key);
|
WriterAndPath wap = logWriters.get(key);
|
||||||
if (wap == null) {
|
if (wap == null) {
|
||||||
Path logfile = new Path(HRegion.getRegionDir(HTableDescriptor
|
Path logfile = new Path(HRegion.getRegionDir(HTableDescriptor
|
||||||
|
@ -1166,7 +1098,7 @@ public class HLog implements HConstants, Syncable {
|
||||||
HRegionInfo.encodeRegionName(key)),
|
HRegionInfo.encodeRegionName(key)),
|
||||||
HREGION_OLDLOGFILE_NAME);
|
HREGION_OLDLOGFILE_NAME);
|
||||||
Path oldlogfile = null;
|
Path oldlogfile = null;
|
||||||
SequenceFile.Reader old = null;
|
Reader old = null;
|
||||||
if (fs.exists(logfile)) {
|
if (fs.exists(logfile)) {
|
||||||
FileStatus stat = fs.getFileStatus(logfile);
|
FileStatus stat = fs.getFileStatus(logfile);
|
||||||
if (stat.getLen() <= 0) {
|
if (stat.getLen() <= 0) {
|
||||||
|
@ -1178,12 +1110,10 @@ public class HLog implements HConstants, Syncable {
|
||||||
"exists. Copying existing file to new file");
|
"exists. Copying existing file to new file");
|
||||||
oldlogfile = new Path(logfile.toString() + ".old");
|
oldlogfile = new Path(logfile.toString() + ".old");
|
||||||
fs.rename(logfile, oldlogfile);
|
fs.rename(logfile, oldlogfile);
|
||||||
old = new SequenceFile.Reader(fs, oldlogfile, conf);
|
old = getReader(fs, oldlogfile, conf);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
SequenceFile.Writer w =
|
Writer w = createWriter(fs, logfile, conf);
|
||||||
SequenceFile.createWriter(fs, conf, logfile,
|
|
||||||
getKeyClass(conf), KeyValue.class, getCompressionType(conf));
|
|
||||||
wap = new WriterAndPath(logfile, w);
|
wap = new WriterAndPath(logfile, w);
|
||||||
logWriters.put(key, wap);
|
logWriters.put(key, wap);
|
||||||
if (LOG.isDebugEnabled()) {
|
if (LOG.isDebugEnabled()) {
|
||||||
|
@ -1193,20 +1123,19 @@ public class HLog implements HConstants, Syncable {
|
||||||
|
|
||||||
if (old != null) {
|
if (old != null) {
|
||||||
// Copy from existing log file
|
// Copy from existing log file
|
||||||
HLogKey oldkey = newKey(conf);
|
HLog.Entry entry;
|
||||||
KeyValue oldval = new KeyValue();
|
for (; (entry = old.next()) != null; count++) {
|
||||||
for (; old.next(oldkey, oldval); count++) {
|
|
||||||
if (LOG.isDebugEnabled() && count > 0
|
if (LOG.isDebugEnabled() && count > 0
|
||||||
&& count % 10000 == 0) {
|
&& count % 10000 == 0) {
|
||||||
LOG.debug("Copied " + count + " edits");
|
LOG.debug("Copied " + count + " edits");
|
||||||
}
|
}
|
||||||
w.append(oldkey, oldval);
|
w.append(entry);
|
||||||
}
|
}
|
||||||
old.close();
|
old.close();
|
||||||
fs.delete(oldlogfile, true);
|
fs.delete(oldlogfile, true);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
wap.w.append(logEntry.getKey(), logEntry.getEdit());
|
wap.w.append(logEntry);
|
||||||
count++;
|
count++;
|
||||||
}
|
}
|
||||||
if (LOG.isDebugEnabled()) {
|
if (LOG.isDebugEnabled()) {
|
||||||
|
@ -1249,18 +1178,24 @@ public class HLog implements HConstants, Syncable {
|
||||||
* Utility class that lets us keep track of the edit with it's key
|
* Utility class that lets us keep track of the edit with it's key
|
||||||
* Only used when splitting logs
|
* Only used when splitting logs
|
||||||
*/
|
*/
|
||||||
public static class HLogEntry {
|
public static class Entry {
|
||||||
private KeyValue edit;
|
private KeyValue edit;
|
||||||
private HLogKey key;
|
private HLogKey key;
|
||||||
|
|
||||||
|
public Entry() {
|
||||||
|
edit = new KeyValue();
|
||||||
|
key = new HLogKey();
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Constructor for both params
|
* Constructor for both params
|
||||||
* @param edit log's edit
|
* @param edit log's edit
|
||||||
* @param key log's key
|
* @param key log's key
|
||||||
*/
|
*/
|
||||||
public HLogEntry(KeyValue edit, HLogKey key) {
|
public Entry(HLogKey key, KeyValue edit) {
|
||||||
super();
|
super();
|
||||||
this.edit = edit;
|
|
||||||
this.key = key;
|
this.key = key;
|
||||||
|
this.edit = edit;
|
||||||
}
|
}
|
||||||
/**
|
/**
|
||||||
* Gets the edit
|
* Gets the edit
|
||||||
|
@ -1360,12 +1295,11 @@ public class HLog implements HConstants, Syncable {
|
||||||
if (!fs.isFile(logPath)) {
|
if (!fs.isFile(logPath)) {
|
||||||
throw new IOException(args[i] + " is not a file");
|
throw new IOException(args[i] + " is not a file");
|
||||||
}
|
}
|
||||||
Reader log = new SequenceFile.Reader(fs, logPath, conf);
|
Reader log = getReader(fs, logPath, conf);
|
||||||
try {
|
try {
|
||||||
HLogKey key = new HLogKey();
|
HLog.Entry entry;
|
||||||
KeyValue val = new KeyValue();
|
while ((entry = log.next()) != null) {
|
||||||
while (log.next(key, val)) {
|
System.out.println(entry.toString());
|
||||||
System.out.println(key.toString() + " " + val.toString());
|
|
||||||
}
|
}
|
||||||
} finally {
|
} finally {
|
||||||
log.close();
|
log.close();
|
||||||
|
@ -1383,15 +1317,4 @@ public class HLog implements HConstants, Syncable {
|
||||||
ClassSize.OBJECT + (5 * ClassSize.REFERENCE) +
|
ClassSize.OBJECT + (5 * ClassSize.REFERENCE) +
|
||||||
ClassSize.ATOMIC_INTEGER + Bytes.SIZEOF_INT + (3 * Bytes.SIZEOF_LONG));
|
ClassSize.ATOMIC_INTEGER + Bytes.SIZEOF_INT + (3 * Bytes.SIZEOF_LONG));
|
||||||
|
|
||||||
static class HLogWriter extends SequenceFile.Writer {
|
|
||||||
public HLogWriter(FileSystem arg0, Configuration arg1, Path arg2,
|
|
||||||
Class<?> arg3, Class<?> arg4, int arg5, short arg6, long arg7,
|
|
||||||
Progressable arg8, Metadata arg9) throws IOException {
|
|
||||||
super(arg0, arg1, arg2, arg3, arg4, arg5, arg6, arg7, arg8, arg9);
|
|
||||||
}
|
|
||||||
|
|
||||||
void flush() {
|
|
||||||
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,110 @@
|
||||||
|
package org.apache.hadoop.hbase.regionserver.wal;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
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.KeyValue;
|
||||||
|
import org.apache.hadoop.hbase.regionserver.wal.HLog;
|
||||||
|
import org.apache.hadoop.io.SequenceFile;
|
||||||
|
|
||||||
|
public class SequenceFileLogReader implements HLog.Reader {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Hack just to set the correct file length up in SequenceFile.Reader.
|
||||||
|
* See HADOOP-6307. The below is all about setting the right length on the
|
||||||
|
* file we are reading. fs.getFileStatus(file).getLen() is passed down to
|
||||||
|
* a private SequenceFile.Reader constructor. This won't work. Need to do
|
||||||
|
* the available on the stream. The below is ugly. It makes getPos, the
|
||||||
|
* first time its called, return length of the file -- i.e. tell a lie -- just
|
||||||
|
* so this line up in SF.Reader's constructor ends up with right answer:
|
||||||
|
*
|
||||||
|
* this.end = in.getPos() + length;
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
private static class WALReader extends SequenceFile.Reader {
|
||||||
|
|
||||||
|
WALReader(final FileSystem fs, final Path p, final Configuration c)
|
||||||
|
throws IOException {
|
||||||
|
super(fs, p, c);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected FSDataInputStream openFile(FileSystem fs, Path file,
|
||||||
|
int bufferSize, long length)
|
||||||
|
throws IOException {
|
||||||
|
return new WALReaderFSDataInputStream(super.openFile(fs, file,
|
||||||
|
bufferSize, length), length);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Override just so can intercept first call to getPos.
|
||||||
|
*/
|
||||||
|
static class WALReaderFSDataInputStream extends FSDataInputStream {
|
||||||
|
private boolean firstGetPosInvocation = true;
|
||||||
|
private long length;
|
||||||
|
|
||||||
|
WALReaderFSDataInputStream(final FSDataInputStream is, final long l)
|
||||||
|
throws IOException {
|
||||||
|
super(is);
|
||||||
|
this.length = l;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getPos() throws IOException {
|
||||||
|
if (this.firstGetPosInvocation) {
|
||||||
|
this.firstGetPosInvocation = false;
|
||||||
|
// Tell a lie. We're doing this just so that this line up in
|
||||||
|
// SequenceFile.Reader constructor comes out with the correct length
|
||||||
|
// on the file:
|
||||||
|
// this.end = in.getPos() + length;
|
||||||
|
long available = this.in.available();
|
||||||
|
// Length gets added up in the SF.Reader constructor so subtract the
|
||||||
|
// difference. If available < this.length, then return this.length.
|
||||||
|
return available >= this.length? available - this.length: this.length;
|
||||||
|
}
|
||||||
|
return super.getPos();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
Configuration conf;
|
||||||
|
WALReader reader;
|
||||||
|
|
||||||
|
public SequenceFileLogReader() { }
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void init(FileSystem fs, Path path, Configuration conf)
|
||||||
|
throws IOException {
|
||||||
|
this.conf = conf;
|
||||||
|
reader = new WALReader(fs, path, conf);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() throws IOException {
|
||||||
|
reader.close();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public HLog.Entry next() throws IOException {
|
||||||
|
return next(null);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public HLog.Entry next(HLog.Entry reuse) throws IOException {
|
||||||
|
if (reuse == null) {
|
||||||
|
HLogKey key = HLog.newKey(conf);
|
||||||
|
KeyValue val = new KeyValue();
|
||||||
|
if (reader.next(key, val)) {
|
||||||
|
return new HLog.Entry(key, val);
|
||||||
|
}
|
||||||
|
} else if (reader.next(reuse.getKey(), reuse.getEdit())) {
|
||||||
|
return reuse;
|
||||||
|
}
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,74 @@
|
||||||
|
package org.apache.hadoop.hbase.regionserver.wal;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.lang.reflect.Field;
|
||||||
|
|
||||||
|
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.KeyValue;
|
||||||
|
import org.apache.hadoop.hbase.regionserver.wal.HLog;
|
||||||
|
import org.apache.hadoop.io.SequenceFile;
|
||||||
|
import org.apache.hadoop.io.SequenceFile.Metadata;
|
||||||
|
import org.apache.hadoop.io.compress.DefaultCodec;
|
||||||
|
|
||||||
|
public class SequenceFileLogWriter implements HLog.Writer {
|
||||||
|
|
||||||
|
SequenceFile.Writer writer;
|
||||||
|
FSDataOutputStream writer_out;
|
||||||
|
|
||||||
|
public SequenceFileLogWriter() { }
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void init(FileSystem fs, Path path, Configuration conf)
|
||||||
|
throws IOException {
|
||||||
|
writer = SequenceFile.createWriter(fs, conf, path,
|
||||||
|
HLog.getKeyClass(conf), KeyValue.class,
|
||||||
|
fs.getConf().getInt("io.file.buffer.size", 4096),
|
||||||
|
(short) conf.getInt("hbase.regionserver.hlog.replication",
|
||||||
|
fs.getDefaultReplication()),
|
||||||
|
conf.getLong("hbase.regionserver.hlog.blocksize",
|
||||||
|
fs.getDefaultBlockSize()),
|
||||||
|
SequenceFile.CompressionType.NONE,
|
||||||
|
new DefaultCodec(),
|
||||||
|
null,
|
||||||
|
new Metadata());
|
||||||
|
|
||||||
|
// Get at the private FSDataOutputStream inside in SequenceFile so we can
|
||||||
|
// call sync on it. Make it accessible. Stash it aside for call up in
|
||||||
|
// the sync method.
|
||||||
|
final Field fields[] = writer.getClass().getDeclaredFields();
|
||||||
|
final String fieldName = "out";
|
||||||
|
for (int i = 0; i < fields.length; ++i) {
|
||||||
|
if (fieldName.equals(fields[i].getName())) {
|
||||||
|
try {
|
||||||
|
fields[i].setAccessible(true);
|
||||||
|
this.writer_out = (FSDataOutputStream)fields[i].get(writer);
|
||||||
|
break;
|
||||||
|
} catch (IllegalAccessException ex) {
|
||||||
|
throw new IOException("Accessing " + fieldName, ex);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void append(HLog.Entry entry) throws IOException {
|
||||||
|
this.writer.append(entry.getKey(), entry.getEdit());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() throws IOException {
|
||||||
|
this.writer.close();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void sync() throws IOException {
|
||||||
|
this.writer.sync();
|
||||||
|
if (this.writer_out != null) {
|
||||||
|
this.writer_out.sync();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -33,9 +33,6 @@ import org.apache.hadoop.hbase.regionserver.wal.HLog;
|
||||||
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
|
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||||
import org.apache.hadoop.io.SequenceFile;
|
|
||||||
import org.apache.hadoop.io.SequenceFile.Reader;
|
|
||||||
|
|
||||||
|
|
||||||
/** JUnit test case for HLog */
|
/** JUnit test case for HLog */
|
||||||
public class TestHLog extends HBaseTestCase implements HConstants {
|
public class TestHLog extends HBaseTestCase implements HConstants {
|
||||||
|
@ -139,10 +136,10 @@ public class TestHLog extends HBaseTestCase implements HConstants {
|
||||||
wal.sync();
|
wal.sync();
|
||||||
// Open a Reader.
|
// Open a Reader.
|
||||||
Path walPath = wal.computeFilename(wal.getFilenum());
|
Path walPath = wal.computeFilename(wal.getFilenum());
|
||||||
SequenceFile.Reader reader = HLog.getReader(this.fs, walPath, this.conf);
|
HLog.Reader reader = HLog.getReader(fs, walPath, conf);
|
||||||
int count = 0;
|
int count = 0;
|
||||||
HLogKey key = new HLogKey();
|
HLog.Entry entry = new HLog.Entry();
|
||||||
while(reader.next(key)) count++;
|
while ((entry = reader.next(entry)) != null) count++;
|
||||||
assertEquals(total, count);
|
assertEquals(total, count);
|
||||||
reader.close();
|
reader.close();
|
||||||
// Add test that checks to see that an open of a Reader works on a file
|
// Add test that checks to see that an open of a Reader works on a file
|
||||||
|
@ -152,16 +149,16 @@ public class TestHLog extends HBaseTestCase implements HConstants {
|
||||||
kvs.add(new KeyValue(Bytes.toBytes(i), bytes, bytes));
|
kvs.add(new KeyValue(Bytes.toBytes(i), bytes, bytes));
|
||||||
wal.append(bytes, bytes, kvs, System.currentTimeMillis());
|
wal.append(bytes, bytes, kvs, System.currentTimeMillis());
|
||||||
}
|
}
|
||||||
reader = HLog.getReader(this.fs, walPath, this.conf);
|
reader = HLog.getReader(fs, walPath, conf);
|
||||||
count = 0;
|
count = 0;
|
||||||
while(reader.next(key)) count++;
|
while((entry = reader.next(entry)) != null) count++;
|
||||||
assertTrue(count >= total);
|
assertTrue(count >= total);
|
||||||
reader.close();
|
reader.close();
|
||||||
// If I sync, should see double the edits.
|
// If I sync, should see double the edits.
|
||||||
wal.sync();
|
wal.sync();
|
||||||
reader = HLog.getReader(this.fs, walPath, this.conf);
|
reader = HLog.getReader(fs, walPath, conf);
|
||||||
count = 0;
|
count = 0;
|
||||||
while(reader.next(key)) count++;
|
while((entry = reader.next(entry)) != null) count++;
|
||||||
assertEquals(total * 2, count);
|
assertEquals(total * 2, count);
|
||||||
// Now do a test that ensures stuff works when we go over block boundary,
|
// Now do a test that ensures stuff works when we go over block boundary,
|
||||||
// especially that we return good length on file.
|
// especially that we return good length on file.
|
||||||
|
@ -173,16 +170,16 @@ public class TestHLog extends HBaseTestCase implements HConstants {
|
||||||
}
|
}
|
||||||
// Now I should have written out lots of blocks. Sync then read.
|
// Now I should have written out lots of blocks. Sync then read.
|
||||||
wal.sync();
|
wal.sync();
|
||||||
reader = HLog.getReader(this.fs, walPath, this.conf);
|
reader = HLog.getReader(fs, walPath, conf);
|
||||||
count = 0;
|
count = 0;
|
||||||
while(reader.next(key)) count++;
|
while((entry = reader.next(entry)) != null) count++;
|
||||||
assertEquals(total * 3, count);
|
assertEquals(total * 3, count);
|
||||||
reader.close();
|
reader.close();
|
||||||
// Close it and ensure that closed, Reader gets right length also.
|
// Close it and ensure that closed, Reader gets right length also.
|
||||||
wal.close();
|
wal.close();
|
||||||
reader = HLog.getReader(this.fs, walPath, this.conf);
|
reader = HLog.getReader(fs, walPath, conf);
|
||||||
count = 0;
|
count = 0;
|
||||||
while(reader.next(key)) count++;
|
while((entry = reader.next(entry)) != null) count++;
|
||||||
assertEquals(total * 3, count);
|
assertEquals(total * 3, count);
|
||||||
reader.close();
|
reader.close();
|
||||||
}
|
}
|
||||||
|
@ -191,14 +188,15 @@ public class TestHLog extends HBaseTestCase implements HConstants {
|
||||||
throws IOException {
|
throws IOException {
|
||||||
assertEquals(howmany, splits.size());
|
assertEquals(howmany, splits.size());
|
||||||
for (int i = 0; i < splits.size(); i++) {
|
for (int i = 0; i < splits.size(); i++) {
|
||||||
SequenceFile.Reader r = HLog.getReader(this.fs, splits.get(i), this.conf);
|
HLog.Reader reader = HLog.getReader(this.fs, splits.get(i), conf);
|
||||||
try {
|
try {
|
||||||
HLogKey key = new HLogKey();
|
|
||||||
KeyValue kv = new KeyValue();
|
|
||||||
int count = 0;
|
int count = 0;
|
||||||
String previousRegion = null;
|
String previousRegion = null;
|
||||||
long seqno = -1;
|
long seqno = -1;
|
||||||
while(r.next(key, kv)) {
|
HLog.Entry entry = new HLog.Entry();
|
||||||
|
while((entry = reader.next(entry)) != null) {
|
||||||
|
HLogKey key = entry.getKey();
|
||||||
|
KeyValue kv = entry.getEdit();
|
||||||
String region = Bytes.toString(key.getRegionName());
|
String region = Bytes.toString(key.getRegionName());
|
||||||
// Assert that all edits are for same region.
|
// Assert that all edits are for same region.
|
||||||
if (previousRegion != null) {
|
if (previousRegion != null) {
|
||||||
|
@ -212,7 +210,7 @@ public class TestHLog extends HBaseTestCase implements HConstants {
|
||||||
}
|
}
|
||||||
assertEquals(howmany * howmany, count);
|
assertEquals(howmany * howmany, count);
|
||||||
} finally {
|
} finally {
|
||||||
r.close();
|
reader.close();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -226,7 +224,7 @@ public class TestHLog extends HBaseTestCase implements HConstants {
|
||||||
final byte [] regionName = Bytes.toBytes("regionname");
|
final byte [] regionName = Bytes.toBytes("regionname");
|
||||||
final byte [] tableName = Bytes.toBytes("tablename");
|
final byte [] tableName = Bytes.toBytes("tablename");
|
||||||
final byte [] row = Bytes.toBytes("row");
|
final byte [] row = Bytes.toBytes("row");
|
||||||
Reader reader = null;
|
HLog.Reader reader = null;
|
||||||
HLog log = new HLog(fs, dir, this.conf, null);
|
HLog log = new HLog(fs, dir, this.conf, null);
|
||||||
try {
|
try {
|
||||||
// Write columns named 1, 2, 3, etc. and then values of single byte
|
// Write columns named 1, 2, 3, etc. and then values of single byte
|
||||||
|
@ -246,17 +244,20 @@ public class TestHLog extends HBaseTestCase implements HConstants {
|
||||||
log = null;
|
log = null;
|
||||||
// Now open a reader on the log and assert append worked.
|
// Now open a reader on the log and assert append worked.
|
||||||
reader = HLog.getReader(fs, filename, conf);
|
reader = HLog.getReader(fs, filename, conf);
|
||||||
HLogKey key = new HLogKey();
|
HLog.Entry entry = new HLog.Entry();
|
||||||
KeyValue val = new KeyValue();
|
|
||||||
for (int i = 0; i < COL_COUNT; i++) {
|
for (int i = 0; i < COL_COUNT; i++) {
|
||||||
reader.next(key, val);
|
reader.next(entry);
|
||||||
|
HLogKey key = entry.getKey();
|
||||||
|
KeyValue val = entry.getEdit();
|
||||||
assertTrue(Bytes.equals(regionName, key.getRegionName()));
|
assertTrue(Bytes.equals(regionName, key.getRegionName()));
|
||||||
assertTrue(Bytes.equals(tableName, key.getTablename()));
|
assertTrue(Bytes.equals(tableName, key.getTablename()));
|
||||||
assertTrue(Bytes.equals(row, val.getRow()));
|
assertTrue(Bytes.equals(row, val.getRow()));
|
||||||
assertEquals((byte)(i + '0'), val.getValue()[0]);
|
assertEquals((byte)(i + '0'), val.getValue()[0]);
|
||||||
System.out.println(key + " " + val);
|
System.out.println(key + " " + val);
|
||||||
}
|
}
|
||||||
while (reader.next(key, val)) {
|
while ((entry = reader.next(null)) != null) {
|
||||||
|
HLogKey key = entry.getKey();
|
||||||
|
KeyValue val = entry.getEdit();
|
||||||
// Assert only one more row... the meta flushed row.
|
// Assert only one more row... the meta flushed row.
|
||||||
assertTrue(Bytes.equals(regionName, key.getRegionName()));
|
assertTrue(Bytes.equals(regionName, key.getRegionName()));
|
||||||
assertTrue(Bytes.equals(tableName, key.getTablename()));
|
assertTrue(Bytes.equals(tableName, key.getTablename()));
|
||||||
|
|
Loading…
Reference in New Issue