You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by nk...@apache.org on 2013/10/02 19:26:59 UTC
svn commit: r1528573 - in /hbase/trunk:
hbase-client/src/main/java/org/apache/hadoop/hbase/client/
hbase-common/src/main/java/org/apache/hadoop/hbase/io/
hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/
hbase-server/src/main/java/org/apa...
Author: nkeywal
Date: Wed Oct 2 17:26:59 2013
New Revision: 1528573
URL: http://svn.apache.org/r1528573
Log:
HBASE-9667 NullOutputStream removed from Guava 15
Added:
hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/LimitInputStream.java
Modified:
hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java
hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/EncodedDataBlock.java
hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java
hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/metrics/ExactCounterMetric.java
hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogReader.java
Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java?rev=1528573&r1=1528572&r2=1528573&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java Wed Oct 2 17:26:59 2013
@@ -29,8 +29,6 @@ import java.util.Map;
import java.util.NavigableMap;
import java.util.TreeMap;
-import javax.annotation.Nullable;
-
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.Cell;
Added: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/LimitInputStream.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/LimitInputStream.java?rev=1528573&view=auto
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/LimitInputStream.java (added)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/LimitInputStream.java Wed Oct 2 17:26:59 2013
@@ -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;
+ }
+}
Modified: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/EncodedDataBlock.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/EncodedDataBlock.java?rev=1528573&r1=1528572&r2=1528573&view=diff
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/EncodedDataBlock.java (original)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/EncodedDataBlock.java Wed Oct 2 17:26:59 2013
@@ -33,10 +33,10 @@ import org.apache.hadoop.hbase.io.compre
import org.apache.hadoop.hbase.io.hfile.HFileContext;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.compress.Compressor;
import com.google.common.base.Preconditions;
-import com.google.common.io.NullOutputStream;
/**
* Encapsulates a data block compressed using a particular encoding algorithm.
@@ -161,7 +161,7 @@ public class EncodedDataBlock {
public static int getCompressedSize(Algorithm algo, Compressor compressor,
byte[] inputBuffer, int offset, int length) throws IOException {
DataOutputStream compressedStream = new DataOutputStream(
- new NullOutputStream());
+ new IOUtils.NullOutputStream());
if (compressor != null) {
compressor.reset();
}
Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java?rev=1528573&r1=1528572&r2=1528573&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java Wed Oct 2 17:26:59 2013
@@ -36,8 +36,6 @@ import org.apache.hadoop.hbase.io.compre
import org.apache.hadoop.hbase.protobuf.generated.HFileProtos;
import org.apache.hadoop.hbase.util.Bytes;
-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
Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/metrics/ExactCounterMetric.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/metrics/ExactCounterMetric.java?rev=1528573&r1=1528572&r2=1528573&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/metrics/ExactCounterMetric.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/metrics/ExactCounterMetric.java Wed Oct 2 17:26:59 2013
@@ -21,9 +21,9 @@ package org.apache.hadoop.hbase.metrics;
import java.util.Collections;
import java.util.Comparator;
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 +33,7 @@ import org.apache.hadoop.metrics.util.Me
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;
@Deprecated
public class ExactCounterMetric extends MetricsBase {
@@ -44,34 +42,25 @@ public class ExactCounterMetric extends
// only publish stats on the topN items (default to DEFAULT_TOP_N)
private final int topN;
- private final Map<String, Counter> counts;
+ private final ConcurrentMap<String, Counter> counts = new ConcurrentHashMap<String, Counter>();
// 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<String, Counter>() {
- @Override
- public Counter apply(String input) {
- return new Counter();
- }
- });
-
this.lock = new ReentrantReadWriteLock();
this.topN = topN;
@@ -89,11 +78,22 @@ public class ExactCounterMetric extends
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();
}
@@ -102,7 +102,7 @@ public class ExactCounterMetric extends
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();
}
Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogReader.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogReader.java?rev=1528573&r1=1528572&r2=1528573&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogReader.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogReader.java Wed Oct 2 17:26:59 2013
@@ -30,12 +30,12 @@ import org.apache.commons.logging.LogFac
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.hbase.codec.Codec;
+import org.apache.hadoop.hbase.io.LimitInputStream;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALKey;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALTrailer;
import org.apache.hadoop.hbase.util.Bytes;
-import com.google.common.io.LimitInputStream;
import com.google.protobuf.CodedInputStream;
import com.google.protobuf.InvalidProtocolBufferException;