HBASE-2233 Support both Hadoop 0.20 and 0.22

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1134089 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2011-06-09 21:27:41 +00:00
parent bfdceb5117
commit 7cd5403b30
18 changed files with 304 additions and 163 deletions

View File

@ -255,7 +255,6 @@ Release 0.91.0 - Unreleased
hbase-1502
HBASE-3071 Graceful decommissioning of a regionserver
NEW FEATURES
HBASE-2001 Coprocessors: Colocate user code with regions (Mingjie Lai via
Andrew Purtell)
@ -285,6 +284,7 @@ Release 0.91.0 - Unreleased
RS web UIs
HBASE-3691 Add compressor support for 'snappy', google's compressor
(Nichole Treadway and Nicholas Telford)
HBASE-2233 Support both Hadoop 0.20 and 0.22
Release 0.90.4 - Unreleased

View File

@ -33,7 +33,6 @@ import org.apache.hadoop.security.UserGroupInformation;
*/
class ConnectionHeader implements Writable {
private String protocol;
private UserGroupInformation ugi = null;
public ConnectionHeader() {}
@ -47,7 +46,6 @@ class ConnectionHeader implements Writable {
*/
public ConnectionHeader(String protocol, UserGroupInformation ugi) {
this.protocol = protocol;
this.ugi = ugi;
}
@Override
@ -56,26 +54,11 @@ class ConnectionHeader implements Writable {
if (protocol.isEmpty()) {
protocol = null;
}
boolean ugiUsernamePresent = in.readBoolean();
if (ugiUsernamePresent) {
String username = in.readUTF();
ugi.readFields(in);
} else {
ugi = null;
}
}
@Override
public void write(DataOutput out) throws IOException {
Text.writeString(out, (protocol == null) ? "" : protocol);
if (ugi != null) {
//Send both effective user and real user for simple auth
out.writeBoolean(true);
out.writeUTF(ugi.getUserName());
} else {
out.writeBoolean(false);
}
}
public String getProtocol() {
@ -83,10 +66,10 @@ class ConnectionHeader implements Writable {
}
public UserGroupInformation getUgi() {
return ugi;
return null;
}
public String toString() {
return protocol + "-" + ugi;
return protocol;
}
}

View File

@ -45,12 +45,12 @@ import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.io.hfile.Compression;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.mapreduce.hadoopbackport.TotalOrderPartitioner;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.Partitioner;
import org.apache.hadoop.mapreduce.RecordWriter;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
@ -271,13 +271,20 @@ public class HFileOutputFormat extends FileOutputFormat<ImmutableBytesWritable,
* The user should be sure to set the map output value class to either KeyValue or Put before
* running this function.
*/
public static void configureIncrementalLoad(Job job, HTable table) throws IOException {
public static void configureIncrementalLoad(Job job, HTable table)
throws IOException {
Configuration conf = job.getConfiguration();
job.setPartitionerClass(TotalOrderPartitioner.class);
Class<? extends Partitioner> topClass;
try {
topClass = getTotalOrderPartitionerClass();
} catch (ClassNotFoundException e) {
throw new IOException("Failed getting TotalOrderPartitioner", e);
}
job.setPartitionerClass(topClass);
job.setOutputKeyClass(ImmutableBytesWritable.class);
job.setOutputValueClass(KeyValue.class);
job.setOutputFormatClass(HFileOutputFormat.class);
// Based on the configured map output class, set the correct reducer to properly
// sort the incoming values.
// TODO it would be nice to pick one or the other of these formats.
@ -288,7 +295,7 @@ public class HFileOutputFormat extends FileOutputFormat<ImmutableBytesWritable,
} else {
LOG.warn("Unknown map output value type:" + job.getMapOutputValueClass());
}
LOG.info("Looking up current regions for table " + table);
List<ImmutableBytesWritable> startKeys = getRegionStartKeys(table);
LOG.info("Configuring " + startKeys.size() + " reduce partitions " +
@ -302,10 +309,14 @@ public class HFileOutputFormat extends FileOutputFormat<ImmutableBytesWritable,
FileSystem fs = partitionsPath.getFileSystem(conf);
writePartitions(conf, partitionsPath, startKeys);
partitionsPath.makeQualified(fs);
URI cacheUri;
try {
// Below we make explicit reference to the bundled TOP. Its cheating.
// We are assume the define in the hbase bundled TOP is as it is in
// hadoop (whether 0.20 or 0.22, etc.)
cacheUri = new URI(partitionsPath.toString() + "#" +
TotalOrderPartitioner.DEFAULT_PATH);
org.apache.hadoop.hbase.mapreduce.hadoopbackport.TotalOrderPartitioner.DEFAULT_PATH);
} catch (URISyntaxException e) {
throw new IOException(e);
}
@ -317,7 +328,28 @@ public class HFileOutputFormat extends FileOutputFormat<ImmutableBytesWritable,
LOG.info("Incremental table output configured.");
}
/**
* If > hadoop 0.20, then we want to use the hadoop TotalOrderPartitioner.
* If 0.20, then we want to use the TOP that we have under hadoopbackport.
* This method is about hbase being able to run on different versions of
* hadoop. In 0.20.x hadoops, we have to use the TOP that is bundled with
* hbase. Otherwise, we use the one in Hadoop.
* @return Instance of the TotalOrderPartitioner class
* @throws ClassNotFoundException If can't find a TotalOrderPartitioner.
*/
private static Class<? extends Partitioner> getTotalOrderPartitionerClass()
throws ClassNotFoundException {
Class<? extends Partitioner> clazz = null;
try {
clazz = (Class<? extends Partitioner>) Class.forName("org.apache.hadoop.mapreduce.lib.partition.TotalOrderPartitioner");
} catch (ClassNotFoundException e) {
clazz =
(Class<? extends Partitioner>) Class.forName("org.apache.hadoop.hbase.mapreduce.hadoopbackport.TotalOrderPartitioner");
}
return clazz;
}
/**
* Run inside the task to deserialize column family to compression algorithm
* map from the

View File

@ -19,6 +19,7 @@
package org.apache.hadoop.hbase.mapreduce.hadoopbackport;
import java.io.IOException;
import java.lang.reflect.Constructor;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
@ -132,8 +133,7 @@ public class InputSampler<K,V> extends Configured implements Tool {
int samplesPerSplit = numSamples / splitsToSample;
long records = 0;
for (int i = 0; i < splitsToSample; ++i) {
TaskAttemptContext samplingContext = new TaskAttemptContext(
job.getConfiguration(), new TaskAttemptID());
TaskAttemptContext samplingContext = getTaskAttemptContext(job);
RecordReader<K,V> reader = inf.createRecordReader(
splits.get(i), samplingContext);
reader.initialize(splits.get(i), samplingContext);
@ -151,6 +151,32 @@ public class InputSampler<K,V> extends Configured implements Tool {
}
}
/**
* This method is about making hbase portable, making it so it can run on
* more than just hadoop 0.20. In later hadoops, TaskAttemptContext became
* an Interface. But in hadoops where TAC is an Interface, we shouldn't
* be using the classes that are in this package; we should be using the
* native Hadoop ones (We'll throw a ClassNotFoundException if end up in
* here when we should be using native hadoop TotalOrderPartitioner).
* @param job
* @return
* @throws IOException
*/
public static TaskAttemptContext getTaskAttemptContext(final Job job)
throws IOException {
Constructor<TaskAttemptContext> c;
try {
c = TaskAttemptContext.class.getConstructor(Configuration.class, TaskAttemptID.class);
} catch (Exception e) {
throw new IOException("Failed getting constructor", e);
}
try {
return c.newInstance(job.getConfiguration(), new TaskAttemptID());
} catch (Exception e) {
throw new IOException("Failed creating instance", e);
}
}
/**
* Sample from random points in the input.
* General-purpose sampler. Takes numSamples / maxSplitsSampled inputs from
@ -214,8 +240,7 @@ public class InputSampler<K,V> extends Configured implements Tool {
// the target sample keyset
for (int i = 0; i < splitsToSample ||
(i < splits.size() && samples.size() < numSamples); ++i) {
TaskAttemptContext samplingContext = new TaskAttemptContext(
job.getConfiguration(), new TaskAttemptID());
TaskAttemptContext samplingContext = getTaskAttemptContext(job);
RecordReader<K,V> reader = inf.createRecordReader(
splits.get(i), samplingContext);
reader.initialize(splits.get(i), samplingContext);
@ -285,8 +310,7 @@ public class InputSampler<K,V> extends Configured implements Tool {
long records = 0;
long kept = 0;
for (int i = 0; i < splitsToSample; ++i) {
TaskAttemptContext samplingContext = new TaskAttemptContext(
job.getConfiguration(), new TaskAttemptID());
TaskAttemptContext samplingContext = getTaskAttemptContext(job);
RecordReader<K,V> reader = inf.createRecordReader(
splits.get(i), samplingContext);
reader.initialize(splits.get(i), samplingContext);

View File

@ -257,6 +257,7 @@ class CatalogJanitor extends Chore {
for (HColumnDescriptor family: split.getTableDesc().getFamilies()) {
Path p = Store.getStoreHomedir(tabledir, split.getEncodedName(),
family.getName());
if (!fs.exists(p)) continue;
// Look for reference files. Call listStatus with anonymous instance of PathFilter.
FileStatus [] ps = fs.listStatus(p,
new PathFilter () {

View File

@ -595,7 +595,7 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
int port = this.conf.getInt("hbase.master.info.port", 60010);
if (port >= 0) {
String a = this.conf.get("hbase.master.info.bindAddress", "0.0.0.0");
this.infoServer = new InfoServer(MASTER, a, port, false);
this.infoServer = new InfoServer(MASTER, a, port, false, this.conf);
this.infoServer.addServlet("status", "/master-status", MasterStatusServlet.class);
this.infoServer.setAttribute(MASTER, this);
this.infoServer.start();

View File

@ -357,7 +357,7 @@ public class ServerManager {
boolean carryingRoot;
try {
ServerName address = ct.getRootLocation();
carryingRoot = address.equals(serverName);
carryingRoot = address != null && address.equals(serverName);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
LOG.info("Interrupted");
@ -369,7 +369,7 @@ public class ServerManager {
// has an inmemory list of who has what. This list will be cleared as we
// process the dead server but should be find asking it now.
ServerName address = ct.getMetaLocation();
boolean carryingMeta = address.equals(serverName);
boolean carryingMeta = address != null && address.equals(serverName);
if (carryingRoot || carryingMeta) {
this.services.getExecutorService().submit(new MetaServerShutdownHandler(this.master,
this.services, this.deadservers, serverName, carryingRoot, carryingMeta));

View File

@ -1288,7 +1288,7 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
boolean auto = this.conf.getBoolean("hbase.regionserver.info.port.auto", false);
while (true) {
try {
this.infoServer = new InfoServer("regionserver", addr, port, false);
this.infoServer = new InfoServer("regionserver", addr, port, false, this.conf);
this.infoServer.addServlet("status", "/rs-status", RSStatusServlet.class);
this.infoServer.setAttribute(REGIONSERVER, this);
this.infoServer.start();

View File

@ -48,6 +48,7 @@ import java.util.regex.Pattern;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@ -139,8 +140,8 @@ public class HLog implements Syncable {
HLog.logReaderClass = null;
}
private OutputStream hdfs_out; // OutputStream associated with the current SequenceFile.writer
private int initialReplication; // initial replication factor of SequenceFile.writer
private FSDataOutputStream hdfs_out; // FSDataOutputStream associated with the current SequenceFile.writer
private int initialReplication; // initial replication factor of SequenceFile.writer
private Method getNumCurrentReplicas; // refers to DFSOutputStream.getNumCurrentReplicas
final static Object [] NO_ARGS = new Object []{};
@ -368,33 +369,42 @@ public class HLog implements Syncable {
rollWriter();
// handle the reflection necessary to call getNumCurrentReplicas()
this.getNumCurrentReplicas = null;
this.getNumCurrentReplicas = getGetNumCurrentReplicas(this.hdfs_out);
logSyncerThread = new LogSyncer(this.optionalFlushInterval);
Threads.setDaemonThreadRunning(logSyncerThread,
Thread.currentThread().getName() + ".logSyncer");
coprocessorHost = new WALCoprocessorHost(this, conf);
}
/**
* Find the 'getNumCurrentReplicas' on the passed <code>os</code> stream.
* @return Method or null.
*/
private Method getGetNumCurrentReplicas(final FSDataOutputStream os) {
Method m = null;
Exception exception = null;
if (this.hdfs_out != null) {
if (os != null) {
try {
this.getNumCurrentReplicas = this.hdfs_out.getClass().
m = os.getWrappedStream().getClass().
getMethod("getNumCurrentReplicas", new Class<?> []{});
this.getNumCurrentReplicas.setAccessible(true);
m.setAccessible(true);
} catch (NoSuchMethodException e) {
// Thrown if getNumCurrentReplicas() function isn't available
exception = e;
} catch (SecurityException e) {
// Thrown if we can't get access to getNumCurrentReplicas()
exception = e;
this.getNumCurrentReplicas = null; // could happen on setAccessible()
m = null; // could happen on setAccessible()
}
}
if (this.getNumCurrentReplicas != null) {
if (m != null) {
LOG.info("Using getNumCurrentReplicas--HDFS-826");
} else {
LOG.info("getNumCurrentReplicas--HDFS-826 not available; hdfs_out=" +
this.hdfs_out + ", exception=" + exception.getMessage());
os + ", exception=" + exception.getMessage());
}
logSyncerThread = new LogSyncer(this.optionalFlushInterval);
Threads.setDaemonThreadRunning(logSyncerThread,
Thread.currentThread().getName() + ".logSyncer");
coprocessorHost = new WALCoprocessorHost(this, conf);
return m;
}
public void registerWALActionsListener (final WALObserver listener) {
@ -436,9 +446,15 @@ public class HLog implements Syncable {
return logSeqNum.get();
}
/**
* Method used internal to this class and for tests only.
* @return The wrapped stream our writer is using; its not the
* writer's 'out' FSDatoOutputStream but the stream that this 'out' wraps
* (In hdfs its an instance of DFSDataOutputStream).
*/
// usage: see TestLogRolling.java
OutputStream getOutputStream() {
return this.hdfs_out;
return this.hdfs_out.getWrappedStream();
}
/**
@ -482,10 +498,9 @@ public class HLog implements Syncable {
// Can we get at the dfsclient outputstream? If an instance of
// SFLW, it'll have done the necessary reflection to get at the
// protected field name.
OutputStream nextHdfsOut = null;
FSDataOutputStream nextHdfsOut = null;
if (nextWriter instanceof SequenceFileLogWriter) {
nextHdfsOut =
((SequenceFileLogWriter)nextWriter).getDFSCOutputStream();
nextHdfsOut = ((SequenceFileLogWriter)nextWriter).getWriterFSDataOutputStream();
}
// Tell our listeners that a new log was created
if (!this.listeners.isEmpty()) {
@ -768,6 +783,7 @@ public class HLog implements Syncable {
*/
public void closeAndDelete() throws IOException {
close();
if (!fs.exists(this.dir)) return;
FileStatus[] files = fs.listStatus(this.dir);
for(FileStatus file : files) {
Path p = getHLogArchivePath(this.oldLogDir, file.getPath());
@ -776,7 +792,7 @@ public class HLog implements Syncable {
}
}
LOG.debug("Moved " + files.length + " log files to " +
FSUtils.getPath(this.oldLogDir));
FSUtils.getPath(this.oldLogDir));
if (!fs.delete(dir, true)) {
LOG.info("Unable to delete " + dir);
}
@ -966,8 +982,7 @@ public class HLog implements Syncable {
}
}
@Override
public void sync() throws IOException {
private void syncer() throws IOException {
synchronized (this.updateLock) {
if (this.closed) {
return;
@ -1027,9 +1042,10 @@ public class HLog implements Syncable {
*
* @throws Exception
*/
int getLogReplication() throws IllegalArgumentException, IllegalAccessException, InvocationTargetException {
if(this.getNumCurrentReplicas != null && this.hdfs_out != null) {
Object repl = this.getNumCurrentReplicas.invoke(this.hdfs_out, NO_ARGS);
int getLogReplication()
throws IllegalArgumentException, IllegalAccessException, InvocationTargetException {
if (this.getNumCurrentReplicas != null && this.hdfs_out != null) {
Object repl = this.getNumCurrentReplicas.invoke(getOutputStream(), NO_ARGS);
if (repl instanceof Integer) {
return ((Integer)repl).intValue();
}
@ -1042,8 +1058,15 @@ public class HLog implements Syncable {
}
public void hsync() throws IOException {
// Not yet implemented up in hdfs so just call hflush.
sync();
syncer();
}
public void hflush() throws IOException {
syncer();
}
public void sync() throws IOException {
syncer();
}
private void requestLogRoll() {
@ -1309,7 +1332,9 @@ public class HLog implements Syncable {
public static NavigableSet<Path> getSplitEditFilesSorted(final FileSystem fs,
final Path regiondir)
throws IOException {
NavigableSet<Path> filesSorted = new TreeSet<Path>();
Path editsdir = getRegionDirRecoveredEditsDir(regiondir);
if (!fs.exists(editsdir)) return filesSorted;
FileStatus[] files = fs.listStatus(editsdir, new PathFilter() {
@Override
public boolean accept(Path p) {
@ -1327,7 +1352,6 @@ public class HLog implements Syncable {
return result;
}
});
NavigableSet<Path> filesSorted = new TreeSet<Path>();
if (files == null) return filesSorted;
for (FileStatus status: files) {
filesSorted.add(status.getPath());

View File

@ -22,7 +22,6 @@ package org.apache.hadoop.hbase.regionserver.wal;
import static org.apache.hadoop.hbase.util.FSUtils.recoverFileLease;
import java.io.EOFException;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.lang.reflect.Constructor;
@ -44,24 +43,23 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.io.HeapSize;
import org.apache.hadoop.hbase.master.SplitLogManager.TaskFinisher.Status;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.RemoteExceptionHandler;
import org.apache.hadoop.hbase.io.HeapSize;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.wal.HLog.Entry;
import org.apache.hadoop.hbase.regionserver.wal.HLog.Reader;
import org.apache.hadoop.hbase.regionserver.wal.HLog.Writer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CancelableProgressable;
import org.apache.hadoop.hbase.util.ClassSize;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.ClassSize;
import org.apache.hadoop.io.MultipleIOException;
import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
import org.apache.hadoop.io.MultipleIOException;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
@ -530,7 +528,7 @@ public class HLogSplitter {
private static List<FileStatus> listAll(FileSystem fs, Path dir)
throws IOException {
List<FileStatus> fset = new ArrayList<FileStatus>(100);
FileStatus [] files = fs.listStatus(dir);
FileStatus [] files = fs.exists(dir)? fs.listStatus(dir): null;
if (files != null) {
for (FileStatus f : files) {
if (f.isDir()) {

View File

@ -43,13 +43,15 @@ public class SequenceFileLogWriter implements HLog.Writer {
private final Log LOG = LogFactory.getLog(this.getClass());
// The sequence file we delegate to.
private SequenceFile.Writer writer;
// The dfsclient out stream gotten made accessible or null if not available.
private OutputStream dfsClient_out;
// The syncFs method from hdfs-200 or null if not available.
private Method syncFs;
// This is the FSDataOutputStream instance that is the 'out' instance
// in the SequenceFile.Writer 'writer' instance above.
private FSDataOutputStream writer_out;
private Class<? extends HLogKey> keyClass;
private Method syncFs = null;
private Method hflush = null;
/**
* Default constructor.
*/
@ -66,10 +68,10 @@ public class SequenceFileLogWriter implements HLog.Writer {
public SequenceFileLogWriter(Class<? extends HLogKey> keyClass) {
this.keyClass = keyClass;
}
@Override
public void init(FileSystem fs, Path path, Configuration conf)
throws IOException {
throws IOException {
if (null == keyClass) {
keyClass = HLog.getKeyClass(conf);
@ -87,10 +89,63 @@ public class SequenceFileLogWriter implements HLog.Writer {
new DefaultCodec(),
null,
new Metadata());
this.writer_out = getSequenceFilePrivateFSDataOutputStreamAccessible();
this.syncFs = getSyncFs();
this.hflush = getHFlush();
String msg =
"syncFs=" + (this.syncFs != null) + ", hflush=" + (this.hflush != null);
if (this.syncFs != null || this.hflush != null) {
LOG.debug(msg);
} else {
LOG.warn("No sync support! " + msg);
}
}
// 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.
/**
* Now do dirty work to see if syncFs is available on the backing this.writer.
* It will be available in branch-0.20-append and in CDH3.
* @return The syncFs method or null if not available.
* @throws IOException
*/
private Method getSyncFs()
throws IOException {
Method m = null;
try {
// function pointer to writer.syncFs() method; present when sync is hdfs-200.
m = this.writer.getClass().getMethod("syncFs", new Class<?> []{});
} catch (SecurityException e) {
throw new IOException("Failed test for syncfs", e);
} catch (NoSuchMethodException e) {
// Not available
}
return m;
}
/**
* See if hflush (0.21 and 0.22 hadoop) is available.
* @return The hflush method or null if not available.
* @throws IOException
*/
private Method getHFlush()
throws IOException {
Method m = null;
try {
Class<? extends OutputStream> c = getWriterFSDataOutputStream().getClass();
m = c.getMethod("hflush", new Class<?> []{});
} catch (SecurityException e) {
throw new IOException("Failed test for hflush", e);
} catch (NoSuchMethodException e) {
// Ignore
}
return m;
}
// Get at the private FSDataOutputStream inside in SequenceFile so we can
// call sync on it. Make it accessible.
private FSDataOutputStream getSequenceFilePrivateFSDataOutputStreamAccessible()
throws IOException {
FSDataOutputStream out = null;
final Field fields [] = this.writer.getClass().getDeclaredFields();
final String fieldName = "out";
for (int i = 0; i < fields.length; ++i) {
@ -98,34 +153,17 @@ public class SequenceFileLogWriter implements HLog.Writer {
try {
// Make the 'out' field up in SF.Writer accessible.
fields[i].setAccessible(true);
FSDataOutputStream out =
(FSDataOutputStream)fields[i].get(this.writer);
this.dfsClient_out = out.getWrappedStream();
out = (FSDataOutputStream)fields[i].get(this.writer);
break;
} catch (IllegalAccessException ex) {
throw new IOException("Accessing " + fieldName, ex);
} catch (SecurityException e) {
// TODO Auto-generated catch block
e.printStackTrace();
}
}
}
// Now do dirty work to see if syncFs is available.
// Test if syncfs is available.
Method m = null;
boolean append = conf.getBoolean("dfs.support.append", false);
if (append) {
try {
// function pointer to writer.syncFs()
m = this.writer.getClass().getMethod("syncFs", new Class<?> []{});
} catch (SecurityException e) {
throw new IOException("Failed test for syncfs", e);
} catch (NoSuchMethodException e) {
// Not available
}
}
this.syncFs = m;
LOG.info((this.syncFs != null)?
"Using syncFs -- HDFS-200":
("syncFs -- HDFS-200 -- not available, dfs.support.append=" + append));
return out;
}
@Override
@ -146,6 +184,12 @@ public class SequenceFileLogWriter implements HLog.Writer {
} catch (Exception e) {
throw new IOException("Reflection", e);
}
} else if (this.hflush != null) {
try {
this.hflush.invoke(getWriterFSDataOutputStream(), HLog.NO_ARGS);
} catch (Exception e) {
throw new IOException("Reflection", e);
}
}
}
@ -158,7 +202,7 @@ public class SequenceFileLogWriter implements HLog.Writer {
* @return The dfsclient out stream up inside SF.Writer made accessible, or
* null if not available.
*/
public OutputStream getDFSCOutputStream() {
return this.dfsClient_out;
public FSDataOutputStream getWriterFSDataOutputStream() {
return this.writer_out;
}
}
}

View File

@ -737,9 +737,12 @@ public class FSUtils {
} catch (NoSuchMethodException e) {
append = false;
}
} else {
}
if (!append) {
// Look for the 0.21, 0.22, new-style append evidence.
try {
FSDataOutputStream.class.getMethod("hflush", new Class<?> []{});
append = true;
} catch (NoSuchMethodException e) {
append = false;
}

View File

@ -20,16 +20,19 @@
package org.apache.hadoop.hbase.util;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.lang.reflect.Method;
import java.net.URL;
import java.util.Map;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.http.HttpServer;
import org.mortbay.jetty.handler.ContextHandlerCollection;
import org.mortbay.jetty.servlet.Context;
import org.mortbay.jetty.servlet.DefaultServlet;
import java.io.IOException;
import java.net.URL;
import java.util.Map;
/**
* Create a Jetty embedded server to answer http requests. The primary goal
* is to serve up status information for the server.
@ -39,6 +42,8 @@ import java.util.Map;
* "/" -> the jsp server code from (src/hbase-webapps/<name>)
*/
public class InfoServer extends HttpServer {
private final Configuration config;
/**
* Create a status server on the given port.
* The jsp scripts are taken from src/hbase-webapps/<code>name<code>.
@ -49,15 +54,19 @@ public class InfoServer extends HttpServer {
* increment by 1 until it finds a free port.
* @throws IOException e
*/
public InfoServer(String name, String bindAddress, int port, boolean findPort)
public InfoServer(String name, String bindAddress, int port, boolean findPort,
final Configuration c)
throws IOException {
super(name, bindAddress, port, findPort, HBaseConfiguration.create());
webServer.addHandler(new ContextHandlerCollection());
this.config = c;
fixupLogsServletLocation();
}
protected void addDefaultApps(ContextHandlerCollection parent, String appDir)
throws IOException {
super.addDefaultApps(parent, appDir);
/**
* Fixup where the logs app points, make it point at hbase logs rather than
* hadoop logs.
*/
private void fixupLogsServletLocation() {
// Must be same as up in hadoop.
final String logsContextPath = "/logs";
// Now, put my logs in place of hadoops... disable old one first.
@ -72,21 +81,39 @@ public class InfoServer extends HttpServer {
this.defaultContexts.put(oldLogsContext, Boolean.FALSE);
}
// Now do my logs.
// set up the context for "/logs/" if "hadoop.log.dir" property is defined.
// Set up the context for "/logs/" if "hbase.log.dir" property is defined.
String logDir = System.getProperty("hbase.log.dir");
if (logDir != null) {
Context logContext = new Context(parent, "/logs");
// This is a little presumptious but seems to work.
Context logContext =
new Context((ContextHandlerCollection)this.webServer.getHandler(),
logsContextPath);
logContext.setResourceBase(logDir);
logContext.addServlet(DefaultServlet.class, "/");
defaultContexts.put(logContext, true);
}
}
/**
* Get the pathname to the webapps files.
* @param appName eg "secondary" or "datanode"
* @return the pathname as a URL
* @throws FileNotFoundException if 'webapps' directory cannot be found on CLASSPATH.
*/
protected String getWebAppsPath(String appName) throws FileNotFoundException {
// Copied from the super-class.
URL url = getClass().getClassLoader().getResource("hbase-webapps/" + appName);
if (url == null)
throw new FileNotFoundException("webapps/" + appName
+ " not found in CLASSPATH");
String urlString = url.toString();
return urlString.substring(0, urlString.lastIndexOf('/'));
}
/**
* Get the pathname to the <code>path</code> files.
* @return the pathname as a URL
*/
@Override
protected String getWebAppsPath() throws IOException {
// Hack: webapps is not a unique enough element to find in CLASSPATH
// We'll more than likely find the hadoop webapps dir. So, instead
@ -95,29 +122,9 @@ public class InfoServer extends HttpServer {
// master webapp resides is where we want this InfoServer picking up
// web applications.
final String master = "master";
String p = getWebAppDir(master);
// Now strip master + the separator off the end of our context
return p.substring(0, p.length() - (master.length() + 1/* The separator*/));
String p = getWebAppsPath(master);
int index = p.lastIndexOf(master);
// Now strip master off the end if it is present
return index == -1? p: p.substring(0, index);
}
private static String getWebAppsPath(final String path)
throws IOException {
URL url = InfoServer.class.getClassLoader().getResource(path);
if (url == null)
throw new IOException("hbase-webapps not found in CLASSPATH: " + path);
return url.toString();
}
/**
* Get the path for this web app
* @param webappName web app
* @return path
* @throws IOException e
*/
public static String getWebAppDir(final String webappName)
throws IOException {
String webappDir;
webappDir = getWebAppsPath("hbase-webapps/" + webappName);
return webappDir;
}
}
}

View File

@ -66,6 +66,7 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.hdfs.DFSClient;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.mapred.MiniMRCluster;
import org.apache.zookeeper.ZooKeeper;
@ -1233,7 +1234,10 @@ public class HBaseTestingUtility {
Field field = this.dfsCluster.getClass().getDeclaredField("nameNode");
field.setAccessible(true);
NameNode nn = (NameNode)field.get(this.dfsCluster);
nn.namesystem.leaseManager.setLeasePeriod(100, 50000);
field = nn.getClass().getDeclaredField("namesystem");
field.setAccessible(true);
FSNamesystem namesystem = (FSNamesystem)field.get(nn);
namesystem.leaseManager.setLeasePeriod(100, 50000);
}
/**

View File

@ -26,6 +26,7 @@ import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import java.io.IOException;
import java.lang.reflect.Constructor;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
@ -165,7 +166,7 @@ public class TestHFileOutputFormat {
*/
@Test
public void test_LATEST_TIMESTAMP_isReplaced()
throws IOException, InterruptedException {
throws Exception {
Configuration conf = new Configuration(this.util.getConfiguration());
RecordWriter<ImmutableBytesWritable, KeyValue> writer = null;
TaskAttemptContext context = null;
@ -174,8 +175,7 @@ public class TestHFileOutputFormat {
try {
Job job = new Job(conf);
FileOutputFormat.setOutputPath(job, dir);
context = new TaskAttemptContext(job.getConfiguration(),
new TaskAttemptID());
context = getTestTaskAttemptContext(job);
HFileOutputFormat hof = new HFileOutputFormat();
writer = hof.getRecordWriter(context);
final byte [] b = Bytes.toBytes("b");
@ -203,6 +203,32 @@ public class TestHFileOutputFormat {
}
}
/**
* @return True if the available mapreduce is post-0.20.
*/
private static boolean isPost020MapReduce() {
// Here is a coarse test for post 0.20 hadoop; TAC became an interface.
return TaskAttemptContext.class.isInterface();
}
private TaskAttemptContext getTestTaskAttemptContext(final Job job)
throws IOException, Exception {
TaskAttemptContext context;
if (isPost020MapReduce()) {
TaskAttemptID id =
TaskAttemptID.forName("attempt_200707121733_0001_m_000000_0");
Class<?> clazz =
Class.forName("org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl");
Constructor<?> c = clazz.
getConstructor(job.getConfiguration().getClass(), TaskAttemptID.class);
context = (TaskAttemptContext)c.newInstance(job.getConfiguration(), id);
} else {
context = org.apache.hadoop.hbase.mapreduce.hadoopbackport.InputSampler.
getTaskAttemptContext(job);
}
return context;
}
/**
* Run small MR job.
*/
@ -454,8 +480,7 @@ public class TestHFileOutputFormat {
* from the column family descriptor
*/
@Test
public void testColumnFamilyCompression()
throws IOException, InterruptedException {
public void testColumnFamilyCompression() throws Exception {
Configuration conf = new Configuration(this.util.getConfiguration());
RecordWriter<ImmutableBytesWritable, KeyValue> writer = null;
TaskAttemptContext context = null;
@ -484,8 +509,7 @@ public class TestHFileOutputFormat {
setupRandomGeneratorMapper(job);
HFileOutputFormat.configureIncrementalLoad(job, table);
FileOutputFormat.setOutputPath(job, dir);
context = new TaskAttemptContext(job.getConfiguration(),
new TaskAttemptID());
context = getTestTaskAttemptContext(job);
HFileOutputFormat hof = new HFileOutputFormat();
writer = hof.getRecordWriter(context);

View File

@ -173,9 +173,8 @@ public class TestHLogSplit {
throws IOException {
AtomicBoolean stop = new AtomicBoolean(false);
FileStatus[] stats = fs.listStatus(new Path("/hbase/t1"));
assertTrue("Previous test should clean up table dir",
stats == null || stats.length == 0);
assertFalse("Previous test should clean up table dir",
fs.exists(new Path("/hbase/t1")));
generateHLogs(-1);
@ -967,8 +966,7 @@ public class TestHLogSplit {
HLogSplitter.moveRecoveredEditsFromTemp("tmpdir", hbaseDir, oldLogDir,
logfile.getPath().toString(), conf);
Path tdir = HTableDescriptor.getTableDir(hbaseDir, TABLE_NAME);
FileStatus [] files = this.fs.listStatus(tdir);
assertTrue(files == null || files.length == 0);
assertFalse(fs.exists(tdir));
assertEquals(0, countHLog(fs.listStatus(oldLogDir)[0].getPath(), fs, conf));
}

View File

@ -59,6 +59,7 @@ public class TestReplication {
private static Configuration conf1;
private static Configuration conf2;
private static Configuration CONF_WITH_LOCALFS;
private static ZooKeeperWatcher zkw1;
private static ZooKeeperWatcher zkw2;
@ -123,7 +124,7 @@ public class TestReplication {
setIsReplication(true);
LOG.info("Setup second Zk");
CONF_WITH_LOCALFS = HBaseConfiguration.create(conf1);
utility1.startMiniCluster(2);
utility2.startMiniCluster(2);
@ -137,7 +138,6 @@ public class TestReplication {
HBaseAdmin admin2 = new HBaseAdmin(conf2);
admin1.createTable(table);
admin2.createTable(table);
htable1 = new HTable(conf1, tableName);
htable1.setWriteBufferSize(1024);
htable2 = new HTable(conf2, tableName);
@ -476,7 +476,7 @@ public class TestReplication {
testSmallBatch();
String[] args = new String[] {"2", Bytes.toString(tableName)};
Job job = VerifyReplication.createSubmittableJob(conf1, args);
Job job = VerifyReplication.createSubmittableJob(CONF_WITH_LOCALFS, args);
if (job == null) {
fail("Job wasn't created, see the log");
}
@ -500,7 +500,7 @@ public class TestReplication {
}
Delete delete = new Delete(put.getRow());
htable2.delete(delete);
job = VerifyReplication.createSubmittableJob(conf1, args);
job = VerifyReplication.createSubmittableJob(CONF_WITH_LOCALFS, args);
if (job == null) {
fail("Job wasn't created, see the log");
}

View File

@ -34,7 +34,6 @@ public class TestFSUtils {
HBaseTestingUtility htu = new HBaseTestingUtility();
htu.getConfiguration().setBoolean("dfs.support.append", false);
assertFalse(FSUtils.isHDFS(htu.getConfiguration()));
assertFalse(FSUtils.isAppendSupported(htu.getConfiguration()));
htu.getConfiguration().setBoolean("dfs.support.append", true);
MiniDFSCluster cluster = null;
try {
@ -45,4 +44,4 @@ public class TestFSUtils {
if (cluster != null) cluster.shutdown();
}
}
}
}