1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.regionserver;
21
22 import java.util.LinkedList;
23 import java.util.concurrent.atomic.AtomicLong;
24
25
26
27
28
29
30
31 public class ReadWriteConsistencyControl {
32 private volatile long memstoreRead = 0;
33 private volatile long memstoreWrite = 0;
34
35 private final Object readWaiters = new Object();
36
37
38 private final LinkedList<WriteEntry> writeQueue =
39 new LinkedList<WriteEntry>();
40
41 private static final ThreadLocal<Long> perThreadReadPoint =
42 new ThreadLocal<Long>();
43
44
45
46
47
48
49 public static long getThreadReadPoint() {
50 return perThreadReadPoint.get();
51 }
52
53
54
55
56
57
58 public static void setThreadReadPoint(long readPoint) {
59 perThreadReadPoint.set(readPoint);
60 }
61
62
63
64
65
66 public static long resetThreadReadPoint(ReadWriteConsistencyControl rwcc) {
67 perThreadReadPoint.set(rwcc.memstoreReadPoint());
68 return getThreadReadPoint();
69 }
70
71
72
73
74 public static void resetThreadReadPoint() {
75 perThreadReadPoint.set(0L);
76 }
77
78 public WriteEntry beginMemstoreInsert() {
79 synchronized (writeQueue) {
80 long nextWriteNumber = ++memstoreWrite;
81 WriteEntry e = new WriteEntry(nextWriteNumber);
82 writeQueue.add(e);
83 return e;
84 }
85 }
86
87 public void completeMemstoreInsert(WriteEntry e) {
88 synchronized (writeQueue) {
89 e.markCompleted();
90
91 long nextReadValue = -1;
92 boolean ranOnce=false;
93 while (!writeQueue.isEmpty()) {
94 ranOnce=true;
95 WriteEntry queueFirst = writeQueue.getFirst();
96
97 if (nextReadValue > 0) {
98 if (nextReadValue+1 != queueFirst.getWriteNumber()) {
99 throw new RuntimeException("invariant in completeMemstoreInsert violated, prev: "
100 + nextReadValue + " next: " + queueFirst.getWriteNumber());
101 }
102 }
103
104 if (queueFirst.isCompleted()) {
105 nextReadValue = queueFirst.getWriteNumber();
106 writeQueue.removeFirst();
107 } else {
108 break;
109 }
110 }
111
112 if (!ranOnce) {
113 throw new RuntimeException("never was a first");
114 }
115
116 if (nextReadValue > 0) {
117 synchronized (readWaiters) {
118 memstoreRead = nextReadValue;
119 readWaiters.notifyAll();
120 }
121
122 }
123 }
124
125 boolean interrupted = false;
126 synchronized (readWaiters) {
127 while (memstoreRead < e.getWriteNumber()) {
128 try {
129 readWaiters.wait(0);
130 } catch (InterruptedException ie) {
131
132
133 interrupted = true;
134 }
135 }
136 }
137 if (interrupted) Thread.currentThread().interrupt();
138 }
139
140 public long memstoreReadPoint() {
141 return memstoreRead;
142 }
143
144
145 public static class WriteEntry {
146 private long writeNumber;
147 private boolean completed = false;
148 WriteEntry(long writeNumber) {
149 this.writeNumber = writeNumber;
150 }
151 void markCompleted() {
152 this.completed = true;
153 }
154 boolean isCompleted() {
155 return this.completed;
156 }
157 long getWriteNumber() {
158 return this.writeNumber;
159 }
160 }
161 }