Index: src/main/java/org/apache/hadoop/hbase/io/encoding/EncodedDataBlock.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/io/encoding/EncodedDataBlock.java (revision 1551262) +++ src/main/java/org/apache/hadoop/hbase/io/encoding/EncodedDataBlock.java (working copy) @@ -24,6 +24,7 @@ import java.nio.ByteBuffer; import java.util.Iterator; +import org.apache.commons.io.output.NullOutputStream; import org.apache.commons.lang.NotImplementedException; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.io.compress.Compressor; Index: src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java (revision 1551262) +++ src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java (working copy) @@ -30,14 +30,13 @@ import java.io.IOException; import java.nio.ByteBuffer; +import org.apache.commons.io.output.NullOutputStream; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.io.RawComparator; -import com.google.common.io.NullOutputStream; - /** * The {@link HFile} has a fixed trailer which contains offsets to other * variable parts of the file. Also includes basic metadata on this file. The Index: src/main/java/org/apache/hadoop/hbase/io/LimitInputStream.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/io/LimitInputStream.java (revision 0) +++ src/main/java/org/apache/hadoop/hbase/io/LimitInputStream.java (revision 0) @@ -0,0 +1,104 @@ +/* + * + * 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.hbase.io; + +import java.io.FilterInputStream; +import java.io.IOException; +import java.io.InputStream; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; + +/** + * 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+. + */ +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; + } +} Index: /Users/tyu/94-hbase/src/main/java/org/apache/hadoop/hbase/metrics/ExactCounterMetric.java =================================================================== --- /Users/tyu/94-hbase/src/main/java/org/apache/hadoop/hbase/metrics/ExactCounterMetric.java (revision 1551273) +++ /Users/tyu/94-hbase/src/main/java/org/apache/hadoop/hbase/metrics/ExactCounterMetric.java (working copy) @@ -23,7 +23,8 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; -import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -33,9 +34,7 @@ import org.apache.hadoop.metrics.util.MetricsRegistry; import org.cliffc.high_scale_lib.Counter; -import com.google.common.base.Function; import com.google.common.collect.Lists; -import com.google.common.collect.MapMaker; public class ExactCounterMetric extends MetricsBase { @@ -43,34 +42,25 @@ // only publish stats on the topN items (default to DEFAULT_TOP_N) private final int topN; - private final Map counts; + private final Map counts = new ConcurrentHashMap(); // all access to the 'counts' map should use this lock. // take a write lock iff you want to guarantee exclusive access // (the map stripes locks internally, so it's already thread safe - // this lock is just so you can take a consistent snapshot of data) private final ReadWriteLock lock; - - - /** - * Constructor to create a new counter metric - * @param nam the name to publish this metric under - * @param registry where the metrics object will be registered - * @param description metrics description - * @param topN how many 'keys' to publish metrics on - */ + + /** + * Constructor to create a new counter metric + * @param nam the name to publish this metric under + * @param registry where the metrics object will be registered + * @param description metrics description + * @param topN how many 'keys' to publish metrics on + */ public ExactCounterMetric(final String nam, final MetricsRegistry registry, final String description, int topN) { super(nam, description); - this.counts = new MapMaker().makeComputingMap( - new Function() { - @Override - public Counter apply(String input) { - return new Counter(); - } - }); - this.lock = new ReentrantReadWriteLock(); this.topN = topN; @@ -88,11 +78,22 @@ this(nam, registry, NO_DESCRIPTION, DEFAULT_TOP_N); } - + /** + * Relies on an external lock on {@link #lock} for thread safety. + */ + private Counter getOrCreateCounter(String type){ + Counter cnt = counts.get(type); + if (cnt == null){ + cnt = new Counter(); + counts.put(type, cnt); + } + return cnt; + } + public void update(String type) { this.lock.readLock().lock(); try { - this.counts.get(type).increment(); + getOrCreateCounter(type).increment(); } finally { this.lock.readLock().unlock(); } @@ -101,7 +102,7 @@ public void update(String type, long count) { this.lock.readLock().lock(); try { - this.counts.get(type).add(count); + getOrCreateCounter(type).add(count); } finally { this.lock.readLock().unlock(); }