View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.regionserver;
20  
21  import java.io.DataInput;
22  import java.io.DataOutput;
23  import java.io.IOException;
24  
25  import org.apache.hadoop.hbase.Cell;
26  import org.apache.hadoop.hbase.CellUtil;
27  import org.apache.hadoop.hbase.classification.InterfaceAudience;
28  import org.apache.hadoop.hbase.io.TimeRange;
29  import org.apache.hadoop.io.Writable;
30  
31  /**
32   * Stores the minimum and maximum timestamp values (both are inclusive).
33   * Can be used to find if any given time range overlaps with its time range
34   * MemStores use this class to track its minimum and maximum timestamps.
35   * When writing StoreFiles, this information is stored in meta blocks and used
36   * at read time to match against the required TimeRange.
37   */
38  @InterfaceAudience.Private
39  public class TimeRangeTracker implements Writable {
40    static final long INITIAL_MINIMUM_TIMESTAMP = Long.MAX_VALUE;
41    long minimumTimestamp = INITIAL_MINIMUM_TIMESTAMP;
42    long maximumTimestamp = -1;
43  
44    /**
45     * Default constructor.
46     * Initializes TimeRange to be null
47     */
48    public TimeRangeTracker() {}
49  
50    /**
51     * Copy Constructor
52     * @param trt source TimeRangeTracker
53     */
54    public TimeRangeTracker(final TimeRangeTracker trt) {
55      set(trt.getMinimumTimestamp(), trt.getMaximumTimestamp());
56    }
57  
58    public TimeRangeTracker(long minimumTimestamp, long maximumTimestamp) {
59      set(minimumTimestamp, maximumTimestamp);
60    }
61  
62    private void set(final long min, final long max) {
63      this.minimumTimestamp = min;
64      this.maximumTimestamp = max;
65    }
66  
67    /**
68     * @param l
69     * @return True if we initialized values
70     */
71    private boolean init(final long l) {
72      if (this.minimumTimestamp != INITIAL_MINIMUM_TIMESTAMP) return false;
73      set(l, l);
74      return true;
75    }
76  
77    /**
78     * Update the current TimestampRange to include the timestamp from Cell
79     * If the Key is of type DeleteColumn or DeleteFamily, it includes the
80     * entire time range from 0 to timestamp of the key.
81     * @param cell the Cell to include
82     */
83    public void includeTimestamp(final Cell cell) {
84      includeTimestamp(cell.getTimestamp());
85      if (CellUtil.isDeleteColumnOrFamily(cell)) {
86        includeTimestamp(0);
87      }
88    }
89  
90    /**
91     * If required, update the current TimestampRange to include timestamp
92     * @param timestamp the timestamp value to include
93     */
94    void includeTimestamp(final long timestamp) {
95      // Do test outside of synchronization block.  Synchronization in here can be problematic
96      // when many threads writing one Store -- they can all pile up trying to add in here.
97      // Happens when doing big write upload where we are hammering on one region.
98      if (timestamp < this.minimumTimestamp) {
99        synchronized (this) {
100         if (!init(timestamp)) {
101           if (timestamp < this.minimumTimestamp) {
102             this.minimumTimestamp = timestamp;
103           }
104         }
105       }
106     } else if (timestamp > this.maximumTimestamp) {
107       synchronized (this) {
108         if (!init(timestamp)) {
109           if (this.maximumTimestamp < timestamp) {
110             this.maximumTimestamp =  timestamp;
111           }
112         }
113       }
114     }
115   }
116 
117   /**
118    * Check if the range has any overlap with TimeRange
119    * @param tr TimeRange
120    * @return True if there is overlap, false otherwise
121    */
122   public synchronized boolean includesTimeRange(final TimeRange tr) {
123     return (this.minimumTimestamp < tr.getMax() && this.maximumTimestamp >= tr.getMin());
124   }
125 
126   /**
127    * @return the minimumTimestamp
128    */
129   public synchronized long getMinimumTimestamp() {
130     return minimumTimestamp;
131   }
132 
133   /**
134    * @return the maximumTimestamp
135    */
136   public synchronized long getMaximumTimestamp() {
137     return maximumTimestamp;
138   }
139 
140   public synchronized void write(final DataOutput out) throws IOException {
141     out.writeLong(minimumTimestamp);
142     out.writeLong(maximumTimestamp);
143   }
144 
145   public synchronized void readFields(final DataInput in) throws IOException {
146     this.minimumTimestamp = in.readLong();
147     this.maximumTimestamp = in.readLong();
148   }
149 
150   @Override
151   public synchronized String toString() {
152     return "[" + minimumTimestamp + "," + maximumTimestamp + "]";
153   }
154 }