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 com.google.common.base.Objects;
22  import com.google.common.base.Objects.ToStringHelper;
23  
24  import java.util.LinkedList;
25  import java.util.concurrent.atomic.AtomicLong;
26  
27  import org.apache.commons.logging.Log;
28  import org.apache.commons.logging.LogFactory;
29  import org.apache.hadoop.hbase.classification.InterfaceAudience;
30  import org.apache.hadoop.hbase.util.Bytes;
31  import org.apache.hadoop.hbase.util.ClassSize;
32  
33  
34  /**
35   * Manages the read/write consistency. This provides an interface for readers to determine what
36   * entries to ignore, and a mechanism for writers to obtain new write numbers, then "commit"
37   * the new writes for readers to read (thus forming atomic transactions).
38   */
39  @InterfaceAudience.Private
40  public class MultiVersionConcurrencyControl {
41    private static final Log LOG = LogFactory.getLog(MultiVersionConcurrencyControl.class);
42    static final long NO_WRITE_NUMBER = 0;
43    private static final long READPOINT_ADVANCE_WAIT_TIME = 10L;
44  
45    final String regionName;
46    final AtomicLong readPoint = new AtomicLong(0);
47    final AtomicLong writePoint = new AtomicLong(0);
48    private final Object readWaiters = new Object();
49    /**
50     * Represents no value, or not set.
51     */
52    public static final long NONE = -1;
53  
54    // This is the pending queue of writes.
55    //
56    // TODO(eclark): Should this be an array of fixed size to
57    // reduce the number of allocations on the write path?
58    // This could be equal to the number of handlers + a small number.
59    // TODO: St.Ack 20150903 Sounds good to me.
60    private final LinkedList<WriteEntry> writeQueue = new LinkedList<WriteEntry>();
61  
62    public MultiVersionConcurrencyControl() {
63      this(null);
64    }
65  
66    public MultiVersionConcurrencyControl(String regionName) {
67      this.regionName = regionName;
68    }
69  
70    /**
71     * Construct and set read point. Write point is uninitialized.
72     */
73    public MultiVersionConcurrencyControl(long startPoint) {
74      this(null);
75      tryAdvanceTo(startPoint, NONE);
76    }
77  
78    /**
79     * Step the MVCC forward on to a new read/write basis.
80     * @param newStartPoint
81     */
82    public void advanceTo(long newStartPoint) {
83      while (true) {
84        long seqId = this.getWritePoint();
85        if (seqId >= newStartPoint) break;
86        if (this.tryAdvanceTo(newStartPoint, seqId)) break;
87      }
88    }
89  
90    /**
91     * Step the MVCC forward on to a new read/write basis.
92     * @param newStartPoint Point to move read and write points to.
93     * @param expected If not -1 (#NONE)
94     * @return Returns false if <code>expected</code> is not equal to the
95     * current <code>readPoint</code> or if <code>startPoint</code> is less than current
96     * <code>readPoint</code>
97     */
98    boolean tryAdvanceTo(long newStartPoint, long expected) {
99      synchronized (writeQueue) {
100       long currentRead = this.readPoint.get();
101       long currentWrite = this.writePoint.get();
102       if (currentRead != currentWrite) {
103         throw new RuntimeException("Already used this mvcc; currentRead=" + currentRead +
104           ", currentWrite=" + currentWrite + "; too late to tryAdvanceTo");
105       }
106       if (expected != NONE && expected != currentRead) {
107         return false;
108       }
109 
110       if (newStartPoint < currentRead) {
111         return false;
112       }
113 
114       readPoint.set(newStartPoint);
115       writePoint.set(newStartPoint);
116     }
117     return true;
118   }
119 
120   /**
121    * Call {@link #begin(Runnable)} with an empty {@link Runnable}.
122    */
123   public WriteEntry begin() {
124     return begin(new Runnable() {
125       @Override public void run() {
126 
127       }
128     });
129   }
130 
131   /**
132    * Start a write transaction. Create a new {@link WriteEntry} with a new write number and add it
133    * to our queue of ongoing writes. Return this WriteEntry instance. To complete the write
134    * transaction and wait for it to be visible, call {@link #completeAndWait(WriteEntry)}. If the
135    * write failed, call {@link #complete(WriteEntry)} so we can clean up AFTER removing ALL trace of
136    * the failed write transaction.
137    * <p>
138    * The {@code action} will be executed under the lock which means it can keep the same order with
139    * mvcc.
140    * @see #complete(WriteEntry)
141    * @see #completeAndWait(WriteEntry)
142    */
143   public WriteEntry begin(Runnable action) {
144     synchronized (writeQueue) {
145       long nextWriteNumber = writePoint.incrementAndGet();
146       WriteEntry e = new WriteEntry(nextWriteNumber);
147       writeQueue.add(e);
148       action.run();
149       return e;
150     }
151   }
152 
153   /**
154    * Wait until the read point catches up to the write point; i.e. wait on all outstanding mvccs
155    * to complete.
156    */
157   public void await() {
158     // Add a write and then wait on reads to catch up to it.
159     completeAndWait(begin());
160   }
161 
162   /**
163    * Complete a {@link WriteEntry} that was created by {@link #begin()} then wait until the
164    * read point catches up to our write.
165    *
166    * At the end of this call, the global read point is at least as large as the write point
167    * of the passed in WriteEntry.  Thus, the write is visible to MVCC readers.
168    */
169   public void completeAndWait(WriteEntry e) {
170     if (!complete(e)) {
171       waitForRead(e);
172     }
173   }
174 
175   /**
176    * Mark the {@link WriteEntry} as complete and advance the read point as much as possible.
177    * Call this even if the write has FAILED (AFTER backing out the write transaction
178    * changes completely) so we can clean up the outstanding transaction.
179    *
180    * How much is the read point advanced?
181    *
182    * Let S be the set of all write numbers that are completed. Set the read point to the highest
183    * numbered write of S.
184    *
185    * @param writeEntry
186    *
187    * @return true if e is visible to MVCC readers (that is, readpoint >= e.writeNumber)
188    */
189   public boolean complete(WriteEntry writeEntry) {
190     synchronized (writeQueue) {
191       writeEntry.markCompleted();
192 
193       long nextReadValue = NONE;
194       boolean ranOnce = false;
195       while (!writeQueue.isEmpty()) {
196         ranOnce = true;
197         WriteEntry queueFirst = writeQueue.getFirst();
198 
199         if (nextReadValue > 0) {
200           if (nextReadValue + 1 != queueFirst.getWriteNumber()) {
201             throw new RuntimeException("Invariant in complete violated, nextReadValue="
202                 + nextReadValue + ", writeNumber=" + queueFirst.getWriteNumber());
203           }
204         }
205 
206         if (queueFirst.isCompleted()) {
207           nextReadValue = queueFirst.getWriteNumber();
208           writeQueue.removeFirst();
209         } else {
210           break;
211         }
212       }
213 
214       if (!ranOnce) {
215         throw new RuntimeException("There is no first!");
216       }
217 
218       if (nextReadValue > 0) {
219         synchronized (readWaiters) {
220           readPoint.set(nextReadValue);
221           readWaiters.notifyAll();
222         }
223       }
224       return readPoint.get() >= writeEntry.getWriteNumber();
225     }
226   }
227 
228   /**
229    * Wait for the global readPoint to advance up to the passed in write entry number.
230    */
231   void waitForRead(WriteEntry e) {
232     boolean interrupted = false;
233     int count = 0;
234     synchronized (readWaiters) {
235       while (readPoint.get() < e.getWriteNumber()) {
236         if (count % 100 == 0 && count > 0) {
237           long totalWaitTillNow = READPOINT_ADVANCE_WAIT_TIME * count;
238           LOG.warn("STUCK for : " + totalWaitTillNow + " millis. " + this);
239         }
240         count++;
241         try {
242           readWaiters.wait(READPOINT_ADVANCE_WAIT_TIME);
243         } catch (InterruptedException ie) {
244           // We were interrupted... finish the loop -- i.e. cleanup --and then
245           // on our way out, reset the interrupt flag.
246           interrupted = true;
247         }
248       }
249     }
250     if (interrupted) {
251       Thread.currentThread().interrupt();
252     }
253   }
254 
255   @Override
256   public String toString() {
257     ToStringHelper helper = Objects.toStringHelper(this).add("readPoint", readPoint)
258         .add("writePoint", writePoint);
259     if (this.regionName != null) {
260       helper.add("regionName", this.regionName);
261     }
262     return helper.toString();
263   }
264 
265   public long getReadPoint() {
266     return readPoint.get();
267   }
268 
269   public long getWritePoint() {
270     return writePoint.get();
271   }
272 
273   /**
274    * Write number and whether write has completed given out at start of a write transaction.
275    * Every created WriteEntry must be completed by calling mvcc#complete or #completeAndWait.
276    */
277   @InterfaceAudience.Private
278   public static class WriteEntry {
279     private final long writeNumber;
280     private boolean completed = false;
281 
282     WriteEntry(long writeNumber) {
283       this.writeNumber = writeNumber;
284     }
285 
286     void markCompleted() {
287       this.completed = true;
288     }
289 
290     boolean isCompleted() {
291       return this.completed;
292     }
293 
294     public long getWriteNumber() {
295       return this.writeNumber;
296     }
297 
298     @Override
299     public String toString() {
300       return this.writeNumber + ", " + this.completed;
301     }
302   }
303 
304   public static final long FIXED_SIZE = ClassSize.align(
305       ClassSize.OBJECT +
306       2 * Bytes.SIZEOF_LONG +
307       2 * ClassSize.REFERENCE);
308 }