001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.procedure2;
019
020import java.io.IOException;
021import java.lang.Thread.UncaughtExceptionHandler;
022import java.util.HashSet;
023import java.util.List;
024import java.util.Optional;
025import java.util.Set;
026import java.util.concurrent.ConcurrentHashMap;
027import java.util.concurrent.DelayQueue;
028import java.util.concurrent.ThreadPoolExecutor;
029import java.util.concurrent.TimeUnit;
030import java.util.concurrent.atomic.AtomicBoolean;
031import org.apache.hadoop.conf.Configuration;
032import org.apache.hadoop.hbase.procedure2.util.DelayedUtil;
033import org.apache.hadoop.hbase.procedure2.util.DelayedUtil.DelayedContainerWithTimestamp;
034import org.apache.hadoop.hbase.procedure2.util.DelayedUtil.DelayedWithTimeout;
035import org.apache.hadoop.hbase.procedure2.util.StringUtils;
036import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
037import org.apache.hadoop.hbase.util.Threads;
038import org.apache.yetus.audience.InterfaceAudience;
039import org.slf4j.Logger;
040import org.slf4j.LoggerFactory;
041
042import org.apache.hbase.thirdparty.com.google.common.collect.ArrayListMultimap;
043import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
044
045/**
046 * A procedure dispatcher that aggregates and sends after elapsed time or after we hit count
047 * threshold. Creates its own threadpool to run RPCs with timeout.
048 * <ul>
049 * <li>Each server queue has a dispatch buffer</li>
050 * <li>Once the dispatch buffer reaches a threshold-size/time we send
051 * <li>
052 * </ul>
053 * <p>
054 * Call {@link #start()} and then {@link #submitTask(Runnable)}. When done, call {@link #stop()}.
055 */
056@InterfaceAudience.Private
057public abstract class RemoteProcedureDispatcher<TEnv, TRemote extends Comparable<TRemote>> {
058  private static final Logger LOG = LoggerFactory.getLogger(RemoteProcedureDispatcher.class);
059
060  public static final String THREAD_POOL_SIZE_CONF_KEY =
061    "hbase.procedure.remote.dispatcher.threadpool.size";
062  private static final int DEFAULT_THREAD_POOL_SIZE = 128;
063
064  public static final String DISPATCH_DELAY_CONF_KEY =
065    "hbase.procedure.remote.dispatcher.delay.msec";
066  private static final int DEFAULT_DISPATCH_DELAY = 150;
067
068  public static final String DISPATCH_MAX_QUEUE_SIZE_CONF_KEY =
069    "hbase.procedure.remote.dispatcher.max.queue.size";
070  private static final int DEFAULT_MAX_QUEUE_SIZE = 32;
071
072  private final AtomicBoolean running = new AtomicBoolean(false);
073  private final ConcurrentHashMap<TRemote, BufferNode> nodeMap =
074    new ConcurrentHashMap<TRemote, BufferNode>();
075
076  private final int operationDelay;
077  private final int queueMaxSize;
078  private final int corePoolSize;
079
080  private TimeoutExecutorThread timeoutExecutor;
081  private ThreadPoolExecutor threadPool;
082
083  protected RemoteProcedureDispatcher(Configuration conf) {
084    this.corePoolSize = conf.getInt(THREAD_POOL_SIZE_CONF_KEY, DEFAULT_THREAD_POOL_SIZE);
085    this.operationDelay = conf.getInt(DISPATCH_DELAY_CONF_KEY, DEFAULT_DISPATCH_DELAY);
086    this.queueMaxSize = conf.getInt(DISPATCH_MAX_QUEUE_SIZE_CONF_KEY, DEFAULT_MAX_QUEUE_SIZE);
087  }
088
089  public boolean start() {
090    if (running.getAndSet(true)) {
091      LOG.warn("Already running");
092      return false;
093    }
094
095    LOG.info("Instantiated, coreThreads={} (allowCoreThreadTimeOut=true), queueMaxSize={}, "
096      + "operationDelay={}", this.corePoolSize, this.queueMaxSize, this.operationDelay);
097
098    // Create the timeout executor
099    timeoutExecutor = new TimeoutExecutorThread();
100    timeoutExecutor.start();
101
102    // Create the thread pool that will execute RPCs
103    threadPool = Threads.getBoundedCachedThreadPool(corePoolSize, 60L, TimeUnit.SECONDS,
104      new ThreadFactoryBuilder().setNameFormat(this.getClass().getSimpleName() + "-pool-%d")
105        .setDaemon(true).setUncaughtExceptionHandler(getUncaughtExceptionHandler()).build());
106    return true;
107  }
108
109  protected void setTimeoutExecutorUncaughtExceptionHandler(UncaughtExceptionHandler eh) {
110    timeoutExecutor.setUncaughtExceptionHandler(eh);
111  }
112
113  public boolean stop() {
114    if (!running.getAndSet(false)) {
115      return false;
116    }
117
118    LOG.info("Stopping procedure remote dispatcher");
119
120    // send stop signals
121    timeoutExecutor.sendStopSignal();
122    threadPool.shutdownNow();
123    return true;
124  }
125
126  public void join() {
127    assert !running.get() : "expected not running";
128
129    // wait the timeout executor
130    timeoutExecutor.awaitTermination();
131    timeoutExecutor = null;
132
133    // wait for the thread pool to terminate
134    threadPool.shutdownNow();
135    try {
136      while (!threadPool.awaitTermination(60, TimeUnit.SECONDS)) {
137        LOG.warn("Waiting for thread-pool to terminate");
138      }
139    } catch (InterruptedException e) {
140      LOG.warn("Interrupted while waiting for thread-pool termination", e);
141    }
142  }
143
144  protected abstract UncaughtExceptionHandler getUncaughtExceptionHandler();
145
146  // ============================================================================================
147  // Node Helpers
148  // ============================================================================================
149  /**
150   * Add a node that will be able to execute remote procedures
151   * @param key the node identifier
152   */
153  public void addNode(final TRemote key) {
154    assert key != null : "Tried to add a node with a null key";
155    final BufferNode newNode = new BufferNode(key);
156    nodeMap.putIfAbsent(key, newNode);
157  }
158
159  /**
160   * Add a remote rpc.
161   * @param key the node identifier
162   */
163  public void addOperationToNode(final TRemote key, RemoteProcedure rp)
164    throws NullTargetServerDispatchException, NoServerDispatchException, NoNodeDispatchException {
165    if (key == null) {
166      throw new NullTargetServerDispatchException(rp.toString());
167    }
168    BufferNode node = nodeMap.get(key);
169    if (node == null) {
170      // If null here, it means node has been removed because it crashed. This happens when server
171      // is expired in ServerManager. ServerCrashProcedure may or may not have run.
172      throw new NoServerDispatchException(key.toString() + "; " + rp.toString());
173    }
174    node.add(rp);
175    // Check our node still in the map; could have been removed by #removeNode.
176    if (!nodeMap.containsValue(node)) {
177      throw new NoNodeDispatchException(key.toString() + "; " + rp.toString());
178    }
179  }
180
181  public void removeCompletedOperation(final TRemote key, RemoteProcedure rp) {
182    BufferNode node = nodeMap.get(key);
183    if (node == null) {
184      LOG.warn("since no node for this key {}, we can't removed the finished remote procedure",
185        key);
186      return;
187    }
188    node.operationCompleted(rp);
189  }
190
191  /**
192   * Remove a remote node
193   * @param key the node identifier
194   */
195  public boolean removeNode(final TRemote key) {
196    final BufferNode node = nodeMap.remove(key);
197    if (node == null) {
198      return false;
199    }
200
201    node.abortOperationsInQueue();
202    return true;
203  }
204
205  // ============================================================================================
206  // Task Helpers
207  // ============================================================================================
208  protected final void submitTask(Runnable task) {
209    threadPool.execute(task);
210  }
211
212  protected final void submitTask(Runnable task, long delay, TimeUnit unit) {
213    timeoutExecutor.add(new DelayedTask(task, delay, unit));
214  }
215
216  protected abstract void remoteDispatch(TRemote key, Set<RemoteProcedure> operations);
217
218  protected abstract void abortPendingOperations(TRemote key, Set<RemoteProcedure> operations);
219
220  /**
221   * Data structure with reference to remote operation.
222   */
223  public static abstract class RemoteOperation {
224    private final RemoteProcedure remoteProcedure;
225
226    protected RemoteOperation(final RemoteProcedure remoteProcedure) {
227      this.remoteProcedure = remoteProcedure;
228    }
229
230    public RemoteProcedure getRemoteProcedure() {
231      return remoteProcedure;
232    }
233  }
234
235  /**
236   * Remote procedure reference.
237   */
238  public interface RemoteProcedure<TEnv, TRemote> {
239    /**
240     * For building the remote operation. May be empty if no need to send remote call. Usually, this
241     * means the RemoteProcedure has been finished already. This is possible, as we may have already
242     * sent the procedure to RS but then the rpc connection is broken so the executeProcedures call
243     * fails, but the RS does receive the procedure and execute it and then report back, before we
244     * retry again.
245     */
246    Optional<RemoteOperation> remoteCallBuild(TEnv env, TRemote remote);
247
248    /**
249     * Called when the executeProcedure call is failed.
250     */
251    void remoteCallFailed(TEnv env, TRemote remote, IOException exception);
252
253    /**
254     * Called when RS tells the remote procedure is succeeded through the
255     * {@code reportProcedureDone} method.
256     */
257    void remoteOperationCompleted(TEnv env);
258
259    /**
260     * Called when RS tells the remote procedure is failed through the {@code reportProcedureDone}
261     * method.
262     */
263    void remoteOperationFailed(TEnv env, RemoteProcedureException error);
264
265    /**
266     * Whether store this remote procedure in dispatched queue only OpenRegionProcedure and
267     * CloseRegionProcedure return false since they are not fully controlled by dispatcher
268     */
269    default boolean storeInDispatchedQueue() {
270      return true;
271    }
272  }
273
274  /**
275   * Account of what procedures are running on remote node.
276   */
277  public interface RemoteNode<TEnv, TRemote> {
278    TRemote getKey();
279
280    void add(RemoteProcedure<TEnv, TRemote> operation);
281
282    void dispatch();
283  }
284
285  protected ArrayListMultimap<Class<?>, RemoteOperation> buildAndGroupRequestByType(final TEnv env,
286    final TRemote remote, final Set<RemoteProcedure> remoteProcedures) {
287    final ArrayListMultimap<Class<?>, RemoteOperation> requestByType = ArrayListMultimap.create();
288    for (RemoteProcedure proc : remoteProcedures) {
289      Optional<RemoteOperation> operation = proc.remoteCallBuild(env, remote);
290      operation.ifPresent(op -> requestByType.put(op.getClass(), op));
291    }
292    return requestByType;
293  }
294
295  protected <T extends RemoteOperation> List<T> fetchType(
296    final ArrayListMultimap<Class<?>, RemoteOperation> requestByType, final Class<T> type) {
297    return (List<T>) requestByType.removeAll(type);
298  }
299
300  // ============================================================================================
301  // Timeout Helpers
302  // ============================================================================================
303  private final class TimeoutExecutorThread extends Thread {
304    private final DelayQueue<DelayedWithTimeout> queue = new DelayQueue<DelayedWithTimeout>();
305
306    public TimeoutExecutorThread() {
307      super("ProcedureDispatcherTimeoutThread");
308    }
309
310    @Override
311    public void run() {
312      while (running.get()) {
313        final DelayedWithTimeout task =
314          DelayedUtil.takeWithoutInterrupt(queue, 20, TimeUnit.SECONDS);
315        if (task == null || task == DelayedUtil.DELAYED_POISON) {
316          if (task == null && queue.size() > 0) {
317            LOG.error("DelayQueue for RemoteProcedureDispatcher is not empty when timed waiting"
318              + " elapsed. If this is repeated consistently, it means no element is getting expired"
319              + " from the queue and it might freeze the system. Queue: {}", queue);
320          }
321          // the executor may be shutting down, and the task is just the shutdown request
322          continue;
323        }
324        if (task instanceof DelayedTask) {
325          threadPool.execute(((DelayedTask) task).getObject());
326        } else {
327          ((BufferNode) task).dispatch();
328        }
329      }
330    }
331
332    public void add(final DelayedWithTimeout delayed) {
333      queue.add(delayed);
334    }
335
336    public void remove(final DelayedWithTimeout delayed) {
337      queue.remove(delayed);
338    }
339
340    public void sendStopSignal() {
341      queue.add(DelayedUtil.DELAYED_POISON);
342    }
343
344    public void awaitTermination() {
345      try {
346        final long startTime = EnvironmentEdgeManager.currentTime();
347        for (int i = 0; isAlive(); ++i) {
348          sendStopSignal();
349          join(250);
350          if (i > 0 && (i % 8) == 0) {
351            LOG.warn("Waiting termination of thread " + getName() + ", "
352              + StringUtils.humanTimeDiff(EnvironmentEdgeManager.currentTime() - startTime));
353          }
354        }
355      } catch (InterruptedException e) {
356        LOG.warn(getName() + " join wait got interrupted", e);
357      }
358    }
359  }
360
361  // ============================================================================================
362  // Internals Helpers
363  // ============================================================================================
364
365  /**
366   * Node that contains a set of RemoteProcedures
367   */
368  protected final class BufferNode extends DelayedContainerWithTimestamp<TRemote>
369    implements RemoteNode<TEnv, TRemote> {
370    private Set<RemoteProcedure> operations;
371    private final Set<RemoteProcedure> dispatchedOperations = new HashSet<>();
372
373    protected BufferNode(final TRemote key) {
374      super(key, 0);
375    }
376
377    @Override
378    public TRemote getKey() {
379      return getObject();
380    }
381
382    @Override
383    public synchronized void add(final RemoteProcedure operation) {
384      if (this.operations == null) {
385        this.operations = new HashSet<>();
386        setTimeout(EnvironmentEdgeManager.currentTime() + operationDelay);
387        timeoutExecutor.add(this);
388      }
389      this.operations.add(operation);
390      if (this.operations.size() > queueMaxSize) {
391        timeoutExecutor.remove(this);
392        dispatch();
393      }
394    }
395
396    @Override
397    public synchronized void dispatch() {
398      if (operations != null) {
399        remoteDispatch(getKey(), operations);
400        operations.stream().filter(operation -> operation.storeInDispatchedQueue())
401          .forEach(operation -> dispatchedOperations.add(operation));
402        this.operations = null;
403      }
404    }
405
406    public synchronized void abortOperationsInQueue() {
407      if (operations != null) {
408        abortPendingOperations(getKey(), operations);
409        this.operations = null;
410      }
411      abortPendingOperations(getKey(), dispatchedOperations);
412      this.dispatchedOperations.clear();
413    }
414
415    public synchronized void operationCompleted(final RemoteProcedure remoteProcedure) {
416      this.dispatchedOperations.remove(remoteProcedure);
417    }
418
419    @Override
420    public String toString() {
421      return super.toString() + ", operations=" + this.operations;
422    }
423  }
424
425  /**
426   * Delayed object that holds a FutureTask.
427   * <p/>
428   * used to submit something later to the thread-pool.
429   */
430  private static final class DelayedTask extends DelayedContainerWithTimestamp<Runnable> {
431    public DelayedTask(Runnable task, long delay, TimeUnit unit) {
432      super(task, EnvironmentEdgeManager.currentTime() + unit.toMillis(delay));
433    }
434  };
435}