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.procedure;
019
020import java.io.Closeable;
021import java.io.IOException;
022import java.util.concurrent.ConcurrentMap;
023import java.util.concurrent.ExecutorService;
024import java.util.concurrent.RejectedExecutionException;
025import java.util.concurrent.SynchronousQueue;
026import java.util.concurrent.ThreadPoolExecutor;
027import java.util.concurrent.TimeUnit;
028import org.apache.hadoop.hbase.errorhandling.ForeignException;
029import org.apache.hadoop.hbase.util.Threads;
030import org.apache.yetus.audience.InterfaceAudience;
031import org.slf4j.Logger;
032import org.slf4j.LoggerFactory;
033
034import org.apache.hbase.thirdparty.com.google.common.collect.MapMaker;
035import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
036
037/**
038 * Process to kick off and manage a running {@link Subprocedure} on a member. This is the
039 * specialized part of a {@link Procedure} that actually does procedure type-specific work and
040 * reports back to the coordinator as it completes each phase.
041 */
042@InterfaceAudience.Private
043public class ProcedureMember implements Closeable {
044  private static final Logger LOG = LoggerFactory.getLogger(ProcedureMember.class);
045
046  final static long KEEP_ALIVE_MILLIS_DEFAULT = 5000;
047
048  private final SubprocedureFactory builder;
049  private final ProcedureMemberRpcs rpcs;
050
051  private final ConcurrentMap<String, Subprocedure> subprocs =
052    new MapMaker().concurrencyLevel(4).weakValues().makeMap();
053  private final ExecutorService pool;
054
055  /**
056   * Instantiate a new ProcedureMember. This is a slave that executes subprocedures.
057   * @param rpcs    controller used to send notifications to the procedure coordinator
058   * @param pool    thread pool to submit subprocedures
059   * @param factory class that creates instances of a subprocedure.
060   */
061  public ProcedureMember(ProcedureMemberRpcs rpcs, ThreadPoolExecutor pool,
062    SubprocedureFactory factory) {
063    this.pool = pool;
064    this.rpcs = rpcs;
065    this.builder = factory;
066  }
067
068  /**
069   * Default thread pool for the procedure n * @param procThreads the maximum number of threads to
070   * allow in the pool
071   */
072  public static ThreadPoolExecutor defaultPool(String memberName, int procThreads) {
073    return defaultPool(memberName, procThreads, KEEP_ALIVE_MILLIS_DEFAULT);
074  }
075
076  /**
077   * Default thread pool for the procedure n * @param procThreads the maximum number of threads to
078   * allow in the pool
079   * @param keepAliveMillis the maximum time (ms) that excess idle threads will wait for new tasks
080   */
081  public static ThreadPoolExecutor defaultPool(String memberName, int procThreads,
082    long keepAliveMillis) {
083    return new ThreadPoolExecutor(1, procThreads, keepAliveMillis, TimeUnit.MILLISECONDS,
084      new SynchronousQueue<>(),
085      new ThreadFactoryBuilder().setNameFormat("member: '" + memberName + "' subprocedure-pool-%d")
086        .setDaemon(true).setUncaughtExceptionHandler(Threads.LOGGING_EXCEPTION_HANDLER).build());
087  }
088
089  /**
090   * Package exposed. Not for public use.
091   * @return reference to the Procedure member's rpcs object
092   */
093  ProcedureMemberRpcs getRpcs() {
094    return rpcs;
095  }
096
097  /**
098   * This is separated from execution so that we can detect and handle the case where the
099   * subprocedure is invalid and inactionable due to bad info (like DISABLED snapshot type being
100   * sent here) nnn
101   */
102  public Subprocedure createSubprocedure(String opName, byte[] data) {
103    return builder.buildSubprocedure(opName, data);
104  }
105
106  /**
107   * Submit an subprocedure for execution. This starts the local acquire phase.
108   * @param subproc the subprocedure to execute.
109   * @return <tt>true</tt> if the subprocedure was started correctly, <tt>false</tt> if it could not
110   *         be started. In the latter case, the subprocedure holds a reference to the exception
111   *         that caused the failure.
112   */
113  public boolean submitSubprocedure(Subprocedure subproc) {
114    // if the submitted subprocedure was null, bail.
115    if (subproc == null) {
116      LOG.warn("Submitted null subprocedure, nothing to run here.");
117      return false;
118    }
119
120    String procName = subproc.getName();
121    if (procName == null || procName.length() == 0) {
122      LOG.error("Subproc name cannot be null or the empty string");
123      return false;
124    }
125
126    // make sure we aren't already running an subprocedure of that name
127    Subprocedure rsub = subprocs.get(procName);
128    if (rsub != null) {
129      if (!rsub.isComplete()) {
130        LOG.error("Subproc '" + procName + "' is already running. Bailing out");
131        return false;
132      }
133      LOG.warn("A completed old subproc " + procName + " is still present, removing");
134      if (!subprocs.remove(procName, rsub)) {
135        LOG.error("Another thread has replaced existing subproc '" + procName + "'. Bailing out");
136        return false;
137      }
138    }
139
140    LOG.debug("Submitting new Subprocedure:" + procName);
141
142    // kick off the subprocedure
143    try {
144      if (subprocs.putIfAbsent(procName, subproc) == null) {
145        this.pool.submit(subproc);
146        return true;
147      } else {
148        LOG.error("Another thread has submitted subproc '" + procName + "'. Bailing out");
149        return false;
150      }
151    } catch (RejectedExecutionException e) {
152      subprocs.remove(procName, subproc);
153
154      // the thread pool is full and we can't run the subprocedure
155      String msg = "Subprocedure pool is full!";
156      subproc.cancel(msg, e.getCause());
157    }
158
159    LOG.error("Failed to start subprocedure '" + procName + "'");
160    return false;
161  }
162
163  /**
164   * Notification that procedure coordinator has reached the global barrier
165   * @param procName name of the subprocedure that should start running the in-barrier phase
166   */
167  public void receivedReachedGlobalBarrier(String procName) {
168    Subprocedure subproc = subprocs.get(procName);
169    if (subproc == null) {
170      LOG.warn("Unexpected reached globa barrier message for Sub-Procedure '" + procName + "'");
171      return;
172    }
173    if (LOG.isTraceEnabled()) {
174      LOG.trace("reached global barrier message for Sub-Procedure '" + procName + "'");
175    }
176    subproc.receiveReachedGlobalBarrier();
177  }
178
179  /**
180   * Best effort attempt to close the threadpool via Thread.interrupt.
181   */
182  @Override
183  public void close() throws IOException {
184    // have to use shutdown now to break any latch waiting
185    pool.shutdownNow();
186  }
187
188  /**
189   * Shutdown the threadpool, and wait for upto timeoutMs millis before bailing
190   * @param timeoutMs timeout limit in millis
191   * @return true if successfully, false if bailed due to timeout. n
192   */
193  boolean closeAndWait(long timeoutMs) throws InterruptedException {
194    pool.shutdown();
195    return pool.awaitTermination(timeoutMs, TimeUnit.MILLISECONDS);
196  }
197
198  /**
199   * The connection to the rest of the procedure group (member and coordinator) has been
200   * broken/lost/failed. This should fail any interested subprocedure, but not attempt to notify
201   * other members since we cannot reach them anymore.
202   * @param message  description of the error
203   * @param cause    the actual cause of the failure
204   * @param procName the name of the procedure we'd cancel due to the error.
205   */
206  public void controllerConnectionFailure(final String message, final Throwable cause,
207    final String procName) {
208    LOG.error(message, cause);
209    if (procName == null) {
210      return;
211    }
212    Subprocedure toNotify = subprocs.get(procName);
213    if (toNotify != null) {
214      toNotify.cancel(message, cause);
215    }
216  }
217
218  /**
219   * Send abort to the specified procedure
220   * @param procName name of the procedure to about
221   * @param ee       exception information about the abort
222   */
223  public void receiveAbortProcedure(String procName, ForeignException ee) {
224    LOG.debug("Request received to abort procedure " + procName, ee);
225    // if we know about the procedure, notify it
226    Subprocedure sub = subprocs.get(procName);
227    if (sub == null) {
228      LOG.info(
229        "Received abort on procedure with no local subprocedure " + procName + ", ignoring it.",
230        ee);
231      return; // Procedure has already completed
232    }
233    String msg = "Propagating foreign exception to subprocedure " + sub.getName();
234    LOG.error(msg, ee);
235    sub.cancel(msg, ee);
236  }
237}