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.master.procedure; 019 020import java.io.IOException; 021import java.lang.Thread.UncaughtExceptionHandler; 022import java.util.List; 023import java.util.Set; 024import java.util.concurrent.TimeUnit; 025import javax.security.sasl.SaslException; 026import org.apache.hadoop.hbase.CallQueueTooBigException; 027import org.apache.hadoop.hbase.DoNotRetryIOException; 028import org.apache.hadoop.hbase.ServerName; 029import org.apache.hadoop.hbase.client.RegionInfo; 030import org.apache.hadoop.hbase.ipc.RpcConnectionConstants; 031import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException; 032import org.apache.hadoop.hbase.master.MasterServices; 033import org.apache.hadoop.hbase.master.ServerListener; 034import org.apache.hadoop.hbase.master.ServerManager; 035import org.apache.hadoop.hbase.procedure2.ProcedureExecutor; 036import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher; 037import org.apache.hadoop.hbase.regionserver.RegionServerAbortedException; 038import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException; 039import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; 040import org.apache.hadoop.ipc.RemoteException; 041import org.apache.yetus.audience.InterfaceAudience; 042import org.slf4j.Logger; 043import org.slf4j.LoggerFactory; 044 045import org.apache.hbase.thirdparty.com.google.common.collect.ArrayListMultimap; 046import org.apache.hbase.thirdparty.com.google.protobuf.ByteString; 047import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; 048 049import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; 050import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; 051import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService; 052import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest; 053import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresRequest; 054import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresResponse; 055import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest; 056import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RemoteProcedureRequest; 057 058/** 059 * A remote procecdure dispatcher for regionservers. 060 */ 061@InterfaceAudience.Private 062public class RSProcedureDispatcher extends RemoteProcedureDispatcher<MasterProcedureEnv, ServerName> 063 implements ServerListener { 064 private static final Logger LOG = LoggerFactory.getLogger(RSProcedureDispatcher.class); 065 066 public static final String RS_RPC_STARTUP_WAIT_TIME_CONF_KEY = 067 "hbase.regionserver.rpc.startup.waittime"; 068 private static final int DEFAULT_RS_RPC_STARTUP_WAIT_TIME = 60000; 069 070 protected final MasterServices master; 071 private final long rsStartupWaitTime; 072 private MasterProcedureEnv procedureEnv; 073 074 public RSProcedureDispatcher(final MasterServices master) { 075 super(master.getConfiguration()); 076 077 this.master = master; 078 this.rsStartupWaitTime = master.getConfiguration().getLong(RS_RPC_STARTUP_WAIT_TIME_CONF_KEY, 079 DEFAULT_RS_RPC_STARTUP_WAIT_TIME); 080 } 081 082 @Override 083 protected UncaughtExceptionHandler getUncaughtExceptionHandler() { 084 return new UncaughtExceptionHandler() { 085 086 @Override 087 public void uncaughtException(Thread t, Throwable e) { 088 LOG.error("Unexpected error caught, this may cause the procedure to hang forever", e); 089 } 090 }; 091 } 092 093 @Override 094 public boolean start() { 095 if (!super.start()) { 096 return false; 097 } 098 setTimeoutExecutorUncaughtExceptionHandler(this::abort); 099 if (master.isStopped()) { 100 LOG.debug("Stopped"); 101 return false; 102 } 103 // Around startup, if failed, some of the below may be set back to null so NPE is possible. 104 ServerManager sm = master.getServerManager(); 105 if (sm == null) { 106 LOG.debug("ServerManager is null"); 107 return false; 108 } 109 sm.registerListener(this); 110 ProcedureExecutor<MasterProcedureEnv> pe = master.getMasterProcedureExecutor(); 111 if (pe == null) { 112 LOG.debug("ProcedureExecutor is null"); 113 return false; 114 } 115 this.procedureEnv = pe.getEnvironment(); 116 if (this.procedureEnv == null) { 117 LOG.debug("ProcedureEnv is null; stopping={}", master.isStopping()); 118 return false; 119 } 120 try { 121 for (ServerName serverName : sm.getOnlineServersList()) { 122 addNode(serverName); 123 } 124 } catch (Exception e) { 125 LOG.info("Failed start", e); 126 return false; 127 } 128 return true; 129 } 130 131 private void abort(Thread t, Throwable e) { 132 LOG.error("Caught error", e); 133 if (!master.isStopped() && !master.isStopping() && !master.isAborted()) { 134 master.abort("Aborting master", e); 135 } 136 } 137 138 @Override 139 public boolean stop() { 140 if (!super.stop()) { 141 return false; 142 } 143 144 master.getServerManager().unregisterListener(this); 145 return true; 146 } 147 148 @Override 149 protected void remoteDispatch(final ServerName serverName, 150 final Set<RemoteProcedure> remoteProcedures) { 151 if (!master.getServerManager().isServerOnline(serverName)) { 152 // fail fast 153 submitTask(new DeadRSRemoteCall(serverName, remoteProcedures)); 154 } else { 155 submitTask(new ExecuteProceduresRemoteCall(serverName, remoteProcedures)); 156 } 157 } 158 159 @Override 160 protected void abortPendingOperations(final ServerName serverName, 161 final Set<RemoteProcedure> operations) { 162 // TODO: Replace with a ServerNotOnlineException() 163 final IOException e = new DoNotRetryIOException("server not online " + serverName); 164 for (RemoteProcedure proc : operations) { 165 proc.remoteCallFailed(procedureEnv, serverName, e); 166 } 167 } 168 169 @Override 170 public void serverAdded(final ServerName serverName) { 171 addNode(serverName); 172 } 173 174 @Override 175 public void serverRemoved(final ServerName serverName) { 176 removeNode(serverName); 177 } 178 179 private interface RemoteProcedureResolver { 180 void dispatchOpenRequests(MasterProcedureEnv env, List<RegionOpenOperation> operations); 181 182 void dispatchCloseRequests(MasterProcedureEnv env, List<RegionCloseOperation> operations); 183 184 void dispatchServerOperations(MasterProcedureEnv env, List<ServerOperation> operations); 185 } 186 187 /** 188 * Fetches {@link org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteOperation}s 189 * from the given {@code remoteProcedures} and groups them by class of the returned operation. 190 * Then {@code resolver} is used to dispatch {@link RegionOpenOperation}s and 191 * {@link RegionCloseOperation}s. 192 * @param serverName RegionServer to which the remote operations are sent 193 * @param operations Remote procedures which are dispatched to the given server 194 * @param resolver Used to dispatch remote procedures to given server. 195 */ 196 public void splitAndResolveOperation(ServerName serverName, Set<RemoteProcedure> operations, 197 RemoteProcedureResolver resolver) { 198 MasterProcedureEnv env = master.getMasterProcedureExecutor().getEnvironment(); 199 ArrayListMultimap<Class<?>, RemoteOperation> reqsByType = 200 buildAndGroupRequestByType(env, serverName, operations); 201 202 List<RegionOpenOperation> openOps = fetchType(reqsByType, RegionOpenOperation.class); 203 if (!openOps.isEmpty()) { 204 resolver.dispatchOpenRequests(env, openOps); 205 } 206 207 List<RegionCloseOperation> closeOps = fetchType(reqsByType, RegionCloseOperation.class); 208 if (!closeOps.isEmpty()) { 209 resolver.dispatchCloseRequests(env, closeOps); 210 } 211 212 List<ServerOperation> refreshOps = fetchType(reqsByType, ServerOperation.class); 213 if (!refreshOps.isEmpty()) { 214 resolver.dispatchServerOperations(env, refreshOps); 215 } 216 217 if (!reqsByType.isEmpty()) { 218 LOG.warn("unknown request type in the queue: " + reqsByType); 219 } 220 } 221 222 private class DeadRSRemoteCall extends ExecuteProceduresRemoteCall { 223 224 public DeadRSRemoteCall(ServerName serverName, Set<RemoteProcedure> remoteProcedures) { 225 super(serverName, remoteProcedures); 226 } 227 228 @Override 229 public void run() { 230 remoteCallFailed(procedureEnv, 231 new RegionServerStoppedException("Server " + getServerName() + " is not online")); 232 } 233 } 234 235 // ========================================================================== 236 // Compatibility calls 237 // ========================================================================== 238 protected class ExecuteProceduresRemoteCall implements RemoteProcedureResolver, Runnable { 239 240 private final ServerName serverName; 241 242 private final Set<RemoteProcedure> remoteProcedures; 243 244 private int numberOfAttemptsSoFar = 0; 245 private long maxWaitTime = -1; 246 247 private final long rsRpcRetryInterval; 248 private static final String RS_RPC_RETRY_INTERVAL_CONF_KEY = 249 "hbase.regionserver.rpc.retry.interval"; 250 private static final int DEFAULT_RS_RPC_RETRY_INTERVAL = 100; 251 252 private ExecuteProceduresRequest.Builder request = null; 253 254 public ExecuteProceduresRemoteCall(final ServerName serverName, 255 final Set<RemoteProcedure> remoteProcedures) { 256 this.serverName = serverName; 257 this.remoteProcedures = remoteProcedures; 258 this.rsRpcRetryInterval = master.getConfiguration().getLong(RS_RPC_RETRY_INTERVAL_CONF_KEY, 259 DEFAULT_RS_RPC_RETRY_INTERVAL); 260 } 261 262 private AdminService.BlockingInterface getRsAdmin() throws IOException { 263 final AdminService.BlockingInterface admin = master.getServerManager().getRsAdmin(serverName); 264 if (admin == null) { 265 throw new IOException("Attempting to send OPEN RPC to server " + getServerName() 266 + " failed because no RPC connection found to this server"); 267 } 268 return admin; 269 } 270 271 protected final ServerName getServerName() { 272 return serverName; 273 } 274 275 private boolean scheduleForRetry(IOException e) { 276 LOG.debug("Request to {} failed, try={}", serverName, numberOfAttemptsSoFar, e); 277 // Should we wait a little before retrying? If the server is starting it's yes. 278 if (e instanceof ServerNotRunningYetException) { 279 long remainingTime = getMaxWaitTime() - EnvironmentEdgeManager.currentTime(); 280 if (remainingTime > 0) { 281 LOG.warn("Waiting a little before retrying {}, try={}, can wait up to {}ms", serverName, 282 numberOfAttemptsSoFar, remainingTime); 283 numberOfAttemptsSoFar++; 284 // Retry every rsRpcRetryInterval millis up to maximum wait time. 285 submitTask(this, rsRpcRetryInterval, TimeUnit.MILLISECONDS); 286 return true; 287 } 288 LOG.warn("{} is throwing ServerNotRunningYetException for {}ms; trying another server", 289 serverName, getMaxWaitTime()); 290 return false; 291 } 292 if (e instanceof DoNotRetryIOException) { 293 LOG.warn("{} tells us DoNotRetry due to {}, try={}, give up", serverName, e.toString(), 294 numberOfAttemptsSoFar); 295 return false; 296 } 297 // This category of exceptions is thrown in the rpc framework, where we can make sure 298 // that the call has not been executed yet, so it is safe to mark it as fail. 299 // Especially for open a region, we'd better choose another region server. 300 // Notice that, it is safe to quit only if this is the first time we send request to region 301 // server. Maybe the region server has accepted our request the first time, and then there is 302 // a network error which prevents we receive the response, and the second time we hit 303 // this category of exceptions, obviously it is not safe to quit here, otherwise it may lead 304 // to a double assign... 305 if (numberOfAttemptsSoFar == 0 && unableToConnectToServer(e)) { 306 return false; 307 } 308 // Always retry for other exception types if the region server is not dead yet. 309 if (!master.getServerManager().isServerOnline(serverName)) { 310 LOG.warn("Request to {} failed due to {}, try={} and the server is not online, give up", 311 serverName, e.toString(), numberOfAttemptsSoFar); 312 return false; 313 } 314 if (e instanceof RegionServerAbortedException || e instanceof RegionServerStoppedException) { 315 // A better way is to return true here to let the upper layer quit, and then schedule a 316 // background task to check whether the region server is dead. And if it is dead, call 317 // remoteCallFailed to tell the upper layer. Keep retrying here does not lead to incorrect 318 // result, but waste some resources. 319 LOG.warn("{} is aborted or stopped, for safety we still need to" 320 + " wait until it is fully dead, try={}", serverName, numberOfAttemptsSoFar); 321 } else { 322 LOG.warn("request to {} failed due to {}, try={}, retrying...", serverName, e.toString(), 323 numberOfAttemptsSoFar); 324 } 325 numberOfAttemptsSoFar++; 326 // Add some backoff here as the attempts rise otherwise if a stuck condition, will fill logs 327 // with failed attempts. None of our backoff classes -- RetryCounter or ClientBackoffPolicy 328 // -- fit here nicely so just do something simple; increment by rsRpcRetryInterval millis * 329 // retry^2 on each try 330 // up to max of 10 seconds (don't want to back off too much in case of situation change). 331 submitTask(this, 332 Math.min(rsRpcRetryInterval * (this.numberOfAttemptsSoFar * this.numberOfAttemptsSoFar), 333 10 * 1000), 334 TimeUnit.MILLISECONDS); 335 return true; 336 } 337 338 /** 339 * The category of exceptions where we can ensure that the request has not yet been received 340 * and/or processed by the target regionserver yet and hence we can determine whether it is safe 341 * to choose different regionserver as the target. 342 * @param e IOException thrown by the underlying rpc framework. 343 * @return true if the exception belongs to the category where the regionserver has not yet 344 * received the request yet. 345 */ 346 private boolean unableToConnectToServer(IOException e) { 347 if (e instanceof CallQueueTooBigException) { 348 LOG.warn("request to {} failed due to {}, try={}, this usually because" 349 + " server is overloaded, give up", serverName, e, numberOfAttemptsSoFar); 350 return true; 351 } 352 if (isSaslError(e)) { 353 LOG.warn("{} is not reachable; give up after first attempt", serverName, e); 354 return true; 355 } 356 return false; 357 } 358 359 private boolean isSaslError(IOException e) { 360 Throwable cause = e; 361 while (true) { 362 if (cause instanceof IOException) { 363 IOException unwrappedCause = unwrapException((IOException) cause); 364 if ( 365 unwrappedCause instanceof SaslException 366 || (unwrappedCause.getMessage() != null && unwrappedCause.getMessage() 367 .contains(RpcConnectionConstants.RELOGIN_IS_IN_PROGRESS)) 368 ) { 369 return true; 370 } 371 } 372 cause = cause.getCause(); 373 if (cause == null) { 374 return false; 375 } 376 } 377 } 378 379 private long getMaxWaitTime() { 380 if (this.maxWaitTime < 0) { 381 // This is the max attempts, not retries, so it should be at least 1. 382 this.maxWaitTime = EnvironmentEdgeManager.currentTime() + rsStartupWaitTime; 383 } 384 return this.maxWaitTime; 385 } 386 387 private IOException unwrapException(IOException e) { 388 if (e instanceof RemoteException) { 389 e = ((RemoteException) e).unwrapRemoteException(); 390 } 391 return e; 392 } 393 394 @Override 395 public void run() { 396 request = ExecuteProceduresRequest.newBuilder(); 397 if (LOG.isTraceEnabled()) { 398 LOG.trace("Building request with operations count=" + remoteProcedures.size()); 399 } 400 splitAndResolveOperation(getServerName(), remoteProcedures, this); 401 402 try { 403 sendRequest(getServerName(), request.build()); 404 } catch (IOException e) { 405 e = unwrapException(e); 406 // TODO: In the future some operation may want to bail out early. 407 // TODO: How many times should we retry (use numberOfAttemptsSoFar) 408 if (!scheduleForRetry(e)) { 409 remoteCallFailed(procedureEnv, e); 410 } 411 } 412 } 413 414 @Override 415 public void dispatchOpenRequests(final MasterProcedureEnv env, 416 final List<RegionOpenOperation> operations) { 417 request.addOpenRegion(buildOpenRegionRequest(env, getServerName(), operations)); 418 } 419 420 @Override 421 public void dispatchCloseRequests(final MasterProcedureEnv env, 422 final List<RegionCloseOperation> operations) { 423 for (RegionCloseOperation op : operations) { 424 request.addCloseRegion(op.buildCloseRegionRequest(getServerName())); 425 } 426 } 427 428 @Override 429 public void dispatchServerOperations(MasterProcedureEnv env, List<ServerOperation> operations) { 430 operations.stream().map(o -> o.buildRequest()).forEachOrdered(request::addProc); 431 } 432 433 // will be overridden in test. 434 protected ExecuteProceduresResponse sendRequest(final ServerName serverName, 435 final ExecuteProceduresRequest request) throws IOException { 436 try { 437 return getRsAdmin().executeProcedures(null, request); 438 } catch (ServiceException se) { 439 throw ProtobufUtil.getRemoteException(se); 440 } 441 } 442 443 protected final void remoteCallFailed(final MasterProcedureEnv env, final IOException e) { 444 for (RemoteProcedure proc : remoteProcedures) { 445 proc.remoteCallFailed(env, getServerName(), e); 446 } 447 } 448 } 449 450 private static OpenRegionRequest buildOpenRegionRequest(final MasterProcedureEnv env, 451 final ServerName serverName, final List<RegionOpenOperation> operations) { 452 final OpenRegionRequest.Builder builder = OpenRegionRequest.newBuilder(); 453 builder.setServerStartCode(serverName.getStartcode()); 454 builder.setMasterSystemTime(EnvironmentEdgeManager.currentTime()); 455 for (RegionOpenOperation op : operations) { 456 builder.addOpenInfo(op.buildRegionOpenInfoRequest(env)); 457 } 458 return builder.build(); 459 } 460 461 // ========================================================================== 462 // RPC Messages 463 // - ServerOperation: refreshConfig, grant, revoke, ... (TODO) 464 // - RegionOperation: open, close, flush, snapshot, ... 465 // ========================================================================== 466 467 public static final class ServerOperation extends RemoteOperation { 468 469 private final long procId; 470 471 private final Class<?> rsProcClass; 472 473 private final byte[] rsProcData; 474 475 public ServerOperation(RemoteProcedure remoteProcedure, long procId, Class<?> rsProcClass, 476 byte[] rsProcData) { 477 super(remoteProcedure); 478 this.procId = procId; 479 this.rsProcClass = rsProcClass; 480 this.rsProcData = rsProcData; 481 } 482 483 public RemoteProcedureRequest buildRequest() { 484 return RemoteProcedureRequest.newBuilder().setProcId(procId) 485 .setProcClass(rsProcClass.getName()).setProcData(ByteString.copyFrom(rsProcData)).build(); 486 } 487 } 488 489 public static abstract class RegionOperation extends RemoteOperation { 490 protected final RegionInfo regionInfo; 491 protected final long procId; 492 493 protected RegionOperation(RemoteProcedure remoteProcedure, RegionInfo regionInfo, long procId) { 494 super(remoteProcedure); 495 this.regionInfo = regionInfo; 496 this.procId = procId; 497 } 498 } 499 500 public static class RegionOpenOperation extends RegionOperation { 501 502 public RegionOpenOperation(RemoteProcedure remoteProcedure, RegionInfo regionInfo, 503 long procId) { 504 super(remoteProcedure, regionInfo, procId); 505 } 506 507 public OpenRegionRequest.RegionOpenInfo 508 buildRegionOpenInfoRequest(final MasterProcedureEnv env) { 509 return RequestConverter.buildRegionOpenInfo(regionInfo, 510 env.getAssignmentManager().getFavoredNodes(regionInfo), procId); 511 } 512 } 513 514 public static class RegionCloseOperation extends RegionOperation { 515 private final ServerName destinationServer; 516 private boolean evictCache; 517 518 public RegionCloseOperation(RemoteProcedure remoteProcedure, RegionInfo regionInfo, long procId, 519 ServerName destinationServer, boolean evictCache) { 520 super(remoteProcedure, regionInfo, procId); 521 this.destinationServer = destinationServer; 522 this.evictCache = evictCache; 523 } 524 525 public ServerName getDestinationServer() { 526 return destinationServer; 527 } 528 529 public CloseRegionRequest buildCloseRegionRequest(final ServerName serverName) { 530 return ProtobufUtil.buildCloseRegionRequest(serverName, regionInfo.getRegionName(), 531 getDestinationServer(), procId, evictCache); 532 533 } 534 } 535}