HBASE-1348 Move 0.20.0 targeted TRUNK to 0.20.0 hadoop

git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@769981 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2009-04-29 22:51:27 +00:00
parent 22f8591fb0
commit e0ac9731ca
44 changed files with 179 additions and 185 deletions

View File

@ -12,6 +12,8 @@ Release 0.20.0 - Unreleased
HBASE-1289 Remove "hbase.fully.distributed" option and update docs
(Nitay Joffe via Stack)
HBASE-1234 Change HBase StoreKey format
HBASE-1348 Move 0.20.0 targeted TRUNK to 0.20.0 hadoop
(Ryan Rawson and Stack)
BUG FIXES
HBASE-1140 "ant clean test" fails (Nitay Joffe via Stack)

Binary file not shown.

BIN
lib/hadoop-0.20.0-core.jar Normal file

Binary file not shown.

Binary file not shown.

Binary file not shown.

Binary file not shown.

BIN
lib/jetty-6.1.14.jar Normal file

Binary file not shown.

Binary file not shown.

Binary file not shown.

Binary file not shown.

BIN
lib/jetty-util-6.1.14.jar Normal file

Binary file not shown.

BIN
lib/jsp-2.1/jsp-2.1.jar Normal file

Binary file not shown.

BIN
lib/jsp-2.1/jsp-api-2.1.jar Normal file

Binary file not shown.

Binary file not shown.

Binary file not shown.

Binary file not shown.

View File

@ -123,6 +123,7 @@ public class HConnectionManager implements HConstants {
private final long pause;
private final int numRetries;
private final int maxRPCAttempts;
private final long rpcTimeout;
private final Object masterLock = new Object();
private volatile boolean closed;
@ -173,6 +174,7 @@ public class HConnectionManager implements HConstants {
this.pause = conf.getLong("hbase.client.pause", 2 * 1000);
this.numRetries = conf.getInt("hbase.client.retries.number", 10);
this.maxRPCAttempts = conf.getInt("hbase.client.rpc.maxattempts", 1);
this.rpcTimeout = conf.getLong("hbase.regionserver.lease.period", 60000);
this.master = null;
this.masterChecked = false;
@ -775,7 +777,7 @@ public class HConnectionManager implements HConstants {
server = (HRegionInterface)HBaseRPC.waitForProxy(
serverInterfaceClass, HBaseRPCProtocolVersion.versionID,
regionServer.getInetSocketAddress(), this.conf,
this.maxRPCAttempts);
this.maxRPCAttempts, this.rpcTimeout);
} catch (RemoteException e) {
throw RemoteExceptionHandler.decodeRemoteException(e);
}

View File

@ -1644,7 +1644,7 @@ public class HTable {
return false;
}
// Let the filter see current row.
this.filter.filterRowKey(endKey);
this.filter.filterRowKey(endKey, 0, endKey.length);
return this.filter.filterAllRemaining();
}

View File

