HBASE-14904 Mark Base[En|De]coder LimitedPrivate and fix binary compat issue

Conflicts:
	hbase-common/src/main/java/org/apache/hadoop/hbase/codec/BaseDecoder.java
	hbase-common/src/main/java/org/apache/hadoop/hbase/codec/BaseEncoder.java
This commit is contained in:
Enis Soztutar 2015-12-03 18:18:20 -08:00
parent b001019d9b
commit edb8edfeb3
5 changed files with 23 additions and 11 deletions

View File

@ -26,17 +26,18 @@ import javax.annotation.Nonnull;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
/**
* TODO javadoc
*/
@InterfaceAudience.Private
@InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
public abstract class BaseDecoder implements Codec.Decoder {
protected static final Log LOG = LogFactory.getLog(BaseDecoder.class);
protected final PBIS in;
protected final InputStream in;
private Cell current = null;
protected static class PBIS extends PushbackInputStream {
@ -60,13 +61,13 @@ public abstract class BaseDecoder implements Codec.Decoder {
if (firstByte == -1) {
return false;
} else {
in.unread(firstByte);
((PBIS)in).unread(firstByte);
}
try {
this.current = parseCell();
} catch (IOException ioEx) {
in.resetBuf(1); // reset the buffer in case the underlying stream is read from upper layers
((PBIS)in).resetBuf(1); // reset the buffer in case the underlying stream is read from upper layers
rethrowEofException(ioEx);
}
return true;
@ -88,6 +89,10 @@ public abstract class BaseDecoder implements Codec.Decoder {
throw eofEx;
}
protected InputStream getInputStream() {
return in;
}
/**
* Extract a Cell.
* @return a parsed Cell or throws an Exception. EOFException or a generic IOException maybe

View File

@ -20,14 +20,16 @@ package org.apache.hadoop.hbase.codec;
import java.io.IOException;
import java.io.OutputStream;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
/**
* TODO javadoc
*/
@InterfaceAudience.Private
@InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
public abstract class BaseEncoder implements Codec.Encoder {
protected final OutputStream out;
// This encoder is 'done' once flush has been called.
@ -37,6 +39,10 @@ public abstract class BaseEncoder implements Codec.Encoder {
this.out = out;
}
protected OutputStream getOuputStream() {
return out;
}
@Override
public abstract void write(Cell cell) throws IOException;

View File

@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.regionserver.wal;
import java.lang.reflect.Constructor;
import java.lang.reflect.InvocationTargetException;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.io.TagCompressionContext;
import org.apache.hadoop.hbase.io.util.Dictionary;
@ -28,10 +29,10 @@ import org.apache.hadoop.hbase.io.util.Dictionary;
/**
* Context that holds the various dictionaries for compression in WAL.
*/
@InterfaceAudience.Private
@InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
public class CompressionContext {
static final String ENABLE_WAL_TAGS_COMPRESSION =
static final String ENABLE_WAL_TAGS_COMPRESSION =
"hbase.regionserver.wal.tags.enablecompression";
// visible only for WALKey, until we move everything into o.a.h.h.wal

View File

@ -22,12 +22,12 @@ import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.codec.BaseDecoder;
import org.apache.hadoop.hbase.codec.BaseEncoder;
import org.apache.hadoop.hbase.codec.Codec;

View File

@ -236,7 +236,7 @@ public class WALPrettyPrinter {
*/
public void processFile(final Configuration conf, final Path p)
throws IOException {
FileSystem fs = FileSystem.get(conf);
FileSystem fs = p.getFileSystem(conf);
if (!fs.exists(p)) {
throw new FileNotFoundException(p.toString());
}