1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.replication.regionserver;
19
20 import java.util.Map;
21 import java.util.Queue;
22 import java.util.concurrent.ConcurrentHashMap;
23 import java.util.concurrent.PriorityBlockingQueue;
24 import org.apache.hadoop.conf.Configuration;
25 import org.apache.hadoop.fs.Path;
26 import org.apache.hadoop.hbase.classification.InterfaceAudience;
27 import org.apache.hadoop.hbase.classification.InterfaceStability;
28 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
29 import org.slf4j.Logger;
30 import org.slf4j.LoggerFactory;
31
32
33
34
35
36 @InterfaceAudience.Private
37 @InterfaceStability.Evolving
38 public class ReplicationSourceLogQueue {
39 private static final Logger LOG = LoggerFactory.getLogger(ReplicationSource.class);
40
41
42 private Map<String, PriorityBlockingQueue<Path>> queues = new ConcurrentHashMap<>();
43 private MetricsSource metrics;
44 private Configuration conf;
45
46 private int queueSizePerGroup;
47
48 private int logQueueWarnThreshold;
49
50 public ReplicationSourceLogQueue(Configuration conf, MetricsSource metrics) {
51 this.conf = conf;
52 this.metrics = metrics;
53 this.queueSizePerGroup = this.conf.getInt("hbase.regionserver.maxlogs", 32);
54 this.logQueueWarnThreshold = this.conf.getInt("replication.source.log.queue.warn", 2);
55 }
56
57
58
59
60
61
62
63 public boolean enqueueLog(Path wal, String walGroupId) {
64 boolean exists = false;
65 PriorityBlockingQueue<Path> queue = queues.get(walGroupId);
66 if (queue == null) {
67 queue = new PriorityBlockingQueue<>(queueSizePerGroup,
68 new ReplicationSource.LogsComparator());
69
70
71 queue.put(wal);
72 queues.put(walGroupId, queue);
73 } else {
74 exists = true;
75 queue.put(wal);
76 }
77
78 this.metrics.incrSizeOfLogQueue();
79
80 this.metrics.setOldestWalAge(getOldestWalAge());
81
82 int queueSize = queue.size();
83 if (queueSize > this.logQueueWarnThreshold) {
84 LOG.warn("WAL group " + walGroupId + " queue size: " + queueSize
85 + " exceeds value of replication.source.log.queue.warn: " + logQueueWarnThreshold);
86 }
87 return exists;
88 }
89
90
91
92
93
94 public int getQueueSize(String walGroupId) {
95 Queue queue = queues.get(walGroupId);
96 if (queue == null) {
97 return 0;
98 }
99 return queue.size();
100 }
101
102
103
104
105 public int getNumQueues() {
106 return queues.size();
107 }
108
109 public Map<String, PriorityBlockingQueue<Path>> getQueues() {
110 return queues;
111 }
112
113
114
115
116
117
118
119 public PriorityBlockingQueue<Path> getQueue(String walGroupId) {
120 return queues.get(walGroupId);
121 }
122
123
124
125
126
127 public void remove(String walGroupId) {
128 PriorityBlockingQueue<Path> queue = getQueue(walGroupId);
129 if (queue == null || queue.isEmpty()) {
130 return;
131 }
132 queue.remove();
133
134 this.metrics.decrSizeOfLogQueue();
135
136 this.metrics.setOldestWalAge(getOldestWalAge());
137 }
138
139
140
141
142
143 public void clear(String walGroupId) {
144 PriorityBlockingQueue<Path> queue = getQueue(walGroupId);
145 while (!queue.isEmpty()) {
146
147 queue.remove();
148 metrics.decrSizeOfLogQueue();
149 }
150 this.metrics.setOldestWalAge(getOldestWalAge());
151 }
152
153
154
155
156 long getOldestWalAge() {
157 long now = EnvironmentEdgeManager.currentTime();
158 long timestamp = getOldestWalTimestamp();
159 if (timestamp == Long.MAX_VALUE) {
160
161
162 timestamp = now;
163 }
164 long age = now - timestamp;
165 return age;
166 }
167
168
169
170
171 private long getOldestWalTimestamp() {
172 long oldestWalTimestamp = Long.MAX_VALUE;
173 for (Map.Entry<String, PriorityBlockingQueue<Path>> entry : queues.entrySet()) {
174 PriorityBlockingQueue<Path> queue = entry.getValue();
175 Path path = queue.peek();
176
177 if (path != null) {
178 oldestWalTimestamp = Math.min(oldestWalTimestamp,
179 ReplicationSource.LogsComparator.getTS(path));
180 }
181 }
182 return oldestWalTimestamp;
183 }
184
185 public MetricsSource getMetrics() {
186 return metrics;
187 }
188 }