HBASE-19651 Remove LimitInputStream

Signed-off-by: Beluga Behr <dam6923@gmail.com>
This commit is contained in:
Michael Stack 2018-01-05 08:31:07 -06:00
parent e35fec284d
commit 4c19e1f4e3
No known key found for this signature in database
GPG Key ID: 9816C7FC8ACC93D2
4 changed files with 5 additions and 11 deletions

View File

@ -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'

View File

@ -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);

View File

@ -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

View File

@ -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=" +