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 0044634..350ad80 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 @@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.regionserver; import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; +import java.util.concurrent.atomic.AtomicLong; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; @@ -38,8 +39,8 @@ import org.apache.hadoop.io.Writable; @InterfaceAudience.Private public class TimeRangeTracker implements Writable { static final long INITIAL_MINIMUM_TIMESTAMP = Long.MAX_VALUE; - long minimumTimestamp = INITIAL_MINIMUM_TIMESTAMP; - long maximumTimestamp = -1; + final AtomicLong minimumTimestamp = new AtomicLong(INITIAL_MINIMUM_TIMESTAMP); + final AtomicLong maximumTimestamp = new AtomicLong(-1); /** * Default constructor. @@ -59,17 +60,17 @@ public class TimeRangeTracker implements Writable { set(minimumTimestamp, maximumTimestamp); } - private void set(final long min, final long max) { - this.minimumTimestamp = min; - this.maximumTimestamp = max; + private synchronized void set(final long min, final long max) { + this.minimumTimestamp.set(min); + this.maximumTimestamp.set(max); } /** * @param l * @return True if we initialized values */ - private boolean init(final long l) { - if (this.minimumTimestamp != INITIAL_MINIMUM_TIMESTAMP) return false; + private synchronized boolean init(final long l) { + if (this.minimumTimestamp.get() != INITIAL_MINIMUM_TIMESTAMP) return false; set(l, l); return true; } @@ -95,19 +96,19 @@ public class TimeRangeTracker implements Writable { // Do test outside of synchronization block. Synchronization in here can be problematic // when many threads writing one Store -- they can all pile up trying to add in here. // Happens when doing big write upload where we are hammering on one region. - if (timestamp < this.minimumTimestamp) { + if (timestamp < this.minimumTimestamp.get()) { synchronized (this) { if (!init(timestamp)) { - if (timestamp < this.minimumTimestamp) { - this.minimumTimestamp = timestamp; + if (timestamp < this.minimumTimestamp.get()) { + this.minimumTimestamp.set(timestamp); } } } - } else if (timestamp > this.maximumTimestamp) { + } else if (timestamp > this.maximumTimestamp.get()) { synchronized (this) { if (!init(timestamp)) { - if (this.maximumTimestamp < timestamp) { - this.maximumTimestamp = timestamp; + if (this.maximumTimestamp.get() < timestamp) { + this.maximumTimestamp.set(timestamp); } } } @@ -119,36 +120,39 @@ public class TimeRangeTracker implements Writable { * @param tr TimeRange * @return True if there is overlap, false otherwise */ - public synchronized boolean includesTimeRange(final TimeRange tr) { - return (this.minimumTimestamp < tr.getMax() && this.maximumTimestamp >= tr.getMin()); + public boolean includesTimeRange(final TimeRange tr) { + // If a concurrent update on min or max timestamps, it will only be to increase the span of + // the range. Presume ok doing this test without tying min and max w/ a synchronization + return (this.minimumTimestamp.get() < tr.getMax() && + this.maximumTimestamp.get() >= tr.getMin()); } /** * @return the minimumTimestamp */ - public synchronized long getMinimumTimestamp() { - return minimumTimestamp; + public long getMinimumTimestamp() { + return minimumTimestamp.get(); } /** * @return the maximumTimestamp */ - public synchronized long getMaximumTimestamp() { - return maximumTimestamp; + public long getMaximumTimestamp() { + return maximumTimestamp.get(); } public synchronized void write(final DataOutput out) throws IOException { - out.writeLong(minimumTimestamp); - out.writeLong(maximumTimestamp); + out.writeLong(minimumTimestamp.get()); + out.writeLong(maximumTimestamp.get()); } public synchronized void readFields(final DataInput in) throws IOException { - this.minimumTimestamp = in.readLong(); - this.maximumTimestamp = in.readLong(); + set(in.readLong(), in.readLong()); } @Override - public synchronized String toString() { + public String toString() { + // Do loose read of min and max independent of each other. return "[" + minimumTimestamp + "," + maximumTimestamp + "]"; } } \ No newline at end of file