You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by an...@apache.org on 2015/06/01 06:29:20 UTC
hbase git commit: HBASE-13805 Use LimitInputStream in hbase-common
instead of ProtobufUtil.LimitedInputStream. (Jingcheng)
Repository: hbase
Updated Branches:
refs/heads/hbase-11339 a84e829e1 -> b5641d8ed
HBASE-13805 Use LimitInputStream in hbase-common instead of ProtobufUtil.LimitedInputStream. (Jingcheng)
Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/b5641d8e
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/b5641d8e
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/b5641d8e
Branch: refs/heads/hbase-11339
Commit: b5641d8edf7c01c73cae006790e4482f8112dd4a
Parents: a84e829
Author: anoopsjohn <an...@gmail.com>
Authored: Mon Jun 1 09:58:56 2015 +0530
Committer: anoopsjohn <an...@gmail.com>
Committed: Mon Jun 1 09:58:56 2015 +0530
----------------------------------------------------------------------
.../hadoop/hbase/protobuf/ProtobufUtil.java | 74 ++++----------------
1 file changed, 12 insertions(+), 62 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hbase/blob/b5641d8e/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
index 908a515..d1c44b4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.protobuf;
import static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType.REGION_NAME;
import java.io.ByteArrayOutputStream;
-import java.io.FilterInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.lang.reflect.Constructor;
@@ -39,7 +38,6 @@ import java.util.Map.Entry;
import java.util.NavigableSet;
import java.util.concurrent.TimeUnit;
-import com.google.protobuf.*;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
@@ -70,6 +68,7 @@ import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.filter.ByteArrayComparable;
import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.io.LimitInputStream;
import org.apache.hadoop.hbase.io.TimeRange;
import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService;
@@ -125,12 +124,12 @@ import org.apache.hadoop.hbase.protobuf.generated.RPCProtos;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
+import org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor.FlushAction;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.EventType;
-import org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor;
import org.apache.hadoop.hbase.quotas.QuotaScope;
import org.apache.hadoop.hbase.quotas.QuotaType;
@@ -157,6 +156,15 @@ import org.apache.hadoop.security.token.Token;
import com.google.common.collect.ArrayListMultimap;
import com.google.common.collect.ListMultimap;
import com.google.common.collect.Lists;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.CodedInputStream;
+import com.google.protobuf.InvalidProtocolBufferException;
+import com.google.protobuf.Message;
+import com.google.protobuf.Parser;
+import com.google.protobuf.RpcChannel;
+import com.google.protobuf.Service;
+import com.google.protobuf.ServiceException;
+import com.google.protobuf.TextFormat;
/**
* Protobufs utility.
@@ -3019,7 +3027,6 @@ public final class ProtobufUtil {
return desc.build();
}
-
/**
* This version of protobuf's mergeDelimitedFrom avoid the hard-coded 64MB limit for decoding
* buffers
@@ -3035,7 +3042,7 @@ public final class ProtobufUtil {
// bail out. (was return false;)
} else {
final int size = CodedInputStream.readRawVarint32(firstByte, in);
- final InputStream limitedInput = new LimitedInputStream(in, size);
+ final InputStream limitedInput = new LimitInputStream(in, size);
final CodedInputStream codedInput = CodedInputStream.newInstance(limitedInput);
codedInput.setSizeLimit(size);
builder.mergeFrom(codedInput);
@@ -3043,63 +3050,6 @@ public final class ProtobufUtil {
}
}
- /**
- * This is cut and paste from protobuf's package private AbstractMessageLite.
- *
- * An InputStream implementations which reads from some other InputStream
- * but is limited to a particular number of bytes. Used by
- * mergeDelimitedFrom(). This is intentionally package-private so that
- * UnknownFieldSet can share it.
- */
- static final class LimitedInputStream extends FilterInputStream {
- private int limit;
-
- LimitedInputStream(InputStream in, int limit) {
- super(in);
- this.limit = limit;
- }
-
- @Override
- public int available() throws IOException {
- return Math.min(super.available(), limit);
- }
-
- @Override
- public int read() throws IOException {
- if (limit <= 0) {
- return -1;
- }
- final int result = super.read();
- if (result >= 0) {
- --limit;
- }
- return result;
- }
-
- @Override
- public int read(final byte[] b, final int off, int len)
- throws IOException {
- if (limit <= 0) {
- return -1;
- }
- len = Math.min(len, limit);
- final int result = super.read(b, off, len);
- if (result >= 0) {
- limit -= result;
- }
- return result;
- }
-
- @Override
- public long skip(final long n) throws IOException {
- final long result = super.skip(Math.min(n, limit));
- if (result >= 0) {
- limit -= result;
- }
- return result;
- }
- }
-
public static ReplicationLoadSink toReplicationLoadSink(
ClusterStatusProtos.ReplicationLoadSink cls) {
return new ReplicationLoadSink(cls.getAgeOfLastAppliedOp(), cls.getTimeStampsOfLastAppliedOp());