@ -33,7 +33,6 @@ import org.apache.hadoop.io.compress.CompressionOutputStream;
import org.apache.hadoop.io.compress.Compressor;
import org.apache.hadoop.io.compress.Decompressor;
import org.apache.hadoop.io.compress.GzipCodec;
import org.apache.hadoop.io.compress.LzoCodec;
/**
* Compression related stuff.
@ -73,61 +72,20 @@ public final class Compression {
*/
public static enum Algorithm {
LZO("lzo") {
private LzoCodec codec;
@Override
CompressionCodec getCodec() {
if (codec == null) {
Configuration conf = new Configuration();
conf.setBoolean("hadoop.native.lib", true);
codec = new LzoCodec();
codec.setConf(conf);
throw new UnsupportedOperationException("LZO compression is disabled for now");
}
return codec;
}
@Override
public synchronized InputStream createDecompressionStream(
InputStream downStream, Decompressor decompressor,
int downStreamBufferSize) throws IOException {
InputStream bis1 = null;
if (downStreamBufferSize > 0) {
bis1 = new BufferedInputStream(downStream, downStreamBufferSize);
public InputStream createDecompressionStream(InputStream downStream, Decompressor decompressor, int downStreamBufferSize) throws IOException {
throw new UnsupportedOperationException("LZO compression is disabled for now");
}
else {
bis1 = downStream;
}
codec.getConf()
.setInt("io.compression.codec.lzo.buffersize", 64 * 1024);
CompressionInputStream cis =
codec.createInputStream(bis1, decompressor);
BufferedInputStream bis2 = new BufferedInputStream(cis, DATA_IBUF_SIZE);
return bis2;
}
@Override
public synchronized OutputStream createCompressionStream(
OutputStream downStream, Compressor compressor,
int downStreamBufferSize) throws IOException {
OutputStream bos1 = null;
if (downStreamBufferSize > 0) {
bos1 = new BufferedOutputStream(downStream, downStreamBufferSize);
}
else {
bos1 = downStream;
}
codec.getConf()
.setInt("io.compression.codec.lzo.buffersize", 64 * 1024);
CompressionOutputStream cos =
codec.createOutputStream(bos1, compressor);
BufferedOutputStream bos2 =
new BufferedOutputStream(new FinishOnFlushCompressionStream(cos),
DATA_OBUF_SIZE);
return bos2;
public OutputStream createCompressionStream(OutputStream downStream, Compressor compressor, int downStreamBufferSize) throws IOException {
throw new UnsupportedOperationException("LZO compression is disabled for now");
}
},
GZ("gz") {
private GzipCodec codec;

View File

@ -1343,9 +1343,9 @@ public class HFile {
this(null);
}
/**
* Constructor
* @param trailer File tail structure with index stats.
* @param c comparator used to compare keys.
*/
BlockIndex(final RawComparator<byte []>c) {
this.comparator = c;

View File

@ -302,7 +302,7 @@ public class HBaseClient {
this.socket = socketFactory.createSocket();
this.socket.setTcpNoDelay(tcpNoDelay);
// connection time out is 20s
this.socket.connect(remoteId.getAddress(), 20000);
NetUtils.connect(this.socket, remoteId.getAddress(), 20000);
this.socket.setSoTimeout(pingInterval);
break;
} catch (SocketTimeoutException toe) {

View File

@ -45,7 +45,6 @@ import org.apache.hadoop.hbase.client.RetriesExhaustedException;
import org.apache.hadoop.hbase.io.HbaseObjectWritable;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.ipc.VersionedProtocol;
import org.apache.hadoop.metrics.util.MetricsTimeVaryingRate;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.UserGroupInformation;
@ -399,15 +398,19 @@ public class HBaseRPC {
long clientVersion,
InetSocketAddress addr,
Configuration conf,
int maxAttempts
int maxAttempts,
long timeout
) throws IOException {
// HBase does limited number of reconnects which is different from hadoop.
long startTime = System.currentTimeMillis();
IOException ioe;
int reconnectAttempts = 0;
while (true) {
try {
return getProxy(protocol, clientVersion, addr, conf);
} catch(ConnectException se) { // namenode has not been started
LOG.info("Server at " + addr + " not available yet, Zzzzz...");
ioe = se;
if (maxAttempts >= 0 && ++reconnectAttempts >= maxAttempts) {
LOG.info("Server at " + addr + " could not be reached after " +
reconnectAttempts + " tries, giving up.");
@ -417,7 +420,14 @@ public class HBaseRPC {
}
} catch(SocketTimeoutException te) { // namenode is busy
LOG.info("Problem connecting to server: " + addr);
ioe = te;
}
// check if timed out
if (System.currentTimeMillis()-timeout >= startTime) {
throw ioe;
}
// wait for retry
try {
Thread.sleep(1000);
} catch (InterruptedException ie) {
@ -639,18 +649,9 @@ public class HBaseRPC {
rpcMetrics.rpcQueueTime.inc(qTime);
rpcMetrics.rpcProcessingTime.inc(processingTime);
}
MetricsTimeVaryingRate m = rpcMetrics.metricsList.get(call.getMethodName());
if (m != null) {
m.inc(processingTime);
}
else {
rpcMetrics.metricsList.put(call.getMethodName(), new MetricsTimeVaryingRate(call.getMethodName()));
m = rpcMetrics.metricsList.get(call.getMethodName());
m.inc(processingTime);
}
rpcMetrics.rpcQueueTime.inc(qTime);
rpcMetrics.rpcProcessingTime.inc(processingTime);
rpcMetrics.inc(call.getMethodName(), processingTime);
if (verbose) log("Return: "+value);
return new HbaseObjectWritable(method.getReturnType(), value);

View File

@ -30,6 +30,7 @@ import org.apache.hadoop.metrics.MetricsRecord;
import org.apache.hadoop.metrics.MetricsUtil;
import org.apache.hadoop.metrics.Updater;
import org.apache.hadoop.metrics.util.MetricsTimeVaryingRate;
import org.apache.hadoop.metrics.util.MetricsRegistry;
/**
*
@ -65,13 +66,28 @@ public class HBaseRpcMetrics implements Updater {
* - they can be set directly by calling their set/inc methods
* -they can also be read directly - e.g. JMX does this.
*/
public MetricsRegistry registry = new MetricsRegistry();
public MetricsTimeVaryingRate rpcQueueTime = new MetricsTimeVaryingRate("RpcQueueTime");
public MetricsTimeVaryingRate rpcProcessingTime = new MetricsTimeVaryingRate("RpcProcessingTime");
public MetricsTimeVaryingRate rpcQueueTime = new MetricsTimeVaryingRate("RpcQueueTime", registry);
public MetricsTimeVaryingRate rpcProcessingTime = new MetricsTimeVaryingRate("RpcProcessingTime", registry);
public Map <String, MetricsTimeVaryingRate> metricsList = Collections.synchronizedMap(new HashMap<String, MetricsTimeVaryingRate>());
//public Map <String, MetricsTimeVaryingRate> metricsList = Collections.synchronizedMap(new HashMap<String, MetricsTimeVaryingRate>());
private MetricsTimeVaryingRate get(String key) {
return (MetricsTimeVaryingRate) registry.get(key);
}
private MetricsTimeVaryingRate create(String key) {
MetricsTimeVaryingRate newMetric = new MetricsTimeVaryingRate(key, this.registry);
return newMetric;
}
public synchronized void inc(String name, int amt) {
MetricsTimeVaryingRate m = get(name);
if (m == null) {
m = create(name);
}
m.inc(amt);
}
/**
* Push the metrics to the monitoring subsystem on doUpdate() call.
@ -81,15 +97,11 @@ public class HBaseRpcMetrics implements Updater {
rpcQueueTime.pushMetric(metricsRecord);
rpcProcessingTime.pushMetric(metricsRecord);
synchronized (metricsList) {
// Iterate through the rpcMetrics hashmap to propogate the different rpc metrics.
Set<String> keys = metricsList.keySet();
synchronized (registry) {
// Iterate through the registry to propogate the different rpc metrics.
Iterator<String> keyIter = keys.iterator();
while (keyIter.hasNext()) {
Object key = keyIter.next();
MetricsTimeVaryingRate value = metricsList.get(key);
for (String metricName : registry.getKeyList() ) {
MetricsTimeVaryingRate value = (MetricsTimeVaryingRate) registry.get(metricName);
value.pushMetric(metricsRecord);
}

View File

@ -297,6 +297,8 @@ public abstract class HBaseServer {
public void run() {
LOG.info(getName() + ": starting");
SERVER.set(HBaseServer.this);
long lastPurgeTime = 0; // last check for old calls.
while (running) {
SelectionKey key = null;
try {

View File

@ -410,7 +410,7 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
LOG.info("Stopping infoServer");
try {
this.infoServer.stop();
} catch (InterruptedException ex) {
} catch (Exception ex) {
ex.printStackTrace();
}
}

View File

@ -125,6 +125,9 @@ class ServerManager implements HConstants {
String serverName = HServerInfo.getServerName(info);
if (serversToServerInfo.containsKey(serverName) ||
deadServers.contains(serverName)) {
LOG.debug("Server start was rejected: " + serverInfo);
LOG.debug("serversToServerInfo.containsKey: " + serversToServerInfo.containsKey(serverName));
LOG.debug("deadServers.contains: " + deadServers.contains(serverName));
throw new Leases.LeaseStillHeldException(serverName);
}
Watcher watcher = new ServerExpirer(serverName, info.getServerAddress());

View File

@ -25,6 +25,7 @@ import org.apache.hadoop.metrics.MetricsUtil;
import org.apache.hadoop.metrics.Updater;
import org.apache.hadoop.metrics.jvm.JvmMetrics;
import org.apache.hadoop.metrics.util.MetricsIntValue;
import org.apache.hadoop.metrics.util.MetricsRegistry;
/**
@ -37,12 +38,12 @@ import org.apache.hadoop.metrics.util.MetricsIntValue;
public class MasterMetrics implements Updater {
private final Log LOG = LogFactory.getLog(this.getClass());
private final MetricsRecord metricsRecord;
private final MetricsRegistry registry = new MetricsRegistry();
/*
* Count of requests to the cluster since last call to metrics update
*/
private final MetricsIntValue cluster_requests =
new MetricsIntValue("cluster_requests");
new MetricsIntValue("cluster_requests", registry);
public MasterMetrics() {
MetricsContext context = MetricsUtil.getContext("hbase");
@ -90,7 +91,7 @@ public class MasterMetrics implements Updater {
*/
public void incrementRequests(final int inc) {
synchronized(this.cluster_requests) {
this.cluster_requests.inc(inc);
this.cluster_requests.set(this.cluster_requests.get() + inc);
}
}
}

View File

@ -220,6 +220,8 @@ public class HRegionServer implements HConstants, HRegionInterface,
// A sleeper that sleeps for msgInterval.
private final Sleeper sleeper;
private final long rpcTimeout;
/**
* Starts a HRegionServer at the default location
* @param conf
@ -316,6 +318,7 @@ public class HRegionServer implements HConstants, HRegionInterface,
for(int i = 0; i < nbBlocks; i++) {
reservedSpace.add(new byte[DEFAULT_SIZE_RESERVATION_BLOCK]);
}
this.rpcTimeout = conf.getLong("hbase.regionserver.lease.period", 60000);
}
/**
@ -523,8 +526,8 @@ public class HRegionServer implements HConstants, HRegionInterface,
LOG.info("Stopping infoServer");
try {
this.infoServer.stop();
} catch (InterruptedException ex) {
ex.printStackTrace();
} catch (Exception e) {
e.printStackTrace();
}
}
@ -1181,7 +1184,7 @@ public class HRegionServer implements HConstants, HRegionInterface,
master = (HMasterRegionInterface)HBaseRPC.waitForProxy(
HMasterRegionInterface.class, HBaseRPCProtocolVersion.versionID,
masterAddress.getInetSocketAddress(),
this.conf, -1);
this.conf, -1, this.rpcTimeout);
} catch (IOException e) {
LOG.warn("Unable to connect to master. Retrying. Error was:", e);
sleeper.sleep();

View File

@ -30,6 +30,9 @@ import org.apache.hadoop.metrics.MetricsUtil;
import org.apache.hadoop.metrics.Updater;
import org.apache.hadoop.metrics.jvm.JvmMetrics;
import org.apache.hadoop.metrics.util.MetricsIntValue;
import org.apache.hadoop.metrics.util.MetricsTimeVaryingRate;
import org.apache.hadoop.metrics.util.MetricsLongValue;
import org.apache.hadoop.metrics.util.MetricsRegistry;
/**
* This class is for maintaining the various regionserver statistics
@ -43,38 +46,43 @@ public class RegionServerMetrics implements Updater {
private final MetricsRecord metricsRecord;
private long lastUpdate = System.currentTimeMillis();
private static final int MB = 1024*1024;
private MetricsRegistry registry = new MetricsRegistry();
public final MetricsTimeVaryingRate atomicIncrementTime =
new MetricsTimeVaryingRate("atomicIncrementTime", registry);
/**
* Count of regions carried by this regionserver
*/
public final MetricsIntValue regions = new MetricsIntValue("hbase_regions");
public final MetricsIntValue regions =
new MetricsIntValue("regions", registry);
/*
* Count of requests to the regionservers since last call to metrics update
*/
private final MetricsRate requests = new MetricsRate("hbase_requests");
private final MetricsRate requests = new MetricsRate("requests");
/**
* Count of stores open on the regionserver.
*/
public final MetricsIntValue stores = new MetricsIntValue("hbase_stores");
public final MetricsIntValue stores = new MetricsIntValue("stores", registry);
/**
* Count of storefiles open on the regionserver.
*/
public final MetricsIntValue storefiles = new MetricsIntValue("hbase_storefiles");
public final MetricsIntValue storefiles = new MetricsIntValue("storefiles", registry);
/**
* Sum of all the storefile index sizes in this regionserver in MB
*/
public final MetricsIntValue storefileIndexSizeMB =
new MetricsIntValue("hbase_storefileIndexSizeMB");
new MetricsIntValue("storefileIndexSizeMB", registry);
/**
* Sum of all the memcache sizes in this regionserver in MB
*/
public final MetricsIntValue memcacheSizeMB =
new MetricsIntValue("hbase_memcacheSizeMB");
new MetricsIntValue("memcacheSizeMB", registry);
public RegionServerMetrics() {
MetricsContext context = MetricsUtil.getContext("hbase");
@ -134,8 +142,7 @@ public class RegionServerMetrics implements Updater {
if (seconds == 0) {
seconds = 1;
}
sb = Strings.appendKeyValue(sb, "request",
Float.valueOf(this.requests.getPreviousIntervalValue()));
sb = Strings.appendKeyValue(sb, "request", Float.valueOf(getRequests()));
sb = Strings.appendKeyValue(sb, "regions",
Integer.valueOf(this.regions.get()));
sb = Strings.appendKeyValue(sb, "stores",

View File

@ -19,15 +19,6 @@
*/
package org.apache.hadoop.hbase.rest;
import java.io.BufferedReader;
import java.io.IOException;
import java.util.Arrays;
import java.util.Map;
import javax.servlet.ServletException;
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseConfiguration;
@ -38,8 +29,20 @@ import org.apache.hadoop.hbase.rest.parser.IHBaseRestParser;
import org.apache.hadoop.hbase.rest.serializer.RestSerializerFactory;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.InfoServer;
import org.mortbay.http.NCSARequestLog;
import org.mortbay.http.SocketListener;
import org.mortbay.jetty.Connector;
import org.mortbay.jetty.NCSARequestLog;
import org.mortbay.jetty.bio.SocketConnector;
import org.mortbay.jetty.handler.RequestLogHandler;
import org.mortbay.jetty.webapp.WebAppContext;
import org.mortbay.thread.QueuedThreadPool;
import javax.servlet.ServletException;
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
import java.io.BufferedReader;
import java.io.IOException;
import java.util.Arrays;
import java.util.Map;
/**
* Servlet implementation class for hbase REST interface. Presumes container
@ -466,15 +469,30 @@ public class Dispatcher extends javax.servlet.http.HttpServlet {
printUsageAndExit();
}
org.mortbay.jetty.Server webServer = new org.mortbay.jetty.Server();
SocketListener listener = new SocketListener();
listener.setPort(port);
listener.setHost(bindAddress);
listener.setMaxThreads(numThreads);
webServer.addListener(listener);
Connector connector = new SocketConnector();
connector.setPort(port);
connector.setHost(bindAddress);
QueuedThreadPool pool = new QueuedThreadPool();
pool.setMaxThreads(numThreads);
webServer.addConnector(connector);
webServer.setThreadPool(pool);
WebAppContext wac = new WebAppContext();
wac.setContextPath("/");
wac.setWar(InfoServer.getWebAppDir("rest"));
NCSARequestLog ncsa = new NCSARequestLog();
ncsa.setLogLatency(true);
webServer.setRequestLog(ncsa);
webServer.addWebApplication("/", InfoServer.getWebAppDir("rest"));
RequestLogHandler rlh = new RequestLogHandler();
rlh.setRequestLog(ncsa);
rlh.setHandler(wac);
webServer.addHandler(rlh);
webServer.start();
}

View File

@ -19,10 +19,14 @@ package org.apache.hadoop.hbase.util;
import java.io.IOException;
import java.net.URL;
import java.util.Map;
import org.apache.hadoop.http.HttpServer;
import org.mortbay.http.HttpContext;
import org.mortbay.http.handler.ResourceHandler;
import org.mortbay.jetty.handler.ContextHandlerCollection;
import org.mortbay.jetty.handler.HandlerCollection;
import org.mortbay.jetty.servlet.Context;
import org.mortbay.jetty.servlet.DefaultServlet;
import org.mortbay.jetty.webapp.WebAppContext;
/**
* Create a Jetty embedded server to answer http requests. The primary goal
@ -47,19 +51,42 @@ public class InfoServer extends HttpServer {
throws IOException {
super(name, bindAddress, port, findPort);
// Set up the context for "/logs/" if "hbase.log.dir" property is defined.
String logDir = System.getProperty("hbase.log.dir");
if (logDir != null) {
HttpContext logContext = new HttpContext();
logContext.setContextPath("/logs/*");
logContext.setResourceBase(logDir);
logContext.addHandler(new ResourceHandler());
webServer.addContext(logContext);
}
HandlerCollection handlers =
new ContextHandlerCollection();
if (name.equals("master")) {
// Put up the rest webapp.
webServer.addWebApplication("/api", getWebAppDir("rest"));
WebAppContext wac = new WebAppContext();
wac.setContextPath("/api");
wac.setWar(getWebAppDir("rest"));
handlers.addHandler(wac);
}
webServer.addHandler(handlers);
}
protected void addDefaultApps(ContextHandlerCollection parent, String appDir)
throws IOException {
super.addDefaultApps(parent, appDir);
// Must be same as up in hadoop.
final String logsContextPath = "/logs";
// Now, put my logs in place of hadoops... disable old one first.
Context oldLogsContext = null;
for (Map.Entry<Context, Boolean> e : defaultContexts.entrySet()) {
if (e.getKey().getContextPath().equals(logsContextPath)) {
oldLogsContext = e.getKey();
break;
}
}
defaultContexts.put(oldLogsContext, Boolean.FALSE);
// Now do my logs.
// set up the context for "/logs/" if "hadoop.log.dir" property is defined.
String logDir = System.getProperty("hbase.log.dir");
if (logDir != null) {
Context logContext = new Context(parent, "/logs");
logContext.setResourceBase(logDir);
logContext.addServlet(DefaultServlet.class, "/");
defaultContexts.put(logContext, true);
}
}

View File

@ -23,18 +23,12 @@ import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.Arrays;
import junit.framework.TestCase;
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.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.RawLocalFileSystem;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestCase;
import org.apache.hadoop.hbase.io.hfile.HFile.Reader;
import org.apache.hadoop.hbase.io.hfile.HFile.Writer;
import org.apache.hadoop.hbase.util.Bytes;
@ -48,24 +42,14 @@ import org.apache.hadoop.io.RawComparator;
* Remove after tfile is committed and use the tfile version of this class
* instead.</p>
*/
public class TestHFile extends TestCase {
public class TestHFile extends HBaseTestCase {
static final Log LOG = LogFactory.getLog(TestHFile.class);
private static String ROOT_DIR =
System.getProperty("test.build.data", "/tmp/TestHFile");
private FileSystem fs;
private Configuration conf;
private final int minBlockSize = 512;
private static String localFormatter = "%010d";
@Override
public void setUp() {
conf = new HBaseConfiguration();
RawLocalFileSystem rawLFS = new RawLocalFileSystem();
rawLFS.setConf(conf);
fs = new LocalFileSystem(rawLFS);
}
// write some records into the tfile
// write them twice
private int writeSomeRecords(Writer writer, int start, int n)
@ -233,7 +217,7 @@ public class TestHFile extends TestCase {
* Make sure the orginals for our compression libs doesn't change on us.
*/
public void testCompressionOrdinance() {
assertTrue(Compression.Algorithm.LZO.ordinal() == 0);
//assertTrue(Compression.Algorithm.LZO.ordinal() == 0);
assertTrue(Compression.Algorithm.GZ.ordinal() == 1);
assertTrue(Compression.Algorithm.NONE.ordinal() == 2);
}

View File

@ -34,7 +34,6 @@ import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.compress.CompressionCodec;
import org.apache.hadoop.io.compress.GzipCodec;
import org.apache.hadoop.io.compress.LzoCodec;
/**
* Set of long-running tests to measure performance of HFile.
@ -171,9 +170,7 @@ public class TestHFilePerformance extends TestCase {
writer.close();
} else if ("SequenceFile".equals(fileType)){
CompressionCodec codec = null;
if ("lzo".equals(codecName))
codec = new LzoCodec();
else if ("gz".equals(codecName))
if ("gz".equals(codecName))
codec = new GzipCodec();
else if (!"none".equals(codecName))
throw new IOException("Codec not supported.");

View File

@ -21,42 +21,19 @@ package org.apache.hadoop.hbase.io.hfile;
import java.io.IOException;
import junit.framework.TestCase;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.RawLocalFileSystem;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.HBaseTestCase;
import org.apache.hadoop.hbase.util.Bytes;
/**
* Test {@link HFileScanner#seekTo(byte[])} and its variants.
*/
public class TestSeekTo extends TestCase {
private static String ROOT_DIR =
System.getProperty("test.build.data", "/tmp/TestHFile");
private HBaseConfiguration conf;
private LocalFileSystem fs;
@Override
public void setUp() {
conf = new HBaseConfiguration();
RawLocalFileSystem rawLFS = new RawLocalFileSystem();
rawLFS.setConf(conf);
fs = new LocalFileSystem(rawLFS);
}
private FSDataOutputStream createFSOutput(Path name) throws IOException {
if (fs.exists(name)) fs.delete(name, true);
FSDataOutputStream fout = fs.create(name);
return fout;
}
public class TestSeekTo extends HBaseTestCase {
Path makeNewFile() throws IOException {
Path ncTFile = new Path(ROOT_DIR, "basic.hfile");
FSDataOutputStream fout = createFSOutput(ncTFile);
Path ncTFile = new Path(this.testDir, "basic.hfile");
FSDataOutputStream fout = this.fs.create(ncTFile);
HFile.Writer writer = new HFile.Writer(fout, 40, "none", null);
// 4 bytes * 3 * 2 for each key/value +
// 3 for keys, 15 for values = 42 (woot)