HADOOP-11286. Copied LimitInputStream from guava-0.14 to hadoop to avoid issues with newer versions of guava in applications. Contributed by Christopher Tubbs.
(cherry picked from commit 6caa8100d5
)
This commit is contained in:
parent
242404b025
commit
1f5bad04ce
|
@ -313,6 +313,10 @@ Release 2.6.0 - UNRELEASED
|
|||
HADOOP-11254. Changed visibility of AccessControlList to be public for
|
||||
consumption by ecosystem. (Zhijie Shen via vinodkv)
|
||||
|
||||
HADOOP-11286. Copied LimitInputStream from guava-0.14 to hadoop to avoid
|
||||
issues with newer versions of guava in applications. (Christopher Tubbs
|
||||
via acmurthy)
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
HADOOP-10838. Byte array native checksumming. (James Thomas via todd)
|
||||
|
|
|
@ -0,0 +1,109 @@
|
|||
/*
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.util;
|
||||
|
||||
import static com.google.common.base.Preconditions.checkArgument;
|
||||
import static com.google.common.base.Preconditions.checkNotNull;
|
||||
|
||||
import java.io.FilterInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||
|
||||
/**
|
||||
* Copied from guava source code v15 (LimitedInputStream)
|
||||
* Guava deprecated LimitInputStream in v14 and removed it in v15. Copying this class here
|
||||
* allows to be compatible with guava 11 to 15+.
|
||||
*
|
||||
* Originally: org.apache.hadoop.hbase.io.LimitInputStream
|
||||
*/
|
||||
@Unstable
|
||||
public final class LimitInputStream extends FilterInputStream {
|
||||
private long left;
|
||||
private long mark = -1;
|
||||
|
||||
public LimitInputStream(InputStream in, long limit) {
|
||||
super(in);
|
||||
checkNotNull(in);
|
||||
checkArgument(limit >= 0, "limit must be non-negative");
|
||||
left = limit;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int available() throws IOException {
|
||||
return (int) Math.min(in.available(), left);
|
||||
}
|
||||
|
||||
// it's okay to mark even if mark isn't supported, as reset won't work
|
||||
@Override
|
||||
public synchronized void mark(int readLimit) {
|
||||
in.mark(readLimit);
|
||||
mark = left;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int read() throws IOException {
|
||||
if (left == 0) {
|
||||
return -1;
|
||||
}
|
||||
|
||||
int result = in.read();
|
||||
if (result != -1) {
|
||||
--left;
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int read(byte[] b, int off, int len) throws IOException {
|
||||
if (left == 0) {
|
||||
return -1;
|
||||
}
|
||||
|
||||
len = (int) Math.min(len, left);
|
||||
int result = in.read(b, off, len);
|
||||
if (result != -1) {
|
||||
left -= result;
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void reset() throws IOException {
|
||||
if (!in.markSupported()) {
|
||||
throw new IOException("Mark not supported");
|
||||
}
|
||||
if (mark == -1) {
|
||||
throw new IOException("Mark not set");
|
||||
}
|
||||
|
||||
in.reset();
|
||||
left = mark;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long skip(long n) throws IOException {
|
||||
n = Math.min(n, left);
|
||||
long skipped = in.skip(n);
|
||||
left -= skipped;
|
||||
return skipped;
|
||||
}
|
||||
}
|
|
@ -62,11 +62,11 @@ import org.apache.hadoop.hdfs.util.MD5FileUtils;
|
|||
import org.apache.hadoop.io.MD5Hash;
|
||||
import org.apache.hadoop.io.compress.CompressionCodec;
|
||||
import org.apache.hadoop.io.compress.CompressorStream;
|
||||
import org.apache.hadoop.util.LimitInputStream;
|
||||
import org.apache.hadoop.util.Time;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.io.LimitInputStream;
|
||||
import com.google.protobuf.CodedOutputStream;
|
||||
|
||||
/**
|
||||
|
|
|
@ -47,12 +47,12 @@ import org.apache.hadoop.hdfs.server.namenode.FSImageUtil;
|
|||
import org.apache.hadoop.hdfs.server.namenode.FsImageProto;
|
||||
import org.apache.hadoop.hdfs.server.namenode.INodeId;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.util.LimitInputStream;
|
||||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.io.LimitInputStream;
|
||||
|
||||
/**
|
||||
* FSImageLoader loads fsimage and provide methods to return JSON formatted
|
||||
|
|
|
@ -31,9 +31,9 @@ import org.apache.hadoop.hdfs.server.namenode.FSImageUtil;
|
|||
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FileSummary;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.util.LimitInputStream;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.io.LimitInputStream;
|
||||
|
||||
/**
|
||||
* This is the tool for analyzing file sizes in the namespace image. In order to
|
||||
|
|
|
@ -51,9 +51,9 @@ import org.apache.hadoop.hdfs.server.namenode.FsImageProto.SnapshotSection;
|
|||
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.StringTableSection;
|
||||
import org.apache.hadoop.hdfs.util.XMLUtils;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.util.LimitInputStream;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.io.LimitInputStream;
|
||||
|
||||
/**
|
||||
* PBImageXmlWriter walks over an fsimage structure and writes out
|
||||
|
|
|
@ -37,8 +37,7 @@ import org.apache.hadoop.io.IOUtils;
|
|||
import org.apache.hadoop.mapreduce.MRJobConfig;
|
||||
import org.apache.hadoop.mapreduce.security.TokenCache;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
|
||||
import com.google.common.io.LimitInputStream;
|
||||
import org.apache.hadoop.util.LimitInputStream;
|
||||
|
||||
/**
|
||||
* This class provides utilities to make it easier to work with Cryptographic
|
||||
|
|
Loading…
Reference in New Issue