Committing broke test so nicholas Sze can take a look...
git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@823696 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
8c7e40825a
commit
5a934ef6e7
|
@ -23,7 +23,11 @@ import java.io.IOException;
|
|||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HBaseTestCase;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
|
@ -31,6 +35,8 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.io.SequenceFile;
|
||||
import org.apache.hadoop.io.SequenceFile.Reader;
|
||||
import org.apache.hadoop.security.UnixUserGroupInformation;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
|
||||
|
||||
/** JUnit test case for HLog */
|
||||
|
@ -40,7 +46,9 @@ public class TestHLog extends HBaseTestCase implements HConstants {
|
|||
|
||||
@Override
|
||||
public void setUp() throws Exception {
|
||||
cluster = new MiniDFSCluster(conf, 2, true, (String[])null);
|
||||
// Enable append for these tests.
|
||||
this.conf.setBoolean("dfs.support.append", true);
|
||||
cluster = new MiniDFSCluster(conf, 3, true, (String[])null);
|
||||
// Set the hbase.rootdir to be the home directory in mini dfs.
|
||||
this.conf.set(HConstants.HBASE_DIR,
|
||||
this.cluster.getFileSystem().getHomeDirectory().toString());
|
||||
|
@ -59,6 +67,68 @@ public class TestHLog extends HBaseTestCase implements HConstants {
|
|||
shutdownDfs(cluster);
|
||||
super.tearDown();
|
||||
}
|
||||
|
||||
/**
|
||||
* Test new HDFS-265 sync.
|
||||
* @throws Exception
|
||||
*/
|
||||
public void testSync() throws Exception {
|
||||
byte [] bytes = Bytes.toBytes(getName());
|
||||
// First verify that using streams all works.
|
||||
Path p = new Path(this.dir, getName() + ".fsdos");
|
||||
FSDataOutputStream out = fs.create(p);
|
||||
out.write(bytes);
|
||||
out.sync();
|
||||
FSDataInputStream in = fs.open(p);
|
||||
assertTrue(in.available() > 0);
|
||||
byte [] buffer = new byte [1024];
|
||||
int read = in.read(buffer);
|
||||
assertEquals(bytes.length, read);
|
||||
out.close();
|
||||
in.close();
|
||||
// To be sure, set our flush to be at 100 edits.
|
||||
this.conf.setInt("hbase.regionserver.flushlogentries", 100);
|
||||
Path subdir = new Path(this.dir, "hlogdir");
|
||||
HLog wal = new HLog(this.fs, subdir, this.conf, null);
|
||||
final int total = 20;
|
||||
for (int i = 0; i < total; i++) {
|
||||
List<KeyValue> kvs = new ArrayList<KeyValue>();
|
||||
kvs.add(new KeyValue(Bytes.toBytes(i), bytes, bytes));
|
||||
wal.append(bytes, bytes, kvs, false, System.currentTimeMillis());
|
||||
}
|
||||
// Assert I cannot read back my edits because a flush has not been called
|
||||
// -- it happens automatically at 100 edits ... see top of this method for
|
||||
// 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();
|
||||
Thread.sleep(70*1000);
|
||||
// Open as another user
|
||||
final HBaseConfiguration conf2 = new HBaseConfiguration(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;
|
||||
HLogKey key = new HLogKey();
|
||||
while(reader.next(key)) count++;
|
||||
assertEquals(total, count);
|
||||
reader.close();
|
||||
// Add test that checks to see that an open of a Reader works on a file
|
||||
// that has had a sync done on it.
|
||||
}
|
||||
|
||||
/**
|
||||
* Just write multiple logs then split. Before fix for HADOOP-2283, this
|
||||
|
@ -131,9 +201,10 @@ public class TestHLog extends HBaseTestCase implements HConstants {
|
|||
}
|
||||
|
||||
/**
|
||||
* Tests that we can write out an edit, close, and then read it back in again.
|
||||
* @throws IOException
|
||||
*/
|
||||
public void testAppend() throws IOException {
|
||||
public void testEditAdd() throws IOException {
|
||||
final int COL_COUNT = 10;
|
||||
final byte [] regionName = Bytes.toBytes("regionname");
|
||||
final byte [] tableName = Bytes.toBytes("tablename");
|
||||
|
@ -151,7 +222,6 @@ public class TestHLog extends HBaseTestCase implements HConstants {
|
|||
timestamp, new byte[] { (byte)(i + '0') }));
|
||||
}
|
||||
log.append(regionName, tableName, cols, false, System.currentTimeMillis());
|
||||
log.sync();
|
||||
long logSeqId = log.startCacheFlush();
|
||||
log.completeCacheFlush(regionName, tableName, logSeqId);
|
||||
log.close();
|
||||
|
|
Loading…
Reference in New Issue