View Javadoc

1   /*
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
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    Class that does enqueueing/dequeuing of wal at one place so that we can update the metrics
34    just at one place.
35   */
36  @InterfaceAudience.Private
37  @InterfaceStability.Evolving
38  public class ReplicationSourceLogQueue {
39    private static final Logger LOG = LoggerFactory.getLogger(ReplicationSource.class);
40    // Queues of logs to process, entry in format of walGroupId->queue,
41    // each presents a queue for one wal group
42    private Map<String, PriorityBlockingQueue<Path>> queues = new ConcurrentHashMap<>();
43    private MetricsSource metrics;
44    private Configuration conf;
45    // per group queue size, keep no more than this number of logs in each wal group
46    private int queueSizePerGroup;
47    // WARN threshold for the number of queued logs, defaults to 2
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     * Enqueue the wal
59     * @param wal wal to be enqueued
60     * @param walGroupId Key for the wal in @queues map
61     * @return boolean whether this is the first time we are seeing this walGroupId.
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        // make sure that we do not use an empty queue when setting up a ReplicationSource, otherwise
70        // the shipper may quit immediately
71        queue.put(wal);
72        queues.put(walGroupId, queue);
73      } else {
74        exists = true;
75        queue.put(wal);
76      }
77      // Increment size of logQueue
78      this.metrics.incrSizeOfLogQueue();
79      // Compute oldest wal age
80      this.metrics.setOldestWalAge(getOldestWalAge());
81      // This will wal a warning for each new wal that gets created above the warn threshold
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     * Get the queue size for the given walGroupId.
92     * @param walGroupId walGroupId
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    * Returns number of queues.
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    * Return queue for the given walGroupId
115    * Please don't add or remove elements from the returned queue.
116    * Use @enqueueLog and @remove methods respectively.
117    * @param walGroupId walGroupId
118    */
119   public PriorityBlockingQueue<Path> getQueue(String walGroupId) {
120     return queues.get(walGroupId);
121   }
122 
123   /**
124    * Remove head from the queue corresponding to given walGroupId.
125    * @param walGroupId walGroupId
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     // Decrease size logQueue.
134     this.metrics.decrSizeOfLogQueue();
135     // Re-compute age of oldest wal metric.
136     this.metrics.setOldestWalAge(getOldestWalAge());
137   }
138 
139   /**
140    * Remove all the elements from the queue corresponding to walGroupId
141    * @param walGroupId walGroupId
142    */
143   public void clear(String walGroupId) {
144     PriorityBlockingQueue<Path> queue = getQueue(walGroupId);
145     while (!queue.isEmpty()) {
146       // Need to iterate since metrics#decrSizeOfLogQueue decrements just by 1.
147       queue.remove();
148       metrics.decrSizeOfLogQueue();
149     }
150     this.metrics.setOldestWalAge(getOldestWalAge());
151   }
152 
153   /*
154     Returns the age of oldest wal.
155    */
156   long getOldestWalAge() {
157     long now = EnvironmentEdgeManager.currentTime();
158     long timestamp = getOldestWalTimestamp();
159     if (timestamp == Long.MAX_VALUE) {
160       // If there are no wals in the queue then set the oldest wal timestamp to current time
161       // so that the oldest wal age will be 0.
162       timestamp = now;
163     }
164     long age = now - timestamp;
165     return age;
166   }
167 
168   /*
169   Get the oldest wal timestamp from all the queues.
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       // Can path ever be null ?
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 }