You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ec...@apache.org on 2016/07/11 23:47:57 UTC

[04/50] [abbrv] hbase git commit: HBASE-16176 Bug fixes/improvements on HBASE-15650 Remove TimeRangeTracker as point of contention when many threads reading a StoreFile Fixes HBASE-16074 ITBLL fails, reports lost big or tiny families broken scanning b

HBASE-16176 Bug fixes/improvements on HBASE-15650 Remove
 TimeRangeTracker as point of contention when many threads reading a StoreFile
 Fixes HBASE-16074 ITBLL fails, reports lost big or tiny families broken
 scanning because of a side effect of a clean up in  HBASE-15650 to make
 TimeRange construction consistent exposed a latent issue in
 TimeRange#compare. See HBASE-16074 for more detail.

Also change HFile Writer constructor so we pass in the TimeRangeTracker, if one,
on construction rather than set later (the flag and reference were not volatile
so could have made for issues in concurrent case). And make sure the construction
of a TimeRange from a TimeRangeTracer on open of an HFile Reader never makes a
bad minimum value, one that would preclude us reading any values from a file
(set min to 0)

M hbase-common/src/main/java/org/apache/hadoop/hbase/io/TimeRange.java
 Call through to next constructor (if minStamp was 0, we'd skip setting
 allTime=true). Add asserts that timestamps are not < 0 cos it messes
 us up if they are (we already were checking for < 0 on construction but
 assert passed in timestamps are not < 0).

M hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
 Add constructor override that takes a TimeRangeTracker (set when flushing
 but not when compacting)

M hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
 Add override creating an HFile in tmp that takes a TimeRangeTracker

M hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
 Add override for HFile Writer that takes a TimeRangeTracker Take it on
 construction instead of having it passed by a setter later (flags and
 reference set by the setter were not volatile... could have been prob
 in concurrent case)

M hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/TimeRangeTracker.java
 Log WARN if bad initial TimeRange value (and then 'fix' it)

M hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTimeRangeTracker.java
 A few tests to prove serialization works as expected and that we'll get a bad min if not constructed properly.

M hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java
 Handle OLDEST_TIMESTAMP explictly. Don't expect TimeRange to do it.

M hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestQueryMatcher.java
 Refactor from junit3 to junit4 and add test for this weird case.


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/496fd983
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/496fd983
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/496fd983

Branch: refs/heads/HBASE-14850
Commit: 496fd9837a0fb199a516758a632fecfe59b0b480
Parents: 3c39cbd
Author: stack <st...@apache.org>
Authored: Fri Jul 8 18:18:01 2016 -0700
Committer: stack <st...@apache.org>
Committed: Fri Jul 8 18:20:31 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/client/Query.java   |  4 +-
 .../org/apache/hadoop/hbase/HConstants.java     |  5 +
 .../org/apache/hadoop/hbase/KeyValueUtil.java   |  2 +-
 .../org/apache/hadoop/hbase/io/TimeRange.java   | 99 ++++++++++++--------
 .../hbase/mob/DefaultMobStoreFlusher.java       |  5 +-
 .../hbase/regionserver/DefaultStoreFlusher.java |  4 +-
 .../regionserver/ExplicitColumnTracker.java     |  4 +-
 .../hadoop/hbase/regionserver/HStore.java       | 27 +++++-
 .../hbase/regionserver/ScanQueryMatcher.java    | 11 ++-
 .../apache/hadoop/hbase/regionserver/Store.java | 20 +++-
 .../hadoop/hbase/regionserver/StoreFile.java    |  1 -
 .../hbase/regionserver/StoreFileReader.java     |  2 +-
 .../hbase/regionserver/StoreFileWriter.java     | 61 ++++++++----
 .../hadoop/hbase/regionserver/StoreScanner.java |  3 +-
 .../hbase/regionserver/StripeStoreFlusher.java  |  4 +-
 .../hbase/regionserver/TimeRangeTracker.java    | 23 ++---
 .../hbase/regionserver/TestQueryMatcher.java    | 35 +++++--
 .../regionserver/TestTimeRangeTracker.java      | 40 +++++++-
 18 files changed, 250 insertions(+), 100 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/496fd983/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Query.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Query.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Query.java
