Return-Path: X-Original-To: apmail-hbase-commits-archive@www.apache.org Delivered-To: apmail-hbase-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id E42B918243 for ; Wed, 20 Apr 2016 22:09:34 +0000 (UTC) Received: (qmail 41384 invoked by uid 500); 20 Apr 2016 22:09:34 -0000 Delivered-To: apmail-hbase-commits-archive@hbase.apache.org Received: (qmail 41337 invoked by uid 500); 20 Apr 2016 22:09:34 -0000 Mailing-List: contact commits-help@hbase.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@hbase.apache.org Delivered-To: mailing list commits@hbase.apache.org Received: (qmail 41327 invoked by uid 99); 20 Apr 2016 22:09:34 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 20 Apr 2016 22:09:34 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 9ADBFDFCE0; Wed, 20 Apr 2016 22:09:34 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: apurtell@apache.org To: commits@hbase.apache.org Date: Wed, 20 Apr 2016 22:09:34 -0000 Message-Id: X-Mailer: ASF-Git Admin Mailer Subject: [1/7] hbase git commit: HBASE-15650 Remove TimeRangeTracker as point of contention when many threads reading a StoreFile Repository: hbase Updated Branches: refs/heads/0.98 8799e1fb4 -> 1b9e1fefe HBASE-15650 Remove TimeRangeTracker as point of contention when many threads reading a StoreFile Refactor so we use the immutable, unsynchronized TimeRange when doing time-based checks at read time rather than use heavily synchronized TimeRangeTracker; let TimeRangeTracker be for write-time only. M hbase-common/src/main/java/org/apache/hadoop/hbase/io/TimeRange.java Make allTime final. Add a includesTimeRange method copied from TimeRangeTracker. M hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/TimeRangeTracker.java Change name of a few methods so they match TimeRange methods that do same thing. (getTimeRangeTracker, getTimeRange, toTimeRange) add utility methods M hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java Change Reader to use TimeRange-based checks instead of TimeRangeTracker. Amending-Author: Andrew Purtell Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/017ab8d7 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/017ab8d7 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/017ab8d7 Branch: refs/heads/0.98 Commit: 017ab8d7f1e9f8acd9bfeb5cbd24e68f418ee4f1 Parents: 8799e1f Author: stack Authored: Fri Apr 15 10:56:56 2016 -0700 Committer: Andrew Purtell Committed: Wed Apr 20 14:19:01 2016 -0700 ---------------------------------------------------------------------- .../org/apache/hadoop/hbase/io/TimeRange.java | 37 +++++++--- .../hbase/io/hfile/HFilePrettyPrinter.java | 3 +- .../hadoop/hbase/regionserver/MemStore.java | 5 +- .../hadoop/hbase/regionserver/StoreFile.java | 59 +++++++--------- .../hbase/regionserver/StoreFileScanner.java | 2 +- .../hbase/regionserver/TimeRangeTracker.java | 74 ++++++++++++++++---- .../hbase/mapreduce/TestHFileOutputFormat.java | 7 +- .../hbase/mapreduce/TestHFileOutputFormat2.java | 7 +- .../hbase/regionserver/MockStoreFile.java | 13 ++-- .../regionserver/TestTimeRangeTracker.java | 2 +- 10 files changed, 129 insertions(+), 80 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hbase/blob/017ab8d7/hbase-common/src/main/java/org/apache/hadoop/hbase/io/TimeRange.java ---------------------------------------------------------------------- diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/TimeRange.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/TimeRange.java index b2c3ebe..95ec6b9 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/TimeRange.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/TimeRange.java @@ -36,11 +36,13 @@ import org.apache.hadoop.hbase.util.Bytes; @InterfaceAudience.Public @InterfaceStability.Stable public class TimeRange { - private static final long MIN_TIME = 0L; - private static final long MAX_TIME = Long.MAX_VALUE; + static final long INITIAL_MIN_TIMESTAMP = 0L; + private static final long MIN_TIME = INITIAL_MIN_TIMESTAMP; + static final long INITIAL_MAX_TIMESTAMP = Long.MAX_VALUE; + static final long MAX_TIME = INITIAL_MAX_TIMESTAMP; private long minStamp = MIN_TIME; private long maxStamp = MAX_TIME; - private boolean allTime = false; + private final boolean allTime; /** * Default constructor. @@ -56,9 +58,7 @@ public class TimeRange { */ public TimeRange(long minStamp) { this.minStamp = minStamp; - if (this.minStamp == MIN_TIME){ - this.allTime = true; - } + this.allTime = this.minStamp == MIN_TIME; } /** @@ -67,6 +67,7 @@ public class TimeRange { */ public TimeRange(byte [] minStamp) { this.minStamp = Bytes.toLong(minStamp); + this.allTime = false; } /** @@ -81,14 +82,12 @@ public class TimeRange { throw new IllegalArgumentException("Timestamp cannot be negative. minStamp:" + minStamp + ", maxStamp" + maxStamp); } - if(maxStamp < minStamp) { + if (maxStamp < minStamp) { throw new IOException("maxStamp is smaller than minStamp"); } this.minStamp = minStamp; this.maxStamp = maxStamp; - if (this.minStamp == MIN_TIME && this.maxStamp == MAX_TIME){ - this.allTime = true; - } + this.allTime = this.minStamp == MIN_TIME && this.maxStamp == MAX_TIME; } /** @@ -134,11 +133,27 @@ public class TimeRange { * @return true if within TimeRange, false if not */ public boolean withinTimeRange(byte [] bytes, int offset) { - if(allTime) return true; + if (allTime) { + return true; + } return withinTimeRange(Bytes.toLong(bytes, offset)); } /** + * Check if the range has any overlap with TimeRange + * @param tr TimeRange + * @return True if there is overlap, false otherwise + */ + // This method came from TimeRangeTracker. We used to go there for this function but better + // to come here to the immutable, unsynchronized datastructure at read time. + public boolean includesTimeRange(final TimeRange tr) { + if (this.allTime) { + return true; + } + return getMin() < tr.getMax() && getMax() >= tr.getMin(); + } + + /** * Check if the specified timestamp is within this TimeRange. *

* Returns true if within interval [minStamp, maxStamp), false http://git-wip-us.apache.org/repos/asf/hbase/blob/017ab8d7/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java index 48f4b42..d5abdc4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java @@ -387,8 +387,7 @@ public class HFilePrettyPrinter extends Configured implements Tool { } else if (Bytes.compareTo(e.getKey(), Bytes.toBytes("TIMERANGE")) == 0) { TimeRangeTracker timeRangeTracker = new TimeRangeTracker(); Writables.copyWritable(e.getValue(), timeRangeTracker); - System.out.println(timeRangeTracker.getMinimumTimestamp() + "...." - + timeRangeTracker.getMaximumTimestamp()); + System.out.println(timeRangeTracker.getMin() + "...." + timeRangeTracker.getMax()); } else if (Bytes.compareTo(e.getKey(), FileInfo.AVG_KEY_LEN) == 0 || Bytes.compareTo(e.getKey(), FileInfo.AVG_VALUE_LEN) == 0) { System.out.println(Bytes.toInt(e.getValue())); http://git-wip-us.apache.org/repos/asf/hbase/blob/017ab8d7/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStore.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStore.java index 549f15e..366a9dd 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStore.java @@ -662,9 +662,8 @@ public class MemStore implements HeapSize { public boolean shouldSeek(Scan scan, long oldestUnexpiredTS) { return (timeRangeTracker.includesTimeRange(scan.getTimeRange()) || snapshotTimeRangeTracker.includesTimeRange(scan.getTimeRange())) - && (Math.max(timeRangeTracker.getMaximumTimestamp(), - snapshotTimeRangeTracker.getMaximumTimestamp()) >= - oldestUnexpiredTS); + && (Math.max(timeRangeTracker.getMax(), snapshotTimeRangeTracker.getMax()) + >= oldestUnexpiredTS); } public TimeRangeTracker getSnapshotTimeRangeTracker() { http://git-wip-us.apache.org/repos/asf/hbase/blob/017ab8d7/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java index b959152..168ebca 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java @@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.KeyValue.KVComparator; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper; +import org.apache.hadoop.hbase.io.TimeRange; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; import org.apache.hadoop.hbase.io.hfile.BlockType; import org.apache.hadoop.hbase.io.hfile.CacheConfig; @@ -468,15 +469,11 @@ public class StoreFile { reader.loadBloomfilter(BlockType.DELETE_FAMILY_BLOOM_META); try { - byte [] timerangeBytes = metadataMap.get(TIMERANGE_KEY); - if (timerangeBytes != null) { - this.reader.timeRangeTracker = new TimeRangeTracker(); - Writables.copyWritable(timerangeBytes, this.reader.timeRangeTracker); - } + this.reader.timeRange = TimeRangeTracker.getTimeRange(metadataMap.get(TIMERANGE_KEY)); } catch (IllegalArgumentException e) { LOG.error("Error reading timestamp range data from meta -- " + "proceeding without", e); - this.reader.timeRangeTracker = null; + this.reader.timeRange = null; } return this.reader; } @@ -694,18 +691,13 @@ public class StoreFile { } public Long getMinimumTimestamp() { - return (getReader().timeRangeTracker == null) ? - null : - getReader().timeRangeTracker.getMinimumTimestamp(); + return getReader().timeRange == null? null: getReader().timeRange.getMin(); } public Long getMaximumTimestamp() { - return (getReader().timeRangeTracker == null) ? - null : - getReader().timeRangeTracker.getMaximumTimestamp(); + return getReader().timeRange == null? null: getReader().timeRange.getMax(); } - /** * Gets the approximate mid-point of this file that is optimal for use in splitting it. * @param comparator Comparator used to compare KVs. @@ -763,13 +755,14 @@ public class StoreFile { protected int bytesPerChecksum; TimeRangeTracker timeRangeTracker = new TimeRangeTracker(); - /* isTimeRangeTrackerSet keeps track if the timeRange has already been set - * When flushing a memstore, we set TimeRange and use this variable to - * indicate that it doesn't need to be calculated again while - * appending KeyValues. - * It is not set in cases of compactions when it is recalculated using only - * the appended KeyValues*/ - boolean isTimeRangeTrackerSet = false; + /** + * timeRangeTrackerSet is used to figure if we were passed a filled-out TimeRangeTracker or not. + * When flushing a memstore, we set the TimeRangeTracker that it accumulated during updates to + * memstore in here into this Writer and use this variable to indicate that we do not need to + * recalculate the timeRangeTracker bounds; it was done already as part of add-to-memstore. + * A completed TimeRangeTracker is not set in cases of compactions when it is recalculated. + */ + boolean timeRangeTrackerSet = false; protected HFile.Writer writer; @@ -853,12 +846,16 @@ public class StoreFile { } /** - * Set TimeRangeTracker - * @param trt + * Set TimeRangeTracker. + * Called when flushing to pass us a pre-calculated TimeRangeTracker, one made during updates + * to memstore so we don't have to make one ourselves as Cells get appended. Call before first + * append. If this method is not called, we will calculate our own range of the Cells that + * comprise this StoreFile (and write them on the end as metadata). It is good to have this stuff + * passed because it is expensive to make. */ public void setTimeRangeTracker(final TimeRangeTracker trt) { this.timeRangeTracker = trt; - isTimeRangeTrackerSet = true; + timeRangeTrackerSet = true; } /** @@ -872,7 +869,7 @@ public class StoreFile { if (KeyValue.Type.Put.getCode() == kv.getTypeByte()) { earliestPutTs = Math.min(earliestPutTs, kv.getTimestamp()); } - if (!isTimeRangeTrackerSet) { + if (!timeRangeTrackerSet) { timeRangeTracker.includeTimestamp(kv); } } @@ -1071,7 +1068,7 @@ public class StoreFile { protected BloomFilter deleteFamilyBloomFilter = null; protected BloomType bloomFilterType; private final HFile.Reader reader; - protected TimeRangeTracker timeRangeTracker = null; + protected TimeRange timeRange; protected long sequenceID = -1; private byte[] lastBloomKey; private long deleteFamilyCnt = -1; @@ -1181,13 +1178,9 @@ public class StoreFile { * determined by the column family's TTL * @return false if queried keys definitely don't exist in this StoreFile */ - boolean passesTimerangeFilter(Scan scan, long oldestUnexpiredTS) { - if (timeRangeTracker == null) { - return true; - } else { - return timeRangeTracker.includesTimeRange(scan.getTimeRange()) && - timeRangeTracker.getMaximumTimestamp() >= oldestUnexpiredTS; - } + boolean passesTimerangeFilter(TimeRange tr, long oldestUnexpiredTS) { + return this.timeRange == null? true: + this.timeRange.includesTimeRange(tr) && this.timeRange.getMax() >= oldestUnexpiredTS; } /** @@ -1579,7 +1572,7 @@ public class StoreFile { } public long getMaxTimestamp() { - return timeRangeTracker == null ? Long.MAX_VALUE : timeRangeTracker.getMaximumTimestamp(); + return timeRange == null ? Long.MAX_VALUE : timeRange.getMax(); } public void setBulkLoaded(boolean bulkLoadResult) { http://git-wip-us.apache.org/repos/asf/hbase/blob/017ab8d7/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java index 0156637..5330ffb 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java @@ -430,7 +430,7 @@ public class StoreFileScanner implements KeyValueScanner { @Override public boolean shouldUseScanner(Scan scan, SortedSet columns, long oldestUnexpiredTS) { - return reader.passesTimerangeFilter(scan, oldestUnexpiredTS) + return reader.passesTimerangeFilter(scan.getTimeRange(), oldestUnexpiredTS) && reader.passesKeyRangeFilter(scan) && reader.passesBloomFilter(scan, columns); } http://git-wip-us.apache.org/repos/asf/hbase/blob/017ab8d7/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/TimeRangeTracker.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/TimeRangeTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/TimeRangeTracker.java index a93f828..55fd92b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/TimeRangeTracker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/TimeRangeTracker.java @@ -27,20 +27,28 @@ import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue.Type; import org.apache.hadoop.hbase.io.TimeRange; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.Writables; import org.apache.hadoop.io.Writable; /** - * Stores the minimum and maximum timestamp values (both are inclusive). - * Can be used to find if any given time range overlaps with its time range - * MemStores use this class to track its minimum and maximum timestamps. - * When writing StoreFiles, this information is stored in meta blocks and used - * at read time to match against the required TimeRange. + * Stores minimum and maximum timestamp values. Both timestamps are inclusive. + * Use this class at write-time ONLY. Too much synchronization to use at read time + * (TODO: there are two scenarios writing, once when lots of concurrency as part of memstore + * updates but then later we can make one as part of a compaction when there is only one thread + * involved -- consider making different version, the synchronized and the unsynchronized). + * Use {@link TimeRange} at read time instead of this. See toTimeRange() to make TimeRange to use. + * MemStores use this class to track minimum and maximum timestamps. The TimeRangeTracker made by + * the MemStore is passed to the StoreFile for it to write out as part a flush in the the file + * metadata. If no memstore involved -- i.e. a compaction -- then the StoreFile will calculate its + * own TimeRangeTracker as it appends. The StoreFile serialized TimeRangeTracker is used + * at read time via an instance of {@link TimeRange} to test if Cells fit the StoreFile TimeRange. */ @InterfaceAudience.Private public class TimeRangeTracker implements Writable { - - long minimumTimestamp = -1; - long maximumTimestamp = -1; + static final long INITIAL_MIN_TIMESTAMP = Long.MAX_VALUE; + long minimumTimestamp = INITIAL_MIN_TIMESTAMP; + static final long INITIAL_MAX_TIMESTAMP = -1; + long maximumTimestamp = INITIAL_MAX_TIMESTAMP; /** * Default constructor. @@ -55,8 +63,8 @@ public class TimeRangeTracker implements Writable { * @param trt source TimeRangeTracker */ public TimeRangeTracker(final TimeRangeTracker trt) { - this.minimumTimestamp = trt.getMinimumTimestamp(); - this.maximumTimestamp = trt.getMaximumTimestamp(); + this.minimumTimestamp = trt.getMin(); + this.maximumTimestamp = trt.getMax(); } public TimeRangeTracker(long minimumTimestamp, long maximumTimestamp) { @@ -123,14 +131,14 @@ public class TimeRangeTracker implements Writable { /** * @return the minimumTimestamp */ - public synchronized long getMinimumTimestamp() { + public synchronized long getMin() { return minimumTimestamp; } /** * @return the maximumTimestamp */ - public synchronized long getMaximumTimestamp() { + public synchronized long getMax() { return maximumTimestamp; } @@ -148,4 +156,46 @@ public class TimeRangeTracker implements Writable { public synchronized String toString() { return "[" + minimumTimestamp + "," + maximumTimestamp + "]"; } + + /** + * @return An instance of TimeRangeTracker filled w/ the content of serialized + * TimeRangeTracker in timeRangeTrackerBytes. + * @throws IOException + */ + public static TimeRangeTracker getTimeRangeTracker(final byte [] timeRangeTrackerBytes) + throws IOException { + if (timeRangeTrackerBytes == null) return null; + TimeRangeTracker trt = new TimeRangeTracker(); + Writables.copyWritable(timeRangeTrackerBytes, trt); + return trt; + } + + /** + * @return An instance of a TimeRange made from the serialized TimeRangeTracker passed in + * timeRangeTrackerBytes. + * @throws IOException + */ + static TimeRange getTimeRange(final byte [] timeRangeTrackerBytes) throws IOException { + TimeRangeTracker trt = getTimeRangeTracker(timeRangeTrackerBytes); + return trt == null? null: trt.toTimeRange(); + } + + private boolean isFreshInstance() { + return getMin() == INITIAL_MIN_TIMESTAMP && getMax() == INITIAL_MAX_TIMESTAMP; + } + + /** + * @return Make a TimeRange from current state of this. + */ + TimeRange toTimeRange() throws IOException { + long min = getMin(); + long max = getMax(); + // Check for the case where the TimeRangeTracker is fresh. In that case it has + // initial values that are antithetical to a TimeRange... Return an uninitialized TimeRange + // if passed an uninitialized TimeRangeTracker. + if (isFreshInstance()) { + return new TimeRange(); + } + return new TimeRange(min, max); + } } http://git-wip-us.apache.org/repos/asf/hbase/blob/017ab8d7/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java index 432080d..68e7288 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java @@ -275,10 +275,9 @@ public class TestHFileOutputFormat { // unmarshall and check values. TimeRangeTracker timeRangeTracker = new TimeRangeTracker(); Writables.copyWritable(range, timeRangeTracker); - LOG.info(timeRangeTracker.getMinimumTimestamp() + - "...." + timeRangeTracker.getMaximumTimestamp()); - assertEquals(1000, timeRangeTracker.getMinimumTimestamp()); - assertEquals(2000, timeRangeTracker.getMaximumTimestamp()); + LOG.info(timeRangeTracker.getMin() + "...." + timeRangeTracker.getMax()); + assertEquals(1000, timeRangeTracker.getMin()); + assertEquals(2000, timeRangeTracker.getMax()); rd.close(); } finally { if (writer != null && context != null) writer.close(context); http://git-wip-us.apache.org/repos/asf/hbase/blob/017ab8d7/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java index fb0ead0..407e23e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java @@ -278,10 +278,9 @@ public class TestHFileOutputFormat2 { // unmarshall and check values. TimeRangeTracker timeRangeTracker = new TimeRangeTracker(); Writables.copyWritable(range, timeRangeTracker); - LOG.info(timeRangeTracker.getMinimumTimestamp() + - "...." + timeRangeTracker.getMaximumTimestamp()); - assertEquals(1000, timeRangeTracker.getMinimumTimestamp()); - assertEquals(2000, timeRangeTracker.getMaximumTimestamp()); + LOG.info(timeRangeTracker.getMin() + "...." + timeRangeTracker.getMax()); + assertEquals(1000, timeRangeTracker.getMin()); + assertEquals(2000, timeRangeTracker.getMax()); rd.close(); } finally { if (writer != null && context != null) writer.close(context); http://git-wip-us.apache.org/repos/asf/hbase/blob/017ab8d7/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MockStoreFile.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MockStoreFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MockStoreFile.java index 3636c48..1b88731 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MockStoreFile.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MockStoreFile.java @@ -109,15 +109,11 @@ public class MockStoreFile extends StoreFile { } public Long getMinimumTimestamp() { - return (timeRangeTracker == null) ? - null : - timeRangeTracker.getMinimumTimestamp(); + return (timeRangeTracker == null) ? null : timeRangeTracker.getMin(); } - + public Long getMaximumTimestamp() { - return (timeRangeTracker == null) ? - null : - timeRangeTracker.getMaximumTimestamp(); + return (timeRangeTracker == null) ? null : timeRangeTracker.getMax(); } @Override @@ -133,7 +129,6 @@ public class MockStoreFile extends StoreFile { @Override public StoreFile.Reader getReader() { final long len = this.length; - final TimeRangeTracker timeRange = this.timeRangeTracker; final long entries = this.entryCount; return new StoreFile.Reader() { @Override @@ -143,7 +138,7 @@ public class MockStoreFile extends StoreFile { @Override public long getMaxTimestamp() { - return timeRange == null ? Long.MAX_VALUE : timeRange.maximumTimestamp; + return timeRange == null ? Long.MAX_VALUE : timeRange.getMax(); } @Override http://git-wip-us.apache.org/repos/asf/hbase/blob/017ab8d7/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTimeRangeTracker.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTimeRangeTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTimeRangeTracker.java index 8fca399..15b9d14 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTimeRangeTracker.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTimeRangeTracker.java @@ -88,7 +88,7 @@ public class TestTimeRangeTracker { for (int i = 0; i < threads.length; i++) { threads[i].join(); } - System.out.println(trr.getMinimumTimestamp() + " " + trr.getMaximumTimestamp() + " " + + System.out.println(trr.getMin() + " " + trr.getMax() + " " + (System.currentTimeMillis() - start)); } }