From 4c19e1f4e35c170865d103a75f50844ba69c1ffe Mon Sep 17 00:00:00 2001 From: Michael Stack Date: Fri, 5 Jan 2018 08:31:07 -0600 Subject: [PATCH] HBASE-19651 Remove LimitInputStream Signed-off-by: Beluga Behr --- NOTICE.txt | 1 - .../apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java | 3 ++- hbase-common/src/main/appended-resources/META-INF/NOTICE | 7 ------- .../hadoop/hbase/regionserver/wal/ProtobufLogReader.java | 5 +++-- 4 files changed, 5 insertions(+), 11 deletions(-) delete mode 100644 hbase-common/src/main/appended-resources/META-INF/NOTICE diff --git a/NOTICE.txt b/NOTICE.txt index 79598575acf..9c238bed136 100755 --- a/NOTICE.txt +++ b/NOTICE.txt @@ -39,7 +39,6 @@ Licensed under the Apache License v2.0 as a part of the Bootstrap project. -- This product includes portions of the Guava project v14 and v21, specifically -'hbase-common/src/main/java/org/apache/hadoop/hbase/io/LimitInputStream.java' 'hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java' 'hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java' diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java index f0a5d16a810..b26802f690e 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java @@ -106,6 +106,7 @@ import org.apache.hadoop.hbase.util.VersionInfo; import org.apache.hadoop.ipc.RemoteException; import org.apache.yetus.audience.InterfaceAudience; +import org.apache.hbase.thirdparty.com.google.common.io.ByteStreams; import org.apache.hbase.thirdparty.com.google.gson.JsonArray; import org.apache.hbase.thirdparty.com.google.gson.JsonElement; import org.apache.hbase.thirdparty.com.google.protobuf.ByteString; @@ -2609,7 +2610,7 @@ public final class ProtobufUtil { final int firstByte = in.read(); if (firstByte != -1) { final int size = CodedInputStream.readRawVarint32(firstByte, in); - final InputStream limitedInput = new BoundedInputStream(in, size); + final InputStream limitedInput = ByteStreams.limit(in, size); final CodedInputStream codedInput = CodedInputStream.newInstance(limitedInput); codedInput.setSizeLimit(size); builder.mergeFrom(codedInput); diff --git a/hbase-common/src/main/appended-resources/META-INF/NOTICE b/hbase-common/src/main/appended-resources/META-INF/NOTICE deleted file mode 100644 index 9176ececdc3..00000000000 --- a/hbase-common/src/main/appended-resources/META-INF/NOTICE +++ /dev/null @@ -1,7 +0,0 @@ --- -This product includes portions of the Guava project v14, specifically -'hbase-common/src/main/java/org/apache/hadoop/hbase/io/LimitInputStream.java' - -Copyright (C) 2007 The Guava Authors - -Licensed under the Apache License, Version 2.0 diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogReader.java index 7babb55e437..ebb6079b088 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogReader.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogReader.java @@ -26,7 +26,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; -import org.apache.commons.io.input.BoundedInputStream; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.FileSystem; @@ -41,9 +40,11 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALTrailer; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.wal.WAL.Entry; import org.apache.yetus.audience.InterfaceAudience; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.common.io.ByteStreams; import org.apache.hbase.thirdparty.com.google.protobuf.CodedInputStream; import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException; @@ -350,7 +351,7 @@ public class ProtobufLogReader extends ReaderBase { "inputStream.available()= " + this.inputStream.available() + ", " + "entry size= " + size + " at offset = " + this.inputStream.getPos()); } - ProtobufUtil.mergeFrom(builder, new BoundedInputStream(this.inputStream, size), + ProtobufUtil.mergeFrom(builder, ByteStreams.limit(this.inputStream, size), (int)size); } catch (InvalidProtocolBufferException ipbe) { throw (EOFException) new EOFException("Invalid PB, EOF? Ignoring; originalPosition=" +