index 53062a0..53dd2c1 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Query.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Query.java
@@ -203,6 +203,4 @@ public abstract class Query extends OperationWithAttributes {
   public Map<byte[], TimeRange> getColumnFamilyTimeRange() {
     return this.colFamTimeRangeMap;
   }
-
-
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/496fd983/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
index 0bc0a07..256c374 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
@@ -544,7 +544,12 @@ public final class HConstants {
 
   /**
    * Timestamp to use when we want to refer to the oldest cell.
+   * Special! Used in fake Cells only. Should never be the timestamp on an actual Cell returned to
+   * a client.
+   * @deprecated Should not be public since hbase-1.3.0. For internal use only. Move internal to
+   * Scanners flagged as special timestamp value never to be returned as timestamp on a Cell.
    */
+  @Deprecated
   public static final long OLDEST_TIMESTAMP = Long.MIN_VALUE;
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/496fd983/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java
index c9da738..6b740a7 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java
@@ -237,7 +237,7 @@ public class KeyValueUtil {
   /**
    * Create a KeyValue for the specified row, family and qualifier that would be
    * larger than or equal to all other possible KeyValues that have the same
-   * row, family, qualifier. Used for reseeking.
+   * row, family, qualifier. Used for reseeking. Should NEVER be returned to a client.
    *
    * @param row
    *          row key

http://git-wip-us.apache.org/repos/asf/hbase/blob/496fd983/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 d5b2509..fed20c4 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
@@ -1,5 +1,4 @@
 /*
- *
  * 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
@@ -26,57 +25,86 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
- * Represents an interval of version timestamps.
+ * Represents an interval of version timestamps. Presumes timestamps between
+ * {@link #INITIAL_MIN_TIMESTAMP} and {@link #INITIAL_MAX_TIMESTAMP} only. Gets freaked out if
+ * passed a timestamp that is < {@link #INITIAL_MIN_TIMESTAMP},
  * <p>
  * Evaluated according to minStamp &lt;= timestamp &lt; maxStamp
  * or [minStamp,maxStamp) in interval notation.
  * <p>
  * Only used internally; should not be accessed directly by clients.
+ *<p>Immutable. Thread-safe.
  */
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public class TimeRange {
-  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;
+  public static final long INITIAL_MIN_TIMESTAMP = 0L;
+  public static final long INITIAL_MAX_TIMESTAMP = Long.MAX_VALUE;
+  private final long minStamp;
+  private final long maxStamp;
   private final boolean allTime;
 
   /**
    * Default constructor.
    * Represents interval [0, Long.MAX_VALUE) (allTime)
+   * @deprecated This is made @InterfaceAudience.Private in the 2.0 line and above
    */
+  @Deprecated
   public TimeRange() {
-    allTime = true;
+    this(INITIAL_MIN_TIMESTAMP, INITIAL_MAX_TIMESTAMP);
   }
 
   /**
    * Represents interval [minStamp, Long.MAX_VALUE)
    * @param minStamp the minimum timestamp value, inclusive
+   * @deprecated This is made @InterfaceAudience.Private in the 2.0 line and above
    */
+  @Deprecated
   public TimeRange(long minStamp) {
-    this.minStamp = minStamp;
-    this.allTime = this.minStamp == MIN_TIME;
+    this(minStamp, INITIAL_MAX_TIMESTAMP);
   }
 
   /**
    * Represents interval [minStamp, Long.MAX_VALUE)
    * @param minStamp the minimum timestamp value, inclusive
+   * @deprecated This is made @InterfaceAudience.Private in the 2.0 line and above
    */
+  @Deprecated
   public TimeRange(byte [] minStamp) {
-    this.minStamp = Bytes.toLong(minStamp);
-    this.allTime = false;
+    this(Bytes.toLong(minStamp));
+  }
+
+  /**
+   * Represents interval [minStamp, maxStamp)
+   * @param minStamp the minimum timestamp, inclusive
+   * @param maxStamp the maximum timestamp, exclusive
+   * @deprecated This is made @InterfaceAudience.Private in the 2.0 line and above
+   */
+  @Deprecated
+  public TimeRange(byte [] minStamp, byte [] maxStamp) {
+    this(Bytes.toLong(minStamp), Bytes.toLong(maxStamp));
   }
 
   /**
    * Represents interval [minStamp, maxStamp)
    * @param minStamp the minimum timestamp, inclusive
    * @param maxStamp the maximum timestamp, exclusive
-   * @throws IllegalArgumentException
+   * @throws IllegalArgumentException if either <0,
+   * @deprecated This is made @InterfaceAudience.Private in the 2.0 line and above
    */
+  @Deprecated
   public TimeRange(long minStamp, long maxStamp) {
+    check(minStamp, maxStamp);
+    this.minStamp = minStamp;
+    this.maxStamp = maxStamp;
+    this.allTime = isAllTime(minStamp, maxStamp);
+  }
+
+  private static boolean isAllTime(long minStamp, long maxStamp) {
+    return minStamp == INITIAL_MIN_TIMESTAMP && maxStamp == INITIAL_MAX_TIMESTAMP;
+  }
+
+  private static void check(long minStamp, long maxStamp) {
     if (minStamp < 0 || maxStamp < 0) {
       throw new IllegalArgumentException("Timestamp cannot be negative. minStamp:" + minStamp
         + ", maxStamp:" + maxStamp);
@@ -84,20 +112,6 @@ public class TimeRange {
     if (maxStamp < minStamp) {
       throw new IllegalArgumentException("maxStamp is smaller than minStamp");
     }
-    this.minStamp = minStamp;
-    this.maxStamp = maxStamp;
-    this.allTime = this.minStamp == MIN_TIME && this.maxStamp == MAX_TIME;
-  }
-
-  /**
-   * Represents interval [minStamp, maxStamp)
-   * @param minStamp the minimum timestamp, inclusive
-   * @param maxStamp the maximum timestamp, exclusive
-   * @throws IOException
-   */
-  public TimeRange(byte [] minStamp, byte [] maxStamp)
-  throws IOException {
-    this(Bytes.toLong(minStamp), Bytes.toLong(maxStamp));
   }
 
   /**
@@ -125,14 +139,15 @@ public class TimeRange {
   /**
    * Check if the specified timestamp is within this TimeRange.
    * <p>
-   * Returns true if within interval [minStamp, maxStamp), false
-   * if not.
+   * Returns true if within interval [minStamp, maxStamp), false if not.
    * @param bytes timestamp to check
    * @param offset offset into the bytes
    * @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));
   }
 
@@ -145,6 +160,7 @@ public class TimeRange {
    * @return true if within TimeRange, false if not
    */
   public boolean withinTimeRange(long timestamp) {
+    assert timestamp >= 0;
     if (this.allTime) {
       return true;
     }
@@ -163,6 +179,7 @@ public class TimeRange {
     if (this.allTime) {
       return true;
     }
+    assert tr.getMin() >= 0;
     return getMin() < tr.getMax() && getMax() >= tr.getMin();
   }
 
@@ -175,27 +192,29 @@ public class TimeRange {
    * @return true if within TimeRange, false if not
    */
   public boolean withinOrAfterTimeRange(long timestamp) {
-    if(allTime) return true;
+    assert timestamp >= 0;
+    if (allTime) {
+      return true;
+    }
     // check if >= minStamp
-    return (timestamp >= minStamp);
+    return timestamp >= minStamp;
   }
 
   /**
-   * Compare the timestamp to timerange
-   * @param timestamp
+   * Compare the timestamp to timerange.
    * @return -1 if timestamp is less than timerange,
    * 0 if timestamp is within timerange,
    * 1 if timestamp is greater than timerange
    */
   public int compare(long timestamp) {
-    if (allTime) return 0;
+    assert timestamp >= 0;
+    if (this.allTime) {
+      return 0;
+    }
     if (timestamp < minStamp) {
       return -1;
-    } else if (timestamp >= maxStamp) {
-      return 1;
-    } else {
-      return 0;
     }
+    return timestamp >= maxStamp? 1: 0;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/496fd983/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java
index 93fa327..eb2564d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java
@@ -99,7 +99,7 @@ public class DefaultMobStoreFlusher extends DefaultStoreFlusher {
   public List<Path> flushSnapshot(MemStoreSnapshot snapshot, long cacheFlushId,
       MonitoredTask status, ThroughputController throughputController) throws IOException {
     ArrayList<Path> result = new ArrayList<Path>();
-    int cellsCount = snapshot.getCellsCount();
+    long cellsCount = snapshot.getCellsCount();
     if (cellsCount == 0) return result; // don't flush if there are no entries
 
     // Use a store scanner to find which rows to flush.
@@ -116,8 +116,7 @@ public class DefaultMobStoreFlusher extends DefaultStoreFlusher {
         status.setStatus("Flushing " + store + ": creating writer");
         // Write the map out to the disk
         writer = store.createWriterInTmp(cellsCount, store.getFamily().getCompression(),
-            false, true, true);
-        writer.setTimeRangeTracker(snapshot.getTimeRangeTracker());
+            false, true, true, false/*default for dropbehind*/, snapshot.getTimeRangeTracker());
         try {
           // It's a mob store, flush the cells in a mob way. This is the difference of flushing
           // between a normal and a mob store.

http://git-wip-us.apache.org/repos/asf/hbase/blob/496fd983/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFlusher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFlusher.java
index bf8794a..079501e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFlusher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFlusher.java
@@ -68,8 +68,8 @@ public class DefaultStoreFlusher extends StoreFlusher {
             /* isCompaction = */ false,
             /* includeMVCCReadpoint = */ true,
             /* includesTags = */ snapshot.isTagsPresent(),
-            /* shouldDropBehind = */ false);
-        writer.setTimeRangeTracker(snapshot.getTimeRangeTracker());
+            /* shouldDropBehind = */ false,
+            snapshot.getTimeRangeTracker());
         IOException e = null;
         try {
           performFlush(scanner, writer, smallestReadPoint, throughputController);

http://git-wip-us.apache.org/repos/asf/hbase/blob/496fd983/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ExplicitColumnTracker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ExplicitColumnTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ExplicitColumnTracker.java
index 86f2fbc..86c8b48 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ExplicitColumnTracker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ExplicitColumnTracker.java
@@ -46,7 +46,7 @@ import org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.MatchCode;
  * believes that the current column should be skipped (by timestamp, filter etc.)</li>
  * </ul>
  * <p>
- * These two methods returns a 
+ * These two methods returns a
  * {@link org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.MatchCode}
  * to define what action should be taken.
  * <p>
@@ -77,7 +77,7 @@ public class ExplicitColumnTracker implements ColumnTracker {
    * @param minVersions minimum number of versions to keep
    * @param maxVersions maximum versions to return per column
    * @param oldestUnexpiredTS the oldest timestamp we are interested in,
-   *  based on TTL 
+   *  based on TTL
    */
   public ExplicitColumnTracker(NavigableSet<byte[]> columns, int minVersions,
       int maxVersions, long oldestUnexpiredTS) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/496fd983/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
index 2d1b9a6..4b79153 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
@@ -958,6 +958,23 @@ public class HStore implements Store {
       boolean isCompaction, boolean includeMVCCReadpoint, boolean includesTag,
       boolean shouldDropBehind)
   throws IOException {
+    return createWriterInTmp(maxKeyCount, compression, isCompaction, includeMVCCReadpoint,
+        includesTag, shouldDropBehind, null);
+  }
+
+  /*
+   * @param maxKeyCount
+   * @param compression Compression algorithm to use
+   * @param isCompaction whether we are creating a new file in a compaction
+   * @param includesMVCCReadPoint - whether to include MVCC or not
+   * @param includesTag - includesTag or not
+   * @return Writer for a new StoreFile in the tmp dir.
+   */
+  @Override
+  public StoreFileWriter createWriterInTmp(long maxKeyCount, Compression.Algorithm compression,
+      boolean isCompaction, boolean includeMVCCReadpoint, boolean includesTag,
+      boolean shouldDropBehind, final TimeRangeTracker trt)
+  throws IOException {
     final CacheConfig writerCacheConf;
     if (isCompaction) {
       // Don't cache data on write on compactions.
@@ -973,7 +990,7 @@ public class HStore implements Store {
     }
     HFileContext hFileContext = createFileContext(compression, includeMVCCReadpoint, includesTag,
       cryptoContext);
-    StoreFileWriter w = new StoreFileWriter.Builder(conf, writerCacheConf,
+    StoreFileWriter.Builder builder = new StoreFileWriter.Builder(conf, writerCacheConf,
         this.getFileSystem())
             .withFilePath(fs.createTempName())
             .withComparator(comparator)
@@ -981,9 +998,11 @@ public class HStore implements Store {
             .withMaxKeyCount(maxKeyCount)
             .withFavoredNodes(favoredNodes)
             .withFileContext(hFileContext)
-            .withShouldDropCacheBehind(shouldDropBehind)
-            .build();
-    return w;
+            .withShouldDropCacheBehind(shouldDropBehind);
+    if (trt != null) {
+      builder.withTimeRangeTracker(trt);
+    }
+    return builder.build();
   }
 
   private HFileContext createFileContext(Compression.Algorithm compression,

http://git-wip-us.apache.org/repos/asf/hbase/blob/496fd983/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java
index bfd399b..d2d0ccb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java
@@ -404,7 +404,16 @@ public class ScanQueryMatcher {
         }
     }
 
-    int timestampComparison = tr.compare(timestamp);
+    // NOTE: Cryptic stuff!
+    // if the timestamp is HConstants.OLDEST_TIMESTAMP, then this is a fake cell made to prime a
+    // Scanner; See KeyValueUTil#createLastOnRow. This Cell should never end up returning out of
+    // here a matchcode of INCLUDE else we will return to the client a fake Cell. If we call
+    // TimeRange, it will return 0 because it doesn't deal in OLDEST_TIMESTAMP and we will fall
+    // into the later code where we could return a matchcode of INCLUDE. See HBASE-16074 "ITBLL
+    // fails, reports lost big or tiny families" for a horror story. Check here for
+    // OLDEST_TIMESTAMP. TimeRange#compare is about more generic timestamps, between 0L and
+    // Long.MAX_LONG. It doesn't do OLDEST_TIMESTAMP weird handling.
+    int timestampComparison = timestamp == HConstants.OLDEST_TIMESTAMP? -1: tr.compare(timestamp);
     if (timestampComparison >= 1) {
       return MatchCode.SKIP;
     } else if (timestampComparison <= -1) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/496fd983/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
index 3419937..5dec59a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
@@ -189,8 +189,24 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf
     boolean shouldDropBehind
   ) throws IOException;
 
-
-
+  /**
+   * @param maxKeyCount
+   * @param compression Compression algorithm to use
+   * @param isCompaction whether we are creating a new file in a compaction
+   * @param includeMVCCReadpoint whether we should out the MVCC readpoint
+   * @param shouldDropBehind should the writer drop caches behind writes
+   * @param trt Ready-made timetracker to use.
+   * @return Writer for a new StoreFile in the tmp dir.
+   */
+  StoreFileWriter createWriterInTmp(
+    long maxKeyCount,
+    Compression.Algorithm compression,
+    boolean isCompaction,
+    boolean includeMVCCReadpoint,
+    boolean includesTags,
+    boolean shouldDropBehind,
+    final TimeRangeTracker trt
+  ) throws IOException;
 
   // Compaction oriented methods
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/496fd983/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 46f0cb8..589d844 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
@@ -198,7 +198,6 @@ public class StoreFile {
     this(fs, new StoreFileInfo(conf, fs, p), conf, cacheConf, cfBloomType);
   }
 
-
   /**
    * Constructor, loads a reader and it's indices, etc. May allocate a
    * substantial amount of ram depending on the underlying files (10-20MB?).

http://git-wip-us.apache.org/repos/asf/hbase/blob/496fd983/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java
index e890381..1582237 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java
@@ -639,7 +639,7 @@ public class StoreFileReader {
   }
 
   public long getMaxTimestamp() {
-    return timeRange == null ? Long.MAX_VALUE : timeRange.getMax();
+    return timeRange == null ? TimeRange.INITIAL_MAX_TIMESTAMP: timeRange.getMax();
   }
 
   boolean isSkipResetSeqId() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/496fd983/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java
index 4a42b7f..442b90d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java
@@ -62,7 +62,6 @@ public class StoreFileWriter implements Compactor.CellSink {
   private Cell lastDeleteFamilyCell = null;
   private long deleteFamilyCnt = 0;
 
-  TimeRangeTracker timeRangeTracker = new TimeRangeTracker();
   /**
    * 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
@@ -70,7 +69,8 @@ public class StoreFileWriter implements Compactor.CellSink {
    * 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;
+   private final boolean timeRangeTrackerSet;
+   final TimeRangeTracker timeRangeTracker;
 
   protected HFile.Writer writer;
   private KeyValue.KeyOnlyKeyValue lastBloomKeyOnlyKV = null;
@@ -91,6 +91,36 @@ public class StoreFileWriter implements Compactor.CellSink {
       final CellComparator comparator, BloomType bloomType, long maxKeys,
       InetSocketAddress[] favoredNodes, HFileContext fileContext)
           throws IOException {
+      this(fs, path, conf, cacheConf, comparator, bloomType, maxKeys, favoredNodes, fileContext,
+          null);
+    }
+
+    /**
+     * Creates an HFile.Writer that also write helpful meta data.
+     * @param fs file system to write to
+     * @param path file name to create
+     * @param conf user configuration
+     * @param comparator key comparator
+     * @param bloomType bloom filter setting
+     * @param maxKeys the expected maximum number of keys to be added. Was used
+     *        for Bloom filter size in {@link HFile} format version 1.
+     * @param favoredNodes
+     * @param fileContext - The HFile context
+   * @param trt Ready-made timetracker to use.
+     * @throws IOException problem writing to FS
+     */
+    private StoreFileWriter(FileSystem fs, Path path,
+        final Configuration conf,
+        CacheConfig cacheConf,
+        final CellComparator comparator, BloomType bloomType, long maxKeys,
+        InetSocketAddress[] favoredNodes, HFileContext fileContext,
+        final TimeRangeTracker trt)
+            throws IOException {
+    // If passed a TimeRangeTracker, use it. Set timeRangeTrackerSet so we don't destroy it.
+    // TODO: put the state of the TRT on the TRT; i.e. make a read-only version (TimeRange) when
+    // it no longer writable.
+    this.timeRangeTrackerSet = trt != null;
+    this.timeRangeTracker = this.timeRangeTrackerSet? trt: new TimeRangeTracker();
     writer = HFile.getWriterFactory(conf, cacheConf)
         .withPath(fs, path)
         .withComparator(comparator)
@@ -171,19 +201,6 @@ public class StoreFileWriter implements Compactor.CellSink {
   }
 
   /**
-   * 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;
-    timeRangeTrackerSet = true;
-  }
-
-  /**
    * Record the earlest Put timestamp.
    *
    * If the timeRangeTracker is not set,
@@ -413,6 +430,7 @@ public class StoreFileWriter implements Compactor.CellSink {
     private Path filePath;
     private InetSocketAddress[] favoredNodes;
     private HFileContext fileContext;
+    private TimeRangeTracker trt;
 
     public Builder(Configuration conf, CacheConfig cacheConf,
         FileSystem fs) {
@@ -422,6 +440,17 @@ public class StoreFileWriter implements Compactor.CellSink {
     }
 
     /**
+     * @param trt A premade TimeRangeTracker to use rather than build one per append (building one
+     * of these is expensive so good to pass one in if you have one).
+     * @return this (for chained invocation)
+     */
+    public Builder withTimeRangeTracker(final TimeRangeTracker trt) {
+      Preconditions.checkNotNull(trt);
+      this.trt = trt;
+      return this;
+    }
+
+    /**
      * Use either this method or {@link #withFilePath}, but not both.
      * @param dir Path to column family directory. The directory is created if
      *          does not exist. The file is given a unique name within this
@@ -514,7 +543,7 @@ public class StoreFileWriter implements Compactor.CellSink {
         comparator = CellComparator.COMPARATOR;
       }
       return new StoreFileWriter(fs, filePath,
-          conf, cacheConf, comparator, bloomType, maxKeyCount, favoredNodes, fileContext);
+          conf, cacheConf, comparator, bloomType, maxKeyCount, favoredNodes, fileContext, trt);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/496fd983/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
index ceb4b8e..080bb95 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
@@ -402,8 +402,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
 
     // We can only exclude store files based on TTL if minVersions is set to 0.
     // Otherwise, we might have to return KVs that have technically expired.
-    long expiredTimestampCutoff = minVersions == 0 ? oldestUnexpiredTS :
-        Long.MIN_VALUE;
+    long expiredTimestampCutoff = minVersions == 0 ? oldestUnexpiredTS: Long.MIN_VALUE;
 
     // include only those scan files which pass all filters
     for (KeyValueScanner kvs : allScanners) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/496fd983/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFlusher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFlusher.java
index cf0de1f..22c3ce7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFlusher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFlusher.java
@@ -114,8 +114,8 @@ public class StripeStoreFlusher extends StoreFlusher {
             /* isCompaction = */ false,
             /* includeMVCCReadpoint = */ true,
             /* includesTags = */ true,
-            /* shouldDropBehind = */ false);
-        writer.setTimeRangeTracker(tracker);
+            /* shouldDropBehind = */ false,
+            tracker);
         return writer;
       }
     };

http://git-wip-us.apache.org/repos/asf/hbase/blob/496fd983/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 48adc9d..1ea3c70 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
@@ -30,7 +30,7 @@ import org.apache.hadoop.hbase.util.Writables;
 import org.apache.hadoop.io.Writable;
 
 /**
- * Stores minimum and maximum timestamp values. Both timestamps are inclusive.
+ * Stores minimum and maximum timestamp values.
  * 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
@@ -45,8 +45,8 @@ import org.apache.hadoop.io.Writable;
 @InterfaceAudience.Private
 public class TimeRangeTracker implements Writable {
   static final long INITIAL_MIN_TIMESTAMP = Long.MAX_VALUE;
+  static final long INITIAL_MAX_TIMESTAMP = -1L;
   long minimumTimestamp = INITIAL_MIN_TIMESTAMP;
-  static final long INITIAL_MAX_TIMESTAMP = -1;
   long maximumTimestamp = INITIAL_MAX_TIMESTAMP;
 
   /**
@@ -125,7 +125,7 @@ public class TimeRangeTracker implements Writable {
   }
 
   /**
-   * Check if the range has any overlap with TimeRange
+   * Check if the range has ANY overlap with TimeRange
    * @param tr TimeRange
    * @return True if there is overlap, false otherwise
    */
@@ -185,22 +185,19 @@ public class TimeRangeTracker implements Writable {
     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 <code>this</code>.
    */
   TimeRange toTimeRange() {
     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();
+    // Initial TimeRangeTracker timestamps are the opposite of what you want for a TimeRange. Fix!
+    if (min == INITIAL_MIN_TIMESTAMP) {
+      min = TimeRange.INITIAL_MIN_TIMESTAMP;
+    }
+    if (max == INITIAL_MAX_TIMESTAMP) {
+      max = TimeRange.INITIAL_MAX_TIMESTAMP;
     }
     return new TimeRange(min, max);
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/496fd983/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestQueryMatcher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestQueryMatcher.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestQueryMatcher.java
index d8de178..db01ee0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestQueryMatcher.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestQueryMatcher.java
@@ -21,6 +21,8 @@ package org.apache.hadoop.hbase.regionserver;
 
 import static org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.MatchCode.INCLUDE;
 import static org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.MatchCode.SKIP;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -28,14 +30,17 @@ import java.util.List;
 import java.util.NavigableSet;
 
 import org.apache.hadoop.hbase.CellComparator;
-import org.apache.hadoop.hbase.HBaseTestCase;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeepDeletedCells;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.KeyValue.Type;
-import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.MatchCode;
@@ -46,8 +51,9 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 @Category({RegionServerTests.class, SmallTests.class})
-public class TestQueryMatcher extends HBaseTestCase {
+public class TestQueryMatcher {
   private static final boolean PRINT = false;
+  private Configuration conf;
 
   private byte[] row1;
   private byte[] row2;
@@ -70,7 +76,7 @@ public class TestQueryMatcher extends HBaseTestCase {
 
   @Before
   public void setUp() throws Exception {
-    super.setUp();
+    this.conf = HBaseConfiguration.create();
     row1 = Bytes.toBytes("row1");
     row2 = Bytes.toBytes("row2");
     row3 = Bytes.toBytes("row3");
@@ -130,6 +136,25 @@ public class TestQueryMatcher extends HBaseTestCase {
     }
   }
 
+  /**
+   * This is a cryptic test. It is checking that we don't include a fake cell, one that has a
+   * timestamp of {@link HConstants#OLDEST_TIMESTAMP}. See HBASE-16074 for background.
+   * @throws IOException
+   */
+  @Test
+  public void testNeverIncludeFakeCell() throws IOException {
+    long now = EnvironmentEdgeManager.currentTime();
+    // Do with fam2 which has a col2 qualifier.
+    ScanQueryMatcher qm = new ScanQueryMatcher(scan,
+        new ScanInfo(this.conf, fam2, 10, 1, ttl, KeepDeletedCells.FALSE, 0, rowComparator),
+        get.getFamilyMap().get(fam2), now - ttl, now);
+    Cell kv = new KeyValue(row1, fam2, col2, 1, data);
+    Cell cell = CellUtil.createLastOnRowCol(kv);
+    qm.setToNewRow(kv);
+    MatchCode code = qm.match(cell);
+    assertFalse(code.compareTo(MatchCode.SEEK_NEXT_COL) != 0);
+    }
+
   @Test
   public void testMatch_ExplicitColumns()
   throws IOException {
@@ -251,7 +276,6 @@ public class TestQueryMatcher extends HBaseTestCase {
     }
   }
 
-
   /**
    * Verify that {@link ScanQueryMatcher} only skips expired KeyValue
    * instances and does not exit early from the row (skipping
@@ -356,4 +380,3 @@ public class TestQueryMatcher extends HBaseTestCase {
     }
   }
 }
-

http://git-wip-us.apache.org/repos/asf/hbase/blob/496fd983/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 3b93bb9..2803baf 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
@@ -20,9 +20,11 @@ package org.apache.hadoop.hbase.regionserver;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
+import java.io.IOException;
 
 import org.apache.hadoop.hbase.io.TimeRange;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Writables;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -32,6 +34,42 @@ import java.io.IOException;
 @Category({RegionServerTests.class, SmallTests.class})
 public class TestTimeRangeTracker {
   @Test
+  public void testExtreme() {
+    TimeRange tr = new TimeRange();
+    assertTrue(tr.includesTimeRange(new TimeRange()));
+    TimeRangeTracker trt = new TimeRangeTracker();
+    assertFalse(trt.includesTimeRange(new TimeRange()));
+    trt.includeTimestamp(1);
+    trt.includeTimestamp(10);
+    assertTrue(trt.includesTimeRange(new TimeRange()));
+  }
+
+  @Test
+  public void testTimeRangeInitialized() {
+    TimeRangeTracker src = new TimeRangeTracker();
+    TimeRange tr = new TimeRange(System.currentTimeMillis());
+    assertFalse(src.includesTimeRange(tr));
+  }
+
+  @Test
+  public void testTimeRangeTrackerNullIsSameAsTimeRangeNull() throws IOException {
+    TimeRangeTracker src = new TimeRangeTracker(1, 2);
+    byte [] bytes = Writables.getBytes(src);
+    TimeRange tgt = TimeRangeTracker.getTimeRange(bytes);
+    assertEquals(src.getMin(), tgt.getMin());
+    assertEquals(src.getMax(), tgt.getMax());
+  }
+
+  @Test
+  public void testSerialization() throws IOException {
+    TimeRangeTracker src = new TimeRangeTracker(1, 2);
+    TimeRangeTracker tgt = new TimeRangeTracker();
+    Writables.copyWritable(src, tgt);
+    assertEquals(src.getMin(), tgt.getMin());
+    assertEquals(src.getMax(), tgt.getMax());
+  }
+
+  @Test
   public void testAlwaysDecrementingSetsMaximum() {
     TimeRangeTracker trr = new TimeRangeTracker();
     trr.includeTimestamp(3);
@@ -145,4 +183,4 @@ public class TestTimeRangeTracker {
     System.out.println(trr.getMin() + " " + trr.getMax() + " " +
       (System.currentTimeMillis() - start));
   }
-}
\ No newline at end of file
+}