001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.io;
019
020import org.apache.hadoop.hbase.util.Bytes;
021import org.apache.yetus.audience.InterfaceAudience;
022
023/**
024 * Represents an interval of version timestamps. Presumes timestamps between
025 * {@link #INITIAL_MIN_TIMESTAMP} and {@link #INITIAL_MAX_TIMESTAMP} only. Gets freaked out if
026 * passed a timestamp that is < {@link #INITIAL_MIN_TIMESTAMP},
027 * <p>
028 * Evaluated according to minStamp &lt;= timestamp &lt; maxStamp or [minStamp,maxStamp) in interval
029 * notation.
030 * <p>
031 * Can be returned and read by clients. Should not be directly created by clients. Thus, all
032 * constructors are purposely @InterfaceAudience.Private.
033 * <p>
034 * Immutable. Thread-safe.
035 */
036@InterfaceAudience.Public
037public class TimeRange {
038  public static final long INITIAL_MIN_TIMESTAMP = 0L;
039  public static final long INITIAL_MAX_TIMESTAMP = Long.MAX_VALUE;
040  private static final TimeRange ALL_TIME =
041    new TimeRange(INITIAL_MIN_TIMESTAMP, INITIAL_MAX_TIMESTAMP);
042
043  public static TimeRange allTime() {
044    return ALL_TIME;
045  }
046
047  public static TimeRange at(long ts) {
048    if (ts < 0 || ts == Long.MAX_VALUE) {
049      throw new IllegalArgumentException("invalid ts:" + ts);
050    }
051    return new TimeRange(ts, ts + 1);
052  }
053
054  /**
055   * Represents the time interval [minStamp, Long.MAX_VALUE)
056   * @param minStamp the minimum timestamp value, inclusive
057   */
058  public static TimeRange from(long minStamp) {
059    check(minStamp, INITIAL_MAX_TIMESTAMP);
060    return new TimeRange(minStamp, INITIAL_MAX_TIMESTAMP);
061  }
062
063  /**
064   * Represents the time interval [0, maxStamp)
065   * @param maxStamp the minimum timestamp value, exclusive
066   */
067  public static TimeRange until(long maxStamp) {
068    check(INITIAL_MIN_TIMESTAMP, maxStamp);
069    return new TimeRange(INITIAL_MIN_TIMESTAMP, maxStamp);
070  }
071
072  /**
073   * Represents the time interval [minStamp, maxStamp)
074   * @param minStamp the minimum timestamp, inclusive
075   * @param maxStamp the maximum timestamp, exclusive
076   */
077  public static TimeRange between(long minStamp, long maxStamp) {
078    check(minStamp, maxStamp);
079    return new TimeRange(minStamp, maxStamp);
080  }
081
082  private final long minStamp;
083  private final long maxStamp;
084  private final boolean allTime;
085
086  /**
087   * Default constructor. Represents interval [0, Long.MAX_VALUE) (allTime)
088   * @deprecated This is made @InterfaceAudience.Private in the 2.0 line and above and may be
089   *             changed to private or removed in 3.0.
090   */
091  @Deprecated
092  @InterfaceAudience.Private
093  public TimeRange() {
094    this(INITIAL_MIN_TIMESTAMP, INITIAL_MAX_TIMESTAMP);
095  }
096
097  /**
098   * Represents interval [minStamp, Long.MAX_VALUE)
099   * @param minStamp the minimum timestamp value, inclusive
100   * @deprecated This is made @InterfaceAudience.Private in the 2.0 line and above and may be
101   *             changed to private or removed in 3.0.
102   */
103  @Deprecated
104  @InterfaceAudience.Private
105  public TimeRange(long minStamp) {
106    this(minStamp, INITIAL_MAX_TIMESTAMP);
107  }
108
109  /**
110   * Represents interval [minStamp, Long.MAX_VALUE)
111   * @param minStamp the minimum timestamp value, inclusive
112   * @deprecated This is made @InterfaceAudience.Private in the 2.0 line and above and may be
113   *             changed to private or removed in 3.0.
114   */
115  @Deprecated
116  @InterfaceAudience.Private
117  public TimeRange(byte[] minStamp) {
118    this(Bytes.toLong(minStamp));
119  }
120
121  /**
122   * Represents interval [minStamp, maxStamp)
123   * @param minStamp the minimum timestamp, inclusive
124   * @param maxStamp the maximum timestamp, exclusive
125   * @deprecated This is made @InterfaceAudience.Private in the 2.0 line and above and may be
126   *             changed to private or removed in 3.0.
127   */
128  @Deprecated
129  @InterfaceAudience.Private
130  public TimeRange(byte[] minStamp, byte[] maxStamp) {
131    this(Bytes.toLong(minStamp), Bytes.toLong(maxStamp));
132  }
133
134  /**
135   * Represents interval [minStamp, maxStamp)
136   * @param minStamp the minimum timestamp, inclusive
137   * @param maxStamp the maximum timestamp, exclusive
138   * @throws IllegalArgumentException if either <0,
139   * @deprecated This is made @InterfaceAudience.Private in the 2.0 line and above and may be
140   *             changed to private or removed in 3.0.
141   */
142  @Deprecated
143  @InterfaceAudience.Private
144  public TimeRange(long minStamp, long maxStamp) {
145    check(minStamp, maxStamp);
146    this.minStamp = minStamp;
147    this.maxStamp = maxStamp;
148    this.allTime = isAllTime(minStamp, maxStamp);
149  }
150
151  private static boolean isAllTime(long minStamp, long maxStamp) {
152    return minStamp == INITIAL_MIN_TIMESTAMP && maxStamp == INITIAL_MAX_TIMESTAMP;
153  }
154
155  private static void check(long minStamp, long maxStamp) {
156    if (minStamp < 0 || maxStamp < 0) {
157      throw new IllegalArgumentException(
158        "Timestamp cannot be negative. minStamp:" + minStamp + ", maxStamp:" + maxStamp);
159    }
160    if (maxStamp < minStamp) {
161      throw new IllegalArgumentException("maxStamp is smaller than minStamp");
162    }
163  }
164
165  /** Returns the smallest timestamp that should be considered */
166  public long getMin() {
167    return minStamp;
168  }
169
170  /** Returns the biggest timestamp that should be considered */
171  public long getMax() {
172    return maxStamp;
173  }
174
175  /**
176   * Check if it is for all time
177   * @return true if it is for all time
178   */
179  public boolean isAllTime() {
180    return allTime;
181  }
182
183  /**
184   * Check if the specified timestamp is within this TimeRange.
185   * <p>
186   * Returns true if within interval [minStamp, maxStamp), false if not.
187   * @param bytes  timestamp to check
188   * @param offset offset into the bytes
189   * @return true if within TimeRange, false if not
190   * @deprecated This is made @InterfaceAudience.Private in the 2.0 line and above and may be
191   *             changed to private or removed in 3.0. Use {@link #withinTimeRange(long)} instead
192   */
193  @Deprecated
194  public boolean withinTimeRange(byte[] bytes, int offset) {
195    if (allTime) {
196      return true;
197    }
198    return withinTimeRange(Bytes.toLong(bytes, offset));
199  }
200
201  /**
202   * Check if the specified timestamp is within this TimeRange.
203   * <p>
204   * Returns true if within interval [minStamp, maxStamp), false if not.
205   * @param timestamp timestamp to check
206   * @return true if within TimeRange, false if not
207   */
208  public boolean withinTimeRange(long timestamp) {
209    assert timestamp >= 0;
210    if (this.allTime) {
211      return true;
212    }
213    // check if >= minStamp
214    return (minStamp <= timestamp && timestamp < maxStamp);
215  }
216
217  /**
218   * Check if the range has any overlap with TimeRange
219   * @param tr TimeRange
220   * @return True if there is overlap, false otherwise
221   */
222  // This method came from TimeRangeTracker. We used to go there for this function but better
223  // to come here to the immutable, unsynchronized datastructure at read time.
224  public boolean includesTimeRange(final TimeRange tr) {
225    if (this.allTime) {
226      return true;
227    }
228    assert tr.getMin() >= 0;
229    return getMin() < tr.getMax() && getMax() >= tr.getMin();
230  }
231
232  /**
233   * Check if the specified timestamp is within or after this TimeRange.
234   * <p>
235   * Returns true if greater than minStamp, false if not.
236   * @param timestamp timestamp to check
237   * @return true if within or after TimeRange, false if not
238   */
239  public boolean withinOrAfterTimeRange(long timestamp) {
240    assert timestamp >= 0;
241    if (allTime) {
242      return true;
243    }
244    // check if >= minStamp
245    return timestamp >= minStamp;
246  }
247
248  /**
249   * Compare the timestamp to timerange.
250   * @return -1 if timestamp is less than timerange, 0 if timestamp is within timerange, 1 if
251   *         timestamp is greater than timerange
252   */
253  public int compare(long timestamp) {
254    assert timestamp >= 0;
255    if (this.allTime) {
256      return 0;
257    }
258    if (timestamp < minStamp) {
259      return -1;
260    }
261    return timestamp >= maxStamp ? 1 : 0;
262  }
263
264  @Override
265  public String toString() {
266    StringBuilder sb = new StringBuilder();
267    sb.append("maxStamp=");
268    sb.append(this.maxStamp);
269    sb.append(", minStamp=");
270    sb.append(this.minStamp);
271    return sb.toString();
272  }
273}