HBASE-1887 Update hbase trunk to latests on hadoop 0.21 branch so we can all test sync/append

git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@823747 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2009-10-10 00:03:45 +00:00
parent 525cf7b559
commit 620bfe40be
7 changed files with 92 additions and 38 deletions

View File

@ -105,6 +105,8 @@ Release 0.21.0 - Unreleased
HBASE-1722 Add support for exporting HBase metrics via JMX HBASE-1722 Add support for exporting HBase metrics via JMX
(Gary Helming via Stack) (Gary Helming via Stack)
HBASE-1899 Use scanner caching in shell count HBASE-1899 Use scanner caching in shell count
HBASE-1887 Update hbase trunk to latests on hadoop 0.21 branch so we can
all test sync/append
OPTIMIZATIONS OPTIMIZATIONS

View File

@ -38,6 +38,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.regionserver.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;
@ -106,7 +107,7 @@ 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 = new SequenceFile.Reader(fileSystem, SequenceFile.Reader logReader = HLog.getReader(fileSystem,
reconstructionLog, conf); reconstructionLog, conf);
try { try {

View File

@ -46,6 +46,8 @@ import java.util.concurrent.locks.ReentrantLock;
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;
@ -55,27 +57,29 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HServerInfo; import org.apache.hadoop.hbase.HServerInfo;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.RemoteExceptionHandler; import org.apache.hadoop.hbase.RemoteExceptionHandler;
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.hbase.KeyValue;
import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.SequenceFile.CompressionType; import org.apache.hadoop.io.SequenceFile.CompressionType;
import org.apache.hadoop.io.SequenceFile.Metadata; import org.apache.hadoop.io.SequenceFile.Metadata;
import org.apache.hadoop.io.SequenceFile.Reader; import org.apache.hadoop.io.SequenceFile.Reader;
import org.apache.hadoop.io.compress.DefaultCodec; import org.apache.hadoop.io.compress.DefaultCodec;
import org.apache.hadoop.util.Progressable; import org.apache.hadoop.util.Progressable;
import org.apache.hadoop.fs.FSDataOutputStream;
/** /**
* HLog stores all the edits to the HStore. * HLog stores all the edits to the HStore. Its the hbase write-ahead-log
* implementation.
* *
* It performs logfile-rolling, so external callers are not aware that the * It performs logfile-rolling, so external callers are not aware that the
* underlying file is being rolled. * underlying file is being rolled.
* *
* <p> * <p>
* A single HLog is used by several HRegions simultaneously. * There is one HLog per RegionServer. All edits for all Regions carried by
* a particular RegionServer are entered first in the HLog.
* *
* <p> * <p>
* Each HRegion is identified by a unique long <code>int</code>. HRegions do * Each HRegion is identified by a unique long <code>int</code>. HRegions do
@ -101,6 +105,8 @@ import org.apache.hadoop.fs.FSDataOutputStream;
* rolling is not. To prevent log rolling taking place during this period, a * rolling is not. To prevent log rolling taking place during this period, a
* separate reentrant lock is used. * separate reentrant lock is used.
* *
* <p>To read an HLog, call {@link #getReader(Path)}.
*
*/ */
public class HLog implements HConstants, Syncable { public class HLog implements HConstants, Syncable {
static final Log LOG = LogFactory.getLog(HLog.class); static final Log LOG = LogFactory.getLog(HLog.class);
@ -352,6 +358,73 @@ public class HLog implements HConstants, Syncable {
return createWriter(path, HLogKey.class, KeyValue.class); return createWriter(path, HLogKey.class, KeyValue.class);
} }
/**
* 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));
}
/**
* Override just so can intercept first call to getPos.
*/
static class WALReaderFSDataInputStream extends FSDataInputStream {
private boolean firstGetPosInvocation = true;
WALReaderFSDataInputStream(final FSDataInputStream is)
throws IOException {
super(is);
}
@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;
return this.in.available();
}
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 path
* @return A WAL Reader. Close when done with it.
* @throws IOException
*/
public static SequenceFile.Reader getReader(final FileSystem fs,
final Path p, final Configuration c)
throws IOException {
return new WALReader(fs, p, c);
}
protected SequenceFile.Writer createWriter(Path path, protected SequenceFile.Writer createWriter(Path path,
Class<? extends HLogKey> keyClass, Class<? extends KeyValue> valueClass) Class<? extends HLogKey> keyClass, Class<? extends KeyValue> valueClass)
throws IOException { throws IOException {
@ -636,6 +709,8 @@ public class HLog implements HConstants, Syncable {
} }
} else { } else {
this.writer.sync(); this.writer.sync();
// Above is sequencefile.writer sync. It doesn't actually synce the
// backing stream. Need to do the below to do that.
if (this.writer_out != null) this.writer_out.sync(); if (this.writer_out != null) this.writer_out.sync();
} }
this.unflushedEntries.set(0); this.unflushedEntries.set(0);

View File

@ -315,8 +315,8 @@ 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 = new SequenceFile.Reader(this.fs, SequenceFile.Reader logReader = HLog.getReader(this.fs, reconstructionLog,
reconstructionLog, this.conf); this.conf);
try { try {
HLogKey key = HLog.newKey(conf); HLogKey key = HLog.newKey(conf);
KeyValue val = new KeyValue(); KeyValue val = new KeyValue();

View File

@ -25,9 +25,7 @@ import java.util.List;
import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestCase; import org.apache.hadoop.hbase.HBaseTestCase;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue;
@ -35,8 +33,6 @@ 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;
import org.apache.hadoop.io.SequenceFile.Reader; import org.apache.hadoop.io.SequenceFile.Reader;
import org.apache.hadoop.security.UnixUserGroupInformation;
import org.apache.hadoop.security.UserGroupInformation;
/** JUnit test case for HLog */ /** JUnit test case for HLog */
@ -96,31 +92,12 @@ 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, false, System.currentTimeMillis()); wal.append(bytes, bytes, kvs, false, System.currentTimeMillis());
} }
// Assert I cannot read back my edits because a flush has not been called // Now call sync and try reading. Opening a Reader before you sync just
// -- it happens automatically at 100 edits ... see top of this method for // gives you EOFE.
// where we set it.
Path walPath = wal.computeFilename(wal.getFilenum());
/**SequenceFile.Reader reader =
new SequenceFile.Reader(this.fs, walPath, this.conf);
int count = 0;
HLogKey key = new HLogKey();
while(reader.next(key)) count++;
assertFalse(count < total);
reader.close();
*/
// Now call sync and retry read.
wal.sync(); wal.sync();
Thread.sleep(70*1000); // Open a Reader.
// Open as another user Path walPath = wal.computeFilename(wal.getFilenum());
final HBaseConfiguration conf2 = new HBaseConfiguration(conf); SequenceFile.Reader reader = HLog.getReader(this.fs, walPath, this.conf);
final String username = UserGroupInformation.getCurrentUGI().getUserName() +
"_" + 1;
UnixUserGroupInformation.saveToConf(conf2,
UnixUserGroupInformation.UGI_PROPERTY_NAME,
new UnixUserGroupInformation(username, new String[]{"supergroup"}));
final FileSystem fs2 = FileSystem.get(conf2);
SequenceFile.Reader reader =
new SequenceFile.Reader(fs2, walPath, conf2);
int count = 0; int count = 0;
HLogKey key = new HLogKey(); HLogKey key = new HLogKey();
while(reader.next(key)) count++; while(reader.next(key)) count++;
@ -173,8 +150,7 @@ 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 = SequenceFile.Reader r = HLog.getReader(this.fs, splits.get(i), this.conf);
new SequenceFile.Reader(this.fs, splits.get(i), this.conf);
try { try {
HLogKey key = new HLogKey(); HLogKey key = new HLogKey();
KeyValue kv = new KeyValue(); KeyValue kv = new KeyValue();
@ -228,7 +204,7 @@ public class TestHLog extends HBaseTestCase implements HConstants {
Path filename = log.computeFilename(log.getFilenum()); Path filename = log.computeFilename(log.getFilenum());
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 = new SequenceFile.Reader(fs, filename, conf); reader = HLog.getReader(fs, filename, conf);
HLogKey key = new HLogKey(); HLogKey key = new HLogKey();
KeyValue val = new KeyValue(); KeyValue val = new KeyValue();
for (int i = 0; i < COL_COUNT; i++) { for (int i = 0; i < COL_COUNT; i++) {