HBASE-3746 Clean up CompressionTest to not directly reference DistributedFileSystem

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1089684 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Todd Lipcon 2011-04-06 23:58:50 +00:00
parent 657ab7f8d6
commit 1596f98835
2 changed files with 41 additions and 41 deletions

View File

@ -169,6 +169,12 @@ Release 0.91.0 - Unreleased
HBASE-3488 Add CellCounter to count multiple versions of rows
(Subbu M. Iyer via Stack)
Release 0.90.3 - Unreleased
BUG FIXES
HBASE-3746 Clean up CompressionTest to not directly reference
DistributedFileSystem (todd)
Release 0.90.2 - Unreleased

View File

@ -22,6 +22,7 @@ package org.apache.hadoop.hbase.util;
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.io.hfile.Compression;
import org.apache.hadoop.hbase.io.hfile.HFile;
@ -92,55 +93,48 @@ public class CompressionTest {
protected static Path path = new Path(".hfile-comp-test");
public static void usage() {
System.err.println("Usage: CompressionTest HDFS_PATH none|gz|lzo");
System.err.println(
"Usage: CompressionTest <path> none|gz|lzo\n" +
"\n" +
"For example:\n" +
" hbase " + CompressionTest.class + " file:///tmp/testfile gz\n");
System.exit(1);
}
protected static DistributedFileSystem openConnection(String urlString)
throws java.net.URISyntaxException, java.io.IOException {
URI dfsUri = new URI(urlString);
Configuration dfsConf = new Configuration();
DistributedFileSystem dfs = new DistributedFileSystem();
dfs.initialize(dfsUri, dfsConf);
return dfs;
}
public static void doSmokeTest(FileSystem fs, Path path, String codec)
throws Exception {
HFile.Writer writer = new HFile.Writer(
fs, path, HFile.DEFAULT_BLOCKSIZE, codec, null);
writer.append(Bytes.toBytes("testkey"), Bytes.toBytes("testval"));
writer.appendFileInfo(Bytes.toBytes("infokey"), Bytes.toBytes("infoval"));
writer.close();
protected static boolean closeConnection(DistributedFileSystem dfs) {
if (dfs != null) {
try {
dfs.close();
} catch (Exception e) {
e.printStackTrace();
}
HFile.Reader reader = new HFile.Reader(fs, path, null, false, false);
reader.loadFileInfo();
byte[] key = reader.getFirstKey();
boolean rc = Bytes.toString(key).equals("testkey");
reader.close();
if (!rc) {
throw new Exception("Read back incorrect result: " +
Bytes.toStringBinary(key));
}
return dfs == null;
}
public static void main(String[] args) {
if (args.length != 2) usage();
public static void main(String[] args) throws Exception {
if (args.length != 2) {
usage();
System.exit(1);
}
Configuration conf = new Configuration();
Path path = new Path(args[0]);
FileSystem fs = path.getFileSystem(conf);
try {
DistributedFileSystem dfs = openConnection(args[0]);
dfs.delete(path, false);
HFile.Writer writer = new HFile.Writer(dfs, path,
HFile.DEFAULT_BLOCKSIZE, args[1], null);
writer.append(Bytes.toBytes("testkey"), Bytes.toBytes("testval"));
writer.appendFileInfo(Bytes.toBytes("infokey"), Bytes.toBytes("infoval"));
writer.close();
HFile.Reader reader = new HFile.Reader(dfs, path, null, false, false);
reader.loadFileInfo();
byte[] key = reader.getFirstKey();
boolean rc = Bytes.toString(key).equals("testkey");
reader.close();
dfs.delete(path, false);
closeConnection(dfs);
if (rc) System.exit(0);
} catch (Exception e) {
e.printStackTrace();
doSmokeTest(fs, path, args[1]);
} finally {
fs.delete(path, false);
}
System.out.println("FAILED");
System.exit(1);
System.out.println("SUCCESS");
}
}