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 070 * @param procThreads the maximum number of threads to 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 078 * @param procThreads the maximum number of threads to 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) 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. 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}