HBASE-19497 Fix findbugs and error-prone warnings in hbase-common (branch-2)

Signed-off-by: Apekshit Sharma <appy@apache.org>
This commit is contained in:
Peter Somogyi 2017-12-11 13:38:44 +01:00 committed by Michael Stack
parent 59529a78f0
commit f9f869f60a
No known key found for this signature in database
GPG Key ID: 9816C7FC8ACC93D2
203 changed files with 1286 additions and 808 deletions

View File

@ -21,9 +21,9 @@ package org.apache.hadoop.hbase;
import java.nio.ByteBuffer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.apache.hadoop.hbase.util.Bytes;
/**
* This is a {@link Tag} implementation in which value is backed by an on heap byte array.
@ -100,6 +100,7 @@ public class ArrayBackedTag implements Tag {
/**
* @return The byte array backing this Tag.
*/
@Override
public byte[] getValueArray() {
return this.bytes;
}
@ -107,6 +108,7 @@ public class ArrayBackedTag implements Tag {
/**
* @return the tag type
*/
@Override
public byte getType() {
return this.type;
}
@ -114,6 +116,7 @@ public class ArrayBackedTag implements Tag {
/**
* @return Length of actual tag bytes within the backed buffer
*/
@Override
public int getValueLength() {
return this.length - INFRASTRUCTURE_SIZE;
}
@ -121,6 +124,7 @@ public class ArrayBackedTag implements Tag {
/**
* @return Offset of actual tag bytes within the backed buffer
*/
@Override
public int getValueOffset() {
return this.offset + INFRASTRUCTURE_SIZE;
}

View File

@ -17,10 +17,10 @@
*/
package org.apache.hadoop.hbase;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.log4j.AsyncAppender;
import org.apache.log4j.ConsoleAppender;
import org.apache.log4j.PatternLayout;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Logger class that buffers before trying to log to the specified console.
@ -40,6 +40,7 @@ public class AsyncConsoleAppender extends AsyncAppender {
consoleAppender.setTarget(value);
}
@Override
public void activateOptions() {
consoleAppender.activateOptions();
super.activateOptions();

View File

@ -24,11 +24,11 @@ import java.net.UnknownHostException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.security.UserProvider;
import org.apache.hadoop.hbase.util.DNS;
import org.apache.hadoop.hbase.util.Strings;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Utility methods for helping with security tasks. Downstream users

View File

@ -17,9 +17,9 @@
*/
package org.apache.hadoop.hbase;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration;
import org.apache.yetus.audience.InterfaceAudience;
/**
* HBase version of Hadoop's Configured class that doesn't initialize the

View File

@ -20,9 +20,9 @@ package org.apache.hadoop.hbase;
import java.nio.ByteBuffer;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
/**
* This is a key only Cell implementation which is identical to {@link KeyValue.KeyOnlyKeyValue}

View File

@ -21,10 +21,10 @@ import java.io.IOException;
import java.io.OutputStream;
import java.nio.ByteBuffer;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClassSize;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
@ -154,6 +154,7 @@ public class ByteBufferKeyValue extends ByteBufferCell implements ExtendedCell {
return this.seqId;
}
@Override
public void setSequenceId(long seqId) {
this.seqId = seqId;
}

View File

@ -21,9 +21,9 @@ package org.apache.hadoop.hbase;
import java.nio.ByteBuffer;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
/**
* This is a {@link Tag} implementation in which value is backed by

View File

@ -21,10 +21,10 @@ package org.apache.hadoop.hbase;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.KeyValue.Type;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.apache.hadoop.hbase.shaded.com.google.common.primitives.Longs;

View File

@ -21,7 +21,6 @@ package org.apache.hadoop.hbase;
import java.io.IOException;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
/**
* An interface for iterating through a sequence of cells. Similar to Java's Iterator, but without

View File

@ -34,6 +34,7 @@ import java.util.List;
import java.util.Map.Entry;
import java.util.NavigableMap;
import java.util.Optional;
import org.apache.hadoop.hbase.KeyValue.Type;
import org.apache.hadoop.hbase.io.HeapSize;
import org.apache.hadoop.hbase.util.ByteBufferUtils;

View File

@ -27,12 +27,13 @@ import java.util.concurrent.ScheduledThreadPoolExecutor;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.ScheduledChore.ChoreServicer;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
/**
* ChoreService is a service that can be used to schedule instances of {@link ScheduledChore} to run
* periodically while sharing threads. The ChoreService is backed by a

View File

@ -30,8 +30,8 @@ import java.util.Map;
import org.apache.commons.collections4.iterators.UnmodifiableIterator;
import org.apache.hadoop.conf.Configuration;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Do a shallow merge of multiple KV configuration pools. This is a very useful

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.hbase;
import java.io.IOException;
import java.io.OutputStream;
import java.nio.ByteBuffer;
import org.apache.hadoop.hbase.io.HeapSize;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
import org.apache.yetus.audience.InterfaceAudience;

View File

@ -25,9 +25,9 @@ import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.VersionInfo;
import org.apache.hadoop.hbase.zookeeper.ZKConfig;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Adds HBase configuration files to a Configuration

View File

@ -19,8 +19,6 @@
package org.apache.hadoop.hbase;
import org.apache.yetus.audience.InterfaceAudience;
import java.util.concurrent.Callable;
import java.util.concurrent.Delayed;
import java.util.concurrent.ExecutionException;
@ -31,6 +29,8 @@ import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import org.apache.yetus.audience.InterfaceAudience;
/**
* ScheduledThreadPoolExecutor that will add some jitter to the RunnableScheduledFuture.getDelay.
*
@ -51,12 +51,13 @@ public class JitterScheduledThreadPoolExecutorImpl extends ScheduledThreadPoolEx
this.spread = spread;
}
@Override
protected <V> java.util.concurrent.RunnableScheduledFuture<V> decorateTask(
Runnable runnable, java.util.concurrent.RunnableScheduledFuture<V> task) {
return new JitteredRunnableScheduledFuture<>(task);
}
@Override
protected <V> java.util.concurrent.RunnableScheduledFuture<V> decorateTask(
Callable<V> callable, java.util.concurrent.RunnableScheduledFuture<V> task) {
return new JitteredRunnableScheduledFuture<>(task);

View File

@ -34,12 +34,13 @@ import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClassSize;
import org.apache.hadoop.io.RawComparator;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
/**
* An HBase Key/Value. This is the fundamental HBase Type.
* <p>
@ -79,7 +80,7 @@ public class KeyValue implements ExtendedCell {
private static final Log LOG = LogFactory.getLog(KeyValue.class);
public static final long FIXED_OVERHEAD = ClassSize.OBJECT + // the KeyValue object itself
public static final int FIXED_OVERHEAD = ClassSize.OBJECT + // the KeyValue object itself
ClassSize.REFERENCE + // pointer to "bytes"
2 * Bytes.SIZEOF_INT + // offset, length
Bytes.SIZEOF_LONG;// memstoreTS
@ -195,9 +196,9 @@ public class KeyValue implements ExtendedCell {
*/
public static long getKeyValueDataStructureSize(int klength, int vlength, int tagsLength) {
if (tagsLength == 0) {
return KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE + klength + vlength;
return (long) KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE + klength + vlength;
}
return KeyValue.KEYVALUE_WITH_TAGS_INFRASTRUCTURE_SIZE + klength + vlength + tagsLength;
return (long) KeyValue.KEYVALUE_WITH_TAGS_INFRASTRUCTURE_SIZE + klength + vlength + tagsLength;
}
/**
@ -211,7 +212,7 @@ public class KeyValue implements ExtendedCell {
* @return the key data structure length
*/
public static long getKeyDataStructureSize(int rlength, int flength, int qlength) {
return KeyValue.KEY_INFRASTRUCTURE_SIZE + rlength + flength + qlength;
return (long) KeyValue.KEY_INFRASTRUCTURE_SIZE + rlength + flength + qlength;
}
/**
@ -1524,6 +1525,7 @@ public class KeyValue implements ExtendedCell {
* Returns any tags embedded in the KeyValue. Used in testcases.
* @return The tags
*/
@Override
public List<Tag> getTags() {
int tagsLength = getTagsLength();
if (tagsLength == 0) {
@ -2282,7 +2284,7 @@ public class KeyValue implements ExtendedCell {
int length = kv.getLength();
out.writeInt(length);
out.write(kv.getBuffer(), kv.getOffset(), length);
return length + Bytes.SIZEOF_INT;
return (long) length + Bytes.SIZEOF_INT;
}
/**
@ -2304,7 +2306,7 @@ public class KeyValue implements ExtendedCell {
public static long oswrite(final KeyValue kv, final OutputStream out, final boolean withTags)
throws IOException {
ByteBufferUtils.putInt(out, kv.getSerializedSize(withTags));
return kv.write(out, withTags) + Bytes.SIZEOF_INT;
return (long) kv.write(out, withTags) + Bytes.SIZEOF_INT;
}
@Override
@ -2350,13 +2352,12 @@ public class KeyValue implements ExtendedCell {
*/
@Override
public long heapSize() {
long sum = FIXED_OVERHEAD;
/*
* Deep object overhead for this KV consists of two parts. The first part is the KV object
* itself, while the second part is the backing byte[]. We will only count the array overhead
* from the byte[] only if this is the first KV in there.
*/
return ClassSize.align(sum) +
return ClassSize.align(FIXED_OVERHEAD) +
(offset == 0
? ClassSize.sizeOfByteArray(length) // count both length and object overhead
: length); // only count the number of bytes

View File

@ -22,10 +22,10 @@ import java.nio.ByteBuffer;
import java.util.Collection;
import java.util.List;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.IterableUtils;
import org.apache.hadoop.hbase.util.Strings;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;

View File

@ -29,13 +29,13 @@ import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.hbase.KeyValue.Type;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.io.util.StreamUtils;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.IterableUtils;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.WritableUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Function;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
@ -515,7 +515,7 @@ public class KeyValueUtil {
int length = kv.getLength();
out.writeInt(length);
out.write(kv.getBuffer(), kv.getOffset(), length);
return length + Bytes.SIZEOF_INT;
return (long) length + Bytes.SIZEOF_INT;
}
/**

View File

@ -26,8 +26,8 @@ import java.util.Set;
import java.util.TreeMap;
import java.util.TreeSet;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Namespace POJO class. Used to represent and define namespaces.

View File

@ -19,8 +19,8 @@ package org.apache.hadoop.hbase;
import java.nio.ByteBuffer;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
import org.apache.yetus.audience.InterfaceAudience;
/**
* An extension of the ByteBufferKeyValue where the tags length is always 0

View File

@ -22,11 +22,11 @@ package org.apache.hadoop.hbase;
import java.io.IOException;
import java.io.OutputStream;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
/**
* An extension of the KeyValue where the tags length is always 0
* An extension of the KeyValue where the tags length is always 0
*/
@InterfaceAudience.Private
public class NoTagsKeyValue extends KeyValue {

View File

@ -135,7 +135,7 @@ public final class PrivateCellUtil {
static class TagRewriteCell implements ExtendedCell {
protected Cell cell;
protected byte[] tags;
private static final long HEAP_SIZE_OVERHEAD = ClassSize.OBJECT + 2 * ClassSize.REFERENCE;
private static final int HEAP_SIZE_OVERHEAD = ClassSize.OBJECT + 2 * ClassSize.REFERENCE;
/**
* @param cell The original Cell which it rewrites
@ -317,7 +317,7 @@ public final class PrivateCellUtil {
protected ByteBufferCell cell;
protected byte[] tags;
private static final long HEAP_SIZE_OVERHEAD = ClassSize.OBJECT + 2 * ClassSize.REFERENCE;
private static final int HEAP_SIZE_OVERHEAD = ClassSize.OBJECT + 2 * ClassSize.REFERENCE;
/**
* @param cell The original ByteBufferCell which it rewrites
@ -1501,7 +1501,7 @@ public final class PrivateCellUtil {
}
private static class FirstOnRowCell extends EmptyCell {
private static final long FIXED_HEAPSIZE =
private static final int FIXED_HEAPSIZE =
ClassSize.OBJECT // object
+ ClassSize.REFERENCE // row array
+ Bytes.SIZEOF_INT // row offset
@ -1829,7 +1829,7 @@ public final class PrivateCellUtil {
}
private static class LastOnRowCell extends EmptyCell {
private static final long FIXED_OVERHEAD =
private static final int FIXED_OVERHEAD =
ClassSize.OBJECT // object
+ ClassSize.REFERENCE // row array
+ Bytes.SIZEOF_INT // row offset

View File

@ -24,10 +24,10 @@ import java.util.List;
import java.util.Locale;
import java.util.regex.Pattern;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.net.Address;
import org.apache.hadoop.hbase.util.Addressing;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.net.InetAddresses;

View File

@ -19,8 +19,8 @@
*/
package org.apache.hadoop.hbase;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
/**
* This class is an extension to KeyValue where rowLen and keyLen are cached.

View File

@ -24,8 +24,8 @@ import java.util.Arrays;
import java.util.Set;
import java.util.concurrent.CopyOnWriteArraySet;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Immutable POJO class for representing a table name.

View File

@ -27,10 +27,10 @@ import org.apache.hadoop.hbase.CellBuilderType;
import org.apache.hadoop.hbase.ExtendedCellBuilder;
import org.apache.hadoop.hbase.ExtendedCellBuilderFactory;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.io.ByteBuffInputStream;
import org.apache.hadoop.hbase.nio.ByteBuff;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Basic Cell codec that just writes out all the individual elements of a Cell. Uses ints

View File

@ -27,10 +27,10 @@ import org.apache.hadoop.hbase.CellBuilderType;
import org.apache.hadoop.hbase.ExtendedCellBuilder;
import org.apache.hadoop.hbase.ExtendedCellBuilderFactory;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.io.ByteBuffInputStream;
import org.apache.hadoop.hbase.nio.ByteBuff;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Basic Cell codec that just writes out all the individual elements of a Cell including the tags.
@ -85,6 +85,7 @@ public class CellCodecWithTags implements Codec {
super(in);
}
@Override
protected Cell parseCell() throws IOException {
byte[] row = readByteArray(this.in);
byte[] family = readByteArray(in);

View File

@ -22,9 +22,9 @@ import java.io.OutputStream;
import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.io.CellOutputStream;
import org.apache.hadoop.hbase.nio.ByteBuff;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Encoder/Decoder for Cell.

View File

@ -27,13 +27,13 @@ import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.NoTagsByteBufferKeyValue;
import org.apache.hadoop.hbase.NoTagsKeyValue;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.nio.ByteBuff;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Codec that does KeyValue version 1 serialization.
*
*
* <p>Encodes Cell as serialized in KeyValue with total length prefix.
* This is how KVs were serialized in Puts, Deletes and Results pre-0.96. Its what would
* happen if you called the Writable#write KeyValue implementation. This encoder will fail
@ -69,6 +69,7 @@ public class KeyValueCodec implements Codec {
super(in);
}
@Override
protected Cell parseCell() throws IOException {
// No tags here
return KeyValueUtil.iscreate(in, false);

View File

@ -22,14 +22,14 @@ import java.io.InputStream;
import java.io.OutputStream;
import java.nio.ByteBuffer;
import org.apache.hadoop.hbase.ByteBufferKeyValue;
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.ByteBufferKeyValue;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.nio.ByteBuff;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Codec that does KeyValue version 1 serialization with serializing tags also.
@ -75,6 +75,7 @@ public class KeyValueCodecWithTags implements Codec {
super(in);
}
@Override
protected Cell parseCell() throws IOException {
// create KeyValue with tags
return KeyValueUtil.iscreate(in, true);

View File

@ -17,8 +17,8 @@
*/
package org.apache.hadoop.hbase.exceptions;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.yetus.audience.InterfaceAudience;
@InterfaceAudience.Private
public class UnexpectedStateException extends HBaseIOException {

View File

@ -20,10 +20,10 @@ package org.apache.hadoop.hbase.filter;
import java.nio.ByteBuffer;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
/** Base class for byte array comparators */

View File

@ -22,9 +22,9 @@ import java.nio.BufferOverflowException;
import java.nio.ByteBuffer;
import java.util.Arrays;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Our own implementation of ByteArrayOutputStream where all methods are NOT synchronized and

View File

@ -19,8 +19,8 @@ package org.apache.hadoop.hbase.io;
import java.io.InputStream;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.nio.ByteBuff;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Not thread safe!
@ -42,6 +42,7 @@ public class ByteBuffInputStream extends InputStream {
* because the end of the stream has been reached, the value <code>-1</code> is returned.
* @return the next byte of data, or <code>-1</code> if the end of the stream has been reached.
*/
@Override
public int read() {
if (this.buf.hasRemaining()) {
return (this.buf.get() & 0xff);
@ -58,6 +59,7 @@ public class ByteBuffInputStream extends InputStream {
* @return the total number of bytes actually read into the buffer, or <code>-1</code> if not even
* 1 byte can be read because the end of the stream has been reached.
*/
@Override
public int read (byte b[], int off, int len) {
int avail = available();
if (avail <= 0) {
@ -81,6 +83,7 @@ public class ByteBuffInputStream extends InputStream {
* @param n the number of bytes to be skipped.
* @return the actual number of bytes skipped.
*/
@Override
public long skip(long n) {
long k = Math.min(n, available());
if (k <= 0) {
@ -94,6 +97,7 @@ public class ByteBuffInputStream extends InputStream {
* @return the number of remaining bytes that can be read (or skipped
* over) from this input stream.
*/
@Override
public int available() {
return this.buf.remaining();
}

View File

@ -20,8 +20,8 @@ package org.apache.hadoop.hbase.io;
import java.io.InputStream;
import java.nio.ByteBuffer;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Not thread safe!

View File

@ -25,8 +25,8 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
import org.apache.yetus.audience.InterfaceAudience;
/**
* An OutputStream which writes data into ByteBuffers. It will try to get ByteBuffer, as and when

View File

@ -27,9 +27,9 @@ import java.nio.ByteOrder;
import java.nio.channels.Channels;
import java.nio.channels.WritableByteChannel;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Not thread safe!
@ -142,6 +142,7 @@ public class ByteBufferOutputStream extends OutputStream
ByteBufferUtils.copyFromArrayToBuffer(curBuf, b, off, len);
}
@Override
public void write(ByteBuffer b, int off, int len) throws IOException {
checkSizeAndGrow(len);
ByteBufferUtils.copyFromBufferToBuffer(b, curBuf, off, len);
@ -153,6 +154,7 @@ public class ByteBufferOutputStream extends OutputStream
* @param i the <code>int</code> to write
* @throws IOException if an I/O error occurs.
*/
@Override
public void writeInt(int i) throws IOException {
checkSizeAndGrow(Bytes.SIZEOF_INT);
ByteBufferUtils.putInt(this.curBuf, i);

View File

@ -22,12 +22,12 @@ import java.util.Queue;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
/**
* Like Hadoops' ByteBufferPool only you do not specify desired size when getting a ByteBuffer. This
* pool keeps an upper bound on the count of ByteBuffers in the pool and a fixed size of ByteBuffer

View File

@ -22,8 +22,8 @@ import java.io.IOException;
import java.io.OutputStream;
import java.nio.ByteBuffer;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Our extension of DataOutputStream which implements ByteBufferWriter

View File

@ -21,9 +21,9 @@ import java.io.IOException;
import java.io.OutputStream;
import java.nio.ByteBuffer;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.io.util.StreamUtils;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
import org.apache.yetus.audience.InterfaceAudience;
/**
* When deal with OutputStream which is not ByteBufferWriter type, wrap it with this class. We will
@ -74,6 +74,7 @@ public class ByteBufferWriterOutputStream extends OutputStream
this.os.write(b);
}
@Override
public void write(byte b[], int off, int len) throws IOException {
this.os.write(b, off, len);
}

View File

@ -24,10 +24,10 @@ import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.WritableComparable;
import org.apache.hadoop.io.WritableComparator;
import org.apache.yetus.audience.InterfaceAudience;
/**
* A byte sequence that is usable as a key or value. Based on
@ -146,6 +146,7 @@ implements WritableComparable<ImmutableBytesWritable> {
return this.offset;
}
@Override
public void readFields(final DataInput in) throws IOException {
this.length = in.readInt();
this.bytes = new byte[this.length];
@ -153,6 +154,7 @@ implements WritableComparable<ImmutableBytesWritable> {
this.offset = 0;
}
@Override
public void write(final DataOutput out) throws IOException {
out.writeInt(this.length);
out.write(this.bytes, this.offset, this.length);
@ -173,6 +175,7 @@ implements WritableComparable<ImmutableBytesWritable> {
* @return Positive if left is bigger than right, 0 if they are equal, and
* negative if left is smaller than right.
*/
@Override
public int compareTo(ImmutableBytesWritable that) {
return WritableComparator.compareBytes(
this.bytes, this.offset, this.length,

View File

@ -26,13 +26,13 @@ import java.lang.reflect.InvocationTargetException;
import java.nio.ByteBuffer;
import org.apache.hadoop.hbase.Tag;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.io.util.Dictionary;
import org.apache.hadoop.hbase.io.util.StreamUtils;
import org.apache.hadoop.hbase.nio.ByteBuff;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.IOUtils;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Context that holds the dictionary for Tag compression and doing the compress/uncompress. This

View File

@ -18,8 +18,8 @@
package org.apache.hadoop.hbase.io;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Represents an interval of version timestamps. Presumes timestamps between

View File

@ -27,7 +27,6 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.compress.CodecPool;
import org.apache.hadoop.io.compress.CompressionCodec;
@ -39,6 +38,7 @@ import org.apache.hadoop.io.compress.DefaultCodec;
import org.apache.hadoop.io.compress.DoNotPool;
import org.apache.hadoop.io.compress.GzipCodec;
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Compression related stuff.
@ -104,7 +104,7 @@ public final class Compression {
LZO("lzo") {
// Use base type to avoid compile-time dependencies.
private volatile transient CompressionCodec lzoCodec;
private transient Object lock = new Object();
private final transient Object lock = new Object();
@Override
CompressionCodec getCodec(Configuration conf) {
@ -131,7 +131,7 @@ public final class Compression {
},
GZ("gz") {
private volatile transient GzipCodec codec;
private transient Object lock = new Object();
private final transient Object lock = new Object();
@Override
DefaultCodec getCodec(Configuration conf) {
@ -183,7 +183,7 @@ public final class Compression {
SNAPPY("snappy") {
// Use base type to avoid compile-time dependencies.
private volatile transient CompressionCodec snappyCodec;
private transient Object lock = new Object();
private final transient Object lock = new Object();
@Override
CompressionCodec getCodec(Configuration conf) {
@ -210,7 +210,7 @@ public final class Compression {
LZ4("lz4") {
// Use base type to avoid compile-time dependencies.
private volatile transient CompressionCodec lz4Codec;
private transient Object lock = new Object();
private final transient Object lock = new Object();
@Override
CompressionCodec getCodec(Configuration conf) {
@ -237,7 +237,7 @@ public final class Compression {
BZIP2("bzip2") {
// Use base type to avoid compile-time dependencies.
private volatile transient CompressionCodec bzipCodec;
private transient Object lock = new Object();
private final transient Object lock = new Object();
@Override
CompressionCodec getCodec(Configuration conf) {
@ -264,7 +264,7 @@ public final class Compression {
ZSTD("zstd") {
// Use base type to avoid compile-time dependencies.
private volatile transient CompressionCodec zStandardCodec;
private transient Object lock = new Object();
private final transient Object lock = new Object();
@Override
CompressionCodec getCodec(Configuration conf) {

View File

@ -24,12 +24,12 @@ import java.util.zip.GZIPOutputStream;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.JVM;
import org.apache.hadoop.io.compress.CompressionOutputStream;
import org.apache.hadoop.io.compress.CompressorStream;
import org.apache.hadoop.io.compress.GzipCodec;
import org.apache.hadoop.io.compress.zlib.ZlibFactory;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Fixes an inefficiency in Hadoop's Gzip codec, allowing to reuse compression

View File

@ -21,8 +21,8 @@ import java.security.Key;
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.MD5Hash;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;

View File

@ -18,8 +18,8 @@ package org.apache.hadoop.hbase.io.crypto;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.io.crypto.aes.CommonsCryptoAES;
import org.apache.yetus.audience.InterfaceAudience;
/**
* The default cipher provider. Supports AES via the Commons Crypto.

View File

@ -18,8 +18,8 @@ package org.apache.hadoop.hbase.io.crypto;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.io.crypto.aes.AES;
import org.apache.yetus.audience.InterfaceAudience;
/**
* The default cipher provider. Supports AES via the JCE.

View File

@ -27,7 +27,6 @@ import java.security.spec.InvalidKeySpecException;
import java.util.Arrays;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import javax.crypto.SecretKeyFactory;
import javax.crypto.spec.PBEKeySpec;
import javax.crypto.spec.SecretKeySpec;
@ -38,10 +37,10 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.yetus.audience.InterfaceAudience;
/**
* A facade for encryption algorithms and related support.

View File

@ -23,18 +23,17 @@ import java.io.OutputStream;
import java.security.GeneralSecurityException;
import java.security.Key;
import java.security.SecureRandom;
import javax.crypto.spec.SecretKeySpec;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.apache.hadoop.hbase.io.crypto.Cipher;
import org.apache.hadoop.hbase.io.crypto.CipherProvider;
import org.apache.hadoop.hbase.io.crypto.Context;
import org.apache.hadoop.hbase.io.crypto.Decryptor;
import org.apache.hadoop.hbase.io.crypto.Encryptor;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;

View File

@ -21,12 +21,11 @@ import java.io.InputStream;
import java.security.InvalidAlgorithmParameterException;
import java.security.InvalidKeyException;
import java.security.Key;
import javax.crypto.spec.IvParameterSpec;
import org.apache.hadoop.hbase.io.crypto.Decryptor;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.apache.hadoop.hbase.io.crypto.Decryptor;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;

View File

@ -22,12 +22,11 @@ import java.security.InvalidAlgorithmParameterException;
import java.security.InvalidKeyException;
import java.security.Key;
import java.security.SecureRandom;
import javax.crypto.spec.IvParameterSpec;
import org.apache.hadoop.hbase.io.crypto.Encryptor;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.apache.hadoop.hbase.io.crypto.Encryptor;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;

View File

@ -24,20 +24,19 @@ import java.security.GeneralSecurityException;
import java.security.Key;
import java.security.SecureRandom;
import java.util.Properties;
import javax.crypto.spec.SecretKeySpec;
import org.apache.commons.crypto.cipher.CryptoCipherFactory;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.apache.hadoop.hbase.io.crypto.Cipher;
import org.apache.hadoop.hbase.io.crypto.CipherProvider;
import org.apache.hadoop.hbase.io.crypto.Context;
import org.apache.hadoop.hbase.io.crypto.Decryptor;
import org.apache.hadoop.hbase.io.crypto.Encryptor;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;

View File

@ -17,17 +17,18 @@
*/
package org.apache.hadoop.hbase.io.crypto.aes;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
import org.apache.commons.crypto.stream.CryptoInputStream;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.apache.hadoop.hbase.io.crypto.Decryptor;
import javax.crypto.spec.IvParameterSpec;
import java.io.IOException;
import java.io.InputStream;
import java.security.Key;
import java.util.Properties;
import javax.crypto.spec.IvParameterSpec;
import org.apache.commons.crypto.stream.CryptoInputStream;
import org.apache.hadoop.hbase.io.crypto.Decryptor;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
@InterfaceAudience.Private
@InterfaceStability.Evolving

View File

@ -17,18 +17,19 @@
*/
package org.apache.hadoop.hbase.io.crypto.aes;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
import org.apache.commons.crypto.stream.CryptoOutputStream;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.apache.hadoop.hbase.io.crypto.Encryptor;
import javax.crypto.spec.IvParameterSpec;
import java.io.IOException;
import java.io.OutputStream;
import java.security.Key;
import java.security.SecureRandom;
import java.util.Properties;
import javax.crypto.spec.IvParameterSpec;
import org.apache.commons.crypto.stream.CryptoOutputStream;
import org.apache.hadoop.hbase.io.crypto.Encryptor;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
@InterfaceAudience.Private
@InterfaceStability.Evolving

View File

@ -18,11 +18,12 @@
package org.apache.hadoop.hbase.io.crypto.aes;
import org.apache.commons.crypto.cipher.CryptoCipher;
import org.apache.commons.crypto.utils.Utils;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import java.io.IOException;
import java.security.InvalidAlgorithmParameterException;
import java.security.InvalidKeyException;
import java.security.NoSuchAlgorithmException;
import java.util.Arrays;
import java.util.Properties;
import javax.crypto.Cipher;
import javax.crypto.Mac;
import javax.crypto.SecretKey;
@ -30,12 +31,11 @@ import javax.crypto.ShortBufferException;
import javax.crypto.spec.IvParameterSpec;
import javax.crypto.spec.SecretKeySpec;
import javax.security.sasl.SaslException;
import java.io.IOException;
import java.security.InvalidAlgorithmParameterException;
import java.security.InvalidKeyException;
import java.security.NoSuchAlgorithmException;
import java.util.Arrays;
import java.util.Properties;
import org.apache.commons.crypto.cipher.CryptoCipher;
import org.apache.commons.crypto.utils.Utils;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
/**
* AES encryption and decryption.

View File

@ -23,9 +23,9 @@ import org.apache.hadoop.hbase.ByteBufferKeyOnlyKeyValue;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.io.hfile.BlockType;
import org.apache.hadoop.hbase.io.hfile.HFileContext;
import org.apache.yetus.audience.InterfaceAudience;
@InterfaceAudience.Private
public abstract class AbstractDataBlockEncoder implements DataBlockEncoder {

View File

@ -21,6 +21,7 @@ import java.io.DataOutputStream;
import java.io.IOException;
import java.io.OutputStream;
import java.nio.ByteBuffer;
import org.apache.hadoop.hbase.ByteBufferCell;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator;

View File

@ -19,8 +19,8 @@ package org.apache.hadoop.hbase.io.encoding;
import java.nio.ByteBuffer;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Stores the state of data block encoder at the beginning of new key.

View File

@ -23,10 +23,10 @@ import java.nio.ByteBuffer;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.nio.ByteBuff;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Just copy data, do not do any kind of compression. Use for comparison and

View File

@ -23,9 +23,9 @@ import java.nio.ByteBuffer;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.io.hfile.HFileContext;
import org.apache.hadoop.hbase.nio.ByteBuff;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Encoding of KeyValue. It aims to be fast and efficient using assumptions:

View File

@ -19,8 +19,8 @@ package org.apache.hadoop.hbase.io.encoding;
import java.io.IOException;
import java.io.OutputStream;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Provide access to all data block encoding algorithms. All of the algorithms
@ -172,16 +172,13 @@ public enum DataBlockEncoding {
return algorithm;
}
protected static DataBlockEncoder createEncoder(String fullyQualifiedClassName){
try {
return (DataBlockEncoder)Class.forName(fullyQualifiedClassName).newInstance();
} catch (InstantiationException e) {
throw new RuntimeException(e);
} catch (IllegalAccessException e) {
throw new RuntimeException(e);
} catch (ClassNotFoundException e) {
throw new IllegalArgumentException(e);
}
protected static DataBlockEncoder createEncoder(String fullyQualifiedClassName) {
try {
return (DataBlockEncoder) Class.forName(fullyQualifiedClassName).getDeclaredConstructor()
.newInstance();
} catch (Exception e) {
throw new RuntimeException(e);
}
}
}

View File

@ -23,14 +23,14 @@ import java.nio.ByteBuffer;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.PrivateCellUtil;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.PrivateCellUtil;
import org.apache.hadoop.hbase.nio.ByteBuff;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ObjectIntPair;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Compress using:
@ -208,7 +208,7 @@ public class DiffKeyDeltaEncoder extends BufferedDataBlockEncoder {
private int compressSingleKeyValue(DataOutputStream out, Cell cell, Cell prevCell)
throws IOException {
byte flag = 0;
int flag = 0; // Do not use more bits that can fit into a byte
int kLength = KeyValueUtil.keyLength(cell);
int vLength = cell.getValueLength();

View File

@ -29,13 +29,13 @@ import org.apache.commons.lang3.NotImplementedException;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
import org.apache.hadoop.hbase.io.hfile.HFileContext;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.compress.Compressor;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
@ -257,7 +257,7 @@ public class EncodedDataBlock {
}
BufferGrabbingByteArrayOutputStream stream = new BufferGrabbingByteArrayOutputStream();
baos.writeTo(stream);
this.dataBlockEncoder.endBlockEncoding(encodingCtx, out, stream.buf);
this.dataBlockEncoder.endBlockEncoding(encodingCtx, out, stream.ourBytes);
} catch (IOException e) {
throw new RuntimeException(String.format(
"Bug in encoding part of algorithm %s. " +
@ -268,11 +268,11 @@ public class EncodedDataBlock {
}
private static class BufferGrabbingByteArrayOutputStream extends ByteArrayOutputStream {
private byte[] buf;
private byte[] ourBytes;
@Override
public void write(byte[] b, int off, int len) {
this.buf = b;
public synchronized void write(byte[] b, int off, int len) {
this.ourBytes = b;
}
}

View File

@ -24,14 +24,14 @@ import java.nio.ByteBuffer;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.PrivateCellUtil;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.PrivateCellUtil;
import org.apache.hadoop.hbase.nio.ByteBuff;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ObjectIntPair;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Encoder similar to {@link DiffKeyDeltaEncoder} but supposedly faster.
@ -253,7 +253,7 @@ public class FastDiffDeltaEncoder extends BufferedDataBlockEncoder {
private int compressSingleKeyValue(DataOutputStream out, Cell cell, Cell prevCell)
throws IOException {
byte flag = 0;
int flag = 0; // Do not use more bits than will fit into a byte
int kLength = KeyValueUtil.keyLength(cell);
int vLength = cell.getValueLength();

View File

@ -18,9 +18,9 @@ package org.apache.hadoop.hbase.io.encoding;
import java.io.IOException;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.io.hfile.HFileContext;
import org.apache.hadoop.hbase.nio.ByteBuff;
import org.apache.yetus.audience.InterfaceAudience;
/**
* A decoding context that is created by a reader's encoder, and is shared

View File

@ -21,7 +21,6 @@ import java.io.IOException;
import java.io.InputStream;
import org.apache.commons.io.IOUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.io.ByteBuffInputStream;
import org.apache.hadoop.hbase.io.TagCompressionContext;
import org.apache.hadoop.hbase.io.compress.Compression;
@ -31,6 +30,7 @@ import org.apache.hadoop.hbase.io.crypto.Encryption;
import org.apache.hadoop.hbase.io.hfile.HFileContext;
import org.apache.hadoop.hbase.nio.ByteBuff;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
/**
* A default implementation of {@link HFileBlockDecodingContext}. It assumes the

View File

@ -24,7 +24,6 @@ import java.io.IOException;
import java.io.InputStream;
import java.security.SecureRandom;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.io.ByteArrayOutputStream;
import org.apache.hadoop.hbase.io.TagCompressionContext;
import org.apache.hadoop.hbase.io.compress.Compression;
@ -36,6 +35,7 @@ import org.apache.hadoop.hbase.io.hfile.HFileContext;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.compress.CompressionOutputStream;
import org.apache.hadoop.io.compress.Compressor;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;

View File

@ -18,10 +18,10 @@ package org.apache.hadoop.hbase.io.encoding;
import java.io.IOException;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.io.hfile.BlockType;
import org.apache.hadoop.hbase.io.hfile.HFileContext;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
/**
* An encoding context that is created by a writer's encoder, and is shared

View File

@ -22,11 +22,11 @@ import java.io.DataOutputStream;
import java.io.IOException;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.PrivateCellUtil;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.PrivateCellUtil;
import org.apache.hadoop.io.WritableUtils;
import org.apache.yetus.audience.InterfaceAudience;
@InterfaceAudience.Private
public class NoneEncoder {

View File

@ -24,13 +24,13 @@ import java.nio.ByteBuffer;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.PrivateCellUtil;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.PrivateCellUtil;
import org.apache.hadoop.hbase.nio.ByteBuff;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Compress key by storing size of common prefix with previous KeyValue

View File

@ -28,13 +28,13 @@ import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.CellComparatorImpl;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.io.ByteArrayOutputStream;
import org.apache.hadoop.hbase.nio.ByteBuff;
import org.apache.hadoop.hbase.nio.SingleByteBuff;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.WritableUtils;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Store cells following every row's start offset, so we can binary search to a row's cells.

View File

@ -16,10 +16,9 @@ import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.CellComparatorImpl;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.io.ByteArrayOutputStream;
import org.apache.yetus.audience.InterfaceAudience;
@InterfaceAudience.Private
public class RowIndexEncoderV1 {

View File

@ -20,21 +20,21 @@ import java.nio.ByteBuffer;
import org.apache.hadoop.hbase.ByteBufferCell;
import org.apache.hadoop.hbase.ByteBufferKeyOnlyKeyValue;
import org.apache.hadoop.hbase.ByteBufferKeyValue;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.PrivateCellUtil;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.ByteBufferKeyValue;
import org.apache.hadoop.hbase.PrivateCellUtil;
import org.apache.hadoop.hbase.SizeCachedKeyValue;
import org.apache.hadoop.hbase.SizeCachedNoTagsKeyValue;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.io.encoding.AbstractDataBlockEncoder.AbstractEncodedSeeker;
import org.apache.hadoop.hbase.nio.ByteBuff;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ObjectIntPair;
import org.apache.yetus.audience.InterfaceAudience;
@InterfaceAudience.Private
public class RowIndexSeekerV1 extends AbstractEncodedSeeker {
@ -365,7 +365,7 @@ public class RowIndexSeekerV1 extends AbstractEncodedSeeker {
public Cell toCell() {
Cell ret;
int cellBufSize = getCellBufSize();
long seqId = 0l;
long seqId = 0L;
if (includesMvcc()) {
seqId = memstoreTS;
}

View File

@ -18,15 +18,16 @@
package org.apache.hadoop.hbase.io.hadoopbackport;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import java.io.IOException;
import java.io.InputStream;
import java.io.InterruptedIOException;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.fs.PositionedReadable;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
/**
* The ThrottleInputStream provides bandwidth throttling on a specified

View File

@ -25,9 +25,9 @@ import java.io.IOException;
import java.io.OutputStream;
import java.nio.ByteBuffer;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.nio.ByteBuff;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Various types of HFile blocks. Ordinal values of these enum constants must not be relied upon.

View File

@ -16,8 +16,8 @@
* limitations under the License.
*/
package org.apache.hadoop.hbase.io.hfile;
import org.apache.hadoop.hbase.HConstants;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.io.HeapSize;
import org.apache.hadoop.hbase.io.compress.Compression;
import org.apache.hadoop.hbase.io.crypto.Encryption;
@ -25,6 +25,7 @@ import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ChecksumType;
import org.apache.hadoop.hbase.util.ClassSize;
import org.apache.yetus.audience.InterfaceAudience;
/**
* This carries the information on some of the meta data about the HFile. This

View File

@ -18,11 +18,11 @@
package org.apache.hadoop.hbase.io.hfile;
import org.apache.hadoop.hbase.HConstants;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
import org.apache.hadoop.hbase.io.crypto.Encryption;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.util.ChecksumType;
import org.apache.yetus.audience.InterfaceAudience;
/**
* A builder that helps in building up the HFileContext

View File

@ -22,8 +22,8 @@ import java.io.IOException;
import java.io.OutputStream;
import java.nio.ByteBuffer;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Dictionary interface

View File

@ -21,9 +21,9 @@ package org.apache.hadoop.hbase.io.util;
import java.nio.ByteBuffer;
import java.util.HashMap;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;

View File

@ -23,9 +23,9 @@ import java.io.InputStream;
import java.io.OutputStream;
import java.nio.ByteBuffer;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.nio.ByteBuff;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;

View File

@ -21,11 +21,11 @@ import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.channels.ReadableByteChannel;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ObjectIntPair;
import org.apache.hadoop.io.WritableUtils;
import org.apache.yetus.audience.InterfaceAudience;
/**
* An abstract class that abstracts out as to how the byte buffers are used,

View File

@ -24,10 +24,10 @@ import java.nio.ByteBuffer;
import java.nio.InvalidMarkException;
import java.nio.channels.ReadableByteChannel;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ObjectIntPair;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
@ -177,6 +177,7 @@ public class MultiByteBuff extends ByteBuff {
* @param index
* @return the int value at the given index
*/
@Override
public int getInt(int index) {
// Mostly the index specified will land within this current item. Short circuit for that
int itemIndex;
@ -207,6 +208,7 @@ public class MultiByteBuff extends ByteBuff {
* @param index
* @return the short value at the given index
*/
@Override
public short getShort(int index) {
// Mostly the index specified will land within this current item. Short circuit for that
int itemIndex;
@ -228,9 +230,9 @@ public class MultiByteBuff extends ByteBuff {
ByteBuffer nextItem = items[itemIndex + 1];
// Get available one byte from this item and remaining one from next
short n = 0;
n ^= ByteBufferUtils.toByte(item, offsetInItem) & 0xFF;
n <<= 8;
n ^= ByteBufferUtils.toByte(nextItem, 0) & 0xFF;
n = (short) (n ^ (ByteBufferUtils.toByte(item, offsetInItem) & 0xFF));
n = (short) (n << 8);
n = (short) (n ^ (ByteBufferUtils.toByte(nextItem, 0) & 0xFF));
return n;
}
@ -287,12 +289,12 @@ public class MultiByteBuff extends ByteBuff {
// Get available bytes from this item and remaining from next
short l = 0;
for (int i = offsetInItem; i < item.capacity(); i++) {
l <<= 8;
l ^= ByteBufferUtils.toByte(item, i) & 0xFF;
l = (short) (l << 8);
l = (short) (l ^ (ByteBufferUtils.toByte(item, i) & 0xFF));
}
for (int i = 0; i < Bytes.SIZEOF_SHORT - remainingLen; i++) {
l <<= 8;
l ^= ByteBufferUtils.toByte(nextItem, i) & 0xFF;
l = (short) (l << 8);
l = (short) (l ^ (ByteBufferUtils.toByte(item, i) & 0xFF));
}
return l;
}
@ -327,6 +329,7 @@ public class MultiByteBuff extends ByteBuff {
* @param index
* @return the long value at the given index
*/
@Override
public long getLong(int index) {
// Mostly the index specified will land within this current item. Short circuit for that
int itemIndex;
@ -520,9 +523,9 @@ public class MultiByteBuff extends ByteBuff {
return this.curItem.getShort();
}
short n = 0;
n ^= get() & 0xFF;
n <<= 8;
n ^= get() & 0xFF;
n = (short) (n ^ (get() & 0xFF));
n = (short) (n << 8);
n = (short) (n ^ (get() & 0xFF));
return n;
}

View File

@ -17,20 +17,19 @@
*/
package org.apache.hadoop.hbase.nio;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.channels.ReadableByteChannel;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
import org.apache.hadoop.hbase.util.ObjectIntPair;
import org.apache.hadoop.hbase.util.UnsafeAccess;
import org.apache.hadoop.hbase.util.UnsafeAvailChecker;
import org.apache.yetus.audience.InterfaceAudience;
import sun.nio.ch.DirectBuffer;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
/**
* An implementation of ByteBuff where a single BB backs the BBI. This just acts
* as a wrapper over a normal BB - offheap or onheap

View File

@ -24,8 +24,8 @@ import java.util.SortedSet;
import java.util.TreeSet;
import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.net.Address;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Stores the group information of region server groups.

View File

@ -19,16 +19,16 @@
package org.apache.hadoop.hbase.security;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.AuthUtil;
import org.apache.yetus.audience.InterfaceAudience;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
/**
* Keeps lists of superusers and super groups loaded from HBase configuration,
* checks if certain user is regarded as superuser.

View File

@ -28,15 +28,17 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ExecutionException;
import org.apache.hadoop.hbase.shaded.com.google.common.cache.LoadingCache;
import org.apache.hadoop.conf.Configuration;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.Methods;
import org.apache.hadoop.security.Groups;
import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.cache.LoadingCache;
/**
* Wrapper to abstract out usage of user and group information in HBase.

View File

@ -24,6 +24,14 @@ import java.util.concurrent.Callable;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.hbase.BaseConfigurable;
import org.apache.hadoop.security.Groups;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.cache.CacheBuilder;
import org.apache.hadoop.hbase.shaded.com.google.common.cache.CacheLoader;
import org.apache.hadoop.hbase.shaded.com.google.common.cache.LoadingCache;
@ -31,13 +39,6 @@ import org.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.Listenab
import org.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.ListeningExecutorService;
import org.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.MoreExecutors;
import org.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.hbase.BaseConfigurable;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.security.Groups;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.ReflectionUtils;
/**
* Provide an instance of a user. Allows custom {@link User} creation.
@ -98,11 +99,11 @@ public class UserProvider extends BaseConfigurable {
}
// Provide the reload function that uses the executor thread.
public ListenableFuture<String[]> reload(final String k,
String[] oldValue) throws Exception {
@Override
public ListenableFuture<String[]> reload(final String k, String[] oldValue)
throws Exception {
return executor.submit(new Callable<String[]>() {
@Override
public String[] call() throws Exception {
return getGroupStrings(k);

View File

@ -42,7 +42,7 @@ public class SpanReceiverHost {
private static enum SingletonHolder {
INSTANCE;
transient Object lock = new Object();
final transient Object lock = new Object();
transient SpanReceiverHost host = null;
}

View File

@ -18,9 +18,6 @@
package org.apache.hadoop.hbase.types;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import java.util.AbstractMap;
import java.util.Collection;
import java.util.Comparator;
@ -32,6 +29,9 @@ import java.util.Set;
import java.util.SortedSet;
import java.util.concurrent.ConcurrentNavigableMap;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
/**
* A Map that keeps a sorted array in order to provide the concurrent map interface.
* Keeping a sorted array means that it's much more cache line friendly, making reads faster
@ -706,7 +706,7 @@ public class CopyOnWriteArrayMap<K, V> extends AbstractMap<K, V>
}
}
private final class ArrayKeyIterator<K, V> implements Iterator<K> {
private static final class ArrayKeyIterator<K, V> implements Iterator<K> {
int index;
private final ArrayHolder<K, V> holder;
@ -732,7 +732,7 @@ public class CopyOnWriteArrayMap<K, V> extends AbstractMap<K, V>
}
}
private final class ArrayValueIterator<K, V> implements Iterator<V> {
private static final class ArrayValueIterator<K, V> implements Iterator<V> {
int index;
private final ArrayHolder<K, V> holder;
@ -758,7 +758,7 @@ public class CopyOnWriteArrayMap<K, V> extends AbstractMap<K, V>
}
}
private final class ArrayEntryIterator<K, V> implements Iterator<Map.Entry<K, V>> {
private static final class ArrayEntryIterator<K, V> implements Iterator<Map.Entry<K, V>> {
int index;
private final ArrayHolder<K, V> holder;
@ -879,8 +879,7 @@ public class CopyOnWriteArrayMap<K, V> extends AbstractMap<K, V>
* Binary search for a given key
* @param needle The key to look for in all of the entries
* @return Same return value as Arrays.binarySearch.
* Positive numbers mean the index.
* Otherwise (-1 * insertion point) - 1
* Positive numbers mean the index. Otherwise (-1 * insertion point) - 1
*/
int find(K needle) {
int begin = startIndex;

View File

@ -17,9 +17,9 @@
*/
package org.apache.hadoop.hbase.types;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.Order;
import org.apache.hadoop.hbase.util.PositionedByteRange;
import org.apache.yetus.audience.InterfaceAudience;
/**
* <p>

View File

@ -17,10 +17,10 @@
*/
package org.apache.hadoop.hbase.types;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.Order;
import org.apache.hadoop.hbase.util.PositionedByteRange;
import org.apache.hadoop.hbase.util.SimplePositionedMutableByteRange;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Wraps an existing {@link DataType} implementation as a fixed-length
@ -47,25 +47,39 @@ public class FixedLengthWrapper<T> implements DataType<T> {
/**
* Retrieve the maximum length (in bytes) of encoded values.
*/
public int getLength() { return length; }
public int getLength() {
return length;
}
@Override
public boolean isOrderPreserving() { return base.isOrderPreserving(); }
public boolean isOrderPreserving() {
return base.isOrderPreserving();
}
@Override
public Order getOrder() { return base.getOrder(); }
public Order getOrder() {
return base.getOrder();
}
@Override
public boolean isNullable() { return base.isNullable(); }
public boolean isNullable() {
return base.isNullable();
}
@Override
public boolean isSkippable() { return true; }
public boolean isSkippable() {
return true;
}
@Override
public int encodedLength(T val) { return length; }
public int encodedLength(T val) {
return length;
}
@Override
public Class<T> encodedClass() { return base.encodedClass(); }
public Class<T> encodedClass() {
return base.encodedClass();
}
@Override
public int skip(PositionedByteRange src) {
@ -99,7 +113,9 @@ public class FixedLengthWrapper<T> implements DataType<T> {
+ ") exceeds max length (" + length + ").");
}
// TODO: is the zero-padding appropriate?
for (; written < length; written++) { dst.put((byte) 0x00); }
for (; written < length; written++) {
dst.put((byte) 0x00);
}
return written;
}
}

View File

@ -17,10 +17,10 @@
*/
package org.apache.hadoop.hbase.types;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.Order;
import org.apache.hadoop.hbase.util.OrderedBytes;
import org.apache.hadoop.hbase.util.PositionedByteRange;
import org.apache.yetus.audience.InterfaceAudience;
/**
* A {@code byte[]} of variable-length. Build on
@ -32,10 +32,14 @@ public class OrderedBlob extends OrderedBytesBase<byte[]> {
public static final OrderedBlob ASCENDING = new OrderedBlob(Order.ASCENDING);
public static final OrderedBlob DESCENDING = new OrderedBlob(Order.DESCENDING);
protected OrderedBlob(Order order) { super(order); }
protected OrderedBlob(Order order) {
super(order);
}
@Override
public boolean isSkippable() { return false; }
public boolean isSkippable() {
return false;
}
@Override
public int encodedLength(byte[] val) {
@ -45,7 +49,9 @@ public class OrderedBlob extends OrderedBytesBase<byte[]> {
}
@Override
public Class<byte[]> encodedClass() { return byte[].class; }
public Class<byte[]> encodedClass() {
return byte[].class;
}
@Override
public byte[] decode(PositionedByteRange src) {

View File

@ -17,10 +17,10 @@
*/
package org.apache.hadoop.hbase.types;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.Order;
import org.apache.hadoop.hbase.util.OrderedBytes;
import org.apache.hadoop.hbase.util.PositionedByteRange;
import org.apache.yetus.audience.InterfaceAudience;
/**
* An alternative to {@link OrderedBlob} for use by {@link Struct} fields that
@ -33,7 +33,9 @@ public class OrderedBlobVar extends OrderedBytesBase<byte[]> {
public static final OrderedBlobVar ASCENDING = new OrderedBlobVar(Order.ASCENDING);
public static final OrderedBlobVar DESCENDING = new OrderedBlobVar(Order.DESCENDING);
protected OrderedBlobVar(Order order) { super(order); }
protected OrderedBlobVar(Order order) {
super(order);
}
@Override
public int encodedLength(byte[] val) {
@ -41,7 +43,9 @@ public class OrderedBlobVar extends OrderedBytesBase<byte[]> {
}
@Override
public Class<byte[]> encodedClass() { return byte[].class; }
public Class<byte[]> encodedClass() {
return byte[].class;
}
@Override
public byte[] decode(PositionedByteRange src) {

View File

@ -17,10 +17,10 @@
*/
package org.apache.hadoop.hbase.types;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.Order;
import org.apache.hadoop.hbase.util.OrderedBytes;
import org.apache.hadoop.hbase.util.PositionedByteRange;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Base class for data types backed by the {@link OrderedBytes} encoding
@ -31,21 +31,31 @@ public abstract class OrderedBytesBase<T> implements DataType<T> {
protected final Order order;
protected OrderedBytesBase(Order order) { this.order = order; }
protected OrderedBytesBase(Order order) {
this.order = order;
}
@Override
public boolean isOrderPreserving() { return true; }
public boolean isOrderPreserving() {
return true;
}
@Override
public Order getOrder() { return order; }
public Order getOrder() {
return order;
}
// almost all OrderedBytes implementations are nullable.
@Override
public boolean isNullable() { return true; }
public boolean isNullable() {
return true;
}
// almost all OrderedBytes implementations are skippable.
@Override
public boolean isSkippable() { return true; }
public boolean isSkippable() {
return true;
}
@Override
public int skip(PositionedByteRange src) {

View File

@ -17,11 +17,10 @@
*/
package org.apache.hadoop.hbase.types;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.Order;
import org.apache.hadoop.hbase.util.OrderedBytes;
import org.apache.hadoop.hbase.util.PositionedByteRange;
import org.apache.yetus.audience.InterfaceAudience;
/**
* A {@code float} of 32-bits using a fixed-length encoding. Based on
@ -33,16 +32,24 @@ public class OrderedFloat32 extends OrderedBytesBase<Float> {
public static final OrderedFloat32 ASCENDING = new OrderedFloat32(Order.ASCENDING);
public static final OrderedFloat32 DESCENDING = new OrderedFloat32(Order.DESCENDING);
protected OrderedFloat32(Order order) { super(order); }
protected OrderedFloat32(Order order) {
super(order);
}
@Override
public boolean isNullable() { return false; }
public boolean isNullable() {
return false;
}
@Override
public int encodedLength(Float val) { return 5; }
public int encodedLength(Float val) {
return 5;
}
@Override
public Class<Float> encodedClass() { return Float.class; }
public Class<Float> encodedClass() {
return Float.class;
}
@Override
public Float decode(PositionedByteRange src) {
@ -51,7 +58,9 @@ public class OrderedFloat32 extends OrderedBytesBase<Float> {
@Override
public int encode(PositionedByteRange dst, Float val) {
if (null == val) throw new IllegalArgumentException("Null values not supported.");
if (null == val) {
throw new IllegalArgumentException("Null values not supported.");
}
return OrderedBytes.encodeFloat32(dst, val, order);
}

View File

@ -17,11 +17,10 @@
*/
package org.apache.hadoop.hbase.types;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.Order;
import org.apache.hadoop.hbase.util.OrderedBytes;
import org.apache.hadoop.hbase.util.PositionedByteRange;
import org.apache.yetus.audience.InterfaceAudience;
/**
* A {@code double} of 64-bits using a fixed-length encoding. Built on
@ -33,16 +32,24 @@ public class OrderedFloat64 extends OrderedBytesBase<Double> {
public static final OrderedFloat64 ASCENDING = new OrderedFloat64(Order.ASCENDING);
public static final OrderedFloat64 DESCENDING = new OrderedFloat64(Order.DESCENDING);
protected OrderedFloat64(Order order) { super(order); }
protected OrderedFloat64(Order order) {
super(order);
}
@Override
public boolean isNullable() { return false; }
public boolean isNullable() {
return false;
}
@Override
public int encodedLength(Double val) { return 9; }
public int encodedLength(Double val) {
return 9;
}
@Override
public Class<Double> encodedClass() { return Double.class; }
public Class<Double> encodedClass() {
return Double.class;
}
@Override
public Double decode(PositionedByteRange src) {
@ -51,7 +58,9 @@ public class OrderedFloat64 extends OrderedBytesBase<Double> {
@Override
public int encode(PositionedByteRange dst, Double val) {
if (null == val) throw new IllegalArgumentException("Null values not supported.");
if (null == val) {
throw new IllegalArgumentException("Null values not supported.");
}
return OrderedBytes.encodeFloat64(dst, val, order);
}

View File

@ -17,11 +17,10 @@
*/
package org.apache.hadoop.hbase.types;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.Order;
import org.apache.hadoop.hbase.util.OrderedBytes;
import org.apache.hadoop.hbase.util.PositionedByteRange;
import org.apache.yetus.audience.InterfaceAudience;
/**
* A {@code short} of 16-bits using a fixed-length encoding. Built on
@ -33,16 +32,24 @@ public class OrderedInt16 extends OrderedBytesBase<Short> {
public static final OrderedInt16 ASCENDING = new OrderedInt16(Order.ASCENDING);
public static final OrderedInt16 DESCENDING = new OrderedInt16(Order.DESCENDING);
protected OrderedInt16(Order order) { super(order); }
protected OrderedInt16(Order order) {
super(order);
}
@Override
public boolean isNullable() { return false; }
public boolean isNullable() {
return false;
}
@Override
public int encodedLength(Short val) { return 3; }
public int encodedLength(Short val) {
return 3;
}
@Override
public Class<Short> encodedClass() { return Short.class; }
public Class<Short> encodedClass() {
return Short.class;
}
@Override
public Short decode(PositionedByteRange src) {
@ -51,7 +58,9 @@ public class OrderedInt16 extends OrderedBytesBase<Short> {
@Override
public int encode(PositionedByteRange dst, Short val) {
if (null == val) throw new IllegalArgumentException("Null values not supported.");
if (null == val) {
throw new IllegalArgumentException("Null values not supported.");
}
return OrderedBytes.encodeInt16(dst, val, order);
}

View File

@ -17,11 +17,10 @@
*/
package org.apache.hadoop.hbase.types;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.Order;
import org.apache.hadoop.hbase.util.OrderedBytes;
import org.apache.hadoop.hbase.util.PositionedByteRange;
import org.apache.yetus.audience.InterfaceAudience;
/**
* An {@code int} of 32-bits using a fixed-length encoding. Built on
@ -33,16 +32,24 @@ public class OrderedInt32 extends OrderedBytesBase<Integer> {
public static final OrderedInt32 ASCENDING = new OrderedInt32(Order.ASCENDING);
public static final OrderedInt32 DESCENDING = new OrderedInt32(Order.DESCENDING);
protected OrderedInt32(Order order) { super(order); }
protected OrderedInt32(Order order) {
super(order);
}
@Override
public boolean isNullable() { return false; }
public boolean isNullable() {
return false;
}
@Override
public int encodedLength(Integer val) { return 5; }
public int encodedLength(Integer val) {
return 5;
}
@Override
public Class<Integer> encodedClass() { return Integer.class; }
public Class<Integer> encodedClass() {
return Integer.class;
}
@Override
public Integer decode(PositionedByteRange src) {
@ -51,7 +58,9 @@ public class OrderedInt32 extends OrderedBytesBase<Integer> {
@Override
public int encode(PositionedByteRange dst, Integer val) {
if (null == val) throw new IllegalArgumentException("Null values not supported.");
if (null == val) {
throw new IllegalArgumentException("Null values not supported.");
}
return OrderedBytes.encodeInt32(dst, val, order);
}

View File

@ -17,11 +17,10 @@
*/
package org.apache.hadoop.hbase.types;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.Order;
import org.apache.hadoop.hbase.util.OrderedBytes;
import org.apache.hadoop.hbase.util.PositionedByteRange;
import org.apache.yetus.audience.InterfaceAudience;
/**
* A {@code long} of 64-bits using a fixed-length encoding. Built on
@ -33,16 +32,24 @@ public class OrderedInt64 extends OrderedBytesBase<Long> {
public static final OrderedInt64 ASCENDING = new OrderedInt64(Order.ASCENDING);
public static final OrderedInt64 DESCENDING = new OrderedInt64(Order.DESCENDING);
protected OrderedInt64(Order order) { super(order); }
protected OrderedInt64(Order order) {
super(order);
}
@Override
public boolean isNullable() { return false; }
public boolean isNullable() {
return false;
}
@Override
public int encodedLength(Long val) { return 9; }
public int encodedLength(Long val) {
return 9;
}
@Override
public Class<Long> encodedClass() { return Long.class; }
public Class<Long> encodedClass() {
return Long.class;
}
@Override
public Long decode(PositionedByteRange src) {
@ -51,7 +58,9 @@ public class OrderedInt64 extends OrderedBytesBase<Long> {
@Override
public int encode(PositionedByteRange dst, Long val) {
if (null == val) throw new IllegalArgumentException("Null values not supported.");
if (null == val) {
throw new IllegalArgumentException("Null values not supported.");
}
return OrderedBytes.encodeInt64(dst, val, order);
}

Some files were not shown because too many files have changed in this diff Show More