From f8fb87291b19b8c051d5ee5aa961ad56739f6fca Mon Sep 17 00:00:00 2001 From: Dave Marion Date: Fri, 31 Jan 2025 07:45:56 -0500 Subject: [PATCH] Enable graceful shutdown of server processes (#5193) Added RPC to the server processes to initiate a graceful shutdown. A new admin command has been created for a user to send this command to a server process. When the server process receives the command to shutdown gracefully it will attempt an orderly shutdown. Additional requests to shutdown gracefully will be ignored. Compactors will finish the major compaction that they are currently working on, then will exit. Scan Servers will return a busy signal to all clients for new scan requests, and will shutdown when all in-progress scans have closed. Tablet Servers will signal the Manager that they are shutting down, which should prevent assignment of tablets to that server, then they will unload all tablets and shut down. The Monitor, Manager, GarbageCollector, and CompactionCoordinator will shut down also. The last step in the shutdown process for all servers is to remove their lock in ZooKeeper. --------- Co-authored-by: Dom G. --- .../fate/zookeeper/ServiceLockSupport.java | 30 +- .../ServerProcessServiceThriftClient.java | 54 + .../core/rpc/clients/ThriftClientTypes.java | 3 + .../core/util/threads/ThreadPoolNames.java | 1 + core/src/main/scripts/generate-thrift.sh | 2 +- core/src/main/spotbugs/exclude-filter.xml | 1 + .../manager/thrift/ManagerClientService.java | 1271 +++++++++++++++++ .../process/thrift/ServerProcessService.java | 619 ++++++++ core/src/main/thrift/manager.thrift | 9 + core/src/main/thrift/process.thrift | 31 + .../MiniAccumuloClusterControl.java | 66 + .../accumulo/server/AbstractServer.java | 66 +- .../server/manager/LiveTServerSet.java | 22 +- .../server/rpc/ThriftProcessorTypes.java | 43 +- .../apache/accumulo/server/util/Admin.java | 34 + .../zookeeper/DistributedWorkQueue.java | 26 +- .../coordinator/CompactionCoordinator.java | 70 +- .../CompactionCoordinatorTest.java | 22 +- .../apache/accumulo/compactor/Compactor.java | 322 +++-- .../accumulo/compactor/CompactorTest.java | 28 +- .../accumulo/gc/SimpleGarbageCollector.java | 249 ++-- .../org/apache/accumulo/manager/Manager.java | 38 +- .../manager/ManagerClientServiceHandler.java | 12 + .../manager/recovery/RecoveryManager.java | 4 +- .../DistributedWorkQueueWorkAssigner.java | 2 +- .../manager/tableOps/bulkVer1/CopyFailed.java | 2 +- .../accumulo/monitor/EmbeddedWebServer.java | 2 +- .../org/apache/accumulo/monitor/Monitor.java | 19 +- .../apache/accumulo/tserver/ScanServer.java | 47 +- .../apache/accumulo/tserver/TabletServer.java | 129 +- .../accumulo/tserver/log/LogSorter.java | 6 +- .../replication/ReplicationWorker.java | 19 +- .../accumulo/tserver/ScanServerTest.java | 8 +- .../accumulo/test/SelfStoppingScanServer.java | 5 +- .../compaction/ExternalCompactionTServer.java | 3 +- .../ExternalDoNothingCompactor.java | 6 +- ...tCompactionCoordinatorForOfflineTable.java | 3 +- .../accumulo/test/fate/FlakyFateManager.java | 3 +- .../test/functional/GracefulShutdownIT.java | 274 ++++ .../functional/HalfDeadServerWatcherIT.java | 4 +- 40 files changed, 3145 insertions(+), 410 deletions(-) create mode 100644 core/src/main/java/org/apache/accumulo/core/rpc/clients/ServerProcessServiceThriftClient.java create mode 100644 core/src/main/thrift-gen-java/org/apache/accumulo/core/process/thrift/ServerProcessService.java create mode 100644 core/src/main/thrift/process.thrift create mode 100644 test/src/main/java/org/apache/accumulo/test/functional/GracefulShutdownIT.java diff --git a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ServiceLockSupport.java b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ServiceLockSupport.java index b4d95a703aa..cd5c400e017 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ServiceLockSupport.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ServiceLockSupport.java @@ -41,16 +41,23 @@ public static class HAServiceLockWatcher implements AccumuloLockWatcher { private static final Logger LOG = LoggerFactory.getLogger(HAServiceLockWatcher.class); private final String serviceName; + private final Supplier shutdownComplete; private volatile boolean acquiredLock = false; private volatile boolean failedToAcquireLock = false; - public HAServiceLockWatcher(String serviceName) { + public HAServiceLockWatcher(String serviceName, Supplier shutdownComplete) { this.serviceName = serviceName; + this.shutdownComplete = shutdownComplete; } @Override public void lostLock(LockLossReason reason) { - Halt.halt(serviceName + " lock in zookeeper lost (reason = " + reason + "), exiting!", -1); + if (shutdownComplete.get()) { + LOG.warn("{} lost lock (reason = {}), not halting because shutdown is complete.", + serviceName, reason); + } else { + Halt.halt(serviceName + " lock in zookeeper lost (reason = " + reason + "), exiting!", -1); + } } @Override @@ -122,24 +129,27 @@ public static class ServiceLockWatcher implements LockWatcher { private static final Logger LOG = LoggerFactory.getLogger(ServiceLockWatcher.class); private final String serviceName; - private final Supplier shuttingDown; + private final Supplier shutdownComplete; private final Consumer lostLockAction; - public ServiceLockWatcher(String serviceName, Supplier shuttingDown, + public ServiceLockWatcher(String serviceName, Supplier shutdownComplete, Consumer lostLockAction) { this.serviceName = serviceName; - this.shuttingDown = shuttingDown; + this.shutdownComplete = shutdownComplete; this.lostLockAction = lostLockAction; } @Override public void lostLock(final LockLossReason reason) { - Halt.halt(1, () -> { - if (!shuttingDown.get()) { + if (shutdownComplete.get()) { + LOG.warn("{} lost lock (reason = {}), not halting because shutdown is complete.", + serviceName, reason); + } else { + Halt.halt(1, () -> { LOG.error("{} lost lock (reason = {}), exiting.", serviceName, reason); - } - lostLockAction.accept(serviceName); - }); + lostLockAction.accept(serviceName); + }); + } } @Override diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/clients/ServerProcessServiceThriftClient.java b/core/src/main/java/org/apache/accumulo/core/rpc/clients/ServerProcessServiceThriftClient.java new file mode 100644 index 00000000000..d512bd2766c --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/core/rpc/clients/ServerProcessServiceThriftClient.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.accumulo.core.rpc.clients; + +import java.io.UncheckedIOException; +import java.net.UnknownHostException; + +import org.apache.accumulo.core.clientImpl.ClientContext; +import org.apache.accumulo.core.process.thrift.ServerProcessService.Client; +import org.apache.accumulo.core.rpc.ThriftUtil; +import org.apache.accumulo.core.util.HostAndPort; +import org.apache.thrift.transport.TTransportException; +import org.slf4j.Logger; + +public class ServerProcessServiceThriftClient extends ThriftClientTypes { + + protected ServerProcessServiceThriftClient(String serviceName) { + super(serviceName, new Client.Factory()); + } + + public Client getServerProcessConnection(ClientContext context, Logger log, String hostname, + int port) { + HostAndPort serverProcess = HostAndPort.fromParts(hostname, port); + try { + // Manager requests can take a long time: don't ever time out + return ThriftUtil.getClientNoTimeout(this, serverProcess, context); + } catch (TTransportException tte) { + Throwable cause = tte.getCause(); + if (cause instanceof UnknownHostException) { + // do not expect to recover from this + throw new UncheckedIOException((UnknownHostException) cause); + } + log.debug("Failed to connect to process at " + serverProcess + ", will retry... ", tte); + return null; + } + + } +} diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/clients/ThriftClientTypes.java b/core/src/main/java/org/apache/accumulo/core/rpc/clients/ThriftClientTypes.java index 95595506fba..5df2223928f 100644 --- a/core/src/main/java/org/apache/accumulo/core/rpc/clients/ThriftClientTypes.java +++ b/core/src/main/java/org/apache/accumulo/core/rpc/clients/ThriftClientTypes.java @@ -58,6 +58,9 @@ public abstract class ThriftClientTypes { public static final TabletScanClientServiceThriftClient TABLET_SCAN = new TabletScanClientServiceThriftClient("scan"); + public static final ServerProcessServiceThriftClient SERVER_PROCESS = + new ServerProcessServiceThriftClient("process"); + /** * execute method with supplied client returning object of type R * diff --git a/core/src/main/java/org/apache/accumulo/core/util/threads/ThreadPoolNames.java b/core/src/main/java/org/apache/accumulo/core/util/threads/ThreadPoolNames.java index bdebd03b2dc..43a4a24b70b 100644 --- a/core/src/main/java/org/apache/accumulo/core/util/threads/ThreadPoolNames.java +++ b/core/src/main/java/org/apache/accumulo/core/util/threads/ThreadPoolNames.java @@ -59,6 +59,7 @@ public enum ThreadPoolNames { TSERVER_COMPACTION_MINOR_POOL("accumulo.pool.tserver.compaction.minor"), TSERVER_MIGRATIONS_POOL("accumulo.pool.tserver.migrations"), TSERVER_MINOR_COMPACTOR_POOL("accumulo.pool.tserver.minor.compactor"), + TSERVER_SHUTDOWN_UNLOAD_TABLET_POOL("accumulo.pool.tserver.shutdown.tablet.unload"), TSERVER_SUMMARY_FILE_RETRIEVER_POOL("accumulo.pool.tserver.summary.file.retriever.pool"), TSERVER_SUMMARY_PARTITION_POOL("accumulo.pool.tserver.summary.partition"), TSERVER_SUMMARY_REMOTE_POOL("accumulo.pool.tserver.summary.remote"), diff --git a/core/src/main/scripts/generate-thrift.sh b/core/src/main/scripts/generate-thrift.sh index 37929eef91f..2c9ad16132a 100755 --- a/core/src/main/scripts/generate-thrift.sh +++ b/core/src/main/scripts/generate-thrift.sh @@ -32,7 +32,7 @@ [[ -z $REQUIRED_THRIFT_VERSION ]] && REQUIRED_THRIFT_VERSION='0.17.0' [[ -z $INCLUDED_MODULES ]] && INCLUDED_MODULES=() [[ -z $BASE_OUTPUT_PACKAGE ]] && BASE_OUTPUT_PACKAGE='org.apache.accumulo.core' -[[ -z $PACKAGES_TO_GENERATE ]] && PACKAGES_TO_GENERATE=(gc master manager tabletserver securityImpl clientImpl dataImpl replication trace compaction) +[[ -z $PACKAGES_TO_GENERATE ]] && PACKAGES_TO_GENERATE=(process gc master manager tabletserver securityImpl clientImpl dataImpl replication trace compaction) [[ -z $BUILD_DIR ]] && BUILD_DIR='target' [[ -z $LANGUAGES_TO_GENERATE ]] && LANGUAGES_TO_GENERATE=(java) [[ -z $FINAL_DIR ]] && FINAL_DIR='src/main' diff --git a/core/src/main/spotbugs/exclude-filter.xml b/core/src/main/spotbugs/exclude-filter.xml index cc0bd6a6801..f1776b30607 100644 --- a/core/src/main/spotbugs/exclude-filter.xml +++ b/core/src/main/spotbugs/exclude-filter.xml @@ -32,6 +32,7 @@ + diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/ManagerClientService.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/ManagerClientService.java index 5e17421edd8..60e46630d5e 100644 --- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/ManagerClientService.java +++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/ManagerClientService.java @@ -51,6 +51,8 @@ public interface Iface { public void shutdownTabletServer(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tabletServer, boolean force) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException; + public void tabletServerStopping(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tabletServer) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException; + public void setSystemProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property, java.lang.String value) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, ThriftPropertyException, org.apache.thrift.TException; public void modifySystemProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.accumulo.core.clientImpl.thrift.ThriftConcurrentModificationException, ThriftPropertyException, org.apache.thrift.TException; @@ -97,6 +99,8 @@ public interface AsyncIface { public void shutdownTabletServer(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tabletServer, boolean force, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void tabletServerStopping(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tabletServer, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void setSystemProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property, java.lang.String value, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; public void modifySystemProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; @@ -515,6 +519,35 @@ public void recv_shutdownTabletServer() throws org.apache.accumulo.core.clientIm return; } + @Override + public void tabletServerStopping(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tabletServer) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException + { + send_tabletServerStopping(tinfo, credentials, tabletServer); + recv_tabletServerStopping(); + } + + public void send_tabletServerStopping(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tabletServer) throws org.apache.thrift.TException + { + tabletServerStopping_args args = new tabletServerStopping_args(); + args.setTinfo(tinfo); + args.setCredentials(credentials); + args.setTabletServer(tabletServer); + sendBase("tabletServerStopping", args); + } + + public void recv_tabletServerStopping() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException + { + tabletServerStopping_result result = new tabletServerStopping_result(); + receiveBase(result, "tabletServerStopping"); + if (result.sec != null) { + throw result.sec; + } + if (result.tnase != null) { + throw result.tnase; + } + return; + } + @Override public void setSystemProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property, java.lang.String value) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, ThriftPropertyException, org.apache.thrift.TException { @@ -1312,6 +1345,48 @@ public Void getResult() throws org.apache.accumulo.core.clientImpl.thrift.Thrift } } + @Override + public void tabletServerStopping(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tabletServer, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + tabletServerStopping_call method_call = new tabletServerStopping_call(tinfo, credentials, tabletServer, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class tabletServerStopping_call extends org.apache.thrift.async.TAsyncMethodCall { + private org.apache.accumulo.core.trace.thrift.TInfo tinfo; + private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; + private java.lang.String tabletServer; + public tabletServerStopping_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tabletServer, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.tinfo = tinfo; + this.credentials = credentials; + this.tabletServer = tabletServer; + } + + @Override + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("tabletServerStopping", org.apache.thrift.protocol.TMessageType.CALL, 0)); + tabletServerStopping_args args = new tabletServerStopping_args(); + args.setTinfo(tinfo); + args.setCredentials(credentials); + args.setTabletServer(tabletServer); + args.write(prot); + prot.writeMessageEnd(); + } + + @Override + public Void getResult() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new java.lang.IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + (new Client(prot)).recv_tabletServerStopping(); + return null; + } + } + @Override public void setSystemProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property, java.lang.String value, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); @@ -1753,6 +1828,7 @@ protected Processor(I iface, java.util.Map extends org.apache.thrift.ProcessFunction { + public tabletServerStopping() { + super("tabletServerStopping"); + } + + @Override + public tabletServerStopping_args getEmptyArgsInstance() { + return new tabletServerStopping_args(); + } + + @Override + protected boolean isOneway() { + return false; + } + + @Override + protected boolean rethrowUnhandledExceptions() { + return false; + } + + @Override + public tabletServerStopping_result getResult(I iface, tabletServerStopping_args args) throws org.apache.thrift.TException { + tabletServerStopping_result result = new tabletServerStopping_result(); + try { + iface.tabletServerStopping(args.tinfo, args.credentials, args.tabletServer); + } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) { + result.sec = sec; + } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException tnase) { + result.tnase = tnase; + } + return result; + } + } + public static class setSystemProperty extends org.apache.thrift.ProcessFunction { public setSystemProperty() { super("setSystemProperty"); @@ -2520,6 +2630,7 @@ protected AsyncProcessor(I iface, java.util.Map extends org.apache.thrift.AsyncProcessFunction { + public tabletServerStopping() { + super("tabletServerStopping"); + } + + @Override + public tabletServerStopping_args getEmptyArgsInstance() { + return new tabletServerStopping_args(); + } + + @Override + public org.apache.thrift.async.AsyncMethodCallback getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new org.apache.thrift.async.AsyncMethodCallback() { + @Override + public void onComplete(Void o) { + tabletServerStopping_result result = new tabletServerStopping_result(); + try { + fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + } catch (org.apache.thrift.transport.TTransportException e) { + _LOGGER.error("TTransportException writing to internal frame buffer", e); + fb.close(); + } catch (java.lang.Exception e) { + _LOGGER.error("Exception writing to internal frame buffer", e); + onError(e); + } + } + @Override + public void onError(java.lang.Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TSerializable msg; + tabletServerStopping_result result = new tabletServerStopping_result(); + if (e instanceof org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) { + result.sec = (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) e; + result.setSecIsSet(true); + msg = result; + } else if (e instanceof org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException) { + result.tnase = (org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException) e; + result.setTnaseIsSet(true); + msg = result; + } else if (e instanceof org.apache.thrift.transport.TTransportException) { + _LOGGER.error("TTransportException inside handler", e); + fb.close(); + return; + } else if (e instanceof org.apache.thrift.TApplicationException) { + _LOGGER.error("TApplicationException inside handler", e); + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TApplicationException)e; + } else { + _LOGGER.error("Exception inside handler", e); + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + } catch (java.lang.Exception ex) { + _LOGGER.error("Exception writing to internal frame buffer", ex); + fb.close(); + } + } + }; + } + + @Override + protected boolean isOneway() { + return false; + } + + @Override + public void start(I iface, tabletServerStopping_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + iface.tabletServerStopping(args.tinfo, args.credentials, args.tabletServer,resultHandler); + } + } + public static class setSystemProperty extends org.apache.thrift.AsyncProcessFunction { public setSystemProperty() { super("setSystemProperty"); @@ -18916,6 +19101,1092 @@ private static S scheme(org.apache. } } + @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) + public static class tabletServerStopping_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("tabletServerStopping_args"); + + private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)2); + private static final org.apache.thrift.protocol.TField TABLET_SERVER_FIELD_DESC = new org.apache.thrift.protocol.TField("tabletServer", org.apache.thrift.protocol.TType.STRING, (short)3); + + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new tabletServerStopping_argsStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new tabletServerStopping_argsTupleSchemeFactory(); + + public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required + public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required + public @org.apache.thrift.annotation.Nullable java.lang.String tabletServer; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + TINFO((short)1, "tinfo"), + CREDENTIALS((short)2, "credentials"), + TABLET_SERVER((short)3, "tabletServer"); + + private static final java.util.Map byName = new java.util.HashMap(); + + static { + for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + @org.apache.thrift.annotation.Nullable + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // TINFO + return TINFO; + case 2: // CREDENTIALS + return CREDENTIALS; + case 3: // TABLET_SERVER + return TABLET_SERVER; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + @org.apache.thrift.annotation.Nullable + public static _Fields findByName(java.lang.String name) { + return byName.get(name); + } + + private final short _thriftId; + private final java.lang.String _fieldName; + + _Fields(short thriftId, java.lang.String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + @Override + public short getThriftFieldId() { + return _thriftId; + } + + @Override + public java.lang.String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class))); + tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class))); + tmpMap.put(_Fields.TABLET_SERVER, new org.apache.thrift.meta_data.FieldMetaData("tabletServer", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(tabletServerStopping_args.class, metaDataMap); + } + + public tabletServerStopping_args() { + } + + public tabletServerStopping_args( + org.apache.accumulo.core.trace.thrift.TInfo tinfo, + org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, + java.lang.String tabletServer) + { + this(); + this.tinfo = tinfo; + this.credentials = credentials; + this.tabletServer = tabletServer; + } + + /** + * Performs a deep copy on other. + */ + public tabletServerStopping_args(tabletServerStopping_args other) { + if (other.isSetTinfo()) { + this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo); + } + if (other.isSetCredentials()) { + this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials); + } + if (other.isSetTabletServer()) { + this.tabletServer = other.tabletServer; + } + } + + @Override + public tabletServerStopping_args deepCopy() { + return new tabletServerStopping_args(this); + } + + @Override + public void clear() { + this.tinfo = null; + this.credentials = null; + this.tabletServer = null; + } + + @org.apache.thrift.annotation.Nullable + public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() { + return this.tinfo; + } + + public tabletServerStopping_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) { + this.tinfo = tinfo; + return this; + } + + public void unsetTinfo() { + this.tinfo = null; + } + + /** Returns true if field tinfo is set (has been assigned a value) and false otherwise */ + public boolean isSetTinfo() { + return this.tinfo != null; + } + + public void setTinfoIsSet(boolean value) { + if (!value) { + this.tinfo = null; + } + } + + @org.apache.thrift.annotation.Nullable + public org.apache.accumulo.core.securityImpl.thrift.TCredentials getCredentials() { + return this.credentials; + } + + public tabletServerStopping_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) { + this.credentials = credentials; + return this; + } + + public void unsetCredentials() { + this.credentials = null; + } + + /** Returns true if field credentials is set (has been assigned a value) and false otherwise */ + public boolean isSetCredentials() { + return this.credentials != null; + } + + public void setCredentialsIsSet(boolean value) { + if (!value) { + this.credentials = null; + } + } + + @org.apache.thrift.annotation.Nullable + public java.lang.String getTabletServer() { + return this.tabletServer; + } + + public tabletServerStopping_args setTabletServer(@org.apache.thrift.annotation.Nullable java.lang.String tabletServer) { + this.tabletServer = tabletServer; + return this; + } + + public void unsetTabletServer() { + this.tabletServer = null; + } + + /** Returns true if field tabletServer is set (has been assigned a value) and false otherwise */ + public boolean isSetTabletServer() { + return this.tabletServer != null; + } + + public void setTabletServerIsSet(boolean value) { + if (!value) { + this.tabletServer = null; + } + } + + @Override + public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) { + switch (field) { + case TINFO: + if (value == null) { + unsetTinfo(); + } else { + setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value); + } + break; + + case CREDENTIALS: + if (value == null) { + unsetCredentials(); + } else { + setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value); + } + break; + + case TABLET_SERVER: + if (value == null) { + unsetTabletServer(); + } else { + setTabletServer((java.lang.String)value); + } + break; + + } + } + + @org.apache.thrift.annotation.Nullable + @Override + public java.lang.Object getFieldValue(_Fields field) { + switch (field) { + case TINFO: + return getTinfo(); + + case CREDENTIALS: + return getCredentials(); + + case TABLET_SERVER: + return getTabletServer(); + + } + throw new java.lang.IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + @Override + public boolean isSet(_Fields field) { + if (field == null) { + throw new java.lang.IllegalArgumentException(); + } + + switch (field) { + case TINFO: + return isSetTinfo(); + case CREDENTIALS: + return isSetCredentials(); + case TABLET_SERVER: + return isSetTabletServer(); + } + throw new java.lang.IllegalStateException(); + } + + @Override + public boolean equals(java.lang.Object that) { + if (that instanceof tabletServerStopping_args) + return this.equals((tabletServerStopping_args)that); + return false; + } + + public boolean equals(tabletServerStopping_args that) { + if (that == null) + return false; + if (this == that) + return true; + + boolean this_present_tinfo = true && this.isSetTinfo(); + boolean that_present_tinfo = true && that.isSetTinfo(); + if (this_present_tinfo || that_present_tinfo) { + if (!(this_present_tinfo && that_present_tinfo)) + return false; + if (!this.tinfo.equals(that.tinfo)) + return false; + } + + boolean this_present_credentials = true && this.isSetCredentials(); + boolean that_present_credentials = true && that.isSetCredentials(); + if (this_present_credentials || that_present_credentials) { + if (!(this_present_credentials && that_present_credentials)) + return false; + if (!this.credentials.equals(that.credentials)) + return false; + } + + boolean this_present_tabletServer = true && this.isSetTabletServer(); + boolean that_present_tabletServer = true && that.isSetTabletServer(); + if (this_present_tabletServer || that_present_tabletServer) { + if (!(this_present_tabletServer && that_present_tabletServer)) + return false; + if (!this.tabletServer.equals(that.tabletServer)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + int hashCode = 1; + + hashCode = hashCode * 8191 + ((isSetTinfo()) ? 131071 : 524287); + if (isSetTinfo()) + hashCode = hashCode * 8191 + tinfo.hashCode(); + + hashCode = hashCode * 8191 + ((isSetCredentials()) ? 131071 : 524287); + if (isSetCredentials()) + hashCode = hashCode * 8191 + credentials.hashCode(); + + hashCode = hashCode * 8191 + ((isSetTabletServer()) ? 131071 : 524287); + if (isSetTabletServer()) + hashCode = hashCode * 8191 + tabletServer.hashCode(); + + return hashCode; + } + + @Override + public int compareTo(tabletServerStopping_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = java.lang.Boolean.compare(isSetTinfo(), other.isSetTinfo()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTinfo()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, other.tinfo); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = java.lang.Boolean.compare(isSetCredentials(), other.isSetCredentials()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetCredentials()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, other.credentials); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = java.lang.Boolean.compare(isSetTabletServer(), other.isSetTabletServer()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTabletServer()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tabletServer, other.tabletServer); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + @org.apache.thrift.annotation.Nullable + @Override + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + scheme(iprot).read(iprot, this); + } + + @Override + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + scheme(oprot).write(oprot, this); + } + + @Override + public java.lang.String toString() { + java.lang.StringBuilder sb = new java.lang.StringBuilder("tabletServerStopping_args("); + boolean first = true; + + sb.append("tinfo:"); + if (this.tinfo == null) { + sb.append("null"); + } else { + sb.append(this.tinfo); + } + first = false; + if (!first) sb.append(", "); + sb.append("credentials:"); + if (this.credentials == null) { + sb.append("null"); + } else { + sb.append(this.credentials); + } + first = false; + if (!first) sb.append(", "); + sb.append("tabletServer:"); + if (this.tabletServer == null) { + sb.append("null"); + } else { + sb.append(this.tabletServer); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (tinfo != null) { + tinfo.validate(); + } + if (credentials != null) { + credentials.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class tabletServerStopping_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + @Override + public tabletServerStopping_argsStandardScheme getScheme() { + return new tabletServerStopping_argsStandardScheme(); + } + } + + private static class tabletServerStopping_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { + + @Override + public void read(org.apache.thrift.protocol.TProtocol iprot, tabletServerStopping_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // TINFO + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(); + struct.tinfo.read(iprot); + struct.setTinfoIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // CREDENTIALS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(); + struct.credentials.read(iprot); + struct.setCredentialsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // TABLET_SERVER + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.tabletServer = iprot.readString(); + struct.setTabletServerIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + + // check for required fields of primitive type, which can't be checked in the validate method + struct.validate(); + } + + @Override + public void write(org.apache.thrift.protocol.TProtocol oprot, tabletServerStopping_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.tinfo != null) { + oprot.writeFieldBegin(TINFO_FIELD_DESC); + struct.tinfo.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.credentials != null) { + oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC); + struct.credentials.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.tabletServer != null) { + oprot.writeFieldBegin(TABLET_SERVER_FIELD_DESC); + oprot.writeString(struct.tabletServer); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class tabletServerStopping_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + @Override + public tabletServerStopping_argsTupleScheme getScheme() { + return new tabletServerStopping_argsTupleScheme(); + } + } + + private static class tabletServerStopping_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, tabletServerStopping_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot; + java.util.BitSet optionals = new java.util.BitSet(); + if (struct.isSetTinfo()) { + optionals.set(0); + } + if (struct.isSetCredentials()) { + optionals.set(1); + } + if (struct.isSetTabletServer()) { + optionals.set(2); + } + oprot.writeBitSet(optionals, 3); + if (struct.isSetTinfo()) { + struct.tinfo.write(oprot); + } + if (struct.isSetCredentials()) { + struct.credentials.write(oprot); + } + if (struct.isSetTabletServer()) { + oprot.writeString(struct.tabletServer); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, tabletServerStopping_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; + java.util.BitSet incoming = iprot.readBitSet(3); + if (incoming.get(0)) { + struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(); + struct.tinfo.read(iprot); + struct.setTinfoIsSet(true); + } + if (incoming.get(1)) { + struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(); + struct.credentials.read(iprot); + struct.setCredentialsIsSet(true); + } + if (incoming.get(2)) { + struct.tabletServer = iprot.readString(); + struct.setTabletServerIsSet(true); + } + } + } + + private static S scheme(org.apache.thrift.protocol.TProtocol proto) { + return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme(); + } + } + + @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) + public static class tabletServerStopping_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("tabletServerStopping_result"); + + private static final org.apache.thrift.protocol.TField SEC_FIELD_DESC = new org.apache.thrift.protocol.TField("sec", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField TNASE_FIELD_DESC = new org.apache.thrift.protocol.TField("tnase", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new tabletServerStopping_resultStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new tabletServerStopping_resultTupleSchemeFactory(); + + public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec; // required + public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException tnase; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SEC((short)1, "sec"), + TNASE((short)2, "tnase"); + + private static final java.util.Map byName = new java.util.HashMap(); + + static { + for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + @org.apache.thrift.annotation.Nullable + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // SEC + return SEC; + case 2: // TNASE + return TNASE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + @org.apache.thrift.annotation.Nullable + public static _Fields findByName(java.lang.String name) { + return byName.get(name); + } + + private final short _thriftId; + private final java.lang.String _fieldName; + + _Fields(short thriftId, java.lang.String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + @Override + public short getThriftFieldId() { + return _thriftId; + } + + @Override + public java.lang.String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SEC, new org.apache.thrift.meta_data.FieldMetaData("sec", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException.class))); + tmpMap.put(_Fields.TNASE, new org.apache.thrift.meta_data.FieldMetaData("tnase", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException.class))); + metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(tabletServerStopping_result.class, metaDataMap); + } + + public tabletServerStopping_result() { + } + + public tabletServerStopping_result( + org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec, + org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException tnase) + { + this(); + this.sec = sec; + this.tnase = tnase; + } + + /** + * Performs a deep copy on other. + */ + public tabletServerStopping_result(tabletServerStopping_result other) { + if (other.isSetSec()) { + this.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(other.sec); + } + if (other.isSetTnase()) { + this.tnase = new org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException(other.tnase); + } + } + + @Override + public tabletServerStopping_result deepCopy() { + return new tabletServerStopping_result(this); + } + + @Override + public void clear() { + this.sec = null; + this.tnase = null; + } + + @org.apache.thrift.annotation.Nullable + public org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException getSec() { + return this.sec; + } + + public tabletServerStopping_result setSec(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) { + this.sec = sec; + return this; + } + + public void unsetSec() { + this.sec = null; + } + + /** Returns true if field sec is set (has been assigned a value) and false otherwise */ + public boolean isSetSec() { + return this.sec != null; + } + + public void setSecIsSet(boolean value) { + if (!value) { + this.sec = null; + } + } + + @org.apache.thrift.annotation.Nullable + public org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException getTnase() { + return this.tnase; + } + + public tabletServerStopping_result setTnase(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException tnase) { + this.tnase = tnase; + return this; + } + + public void unsetTnase() { + this.tnase = null; + } + + /** Returns true if field tnase is set (has been assigned a value) and false otherwise */ + public boolean isSetTnase() { + return this.tnase != null; + } + + public void setTnaseIsSet(boolean value) { + if (!value) { + this.tnase = null; + } + } + + @Override + public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) { + switch (field) { + case SEC: + if (value == null) { + unsetSec(); + } else { + setSec((org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException)value); + } + break; + + case TNASE: + if (value == null) { + unsetTnase(); + } else { + setTnase((org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException)value); + } + break; + + } + } + + @org.apache.thrift.annotation.Nullable + @Override + public java.lang.Object getFieldValue(_Fields field) { + switch (field) { + case SEC: + return getSec(); + + case TNASE: + return getTnase(); + + } + throw new java.lang.IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + @Override + public boolean isSet(_Fields field) { + if (field == null) { + throw new java.lang.IllegalArgumentException(); + } + + switch (field) { + case SEC: + return isSetSec(); + case TNASE: + return isSetTnase(); + } + throw new java.lang.IllegalStateException(); + } + + @Override + public boolean equals(java.lang.Object that) { + if (that instanceof tabletServerStopping_result) + return this.equals((tabletServerStopping_result)that); + return false; + } + + public boolean equals(tabletServerStopping_result that) { + if (that == null) + return false; + if (this == that) + return true; + + boolean this_present_sec = true && this.isSetSec(); + boolean that_present_sec = true && that.isSetSec(); + if (this_present_sec || that_present_sec) { + if (!(this_present_sec && that_present_sec)) + return false; + if (!this.sec.equals(that.sec)) + return false; + } + + boolean this_present_tnase = true && this.isSetTnase(); + boolean that_present_tnase = true && that.isSetTnase(); + if (this_present_tnase || that_present_tnase) { + if (!(this_present_tnase && that_present_tnase)) + return false; + if (!this.tnase.equals(that.tnase)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + int hashCode = 1; + + hashCode = hashCode * 8191 + ((isSetSec()) ? 131071 : 524287); + if (isSetSec()) + hashCode = hashCode * 8191 + sec.hashCode(); + + hashCode = hashCode * 8191 + ((isSetTnase()) ? 131071 : 524287); + if (isSetTnase()) + hashCode = hashCode * 8191 + tnase.hashCode(); + + return hashCode; + } + + @Override + public int compareTo(tabletServerStopping_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = java.lang.Boolean.compare(isSetSec(), other.isSetSec()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSec()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sec, other.sec); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = java.lang.Boolean.compare(isSetTnase(), other.isSetTnase()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTnase()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tnase, other.tnase); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + @org.apache.thrift.annotation.Nullable + @Override + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + scheme(iprot).read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + scheme(oprot).write(oprot, this); + } + + @Override + public java.lang.String toString() { + java.lang.StringBuilder sb = new java.lang.StringBuilder("tabletServerStopping_result("); + boolean first = true; + + sb.append("sec:"); + if (this.sec == null) { + sb.append("null"); + } else { + sb.append(this.sec); + } + first = false; + if (!first) sb.append(", "); + sb.append("tnase:"); + if (this.tnase == null) { + sb.append("null"); + } else { + sb.append(this.tnase); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class tabletServerStopping_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + @Override + public tabletServerStopping_resultStandardScheme getScheme() { + return new tabletServerStopping_resultStandardScheme(); + } + } + + private static class tabletServerStopping_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { + + @Override + public void read(org.apache.thrift.protocol.TProtocol iprot, tabletServerStopping_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // SEC + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(); + struct.sec.read(iprot); + struct.setSecIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // TNASE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.tnase = new org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException(); + struct.tnase.read(iprot); + struct.setTnaseIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + + // check for required fields of primitive type, which can't be checked in the validate method + struct.validate(); + } + + @Override + public void write(org.apache.thrift.protocol.TProtocol oprot, tabletServerStopping_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.sec != null) { + oprot.writeFieldBegin(SEC_FIELD_DESC); + struct.sec.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.tnase != null) { + oprot.writeFieldBegin(TNASE_FIELD_DESC); + struct.tnase.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class tabletServerStopping_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + @Override + public tabletServerStopping_resultTupleScheme getScheme() { + return new tabletServerStopping_resultTupleScheme(); + } + } + + private static class tabletServerStopping_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, tabletServerStopping_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot; + java.util.BitSet optionals = new java.util.BitSet(); + if (struct.isSetSec()) { + optionals.set(0); + } + if (struct.isSetTnase()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetSec()) { + struct.sec.write(oprot); + } + if (struct.isSetTnase()) { + struct.tnase.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, tabletServerStopping_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; + java.util.BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + struct.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(); + struct.sec.read(iprot); + struct.setSecIsSet(true); + } + if (incoming.get(1)) { + struct.tnase = new org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException(); + struct.tnase.read(iprot); + struct.setTnaseIsSet(true); + } + } + } + + private static S scheme(org.apache.thrift.protocol.TProtocol proto) { + return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme(); + } + } + @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) public static class setSystemProperty_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("setSystemProperty_args"); diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/process/thrift/ServerProcessService.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/process/thrift/ServerProcessService.java new file mode 100644 index 00000000000..273d44e49fa --- /dev/null +++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/process/thrift/ServerProcessService.java @@ -0,0 +1,619 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +/** + * Autogenerated by Thrift Compiler (0.17.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.accumulo.core.process.thrift; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) +public class ServerProcessService { + + public interface Iface { + + public void gracefulShutdown(org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException; + + } + + public interface AsyncIface { + + public void gracefulShutdown(org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + } + + public static class Client extends org.apache.thrift.TServiceClient implements Iface { + public static class Factory implements org.apache.thrift.TServiceClientFactory { + public Factory() {} + @Override + public Client getClient(org.apache.thrift.protocol.TProtocol prot) { + return new Client(prot); + } + @Override + public Client getClient(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) { + return new Client(iprot, oprot); + } + } + + public Client(org.apache.thrift.protocol.TProtocol prot) + { + super(prot, prot); + } + + public Client(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) { + super(iprot, oprot); + } + + @Override + public void gracefulShutdown(org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException + { + send_gracefulShutdown(credentials); + } + + public void send_gracefulShutdown(org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException + { + gracefulShutdown_args args = new gracefulShutdown_args(); + args.setCredentials(credentials); + sendBaseOneway("gracefulShutdown", args); + } + + } + public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface { + public static class Factory implements org.apache.thrift.async.TAsyncClientFactory { + private org.apache.thrift.async.TAsyncClientManager clientManager; + private org.apache.thrift.protocol.TProtocolFactory protocolFactory; + public Factory(org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.protocol.TProtocolFactory protocolFactory) { + this.clientManager = clientManager; + this.protocolFactory = protocolFactory; + } + @Override + public AsyncClient getAsyncClient(org.apache.thrift.transport.TNonblockingTransport transport) { + return new AsyncClient(protocolFactory, clientManager, transport); + } + } + + public AsyncClient(org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.transport.TNonblockingTransport transport) { + super(protocolFactory, clientManager, transport); + } + + @Override + public void gracefulShutdown(org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + gracefulShutdown_call method_call = new gracefulShutdown_call(credentials, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class gracefulShutdown_call extends org.apache.thrift.async.TAsyncMethodCall { + private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; + public gracefulShutdown_call(org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, true); + this.credentials = credentials; + } + + @Override + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("gracefulShutdown", org.apache.thrift.protocol.TMessageType.ONEWAY, 0)); + gracefulShutdown_args args = new gracefulShutdown_args(); + args.setCredentials(credentials); + args.write(prot); + prot.writeMessageEnd(); + } + + @Override + public Void getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new java.lang.IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return null; + } + } + + } + + public static class Processor extends org.apache.thrift.TBaseProcessor implements org.apache.thrift.TProcessor { + private static final org.slf4j.Logger _LOGGER = org.slf4j.LoggerFactory.getLogger(Processor.class.getName()); + public Processor(I iface) { + super(iface, getProcessMap(new java.util.HashMap>())); + } + + protected Processor(I iface, java.util.Map> processMap) { + super(iface, getProcessMap(processMap)); + } + + private static java.util.Map> getProcessMap(java.util.Map> processMap) { + processMap.put("gracefulShutdown", new gracefulShutdown()); + return processMap; + } + + public static class gracefulShutdown extends org.apache.thrift.ProcessFunction { + public gracefulShutdown() { + super("gracefulShutdown"); + } + + @Override + public gracefulShutdown_args getEmptyArgsInstance() { + return new gracefulShutdown_args(); + } + + @Override + protected boolean isOneway() { + return true; + } + + @Override + protected boolean rethrowUnhandledExceptions() { + return false; + } + + @Override + public org.apache.thrift.TBase getResult(I iface, gracefulShutdown_args args) throws org.apache.thrift.TException { + iface.gracefulShutdown(args.credentials); + return null; + } + } + + } + + public static class AsyncProcessor extends org.apache.thrift.TBaseAsyncProcessor { + private static final org.slf4j.Logger _LOGGER = org.slf4j.LoggerFactory.getLogger(AsyncProcessor.class.getName()); + public AsyncProcessor(I iface) { + super(iface, getProcessMap(new java.util.HashMap>())); + } + + protected AsyncProcessor(I iface, java.util.Map> processMap) { + super(iface, getProcessMap(processMap)); + } + + private static java.util.Map> getProcessMap(java.util.Map> processMap) { + processMap.put("gracefulShutdown", new gracefulShutdown()); + return processMap; + } + + public static class gracefulShutdown extends org.apache.thrift.AsyncProcessFunction { + public gracefulShutdown() { + super("gracefulShutdown"); + } + + @Override + public gracefulShutdown_args getEmptyArgsInstance() { + return new gracefulShutdown_args(); + } + + @Override + public org.apache.thrift.async.AsyncMethodCallback getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new org.apache.thrift.async.AsyncMethodCallback() { + @Override + public void onComplete(Void o) { + } + @Override + public void onError(java.lang.Exception e) { + if (e instanceof org.apache.thrift.transport.TTransportException) { + _LOGGER.error("TTransportException inside handler", e); + fb.close(); + } else { + _LOGGER.error("Exception inside oneway handler", e); + } + } + }; + } + + @Override + protected boolean isOneway() { + return true; + } + + @Override + public void start(I iface, gracefulShutdown_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + iface.gracefulShutdown(args.credentials,resultHandler); + } + } + + } + + @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) + public static class gracefulShutdown_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("gracefulShutdown_args"); + + private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new gracefulShutdown_argsStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new gracefulShutdown_argsTupleSchemeFactory(); + + public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + CREDENTIALS((short)1, "credentials"); + + private static final java.util.Map byName = new java.util.HashMap(); + + static { + for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + @org.apache.thrift.annotation.Nullable + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // CREDENTIALS + return CREDENTIALS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + @org.apache.thrift.annotation.Nullable + public static _Fields findByName(java.lang.String name) { + return byName.get(name); + } + + private final short _thriftId; + private final java.lang.String _fieldName; + + _Fields(short thriftId, java.lang.String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + @Override + public short getThriftFieldId() { + return _thriftId; + } + + @Override + public java.lang.String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class))); + metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(gracefulShutdown_args.class, metaDataMap); + } + + public gracefulShutdown_args() { + } + + public gracefulShutdown_args( + org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) + { + this(); + this.credentials = credentials; + } + + /** + * Performs a deep copy on other. + */ + public gracefulShutdown_args(gracefulShutdown_args other) { + if (other.isSetCredentials()) { + this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials); + } + } + + @Override + public gracefulShutdown_args deepCopy() { + return new gracefulShutdown_args(this); + } + + @Override + public void clear() { + this.credentials = null; + } + + @org.apache.thrift.annotation.Nullable + public org.apache.accumulo.core.securityImpl.thrift.TCredentials getCredentials() { + return this.credentials; + } + + public gracefulShutdown_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) { + this.credentials = credentials; + return this; + } + + public void unsetCredentials() { + this.credentials = null; + } + + /** Returns true if field credentials is set (has been assigned a value) and false otherwise */ + public boolean isSetCredentials() { + return this.credentials != null; + } + + public void setCredentialsIsSet(boolean value) { + if (!value) { + this.credentials = null; + } + } + + @Override + public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) { + switch (field) { + case CREDENTIALS: + if (value == null) { + unsetCredentials(); + } else { + setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value); + } + break; + + } + } + + @org.apache.thrift.annotation.Nullable + @Override + public java.lang.Object getFieldValue(_Fields field) { + switch (field) { + case CREDENTIALS: + return getCredentials(); + + } + throw new java.lang.IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + @Override + public boolean isSet(_Fields field) { + if (field == null) { + throw new java.lang.IllegalArgumentException(); + } + + switch (field) { + case CREDENTIALS: + return isSetCredentials(); + } + throw new java.lang.IllegalStateException(); + } + + @Override + public boolean equals(java.lang.Object that) { + if (that instanceof gracefulShutdown_args) + return this.equals((gracefulShutdown_args)that); + return false; + } + + public boolean equals(gracefulShutdown_args that) { + if (that == null) + return false; + if (this == that) + return true; + + boolean this_present_credentials = true && this.isSetCredentials(); + boolean that_present_credentials = true && that.isSetCredentials(); + if (this_present_credentials || that_present_credentials) { + if (!(this_present_credentials && that_present_credentials)) + return false; + if (!this.credentials.equals(that.credentials)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + int hashCode = 1; + + hashCode = hashCode * 8191 + ((isSetCredentials()) ? 131071 : 524287); + if (isSetCredentials()) + hashCode = hashCode * 8191 + credentials.hashCode(); + + return hashCode; + } + + @Override + public int compareTo(gracefulShutdown_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = java.lang.Boolean.compare(isSetCredentials(), other.isSetCredentials()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetCredentials()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, other.credentials); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + @org.apache.thrift.annotation.Nullable + @Override + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + scheme(iprot).read(iprot, this); + } + + @Override + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + scheme(oprot).write(oprot, this); + } + + @Override + public java.lang.String toString() { + java.lang.StringBuilder sb = new java.lang.StringBuilder("gracefulShutdown_args("); + boolean first = true; + + sb.append("credentials:"); + if (this.credentials == null) { + sb.append("null"); + } else { + sb.append(this.credentials); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (credentials != null) { + credentials.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class gracefulShutdown_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + @Override + public gracefulShutdown_argsStandardScheme getScheme() { + return new gracefulShutdown_argsStandardScheme(); + } + } + + private static class gracefulShutdown_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { + + @Override + public void read(org.apache.thrift.protocol.TProtocol iprot, gracefulShutdown_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // CREDENTIALS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(); + struct.credentials.read(iprot); + struct.setCredentialsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + + // check for required fields of primitive type, which can't be checked in the validate method + struct.validate(); + } + + @Override + public void write(org.apache.thrift.protocol.TProtocol oprot, gracefulShutdown_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.credentials != null) { + oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC); + struct.credentials.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class gracefulShutdown_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + @Override + public gracefulShutdown_argsTupleScheme getScheme() { + return new gracefulShutdown_argsTupleScheme(); + } + } + + private static class gracefulShutdown_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, gracefulShutdown_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot; + java.util.BitSet optionals = new java.util.BitSet(); + if (struct.isSetCredentials()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetCredentials()) { + struct.credentials.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, gracefulShutdown_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; + java.util.BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(); + struct.credentials.read(iprot); + struct.setCredentialsIsSet(true); + } + } + } + + private static S scheme(org.apache.thrift.protocol.TProtocol proto) { + return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme(); + } + } + + private static void unusedMethod() {} +} diff --git a/core/src/main/thrift/manager.thrift b/core/src/main/thrift/manager.thrift index b601bca6847..139e55fc1e4 100644 --- a/core/src/main/thrift/manager.thrift +++ b/core/src/main/thrift/manager.thrift @@ -290,6 +290,15 @@ service ManagerClientService { 1:client.ThriftSecurityException sec 2:client.ThriftNotActiveServiceException tnase ) + + void tabletServerStopping( + 1:trace.TInfo tinfo + 2:security.TCredentials credentials + 3:string tabletServer + ) throws ( + 1:client.ThriftSecurityException sec + 2:client.ThriftNotActiveServiceException tnase + ) void setSystemProperty( 1:trace.TInfo tinfo diff --git a/core/src/main/thrift/process.thrift b/core/src/main/thrift/process.thrift new file mode 100644 index 00000000000..c35984148b2 --- /dev/null +++ b/core/src/main/thrift/process.thrift @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +namespace java org.apache.accumulo.core.process.thrift +namespace cpp org.apache.accumulo.core.process.thrift + +include "security.thrift" + +service ServerProcessService { + + oneway void gracefulShutdown( + 1:security.TCredentials credentials + ) + +} + \ No newline at end of file diff --git a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterControl.java b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterControl.java index b0c46aadf97..e34a8ae297a 100644 --- a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterControl.java +++ b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterControl.java @@ -28,6 +28,7 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Optional; +import java.util.Set; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -528,4 +529,69 @@ public void kill(ServerType server, String hostname) throws IOException { stop(server, hostname); } + public void refreshProcesses(ServerType type) { + switch (type) { + case COMPACTION_COORDINATOR: + if (!coordinatorProcess.isAlive()) { + coordinatorProcess = null; + } + break; + case COMPACTOR: + compactorProcesses.removeIf(process -> !process.isAlive()); + break; + case GARBAGE_COLLECTOR: + if (!gcProcess.isAlive()) { + gcProcess = null; + } + break; + case MANAGER: + case MASTER: + if (!managerProcess.isAlive()) { + managerProcess = null; + } + break; + case MONITOR: + if (!monitor.isAlive()) { + monitor = null; + } + break; + case SCAN_SERVER: + scanServerProcesses.removeIf(process -> !process.isAlive()); + break; + case TABLET_SERVER: + tabletServerProcesses.removeIf(process -> !process.isAlive()); + break; + case ZOOKEEPER: + if (!zooKeeperProcess.isAlive()) { + zooKeeperProcess = null; + } + break; + default: + throw new IllegalArgumentException("Unhandled type: " + type); + } + } + + public Set getProcesses(ServerType type) { + switch (type) { + case COMPACTION_COORDINATOR: + return coordinatorProcess == null ? Set.of() : Set.of(coordinatorProcess); + case COMPACTOR: + return Set.copyOf(compactorProcesses); + case GARBAGE_COLLECTOR: + return gcProcess == null ? Set.of() : Set.of(gcProcess); + case MANAGER: + case MASTER: + return managerProcess == null ? Set.of() : Set.of(managerProcess); + case MONITOR: + return monitor == null ? Set.of() : Set.of(monitor); + case SCAN_SERVER: + return Set.copyOf(scanServerProcesses); + case TABLET_SERVER: + return Set.copyOf(tabletServerProcesses); + case ZOOKEEPER: + return zooKeeperProcess == null ? Set.of() : Set.of(zooKeeperProcess); + default: + throw new IllegalArgumentException("Unhandled type: " + type); + } + } } diff --git a/server/base/src/main/java/org/apache/accumulo/server/AbstractServer.java b/server/base/src/main/java/org/apache/accumulo/server/AbstractServer.java index c65314a0f12..69378845e36 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/AbstractServer.java +++ b/server/base/src/main/java/org/apache/accumulo/server/AbstractServer.java @@ -21,14 +21,18 @@ import java.util.Objects; import java.util.OptionalInt; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.classloader.ClassLoaderUtil; +import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException; import org.apache.accumulo.core.conf.AccumuloConfiguration; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.fate.zookeeper.ServiceLock; import org.apache.accumulo.core.metrics.MetricsProducer; +import org.apache.accumulo.core.process.thrift.ServerProcessService; +import org.apache.accumulo.core.securityImpl.thrift.TCredentials; import org.apache.accumulo.core.trace.TraceUtil; import org.apache.accumulo.core.util.Halt; import org.apache.accumulo.core.util.threads.Threads; @@ -41,7 +45,8 @@ import io.micrometer.core.instrument.MeterRegistry; -public abstract class AbstractServer implements AutoCloseable, MetricsProducer, Runnable { +public abstract class AbstractServer + implements AutoCloseable, MetricsProducer, Runnable, ServerProcessService.Iface { private final ServerContext context; protected final String applicationName; @@ -52,6 +57,8 @@ public abstract class AbstractServer implements AutoCloseable, MetricsProducer, private volatile long idlePeriodStartNanos = 0L; private volatile Thread serverThread; private volatile Thread verificationThread; + private final AtomicBoolean shutdownRequested = new AtomicBoolean(false); + private final AtomicBoolean shutdownComplete = new AtomicBoolean(false); protected AbstractServer(String appName, ServerOpts opts, String[] args) { this.log = LoggerFactory.getLogger(getClass().getName()); @@ -102,8 +109,62 @@ protected void updateIdleStatus(boolean isIdle) { } } + @Override + public void gracefulShutdown(TCredentials credentials) { + + try { + if (!context.getSecurityOperation().canPerformSystemActions(credentials)) { + log.warn("Ignoring shutdown request, user " + credentials.getPrincipal() + + " does not have the appropriate permissions."); + } + } catch (ThriftSecurityException e) { + log.error( + "Error trying to determine if user has permissions to shutdown server, ignoring request", + e); + return; + } + + if (shutdownRequested.compareAndSet(false, true)) { + // Don't interrupt the server thread, that will cause + // IO operations to fail as the servers are finishing + // their work. + log.info("Graceful shutdown initiated."); + } else { + log.warn("Graceful shutdown previously requested."); + } + } + + public boolean isShutdownRequested() { + return shutdownRequested.get(); + } + + public AtomicBoolean getShutdownComplete() { + return shutdownComplete; + } + /** - * Run this server in a main thread + * Run this server in a main thread. The server's run method should set up the server, then wait + * on isShutdownRequested() to return false, like so: + * + *
+   * public void run() {
+   *   // setup server and start threads
+   *   while (!isShutdownRequested()) {
+   *     if (Thread.currentThread().isInterrupted()) {
+   *       LOG.info("Server process thread has been interrupted, shutting down");
+   *       break;
+   *     }
+   *     try {
+   *       // sleep or other things
+   *     } catch (InterruptedException e) {
+   *       gracefulShutdown();
+   *     }
+   *   }
+   *   // shut down server
+   *   getShutdownComplete().set(true);
+   *   ServiceLock.unlock(serverLock);
+   * }
+   * 
*/ public void runServer() throws Exception { final AtomicReference err = new AtomicReference<>(); @@ -115,6 +176,7 @@ public void runServer() throws Exception { verificationThread.interrupt(); verificationThread.join(); } + log.info(getClass().getSimpleName() + " process shut down."); Throwable thrown = err.get(); if (thrown != null) { if (thrown instanceof Error) { diff --git a/server/base/src/main/java/org/apache/accumulo/server/manager/LiveTServerSet.java b/server/base/src/main/java/org/apache/accumulo/server/manager/LiveTServerSet.java index 52733792b76..572c5954c15 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/manager/LiveTServerSet.java +++ b/server/base/src/main/java/org/apache/accumulo/server/manager/LiveTServerSet.java @@ -28,6 +28,7 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; import org.apache.accumulo.core.Constants; @@ -242,6 +243,9 @@ static class TServerInfo { // as above, indexed by TServerInstance private final Map currentInstances = new HashMap<>(); + private final ConcurrentHashMap serversShuttingDown = + new ConcurrentHashMap<>(); + // The set of entries in zookeeper without locks, and the first time each was noticed private final Map locklessServers = new HashMap<>(); @@ -264,6 +268,19 @@ public synchronized void startListeningForTabletServerChanges() { .scheduleWithFixedDelay(this::scanServers, 0, 5000, TimeUnit.MILLISECONDS)); } + public void tabletServerShuttingDown(String server) { + + TServerInfo info = null; + synchronized (this) { + info = current.get(server); + } + if (info != null) { + serversShuttingDown.put(server, info); + } else { + log.info("Tablet Server reported it's shutting down, but not in list of current servers"); + } + } + public synchronized void scanServers() { try { final Set updates = new HashSet<>(); @@ -312,6 +329,7 @@ private synchronized void checkServer(final Set updates, doomed.add(info.instance); current.remove(zPath); currentInstances.remove(info.instance); + serversShuttingDown.remove(zPath); } Long firstSeen = locklessServers.get(zPath); @@ -389,7 +407,9 @@ public synchronized TServerConnection getConnection(TServerInstance server) { } public synchronized Set getCurrentServers() { - return new HashSet<>(currentInstances.keySet()); + Set current = currentInstances.keySet(); + serversShuttingDown.values().forEach(tsi -> current.remove(tsi.instance)); + return new HashSet<>(current); } public synchronized int size() { diff --git a/server/base/src/main/java/org/apache/accumulo/server/rpc/ThriftProcessorTypes.java b/server/base/src/main/java/org/apache/accumulo/server/rpc/ThriftProcessorTypes.java index bb05a4608d6..6b2aa9a2c84 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/rpc/ThriftProcessorTypes.java +++ b/server/base/src/main/java/org/apache/accumulo/server/rpc/ThriftProcessorTypes.java @@ -24,6 +24,7 @@ import org.apache.accumulo.core.gc.thrift.GCMonitorService; import org.apache.accumulo.core.manager.thrift.FateService; import org.apache.accumulo.core.manager.thrift.ManagerClientService; +import org.apache.accumulo.core.process.thrift.ServerProcessService; import org.apache.accumulo.core.replication.thrift.ReplicationCoordinator; import org.apache.accumulo.core.replication.thrift.ReplicationServicer; import org.apache.accumulo.core.rpc.clients.ThriftClientTypes; @@ -95,34 +96,52 @@ public > TProcessor getTProcessor( public static final ThriftProcessorTypes TABLET_SERVER_SCAN = new ThriftProcessorTypes<>(ThriftClientTypes.TABLET_SCAN); - public static TMultiplexedProcessor getCompactorTProcessor(CompactorService.Iface serviceHandler, + public static final ThriftProcessorTypes SERVER_PROCESS = + new ThriftProcessorTypes<>(ThriftClientTypes.SERVER_PROCESS); + + public static TMultiplexedProcessor getCompactorTProcessor( + ServerProcessService.Iface processHandler, CompactorService.Iface serviceHandler, ServerContext context) { TMultiplexedProcessor muxProcessor = new TMultiplexedProcessor(); + muxProcessor.registerProcessor(SERVER_PROCESS.getServiceName(), + SERVER_PROCESS.getTProcessor(ServerProcessService.Processor.class, + ServerProcessService.Iface.class, processHandler, context)); muxProcessor.registerProcessor(COMPACTOR.getServiceName(), COMPACTOR.getTProcessor( CompactorService.Processor.class, CompactorService.Iface.class, serviceHandler, context)); return muxProcessor; } public static TMultiplexedProcessor getCoordinatorTProcessor( - CompactionCoordinatorService.Iface serviceHandler, ServerContext context) { + ServerProcessService.Iface processHandler, CompactionCoordinatorService.Iface serviceHandler, + ServerContext context) { TMultiplexedProcessor muxProcessor = new TMultiplexedProcessor(); + muxProcessor.registerProcessor(SERVER_PROCESS.getServiceName(), + SERVER_PROCESS.getTProcessor(ServerProcessService.Processor.class, + ServerProcessService.Iface.class, processHandler, context)); muxProcessor.registerProcessor(COORDINATOR.getServiceName(), COORDINATOR.getTProcessor(CompactionCoordinatorService.Processor.class, CompactionCoordinatorService.Iface.class, serviceHandler, context)); return muxProcessor; } - public static TMultiplexedProcessor getGcTProcessor(GCMonitorService.Iface serviceHandler, - ServerContext context) { + public static TMultiplexedProcessor getGcTProcessor(ServerProcessService.Iface processHandler, + GCMonitorService.Iface serviceHandler, ServerContext context) { TMultiplexedProcessor muxProcessor = new TMultiplexedProcessor(); + muxProcessor.registerProcessor(SERVER_PROCESS.getServiceName(), + SERVER_PROCESS.getTProcessor(ServerProcessService.Processor.class, + ServerProcessService.Iface.class, processHandler, context)); muxProcessor.registerProcessor(GC.getServiceName(), GC.getTProcessor( GCMonitorService.Processor.class, GCMonitorService.Iface.class, serviceHandler, context)); return muxProcessor; } - public static TMultiplexedProcessor getManagerTProcessor(FateService.Iface fateServiceHandler, + public static TMultiplexedProcessor getManagerTProcessor( + ServerProcessService.Iface processHandler, FateService.Iface fateServiceHandler, ManagerClientService.Iface managerServiceHandler, ServerContext context) { TMultiplexedProcessor muxProcessor = new TMultiplexedProcessor(); + muxProcessor.registerProcessor(SERVER_PROCESS.getServiceName(), + SERVER_PROCESS.getTProcessor(ServerProcessService.Processor.class, + ServerProcessService.Iface.class, processHandler, context)); muxProcessor.registerProcessor(FATE.getServiceName(), FATE.getTProcessor( FateService.Processor.class, FateService.Iface.class, fateServiceHandler, context)); muxProcessor.registerProcessor(MANAGER.getServiceName(), @@ -149,19 +168,27 @@ public static TMultiplexedProcessor getReplicationClientTProcessor( return muxProcessor; } - public static TMultiplexedProcessor - getScanServerTProcessor(TabletScanClientService.Iface tserverHandler, ServerContext context) { + public static TMultiplexedProcessor getScanServerTProcessor( + ServerProcessService.Iface processHandler, TabletScanClientService.Iface tserverHandler, + ServerContext context) { TMultiplexedProcessor muxProcessor = new TMultiplexedProcessor(); + muxProcessor.registerProcessor(SERVER_PROCESS.getServiceName(), + SERVER_PROCESS.getTProcessor(ServerProcessService.Processor.class, + ServerProcessService.Iface.class, processHandler, context)); muxProcessor.registerProcessor(TABLET_SERVER_SCAN.getServiceName(), TABLET_SERVER_SCAN.getTProcessor(TabletScanClientService.Processor.class, TabletScanClientService.Iface.class, tserverHandler, context)); return muxProcessor; } - public static TMultiplexedProcessor getTabletServerTProcessor(ClientServiceHandler clientHandler, + public static TMultiplexedProcessor getTabletServerTProcessor( + ServerProcessService.Iface processHandler, ClientServiceHandler clientHandler, TabletClientService.Iface tserverHandler, TabletScanClientService.Iface tserverScanHandler, ServerContext context) { TMultiplexedProcessor muxProcessor = new TMultiplexedProcessor(); + muxProcessor.registerProcessor(SERVER_PROCESS.getServiceName(), + SERVER_PROCESS.getTProcessor(ServerProcessService.Processor.class, + ServerProcessService.Iface.class, processHandler, context)); muxProcessor.registerProcessor(CLIENT.getServiceName(), CLIENT.getTProcessor( ClientService.Processor.class, ClientService.Iface.class, clientHandler, context)); muxProcessor.registerProcessor(TABLET_SERVER.getServiceName(), diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java b/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java index a5e866c218b..e3fb627dbaa 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java @@ -35,6 +35,7 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.Objects; import java.util.Set; import java.util.SortedSet; import java.util.TreeMap; @@ -62,6 +63,7 @@ import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.manager.thrift.FateService; import org.apache.accumulo.core.metadata.MetadataTable; +import org.apache.accumulo.core.process.thrift.ServerProcessService; import org.apache.accumulo.core.rpc.ThriftUtil; import org.apache.accumulo.core.rpc.clients.ThriftClientTypes; import org.apache.accumulo.core.security.Authorizations; @@ -79,6 +81,7 @@ import org.apache.accumulo.server.security.SecurityUtil; import org.apache.accumulo.server.util.fateCommand.FateSummaryReport; import org.apache.accumulo.start.spi.KeywordExecutable; +import org.apache.thrift.TException; import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -101,6 +104,13 @@ private static class SubCommandOpts { public boolean help = false; } + @Parameters( + commandDescription = "signal the server process to shutdown normally, finishing anything it might be working on, but not starting any new tasks") + static class GracefulShutdownCommand extends SubCommandOpts { + @Parameter(required = true, names = {"-a", "--address"}, description = "") + String address = null; + } + @Parameters(commandDescription = "stop the tablet server on the given hosts") static class StopCommand extends SubCommandOpts { @Parameter(names = {"-f", "--force"}, @@ -311,6 +321,9 @@ public void execute(final String[] args) { FateOpsCommand fateOpsCommand = new FateOpsCommand(); cl.addCommand("fate", fateOpsCommand); + GracefulShutdownCommand gracefulShutdownCommand = new GracefulShutdownCommand(); + cl.addCommand("signalShutdown", gracefulShutdownCommand); + ListInstancesCommand listInstancesOpts = new ListInstancesCommand(); cl.addCommand("listInstances", listInstancesOpts); @@ -401,6 +414,8 @@ public void execute(final String[] args) { } else if (cl.getParsedCommand().equals("stop")) { stopTabletServer(context, stopOpts.args, stopOpts.force); + } else if (cl.getParsedCommand().equals("signalShutdown")) { + signalGracefulShutdown(context, gracefulShutdownCommand.address); } else if (cl.getParsedCommand().equals("dumpConfig")) { printConfig(context, dumpConfigCommand); } else if (cl.getParsedCommand().equals("volumes")) { @@ -548,6 +563,25 @@ private static void stopServer(final ClientContext context, final boolean tablet client -> client.shutdown(TraceUtil.traceInfo(), context.rpcCreds(), tabletServersToo)); } + // Visible for tests + public static void signalGracefulShutdown(final ClientContext context, String address) { + + Objects.requireNonNull(address, "address not set"); + final HostAndPort hp = HostAndPort.fromString(address); + ServerProcessService.Client client = null; + try { + client = ThriftClientTypes.SERVER_PROCESS.getServerProcessConnection(context, log, + hp.getHost(), hp.getPort()); + client.gracefulShutdown(context.rpcCreds()); + } catch (TException e) { + throw new RuntimeException("Error invoking graceful shutdown for server: " + hp, e); + } finally { + if (client != null) { + ThriftUtil.returnClient(client, context); + } + } + } + private static void stopTabletServer(final ClientContext context, List servers, final boolean force) throws AccumuloException, AccumuloSecurityException { if (context.getManagerLocations().isEmpty()) { diff --git a/server/base/src/main/java/org/apache/accumulo/server/zookeeper/DistributedWorkQueue.java b/server/base/src/main/java/org/apache/accumulo/server/zookeeper/DistributedWorkQueue.java index 595a04a8f18..c916090982d 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/zookeeper/DistributedWorkQueue.java +++ b/server/base/src/main/java/org/apache/accumulo/server/zookeeper/DistributedWorkQueue.java @@ -36,6 +36,7 @@ import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeExistsPolicy; import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeMissingPolicy; import org.apache.accumulo.core.util.threads.ThreadPools; +import org.apache.accumulo.server.AbstractServer; import org.apache.accumulo.server.ServerContext; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.NodeExistsException; @@ -61,7 +62,7 @@ public class DistributedWorkQueue { private ThreadPoolExecutor threadPool; private final ZooReaderWriter zoo; private final String path; - private final ServerContext context; + private final AbstractServer server; private final long timerInitialDelay; private final long timerPeriod; @@ -80,6 +81,11 @@ private void lookForWork(final Processor processor, List children) { try { for (final String child : children) { + // Don't accept work if the server is shutting down + if (server.isShutdownRequested()) { + return; + } + if (child.equals(LOCKS_NODE)) { continue; } @@ -169,23 +175,27 @@ public interface Processor { void process(String workID, byte[] data); } - public DistributedWorkQueue(String path, AccumuloConfiguration config, ServerContext context) { + public DistributedWorkQueue(String path, AccumuloConfiguration config, AbstractServer server) { // Preserve the old delay and period - this(path, config, context, random.nextInt(toIntExact(MINUTES.toMillis(1))), + this(path, config, server, random.nextInt(toIntExact(MINUTES.toMillis(1))), MINUTES.toMillis(1)); } - public DistributedWorkQueue(String path, AccumuloConfiguration config, ServerContext context, + public DistributedWorkQueue(String path, AccumuloConfiguration config, AbstractServer server, long timerInitialDelay, long timerPeriod) { this.path = path; - this.context = context; + this.server = server; this.timerInitialDelay = timerInitialDelay; this.timerPeriod = timerPeriod; - zoo = context.getZooReaderWriter(); + zoo = server.getContext().getZooReaderWriter(); } public ServerContext getContext() { - return context; + return server.getContext(); + } + + public AbstractServer getServer() { + return server; } public void startProcessing(final Processor processor, ThreadPoolExecutor executorService) @@ -225,7 +235,7 @@ public void process(WatchedEvent event) { // Add a little jitter to avoid all the tservers slamming zookeeper at once ThreadPools.watchCriticalScheduledTask( - context.getScheduledExecutor().scheduleWithFixedDelay(new Runnable() { + server.getContext().getScheduledExecutor().scheduleWithFixedDelay(new Runnable() { @Override public void run() { log.debug("Looking for work in {}", path); diff --git a/server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java b/server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java index 7d853b48089..5aad9192061 100644 --- a/server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java +++ b/server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java @@ -65,6 +65,7 @@ import org.apache.accumulo.core.metadata.schema.ExternalCompactionId; import org.apache.accumulo.core.metadata.schema.TabletMetadata; import org.apache.accumulo.core.metrics.MetricsInfo; +import org.apache.accumulo.core.process.thrift.ServerProcessService; import org.apache.accumulo.core.rpc.ThriftUtil; import org.apache.accumulo.core.rpc.clients.ThriftClientTypes; import org.apache.accumulo.core.securityImpl.thrift.TCredentials; @@ -75,7 +76,6 @@ import org.apache.accumulo.core.trace.TraceUtil; import org.apache.accumulo.core.trace.thrift.TInfo; import org.apache.accumulo.core.util.HostAndPort; -import org.apache.accumulo.core.util.UtilWaitThread; import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil; import org.apache.accumulo.core.util.compaction.RunningCompaction; import org.apache.accumulo.core.util.threads.ThreadPools; @@ -103,8 +103,8 @@ import io.micrometer.core.instrument.Tag; -public class CompactionCoordinator extends AbstractServer - implements CompactionCoordinatorService.Iface, LiveTServerSet.Listener { +public class CompactionCoordinator extends AbstractServer implements + CompactionCoordinatorService.Iface, LiveTServerSet.Listener, ServerProcessService.Iface { private static final Logger LOG = LoggerFactory.getLogger(CompactionCoordinator.class); @@ -137,9 +137,6 @@ public class CompactionCoordinator extends AbstractServer private ServiceLock coordinatorLock; - // Exposed for tests - protected volatile Boolean shutdown = false; - private final ScheduledThreadPoolExecutor schedExecutor; private final LoadingCache compactorCounts; @@ -222,9 +219,10 @@ protected void getCoordinatorLock(HostAndPort clientAddress) coordinatorLock = new ServiceLock(getContext().getZooReaderWriter().getZooKeeper(), ServiceLock.path(lockPath), zooLockUUID); + HAServiceLockWatcher coordinatorLockWatcher = + new HAServiceLockWatcher("coordinator", () -> getShutdownComplete().get()); while (true) { - HAServiceLockWatcher coordinatorLockWatcher = new HAServiceLockWatcher("coordinator"); coordinatorLock.lock(coordinatorLockWatcher, coordinatorClientAddress.getBytes(UTF_8)); coordinatorLockWatcher.waitForChange(); @@ -247,7 +245,7 @@ protected void getCoordinatorLock(HostAndPort clientAddress) * @throws UnknownHostException host unknown */ protected ServerAddress startCoordinatorClientService() throws UnknownHostException { - var processor = ThriftProcessorTypes.getCoordinatorTProcessor(this, getContext()); + var processor = ThriftProcessorTypes.getCoordinatorTProcessor(this, this, getContext()); @SuppressWarnings("deprecation") var maxMessageSizeProperty = getConfiguration().resolve(Property.RPC_MAX_MESSAGE_SIZE, Property.GENERAL_MAX_MESSAGE_SIZE); @@ -310,31 +308,51 @@ public void run() { startDeadCompactionDetector(); LOG.info("Starting loop to check tservers for compaction summaries"); - while (!shutdown) { - long start = System.currentTimeMillis(); + while (!isShutdownRequested()) { + if (Thread.currentThread().isInterrupted()) { + LOG.info("Server process thread has been interrupted, shutting down"); + break; + } + try { + long start = System.currentTimeMillis(); - updateSummaries(); + updateSummaries(); - long now = System.currentTimeMillis(); + long now = System.currentTimeMillis(); - Map> idleCompactors = getIdleCompactors(); - TIME_COMPACTOR_LAST_CHECKED.forEach((queue, lastCheckTime) -> { - if ((now - lastCheckTime) > getMissingCompactorWarningTime() - && QUEUE_SUMMARIES.isCompactionsQueued(queue) && idleCompactors.containsKey(queue)) { - LOG.warn("No compactors have checked in with coordinator for queue {} in {}ms", queue, - getMissingCompactorWarningTime()); - } - }); + Map> idleCompactors = getIdleCompactors(); + TIME_COMPACTOR_LAST_CHECKED.forEach((queue, lastCheckTime) -> { + if ((now - lastCheckTime) > getMissingCompactorWarningTime() + && QUEUE_SUMMARIES.isCompactionsQueued(queue) && idleCompactors.containsKey(queue)) { + LOG.warn("No compactors have checked in with coordinator for queue {} in {}ms", queue, + getMissingCompactorWarningTime()); + } + }); - long checkInterval = getTServerCheckInterval(); - long duration = (System.currentTimeMillis() - start); - if (checkInterval - duration > 0) { - LOG.debug("Waiting {}ms for next tserver check", (checkInterval - duration)); - UtilWaitThread.sleep(checkInterval - duration); + long checkInterval = getTServerCheckInterval(); + long duration = (System.currentTimeMillis() - start); + if (checkInterval - duration > 0) { + LOG.debug("Waiting {}ms for next tserver check", (checkInterval - duration)); + Thread.sleep(checkInterval - duration); + } + } catch (InterruptedException e) { + LOG.info("Interrupt Exception received, shutting down"); + gracefulShutdown(getContext().rpcCreds()); } } - LOG.info("Shutting down"); + LOG.debug("Stopping Thrift Servers"); + if (coordinatorAddress.server != null) { + coordinatorAddress.server.stop(); + } + getShutdownComplete().set(true); + LOG.info("stop requested. exiting ... "); + try { + coordinatorLock.unlock(); + } catch (Exception e) { + LOG.warn("Failed to release Coordinator lock", e); + } + } private Map> getIdleCompactors() { diff --git a/server/compaction-coordinator/src/test/java/org/apache/accumulo/coordinator/CompactionCoordinatorTest.java b/server/compaction-coordinator/src/test/java/org/apache/accumulo/coordinator/CompactionCoordinatorTest.java index 9c6d0064796..8c420642749 100644 --- a/server/compaction-coordinator/src/test/java/org/apache/accumulo/coordinator/CompactionCoordinatorTest.java +++ b/server/compaction-coordinator/src/test/java/org/apache/accumulo/coordinator/CompactionCoordinatorTest.java @@ -38,6 +38,7 @@ import java.util.TreeSet; import java.util.UUID; import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.atomic.AtomicBoolean; import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException; import org.apache.accumulo.core.compaction.thrift.TExternalCompaction; @@ -50,6 +51,7 @@ import org.apache.accumulo.core.metadata.TServerInstance; import org.apache.accumulo.core.metadata.schema.ExternalCompactionId; import org.apache.accumulo.core.metrics.MetricsInfo; +import org.apache.accumulo.core.process.thrift.ServerProcessService; import org.apache.accumulo.core.rpc.ThriftUtil; import org.apache.accumulo.core.securityImpl.thrift.TCredentials; import org.apache.accumulo.core.tabletserver.thrift.TCompactionQueueSummary; @@ -91,11 +93,13 @@ "com.sun.org.apache.xerces.*"}) public class CompactionCoordinatorTest { - public class TestCoordinator extends CompactionCoordinator { + public class TestCoordinator extends CompactionCoordinator implements ServerProcessService.Iface { private final ServerContext context; private final ServerAddress client; private final TabletClientService.Client tabletServerClient; + private final AtomicBoolean shutdown = new AtomicBoolean(false); + private final AtomicBoolean shutdownComplete = new AtomicBoolean(false); private Set metadataCompactionIds = null; @@ -116,10 +120,24 @@ protected void startDeadCompactionDetector() {} @Override protected long getTServerCheckInterval() { - this.shutdown = true; + gracefulShutdown(null); return 0L; } + @Override + public void gracefulShutdown(TCredentials credentials) { + shutdown.set(true); + } + + @Override + public boolean isShutdownRequested() { + return shutdown.get(); + } + + public AtomicBoolean getShutdownComplete() { + return shutdownComplete; + } + @Override protected void startCompactionCleaner(ScheduledThreadPoolExecutor schedExecutor) {} diff --git a/server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java b/server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java index f95fad41ab2..109f287ffa1 100644 --- a/server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java +++ b/server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java @@ -81,6 +81,7 @@ import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType; import org.apache.accumulo.core.metrics.MetricsInfo; import org.apache.accumulo.core.metrics.MetricsProducer; +import org.apache.accumulo.core.process.thrift.ServerProcessService; import org.apache.accumulo.core.rpc.ThriftUtil; import org.apache.accumulo.core.rpc.clients.ThriftClientTypes; import org.apache.accumulo.core.securityImpl.thrift.TCredentials; @@ -127,7 +128,8 @@ import io.micrometer.core.instrument.MeterRegistry; import io.micrometer.core.instrument.Tag; -public class Compactor extends AbstractServer implements MetricsProducer, CompactorService.Iface { +public class Compactor extends AbstractServer + implements MetricsProducer, CompactorService.Iface, ServerProcessService.Iface { public interface FileCompactorRunnable extends Runnable { /** @@ -168,9 +170,6 @@ public String getQueueName() { private ServiceLock compactorLock; private ServerAddress compactorAddress = null; - // Exposed for tests - protected volatile boolean shutdown = false; - private final AtomicBoolean compactionRunning = new AtomicBoolean(false); protected Compactor(CompactorServerOpts opts, String[] args) { @@ -288,7 +287,7 @@ protected void announceExistence(HostAndPort clientAddress) compactorLock = new ServiceLock(getContext().getZooReaderWriter().getZooKeeper(), ServiceLock.path(zPath), compactorId); - LockWatcher lw = new ServiceLockWatcher("compactor", () -> false, + LockWatcher lw = new ServiceLockWatcher("compactor", () -> getShutdownComplete().get(), (name) -> gcLogger.logGCInfo(getConfiguration())); try { @@ -320,7 +319,7 @@ protected void announceExistence(HostAndPort clientAddress) * @throws UnknownHostException host unknown */ protected ServerAddress startCompactorClientService() throws UnknownHostException { - var processor = ThriftProcessorTypes.getCompactorTProcessor(this, getContext()); + var processor = ThriftProcessorTypes.getCompactorTProcessor(this, this, getContext()); @SuppressWarnings("deprecation") var maxMessageSizeProperty = getConfiguration().resolve(Property.RPC_MAX_MESSAGE_SIZE, Property.GENERAL_MAX_MESSAGE_SIZE); @@ -698,182 +697,188 @@ public void run() { final AtomicReference err = new AtomicReference<>(); - while (!shutdown) { - - // mark compactor as idle while not in the compaction loop - updateIdleStatus(true); + while (!isShutdownRequested()) { + if (Thread.currentThread().isInterrupted()) { + LOG.info("Server process thread has been interrupted, shutting down"); + break; + } + try { + // mark compactor as idle while not in the compaction loop + updateIdleStatus(true); - currentCompactionId.set(null); - err.set(null); - JOB_HOLDER.reset(); + currentCompactionId.set(null); + err.set(null); + JOB_HOLDER.reset(); - TExternalCompactionJob job; - try { - TNextCompactionJob next = getNextJob(getNextId()); - job = next.getJob(); - if (!job.isSetExternalCompactionId()) { - LOG.trace("No external compactions in queue {}", this.queueName); - UtilWaitThread.sleep(getWaitTimeBetweenCompactionChecks(next.getCompactorCount())); + TExternalCompactionJob job; + try { + TNextCompactionJob next = getNextJob(getNextId()); + job = next.getJob(); + if (!job.isSetExternalCompactionId()) { + LOG.trace("No external compactions in queue {}", this.queueName); + UtilWaitThread.sleep(getWaitTimeBetweenCompactionChecks(next.getCompactorCount())); + continue; + } + if (!job.getExternalCompactionId().equals(currentCompactionId.get().toString())) { + throw new IllegalStateException("Returned eci " + job.getExternalCompactionId() + + " does not match supplied eci " + currentCompactionId.get()); + } + } catch (RetriesExceededException e2) { + LOG.warn("Retries exceeded getting next job. Retrying..."); continue; } - if (!job.getExternalCompactionId().equals(currentCompactionId.get().toString())) { - throw new IllegalStateException("Returned eci " + job.getExternalCompactionId() - + " does not match supplied eci " + currentCompactionId.get()); - } - } catch (RetriesExceededException e2) { - LOG.warn("Retries exceeded getting next job. Retrying..."); - continue; - } - LOG.debug("Received next compaction job: {}", job); + LOG.debug("Received next compaction job: {}", job); - final LongAdder totalInputEntries = new LongAdder(); - final LongAdder totalInputBytes = new LongAdder(); - final CountDownLatch started = new CountDownLatch(1); - final CountDownLatch stopped = new CountDownLatch(1); + final LongAdder totalInputEntries = new LongAdder(); + final LongAdder totalInputBytes = new LongAdder(); + final CountDownLatch started = new CountDownLatch(1); + final CountDownLatch stopped = new CountDownLatch(1); - final FileCompactorRunnable fcr = - createCompactionJob(job, totalInputEntries, totalInputBytes, started, stopped, err); + final FileCompactorRunnable fcr = + createCompactionJob(job, totalInputEntries, totalInputBytes, started, stopped, err); - final Thread compactionThread = - Threads.createThread("Compaction job for tablet " + job.getExtent().toString(), fcr); + final Thread compactionThread = + Threads.createThread("Compaction job for tablet " + job.getExtent().toString(), fcr); - JOB_HOLDER.set(job, compactionThread, fcr.getFileCompactor()); + JOB_HOLDER.set(job, compactionThread, fcr.getFileCompactor()); - try { - // mark compactor as busy while compacting - updateIdleStatus(false); - - // Need to call FileCompactorRunnable.initialize after calling JOB_HOLDER.set - fcr.initialize(); - - compactionThread.start(); // start the compactionThread - started.await(); // wait until the compactor is started - final long inputEntries = totalInputEntries.sum(); - final long waitTime = calculateProgressCheckTime(totalInputBytes.sum()); - LOG.debug("Progress checks will occur every {} seconds", waitTime); - String percentComplete = "unknown"; - - while (!stopped.await(waitTime, TimeUnit.SECONDS)) { - List running = - org.apache.accumulo.server.compaction.FileCompactor.getRunningCompactions(); - if (!running.isEmpty()) { - // Compaction has started. There should only be one in the list - CompactionInfo info = running.get(0); - if (info != null) { - final long entriesRead = info.getEntriesRead(); - final long entriesWritten = info.getEntriesWritten(); - if (inputEntries > 0) { - percentComplete = Float.toString((entriesRead / (float) inputEntries) * 100); - } - String message = String.format( - "Compaction in progress, read %d of %d input entries ( %s %s ), written %d entries", - entriesRead, inputEntries, percentComplete, "%", entriesWritten); - watcher.run(); - try { - LOG.debug("Updating coordinator with compaction progress: {}.", message); - TCompactionStatusUpdate update = new TCompactionStatusUpdate( - TCompactionState.IN_PROGRESS, message, inputEntries, entriesRead, - entriesWritten, fcr.getCompactionAge().toNanos()); - updateCompactionState(job, update); - } catch (RetriesExceededException e) { - LOG.warn("Error updating coordinator with compaction progress, error: {}", - e.getMessage()); + try { + // mark compactor as busy while compacting + updateIdleStatus(false); + + // Need to call FileCompactorRunnable.initialize after calling JOB_HOLDER.set + fcr.initialize(); + + compactionThread.start(); // start the compactionThread + started.await(); // wait until the compactor is started + final long inputEntries = totalInputEntries.sum(); + final long waitTime = calculateProgressCheckTime(totalInputBytes.sum()); + LOG.debug("Progress checks will occur every {} seconds", waitTime); + String percentComplete = "unknown"; + + while (!stopped.await(waitTime, TimeUnit.SECONDS)) { + List running = + org.apache.accumulo.server.compaction.FileCompactor.getRunningCompactions(); + if (!running.isEmpty()) { + // Compaction has started. There should only be one in the list + CompactionInfo info = running.get(0); + if (info != null) { + final long entriesRead = info.getEntriesRead(); + final long entriesWritten = info.getEntriesWritten(); + if (inputEntries > 0) { + percentComplete = Float.toString((entriesRead / (float) inputEntries) * 100); + } + String message = String.format( + "Compaction in progress, read %d of %d input entries ( %s %s ), written %d entries", + entriesRead, inputEntries, percentComplete, "%", entriesWritten); + watcher.run(); + try { + LOG.debug("Updating coordinator with compaction progress: {}.", message); + TCompactionStatusUpdate update = new TCompactionStatusUpdate( + TCompactionState.IN_PROGRESS, message, inputEntries, entriesRead, + entriesWritten, fcr.getCompactionAge().toNanos()); + updateCompactionState(job, update); + } catch (RetriesExceededException e) { + LOG.warn("Error updating coordinator with compaction progress, error: {}", + e.getMessage()); + } } + } else { + LOG.debug("Waiting on compaction thread to finish, but no RUNNING compaction"); } - } else { - LOG.debug("Waiting on compaction thread to finish, but no RUNNING compaction"); - } - } - compactionThread.join(); - LOG.trace("Compaction thread finished."); - // Run the watcher again to clear out the finished compaction and set the - // stuck count to zero. - watcher.run(); - - if (err.get() != null) { - // maybe the error occured because the table was deleted or something like that, so - // force a cancel check to possibly reduce noise in the logs - checkIfCanceled(); - } - - if (compactionThread.isInterrupted() || JOB_HOLDER.isCancelled() - || (err.get() != null && err.get().getClass().equals(InterruptedException.class))) { - LOG.warn("Compaction thread was interrupted, sending CANCELLED state"); - try { - TCompactionStatusUpdate update = - new TCompactionStatusUpdate(TCompactionState.CANCELLED, "Compaction cancelled", - -1, -1, -1, fcr.getCompactionAge().toNanos()); - updateCompactionState(job, update); - updateCompactionFailed(job); - } catch (RetriesExceededException e) { - LOG.error("Error updating coordinator with compaction cancellation.", e); - } finally { - currentCompactionId.set(null); } - } else if (err.get() != null) { - KeyExtent fromThriftExtent = KeyExtent.fromThrift(job.getExtent()); - try { - LOG.info("Updating coordinator with compaction failure: id: {}, extent: {}", - job.getExternalCompactionId(), fromThriftExtent); - TCompactionStatusUpdate update = new TCompactionStatusUpdate(TCompactionState.FAILED, - "Compaction failed due to: " + err.get().getMessage(), -1, -1, -1, - fcr.getCompactionAge().toNanos()); - updateCompactionState(job, update); - updateCompactionFailed(job); - } catch (RetriesExceededException e) { - LOG.error("Error updating coordinator with compaction failure: id: {}, extent: {}", - job.getExternalCompactionId(), fromThriftExtent, e); - } finally { - currentCompactionId.set(null); + compactionThread.join(); + LOG.trace("Compaction thread finished."); + // Run the watcher again to clear out the finished compaction and set the + // stuck count to zero. + watcher.run(); + + if (err.get() != null) { + // maybe the error occured because the table was deleted or something like that, so + // force a cancel check to possibly reduce noise in the logs + checkIfCanceled(); } - } else { - try { - LOG.trace("Updating coordinator with compaction completion."); - updateCompactionCompleted(job, JOB_HOLDER.getStats()); - } catch (RetriesExceededException e) { - LOG.error( - "Error updating coordinator with compaction completion, cancelling compaction.", - e); + + if (compactionThread.isInterrupted() || JOB_HOLDER.isCancelled() + || (err.get() != null && err.get().getClass().equals(InterruptedException.class))) { + LOG.warn("Compaction thread was interrupted, sending CANCELLED state"); + try { + TCompactionStatusUpdate update = + new TCompactionStatusUpdate(TCompactionState.CANCELLED, "Compaction cancelled", + -1, -1, -1, fcr.getCompactionAge().toNanos()); + updateCompactionState(job, update); + updateCompactionFailed(job); + } catch (RetriesExceededException e) { + LOG.error("Error updating coordinator with compaction cancellation.", e); + } finally { + currentCompactionId.set(null); + } + } else if (err.get() != null) { + KeyExtent fromThriftExtent = KeyExtent.fromThrift(job.getExtent()); + try { + LOG.info("Updating coordinator with compaction failure: id: {}, extent: {}", + job.getExternalCompactionId(), fromThriftExtent); + TCompactionStatusUpdate update = new TCompactionStatusUpdate( + TCompactionState.FAILED, "Compaction failed due to: " + err.get().getMessage(), + -1, -1, -1, fcr.getCompactionAge().toNanos()); + updateCompactionState(job, update); + updateCompactionFailed(job); + } catch (RetriesExceededException e) { + LOG.error("Error updating coordinator with compaction failure: id: {}, extent: {}", + job.getExternalCompactionId(), fromThriftExtent, e); + } finally { + currentCompactionId.set(null); + } + } else { try { - cancel(job.getExternalCompactionId()); - } catch (TException e1) { - LOG.error("Error cancelling compaction.", e1); + LOG.trace("Updating coordinator with compaction completion."); + updateCompactionCompleted(job, JOB_HOLDER.getStats()); + } catch (RetriesExceededException e) { + LOG.error( + "Error updating coordinator with compaction completion, cancelling compaction.", + e); + try { + cancel(job.getExternalCompactionId()); + } catch (TException e1) { + LOG.error("Error cancelling compaction.", e1); + } + } finally { + currentCompactionId.set(null); } - } finally { - currentCompactionId.set(null); } - } - } catch (RuntimeException e1) { - LOG.error( - "Compactor thread was interrupted waiting for compaction to start, cancelling job", - e1); - try { - cancel(job.getExternalCompactionId()); - } catch (TException e2) { - LOG.error("Error cancelling compaction.", e2); - } - } finally { - currentCompactionId.set(null); + } catch (RuntimeException e1) { + LOG.error( + "Compactor thread was interrupted waiting for compaction to start, cancelling job", + e1); + try { + cancel(job.getExternalCompactionId()); + } catch (TException e2) { + LOG.error("Error cancelling compaction.", e2); + } + } finally { + currentCompactionId.set(null); - // mark compactor as idle after compaction completes - updateIdleStatus(true); + // mark compactor as idle after compaction completes + updateIdleStatus(true); - // In the case where there is an error in the foreground code the background compaction - // may still be running. Must cancel it before starting another iteration of the loop to - // avoid multiple threads updating shared state. - while (compactionThread.isAlive()) { - compactionThread.interrupt(); - compactionThread.join(1000); + // In the case where there is an error in the foreground code the background compaction + // may still be running. Must cancel it before starting another iteration of the loop to + // avoid multiple threads updating shared state. + while (compactionThread.isAlive()) { + compactionThread.interrupt(); + compactionThread.join(1000); + } } + } catch (InterruptedException e) { + LOG.info("Interrupt Exception received, shutting down"); + gracefulShutdown(getContext().rpcCreds()); } - - } - + } // end while } catch (Exception e) { LOG.error("Unhandled error occurred in Compactor", e); } finally { // Shutdown local thrift server - LOG.info("Stopping Thrift Servers"); + LOG.debug("Stopping Thrift Servers"); if (compactorAddress.server != null) { compactorAddress.server.stop(); } @@ -889,6 +894,7 @@ public void run() { } gcLogger.logGCInfo(getConfiguration()); + getShutdownComplete().set(true); LOG.info("stop requested. exiting ... "); try { if (null != compactorLock) { diff --git a/server/compactor/src/test/java/org/apache/accumulo/compactor/CompactorTest.java b/server/compactor/src/test/java/org/apache/accumulo/compactor/CompactorTest.java index e6d4a32b4ed..74aaf09fa3d 100644 --- a/server/compactor/src/test/java/org/apache/accumulo/compactor/CompactorTest.java +++ b/server/compactor/src/test/java/org/apache/accumulo/compactor/CompactorTest.java @@ -33,6 +33,7 @@ import java.util.UUID; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.atomic.LongAdder; import java.util.function.Supplier; @@ -50,6 +51,8 @@ import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.metadata.schema.ExternalCompactionId; import org.apache.accumulo.core.metrics.MetricsInfo; +import org.apache.accumulo.core.process.thrift.ServerProcessService; +import org.apache.accumulo.core.securityImpl.thrift.TCredentials; import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats; import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob; import org.apache.accumulo.core.util.Halt; @@ -183,7 +186,7 @@ public void run() { } - public class SuccessfulCompactor extends Compactor { + public class SuccessfulCompactor extends Compactor implements ServerProcessService.Iface { private final Logger LOG = LoggerFactory.getLogger(SuccessfulCompactor.class); @@ -195,6 +198,8 @@ public class SuccessfulCompactor extends Compactor { private volatile boolean completedCalled = false; private volatile boolean failedCalled = false; private TCompactionStatusUpdate latestState = null; + private final AtomicBoolean shutdown = new AtomicBoolean(false); + private final AtomicBoolean shutdownComplete = new AtomicBoolean(false); SuccessfulCompactor(Supplier uuid, ServerAddress address, TExternalCompactionJob job, ServerContext context, ExternalCompactionId eci, CompactorServerOpts compactorServerOpts) { @@ -227,10 +232,24 @@ protected ServerAddress startCompactorClientService() throws UnknownHostExceptio protected TNextCompactionJob getNextJob(Supplier uuid) throws RetriesExceededException { LOG.info("Attempting to get next job, eci = {}", eci); currentCompactionId.set(eci); - this.shutdown = true; + gracefulShutdown(null); return new TNextCompactionJob(job, 1); } + @Override + public void gracefulShutdown(TCredentials creds) { + shutdown.set(true); + } + + @Override + public boolean isShutdownRequested() { + return shutdown.get(); + } + + public AtomicBoolean getShutdownComplete() { + return shutdownComplete; + } + @Override protected synchronized void checkIfCanceled() {} @@ -283,7 +302,7 @@ protected Collection getServiceTags(HostAndPort clientAddress) { } - public class FailedCompactor extends SuccessfulCompactor { + public class FailedCompactor extends SuccessfulCompactor implements ServerProcessService.Iface { FailedCompactor(Supplier uuid, ServerAddress address, TExternalCompactionJob job, ServerContext context, ExternalCompactionId eci, CompactorServerOpts compactorServerOpts) { @@ -298,7 +317,8 @@ protected FileCompactorRunnable createCompactionJob(TExternalCompactionJob job, } } - public class InterruptedCompactor extends SuccessfulCompactor { + public class InterruptedCompactor extends SuccessfulCompactor + implements ServerProcessService.Iface { InterruptedCompactor(Supplier uuid, ServerAddress address, TExternalCompactionJob job, ServerContext context, ExternalCompactionId eci, CompactorServerOpts compactorServerOpts) { diff --git a/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java b/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java index 8ea304a8fd3..7d7096a8604 100644 --- a/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java +++ b/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java @@ -40,6 +40,7 @@ import org.apache.accumulo.core.metadata.RootTable; import org.apache.accumulo.core.metadata.schema.Ample.DataLevel; import org.apache.accumulo.core.metrics.MetricsInfo; +import org.apache.accumulo.core.process.thrift.ServerProcessService; import org.apache.accumulo.core.securityImpl.thrift.TCredentials; import org.apache.accumulo.core.trace.TraceUtil; import org.apache.accumulo.core.trace.thrift.TInfo; @@ -68,7 +69,8 @@ // Could/Should implement HighlyAvailableService but the Thrift server is already started before // the ZK lock is acquired. The server is only for metrics, there are no concerns about clients // using the service before the lock is acquired. -public class SimpleGarbageCollector extends AbstractServer implements Iface { +public class SimpleGarbageCollector extends AbstractServer + implements Iface, ServerProcessService.Iface { private static final Logger log = LoggerFactory.getLogger(SimpleGarbageCollector.class); @@ -191,138 +193,156 @@ public void run() { } }); - while (true) { - Span outerSpan = TraceUtil.startSpan(this.getClass(), "gc"); - try (Scope outerScope = outerSpan.makeCurrent()) { - Span innerSpan = TraceUtil.startSpan(this.getClass(), "loop"); - try (Scope innerScope = innerSpan.makeCurrent()) { - final long tStart = System.nanoTime(); + while (!isShutdownRequested()) { + if (Thread.currentThread().isInterrupted()) { + LOG.info("Server process thread has been interrupted, shutting down"); + break; + } + try { + Span outerSpan = TraceUtil.startSpan(this.getClass(), "gc"); + try (Scope outerScope = outerSpan.makeCurrent()) { + Span innerSpan = TraceUtil.startSpan(this.getClass(), "loop"); + try (Scope innerScope = innerSpan.makeCurrent()) { + final long tStart = System.nanoTime(); + try { + System.gc(); // make room + + status.current.started = System.currentTimeMillis(); + var rootGC = new GCRun(DataLevel.ROOT, getContext()); + var mdGC = new GCRun(DataLevel.METADATA, getContext()); + var userGC = new GCRun(DataLevel.USER, getContext()); + + log.info("Starting Root table Garbage Collection."); + status.current.bulks += new GarbageCollectionAlgorithm().collect(rootGC); + incrementStatsForRun(rootGC); + logStats(); + + log.info("Starting Metadata table Garbage Collection."); + status.current.bulks += new GarbageCollectionAlgorithm().collect(mdGC); + incrementStatsForRun(mdGC); + logStats(); + + log.info("Starting User table Garbage Collection."); + status.current.bulks += new GarbageCollectionAlgorithm().collect(userGC); + incrementStatsForRun(userGC); + logStats(); + + } catch (Exception e) { + TraceUtil.setException(innerSpan, e, false); + log.error("{}", e.getMessage(), e); + } finally { + status.current.finished = System.currentTimeMillis(); + status.last = status.current; + gcCycleMetrics.setLastCollect(status.current); + status.current = new GcCycleStats(); + } + + final long tStop = System.nanoTime(); + log.info(String.format("Collect cycle took %.2f seconds", + (TimeUnit.NANOSECONDS.toMillis(tStop - tStart) / 1000.0))); + + /* + * We want to prune references to fully-replicated WALs from the replication table which + * are no longer referenced in the metadata table before running + * GarbageCollectWriteAheadLogs to ensure we delete as many files as possible. + */ + Span replSpan = TraceUtil.startSpan(this.getClass(), "replicationClose"); + try (Scope replScope = replSpan.makeCurrent()) { + @SuppressWarnings("deprecation") + Runnable closeWals = + new org.apache.accumulo.gc.replication.CloseWriteAheadLogReferences(getContext()); + closeWals.run(); + } catch (Exception e) { + TraceUtil.setException(replSpan, e, false); + log.error("Error trying to close write-ahead logs for replication table", e); + } finally { + replSpan.end(); + } + + // Clean up any unused write-ahead logs + Span walSpan = TraceUtil.startSpan(this.getClass(), "walogs"); + try (Scope walScope = walSpan.makeCurrent()) { + GarbageCollectWriteAheadLogs walogCollector = new GarbageCollectWriteAheadLogs( + getContext(), fs, liveTServerSet, isUsingTrash()); + log.info("Beginning garbage collection of write-ahead logs"); + walogCollector.collect(status); + gcCycleMetrics.setLastWalCollect(status.lastLog); + } catch (Exception e) { + TraceUtil.setException(walSpan, e, false); + log.error("{}", e.getMessage(), e); + } finally { + walSpan.end(); + } + } catch (Exception e) { + TraceUtil.setException(innerSpan, e, true); + throw e; + } finally { + innerSpan.end(); + } + + // we just made a lot of metadata changes: flush them out try { - System.gc(); // make room + AccumuloClient accumuloClient = getContext(); - status.current.started = System.currentTimeMillis(); - var rootGC = new GCRun(DataLevel.ROOT, getContext()); - var mdGC = new GCRun(DataLevel.METADATA, getContext()); - var userGC = new GCRun(DataLevel.USER, getContext()); + final long actionStart = System.nanoTime(); - log.info("Starting Root table Garbage Collection."); - status.current.bulks += new GarbageCollectionAlgorithm().collect(rootGC); - incrementStatsForRun(rootGC); - logStats(); + String action = getConfiguration().get(Property.GC_USE_FULL_COMPACTION); + log.debug("gc post action {} started", action); - log.info("Starting Metadata table Garbage Collection."); - status.current.bulks += new GarbageCollectionAlgorithm().collect(mdGC); - incrementStatsForRun(mdGC); - logStats(); + switch (action) { + case "compact": + accumuloClient.tableOperations().compact(MetadataTable.NAME, null, null, true, + true); + accumuloClient.tableOperations().compact(RootTable.NAME, null, null, true, true); + break; + case "flush": + accumuloClient.tableOperations().flush(MetadataTable.NAME, null, null, true); + accumuloClient.tableOperations().flush(RootTable.NAME, null, null, true); + break; + default: + log.trace("'none - no action' or invalid value provided: {}", action); + } - log.info("Starting User table Garbage Collection."); - status.current.bulks += new GarbageCollectionAlgorithm().collect(userGC); - incrementStatsForRun(userGC); - logStats(); + final long actionComplete = System.nanoTime(); - } catch (Exception e) { - TraceUtil.setException(innerSpan, e, false); - log.error("{}", e.getMessage(), e); - } finally { - status.current.finished = System.currentTimeMillis(); - status.last = status.current; - gcCycleMetrics.setLastCollect(status.current); - status.current = new GcCycleStats(); - } + gcCycleMetrics.setPostOpDurationNanos(actionComplete - actionStart); - final long tStop = System.nanoTime(); - log.info(String.format("Collect cycle took %.2f seconds", - (TimeUnit.NANOSECONDS.toMillis(tStop - tStart) / 1000.0))); - - /* - * We want to prune references to fully-replicated WALs from the replication table which - * are no longer referenced in the metadata table before running - * GarbageCollectWriteAheadLogs to ensure we delete as many files as possible. - */ - Span replSpan = TraceUtil.startSpan(this.getClass(), "replicationClose"); - try (Scope replScope = replSpan.makeCurrent()) { - @SuppressWarnings("deprecation") - Runnable closeWals = - new org.apache.accumulo.gc.replication.CloseWriteAheadLogReferences(getContext()); - closeWals.run(); - } catch (Exception e) { - TraceUtil.setException(replSpan, e, false); - log.error("Error trying to close write-ahead logs for replication table", e); - } finally { - replSpan.end(); - } + log.info("gc post action {} completed in {} seconds", action, String.format("%.2f", + (TimeUnit.NANOSECONDS.toMillis(actionComplete - actionStart) / 1000.0))); - // Clean up any unused write-ahead logs - Span walSpan = TraceUtil.startSpan(this.getClass(), "walogs"); - try (Scope walScope = walSpan.makeCurrent()) { - GarbageCollectWriteAheadLogs walogCollector = - new GarbageCollectWriteAheadLogs(getContext(), fs, liveTServerSet, isUsingTrash()); - log.info("Beginning garbage collection of write-ahead logs"); - walogCollector.collect(status); - gcCycleMetrics.setLastWalCollect(status.lastLog); } catch (Exception e) { - TraceUtil.setException(walSpan, e, false); - log.error("{}", e.getMessage(), e); - } finally { - walSpan.end(); + TraceUtil.setException(outerSpan, e, false); + log.warn("{}", e.getMessage(), e); } } catch (Exception e) { - TraceUtil.setException(innerSpan, e, true); + TraceUtil.setException(outerSpan, e, true); throw e; } finally { - innerSpan.end(); + outerSpan.end(); } - - // we just made a lot of metadata changes: flush them out try { - AccumuloClient accumuloClient = getContext(); - - final long actionStart = System.nanoTime(); - - String action = getConfiguration().get(Property.GC_USE_FULL_COMPACTION); - log.debug("gc post action {} started", action); - - switch (action) { - case "compact": - accumuloClient.tableOperations().compact(MetadataTable.NAME, null, null, true, true); - accumuloClient.tableOperations().compact(RootTable.NAME, null, null, true, true); - break; - case "flush": - accumuloClient.tableOperations().flush(MetadataTable.NAME, null, null, true); - accumuloClient.tableOperations().flush(RootTable.NAME, null, null, true); - break; - default: - log.trace("'none - no action' or invalid value provided: {}", action); - } - - final long actionComplete = System.nanoTime(); - gcCycleMetrics.setPostOpDurationNanos(actionComplete - actionStart); - - log.info("gc post action {} completed in {} seconds", action, String.format("%.2f", - (TimeUnit.NANOSECONDS.toMillis(actionComplete - actionStart) / 1000.0))); - - } catch (Exception e) { - TraceUtil.setException(outerSpan, e, false); + gcCycleMetrics.incrementRunCycleCount(); + long gcDelay = getConfiguration().getTimeInMillis(Property.GC_CYCLE_DELAY); + log.debug("Sleeping for {} milliseconds", gcDelay); + Thread.sleep(gcDelay); + } catch (InterruptedException e) { log.warn("{}", e.getMessage(), e); + throw e; } - } catch (Exception e) { - TraceUtil.setException(outerSpan, e, true); - throw e; - } finally { - outerSpan.end(); - } - try { - - gcCycleMetrics.incrementRunCycleCount(); - long gcDelay = getConfiguration().getTimeInMillis(Property.GC_CYCLE_DELAY); - log.debug("Sleeping for {} milliseconds", gcDelay); - Thread.sleep(gcDelay); } catch (InterruptedException e) { - log.warn("{}", e.getMessage(), e); - return; + log.info("Interrupt Exception received, shutting down"); + gracefulShutdown(getContext().rpcCreds()); } } + getShutdownComplete().set(true); + log.info("stop requested. exiting ... "); + try { + gcLock.unlock(); + } catch (Exception e) { + log.warn("Failed to release GarbageCollector lock", e); + } + } private void incrementStatsForRun(GCRun gcRun) { @@ -365,7 +385,8 @@ private void getZooLock(HostAndPort addr) throws KeeperException, InterruptedExc UUID zooLockUUID = UUID.randomUUID(); gcLock = new ServiceLock(getContext().getZooReaderWriter().getZooKeeper(), path, zooLockUUID); - HAServiceLockWatcher gcLockWatcher = new HAServiceLockWatcher("gc"); + HAServiceLockWatcher gcLockWatcher = + new HAServiceLockWatcher("gc", () -> getShutdownComplete().get()); while (true) { gcLock.lock(gcLockWatcher, @@ -392,7 +413,7 @@ private void getZooLock(HostAndPort addr) throws KeeperException, InterruptedExc } private HostAndPort startStatsService() { - var processor = ThriftProcessorTypes.getGcTProcessor(this, getContext()); + var processor = ThriftProcessorTypes.getGcTProcessor(this, this, getContext()); IntStream port = getConfiguration().getPortStream(Property.GC_PORT); HostAndPort[] addresses = TServerUtils.getHostAndPorts(getHostname(), port); @SuppressWarnings("deprecation") diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java b/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java index 7ca35732cca..aa47e713e4e 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java @@ -96,6 +96,7 @@ import org.apache.accumulo.core.metadata.schema.TabletMetadata; import org.apache.accumulo.core.metrics.MetricsInfo; import org.apache.accumulo.core.metrics.MetricsProducer; +import org.apache.accumulo.core.process.thrift.ServerProcessService; import org.apache.accumulo.core.replication.thrift.ReplicationCoordinator; import org.apache.accumulo.core.spi.balancer.BalancerEnvironment; import org.apache.accumulo.core.spi.balancer.SimpleLoadBalancer; @@ -164,8 +165,8 @@ *

* The manager will also coordinate log recoveries and reports general status. */ -public class Manager extends AbstractServer - implements LiveTServerSet.Listener, TableObserver, CurrentState, HighlyAvailableService { +public class Manager extends AbstractServer implements LiveTServerSet.Listener, TableObserver, + CurrentState, HighlyAvailableService, ServerProcessService.Iface { static final Logger log = LoggerFactory.getLogger(Manager.class); @@ -1243,7 +1244,7 @@ public void run() { ServerAddress sa; var processor = - ThriftProcessorTypes.getManagerTProcessor(fateServiceHandler, haProxy, getContext()); + ThriftProcessorTypes.getManagerTProcessor(this, fateServiceHandler, haProxy, getContext()); try { @SuppressWarnings("deprecation") @@ -1441,10 +1442,23 @@ boolean canSuspendTablets() { // The manager is fully initialized. Clients are allowed to connect now. managerInitialized.set(true); - while (clientService.isServing()) { - sleepUninterruptibly(500, MILLISECONDS); + while (!isShutdownRequested() && clientService.isServing()) { + if (Thread.currentThread().isInterrupted()) { + LOG.info("Server process thread has been interrupted, shutting down"); + break; + } + try { + Thread.sleep(500); + } catch (InterruptedException e) { + log.info("Interrupt Exception received, shutting down"); + gracefulShutdown(context.rpcCreds()); + } } - log.info("Shutting down fate."); + + LOG.debug("Stopping Thrift Servers"); + sa.server.stop(); + + log.debug("Shutting down fate."); fate().shutdown(); final long deadline = System.currentTimeMillis() + MAX_CLEANUP_WAIT_TIME; @@ -1485,7 +1499,13 @@ boolean canSuspendTablets() { throw new IllegalStateException("Exception waiting on watcher", e); } } - log.info("exiting"); + getShutdownComplete().set(true); + log.info("stop requested. exiting ... "); + try { + managerLock.unlock(); + } catch (Exception e) { + log.warn("Failed to release Manager lock", e); + } } @Deprecated @@ -1645,9 +1665,11 @@ private void getManagerLock(final ServiceLockPath zManagerLoc) UUID zooLockUUID = UUID.randomUUID(); managerLock = new ServiceLock(zooKeeper, zManagerLoc, zooLockUUID); + HAServiceLockWatcher managerLockWatcher = + new HAServiceLockWatcher("manager", () -> getShutdownComplete().get()); + while (true) { - HAServiceLockWatcher managerLockWatcher = new HAServiceLockWatcher("manager"); managerLock.lock(managerLockWatcher, managerClientAddress.getBytes(UTF_8)); managerLockWatcher.waitForChange(); diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/ManagerClientServiceHandler.java b/server/manager/src/main/java/org/apache/accumulo/manager/ManagerClientServiceHandler.java index ee914703ccf..bc734022571 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/ManagerClientServiceHandler.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/ManagerClientServiceHandler.java @@ -49,6 +49,7 @@ import org.apache.accumulo.core.clientImpl.thrift.TableOperation; import org.apache.accumulo.core.clientImpl.thrift.TableOperationExceptionType; import org.apache.accumulo.core.clientImpl.thrift.ThriftConcurrentModificationException; +import org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException; import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException; import org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException; import org.apache.accumulo.core.conf.DeprecatedPropertyUtil; @@ -342,6 +343,17 @@ public void shutdownTabletServer(TInfo info, TCredentials c, String tabletServer log.debug("FATE op shutting down " + tabletServer + " finished"); } + @Override + public void tabletServerStopping(TInfo tinfo, TCredentials credentials, String tabletServer) + throws ThriftSecurityException, ThriftNotActiveServiceException, TException { + if (!manager.security.canPerformSystemActions(credentials)) { + throw new ThriftSecurityException(credentials.getPrincipal(), + SecurityErrorCode.PERMISSION_DENIED); + } + log.info("Tablet Server {} has reported it's shutting down", tabletServer); + manager.tserverSet.tabletServerShuttingDown(tabletServer); + } + @Override public void reportSplitExtent(TInfo info, TCredentials credentials, String serverName, TabletSplit split) throws ThriftSecurityException { diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/recovery/RecoveryManager.java b/server/manager/src/main/java/org/apache/accumulo/manager/recovery/RecoveryManager.java index dc1f00ba4a3..b1117eba3fe 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/recovery/RecoveryManager.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/recovery/RecoveryManager.java @@ -79,7 +79,7 @@ public RecoveryManager(Manager manager, long timeToCacheExistsInMillis) { try { List workIDs = new DistributedWorkQueue(manager.getZooKeeperRoot() + Constants.ZRECOVERY, - manager.getConfiguration(), manager.getContext()).getWorkQueued(); + manager.getConfiguration(), manager).getWorkQueued(); sortsQueued.addAll(workIDs); } catch (Exception e) { log.warn("{}", e.getMessage(), e); @@ -132,7 +132,7 @@ private void initiateSort(String sortId, String source, final String destination throws KeeperException, InterruptedException { String work = source + "|" + destination; new DistributedWorkQueue(manager.getZooKeeperRoot() + Constants.ZRECOVERY, - manager.getConfiguration(), manager.getContext()).addWork(sortId, work.getBytes(UTF_8)); + manager.getConfiguration(), manager).addWork(sortId, work.getBytes(UTF_8)); synchronized (this) { sortsQueued.add(sortId); diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/replication/DistributedWorkQueueWorkAssigner.java b/server/manager/src/main/java/org/apache/accumulo/manager/replication/DistributedWorkQueueWorkAssigner.java index 7e60c344f2b..a87a78c3179 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/replication/DistributedWorkQueueWorkAssigner.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/replication/DistributedWorkQueueWorkAssigner.java @@ -92,7 +92,7 @@ protected void setZooCache(ZooCache zooCache) { protected void initializeWorkQueue(AccumuloConfiguration conf) { workQueue = new DistributedWorkQueue(ZooUtil.getRoot(client.instanceOperations().getInstanceId()) - + ReplicationConstants.ZOO_WORK_QUEUE, conf, this.workQueue.getContext()); + + ReplicationConstants.ZOO_WORK_QUEUE, conf, this.workQueue.getServer()); } @Override diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/CopyFailed.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/CopyFailed.java index 53d5dd40022..b53d54edacf 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/CopyFailed.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/CopyFailed.java @@ -147,7 +147,7 @@ public Repo call(long tid, Manager manager) throws Exception { if (!loadedFailures.isEmpty()) { DistributedWorkQueue bifCopyQueue = new DistributedWorkQueue( Constants.ZROOT + "/" + manager.getInstanceID() + Constants.ZBULK_FAILED_COPYQ, - manager.getConfiguration(), manager.getContext()); + manager.getConfiguration(), manager); HashSet workIds = new HashSet<>(); diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/EmbeddedWebServer.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/EmbeddedWebServer.java index 6784206b686..300534858aa 100644 --- a/server/monitor/src/main/java/org/apache/accumulo/monitor/EmbeddedWebServer.java +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/EmbeddedWebServer.java @@ -127,7 +127,7 @@ public void start() { } } - private void stop() { + public void stop() { try { server.stop(); server.join(); diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java index 548d2a985c7..fdd077713d8 100644 --- a/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java @@ -526,6 +526,22 @@ public void run() { }).start(); monitorInitialized.set(true); + + while (!isShutdownRequested()) { + if (Thread.currentThread().isInterrupted()) { + LOG.info("Server process thread has been interrupted, shutting down"); + break; + } + try { + Thread.sleep(1000); + } catch (InterruptedException e) { + LOG.info("Interrupt Exception received, shutting down"); + gracefulShutdown(context.rpcCreds()); + } + } + + server.stop(); + log.info("stop requested. exiting ... "); } private ServletHolder getDefaultServlet() { @@ -826,9 +842,10 @@ private void getMonitorLock() throws KeeperException, InterruptedException { // Get a ZooLock for the monitor UUID zooLockUUID = UUID.randomUUID(); monitorLock = new ServiceLock(zoo.getZooKeeper(), monitorLockPath, zooLockUUID); + HAServiceLockWatcher monitorLockWatcher = + new HAServiceLockWatcher("monitor", () -> isShutdownRequested()); while (true) { - HAServiceLockWatcher monitorLockWatcher = new HAServiceLockWatcher("monitor"); monitorLock.lock(monitorLockWatcher, new byte[0]); monitorLockWatcher.waitForChange(); diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/ScanServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/ScanServer.java index bfbcdeed9b0..245302e14f2 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/ScanServer.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/ScanServer.java @@ -75,6 +75,7 @@ import org.apache.accumulo.core.metadata.schema.Ample; import org.apache.accumulo.core.metadata.schema.TabletMetadata; import org.apache.accumulo.core.metrics.MetricsInfo; +import org.apache.accumulo.core.process.thrift.ServerProcessService; import org.apache.accumulo.core.securityImpl.thrift.TCredentials; import org.apache.accumulo.core.spi.scan.ScanServerSelector; import org.apache.accumulo.core.tabletserver.thrift.ActiveScan; @@ -127,7 +128,7 @@ import com.google.common.collect.Sets; public class ScanServer extends AbstractServer - implements TabletScanClientService.Iface, TabletHostingServer { + implements TabletScanClientService.Iface, TabletHostingServer, ServerProcessService.Iface { public static class ScanServerOpts extends ServerOpts { @Parameter(required = false, names = {"-g", "--group"}, @@ -201,7 +202,6 @@ private TabletMetadataLoader(Ample ample) { HostAndPort clientAddress; private final GarbageCollectionLogger gcLogger = new GarbageCollectionLogger(); - protected volatile boolean serverStopRequested = false; private ServiceLock scanServerLock; protected TabletServerScanMetrics scanMetrics; private ScanServerMetrics scanServerMetrics; @@ -304,7 +304,7 @@ protected ServerAddress startScanServerClientService() throws UnknownHostExcepti // This class implements TabletClientService.Iface and then delegates calls. Be sure // to set up the ThriftProcessor using this class, not the delegate. - TProcessor processor = ThriftProcessorTypes.getScanServerTProcessor(this, getContext()); + TProcessor processor = ThriftProcessorTypes.getScanServerTProcessor(this, this, getContext()); @SuppressWarnings("deprecation") var maxMessageSizeProperty = getConfiguration().resolve(Property.RPC_MAX_MESSAGE_SIZE, @@ -348,7 +348,7 @@ private ServiceLock announceExistence() { serverLockUUID = UUID.randomUUID(); scanServerLock = new ServiceLock(zoo.getZooKeeper(), zLockPath, serverLockUUID); - LockWatcher lw = new ServiceLockWatcher("scan server", () -> serverStopRequested, + LockWatcher lw = new ServiceLockWatcher("scan server", () -> getShutdownComplete().get(), (name) -> gcLogger.logGCInfo(getConfiguration())); // Don't use the normal ServerServices lock content, instead put the server UUID here. @@ -402,17 +402,33 @@ public void run() { ServiceLock lock = announceExistence(); try { - while (!serverStopRequested) { - UtilWaitThread.sleep(1000); - updateIdleStatus( - sessionManager.getActiveScans().isEmpty() && tabletMetadataCache.estimatedSize() == 0); + while (!isShutdownRequested()) { + if (Thread.currentThread().isInterrupted()) { + LOG.info("Server process thread has been interrupted, shutting down"); + break; + } + try { + Thread.sleep(1000); + updateIdleStatus(sessionManager.getActiveScans().isEmpty() + && tabletMetadataCache.estimatedSize() == 0); + } catch (InterruptedException e) { + LOG.info("Interrupt Exception received, shutting down"); + gracefulShutdown(getContext().rpcCreds()); + } } } finally { - LOG.info("Stopping Thrift Servers"); + // Wait for scans to got to zero + while (!sessionManager.getActiveScans().isEmpty()) { + LOG.debug("Waiting on {} active scans to complete.", + sessionManager.getActiveScans().size()); + UtilWaitThread.sleep(1000); + } + + LOG.debug("Stopping Thrift Servers"); address.server.stop(); try { - LOG.info("Removing server scan references"); + LOG.debug("Removing server scan references"); this.getContext().getAmple().deleteScanServerFileReferences(clientAddress.toString(), serverLockUUID); } catch (Exception e) { @@ -435,6 +451,7 @@ public void run() { } gcLogger.logGCInfo(getConfiguration()); + getShutdownComplete().set(true); LOG.info("stop requested. exiting ... "); try { if (null != lock) { @@ -929,6 +946,11 @@ public InitialScan startScan(TInfo tinfo, TCredentials credentials, TKeyExtent t long busyTimeout) throws ThriftSecurityException, NotServingTabletException, TooManyFilesException, TSampleNotPresentException, TException { + if (isShutdownRequested()) { + // Prevent scans from starting if shutting down + throw new ScanServerBusyException(); + } + KeyExtent extent = getKeyExtent(textent); if (extent.isMeta() && !isSystemUser(credentials)) { @@ -989,6 +1011,11 @@ public InitialMultiScan startMultiScan(TInfo tinfo, TCredentials credentials, Map executionHints, long busyTimeout) throws ThriftSecurityException, TSampleNotPresentException, TException { + if (isShutdownRequested()) { + // Prevent scans from starting if shutting down + throw new ScanServerBusyException(); + } + if (tbatch.size() == 0) { throw new TException("Scan Server batch must include at least one extent"); } diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java index beb2e550a68..2d4749b90bd 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java @@ -54,6 +54,7 @@ import java.util.UUID; import java.util.concurrent.BlockingDeque; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Future; import java.util.concurrent.LinkedBlockingDeque; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.ThreadPoolExecutor; @@ -87,14 +88,18 @@ import org.apache.accumulo.core.metadata.MetadataTable; import org.apache.accumulo.core.metadata.RootTable; import org.apache.accumulo.core.metadata.TServerInstance; +import org.apache.accumulo.core.metadata.schema.Ample.DataLevel; import org.apache.accumulo.core.metadata.schema.TabletsMetadata; import org.apache.accumulo.core.metrics.MetricsInfo; +import org.apache.accumulo.core.process.thrift.ServerProcessService; import org.apache.accumulo.core.rpc.ThriftUtil; import org.apache.accumulo.core.rpc.clients.ThriftClientTypes; import org.apache.accumulo.core.spi.fs.VolumeChooserEnvironment; import org.apache.accumulo.core.tabletserver.log.LogEntry; +import org.apache.accumulo.core.tabletserver.thrift.TUnloadTabletGoal; import org.apache.accumulo.core.trace.TraceUtil; import org.apache.accumulo.core.util.ComparablePair; +import org.apache.accumulo.core.util.Halt; import org.apache.accumulo.core.util.HostAndPort; import org.apache.accumulo.core.util.MapCounter; import org.apache.accumulo.core.util.Pair; @@ -103,6 +108,7 @@ import org.apache.accumulo.core.util.ServerServices; import org.apache.accumulo.core.util.ServerServices.Service; import org.apache.accumulo.core.util.UtilWaitThread; +import org.apache.accumulo.core.util.threads.ThreadPoolNames; import org.apache.accumulo.core.util.threads.ThreadPools; import org.apache.accumulo.core.util.threads.Threads; import org.apache.accumulo.server.AbstractServer; @@ -169,7 +175,8 @@ import io.opentelemetry.api.trace.Span; import io.opentelemetry.context.Scope; -public class TabletServer extends AbstractServer implements TabletHostingServer { +public class TabletServer extends AbstractServer + implements TabletHostingServer, ServerProcessService.Iface { private static final SecureRandom random = new SecureRandom(); private static final Logger log = LoggerFactory.getLogger(TabletServer.class); @@ -218,9 +225,6 @@ public TabletServerMinCMetrics getMinCMetrics() { volatile HostAndPort clientAddress; - private volatile boolean serverStopRequested = false; - private volatile boolean shutdownComplete = false; - private ServiceLock tabletServerLock; private TServer server; @@ -253,7 +257,7 @@ protected TabletServer(ServerOpts opts, String[] args) { this.sessionManager = new SessionManager(context); this.logSorter = new LogSorter(context, aconf); @SuppressWarnings("deprecation") - var replWorker = new org.apache.accumulo.tserver.replication.ReplicationWorker(context); + var replWorker = new org.apache.accumulo.tserver.replication.ReplicationWorker(this); this.replWorker = replWorker; this.statsKeeper = new TabletStatsKeeper(); final int numBusyTabletsToLog = aconf.getCount(Property.TSERV_LOG_BUSY_TABLETS_COUNT); @@ -400,7 +404,7 @@ String getLockID() { void requestStop() { log.info("Stop requested."); - serverStopRequested = true; + gracefulShutdown(getContext().rpcCreds()); } private class SplitRunner implements Runnable { @@ -616,7 +620,7 @@ private HostAndPort startTabletClientService() throws UnknownHostException { thriftClientHandler = newTabletClientHandler(watcher, writeTracker); scanClientHandler = newThriftScanClientHandler(writeTracker); - TProcessor processor = ThriftProcessorTypes.getTabletServerTProcessor(clientHandler, + TProcessor processor = ThriftProcessorTypes.getTabletServerTProcessor(this, clientHandler, thriftClientHandler, scanClientHandler, getContext()); HostAndPort address = startServer(clientAddress.getHost(), processor); log.info("address = {}", address); @@ -683,7 +687,7 @@ private void announceExistence() { tabletServerLock = new ServiceLock(zoo.getZooKeeper(), zLockPath, UUID.randomUUID()); - LockWatcher lw = new ServiceLockWatcher("tablet server", () -> serverStopRequested, + LockWatcher lw = new ServiceLockWatcher("tablet server", () -> getShutdownComplete().get(), (name) -> gcLogger.logGCInfo(getConfiguration())); byte[] lockContent = new ServerServices(getClientAddressString(), Service.TSERV_CLIENT) @@ -785,9 +789,8 @@ public void run() { .createExecutorService(getConfiguration(), Property.TSERV_WORKQ_THREADS, true); // TODO: Remove when Property.TSERV_WORKQ_THREADS is removed - DistributedWorkQueue bulkFailedCopyQ = - new DistributedWorkQueue(getContext().getZooKeeperRoot() + Constants.ZBULK_FAILED_COPYQ, - getConfiguration(), getContext()); + DistributedWorkQueue bulkFailedCopyQ = new DistributedWorkQueue( + getContext().getZooKeeperRoot() + Constants.ZBULK_FAILED_COPYQ, getConfiguration(), this); try { bulkFailedCopyQ.startProcessing(new BulkFailedCopyProcessor(getContext()), distWorkQThreadPool); @@ -796,7 +799,7 @@ public void run() { } try { - logSorter.startWatchingForRecoveryLogs(); + logSorter.startWatchingForRecoveryLogs(this); } catch (Exception ex) { log.error("Error setting watches for recoveries"); throw new RuntimeException(ex); @@ -861,7 +864,11 @@ public void run() { CLEANUP_BULK_LOADED_CACHE_MILLIS, TimeUnit.MILLISECONDS)); HostAndPort managerHost; - while (!serverStopRequested) { + while (!isShutdownRequested()) { + if (Thread.currentThread().isInterrupted()) { + LOG.info("Server process thread has been interrupted, shutting down"); + break; + } updateIdleStatus(getOnlineTablets().isEmpty()); @@ -873,7 +880,7 @@ public void run() { try { // wait until a message is ready to send, or a server stop // was requested - while (mm == null && !serverStopRequested) { + while (mm == null && !isShutdownRequested() && !Thread.currentThread().isInterrupted()) { mm = managerMessages.poll(1, TimeUnit.SECONDS); updateIdleStatus(getOnlineTablets().isEmpty()); } @@ -886,8 +893,8 @@ public void run() { // if while loop does not execute at all and mm != null, // then finally block should place mm back on queue - while (!serverStopRequested && mm != null && client != null - && client.getOutputProtocol() != null + while (!Thread.currentThread().isInterrupted() && !isShutdownRequested() && mm != null + && client != null && client.getOutputProtocol() != null && client.getOutputProtocol().getTransport() != null && client.getOutputProtocol().getTransport().isOpen()) { try { @@ -917,7 +924,7 @@ public void run() { } } catch (InterruptedException e) { log.info("Interrupt Exception received, shutting down"); - serverStopRequested = true; + gracefulShutdown(getContext().rpcCreds()); } catch (Exception e) { // may have lost connection with manager // loop back to the beginning and wait for a new one @@ -926,26 +933,84 @@ public void run() { } } - // wait for shutdown - // if the main thread exits oldServer the manager listener, the JVM will - // kill the other threads and finalize objects. We want the shutdown that is - // running in the manager listener thread to complete oldServer this happens. - // consider making other threads daemon threads so that objects don't - // get prematurely finalized - synchronized (this) { - while (!shutdownComplete) { - try { - this.wait(1000); - } catch (InterruptedException e) { - log.error(e.toString()); - } - } + // Tell the Manager we are shutting down so that it doesn't try + // to assign tablets. + ManagerClientService.Client iface = managerConnection(getManagerAddress()); + try { + iface.tabletServerStopping(TraceUtil.traceInfo(), getContext().rpcCreds(), + getClientAddressString()); + } catch (TException e) { + LOG.error("Error informing Manager that we are shutting down, halting server", e); + Halt.halt("Error informing Manager that we are shutting down, exiting!", -1); + } finally { + returnManagerConnection(iface); } + log.debug("Stopping Replication Server"); if (this.replServer != null) { this.replServer.stop(); } + // Best-effort attempt at unloading tablets. + log.debug("Unloading tablets"); + final List> futures = new ArrayList<>(); + final ThreadPoolExecutor tpe = getContext().threadPools() + .getPoolBuilder(ThreadPoolNames.TSERVER_SHUTDOWN_UNLOAD_TABLET_POOL).numCoreThreads(8) + .numMaxThreads(16).build(); + + iface = managerConnection(getManagerAddress()); + boolean managerDown = false; + + try { + for (DataLevel level : new DataLevel[] {DataLevel.USER, DataLevel.METADATA, DataLevel.ROOT}) { + getOnlineTablets().keySet().forEach(ke -> { + if (DataLevel.of(ke.tableId()) == level) { + futures.add( + tpe.submit(new UnloadTabletHandler(this, ke, TUnloadTabletGoal.UNASSIGNED, 5000))); + } + }); + while (!futures.isEmpty()) { + Iterator> unloads = futures.iterator(); + while (unloads.hasNext()) { + Future f = unloads.next(); + if (f.isDone()) { + if (!managerDown) { + ManagerMessage mm = managerMessages.poll(); + try { + mm.send(getContext().rpcCreds(), getClientAddressString(), iface); + } catch (TException e) { + managerDown = true; + LOG.debug("Error sending message to Manager during tablet unloading, msg: {}", + e.getMessage()); + } + } + unloads.remove(); + } + } + log.debug("Waiting on {} {} tablets to close.", futures.size(), level); + UtilWaitThread.sleep(1000); + } + log.debug("All {} tablets unloaded", level); + } + } finally { + if (!managerDown) { + try { + ManagerMessage mm = managerMessages.poll(); + do { + if (mm != null) { + mm.send(getContext().rpcCreds(), getClientAddressString(), iface); + } + mm = managerMessages.poll(); + } while (mm != null); + } catch (TException e) { + LOG.debug("Error sending message to Manager during tablet unloading, msg: {}", + e.getMessage()); + } + } + returnManagerConnection(iface); + tpe.shutdown(); + } + log.debug("Stopping Thrift Servers"); if (server != null) { server.stop(); @@ -960,8 +1025,8 @@ public void run() { gcLogger.logGCInfo(getConfiguration()); + getShutdownComplete().set(true); log.info("TServerInfo: stop requested. exiting ... "); - try { tabletServerLock.unlock(); } catch (Exception e) { diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/LogSorter.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/LogSorter.java index 32a249e7b64..442b2f85f93 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/LogSorter.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/LogSorter.java @@ -46,6 +46,7 @@ import org.apache.accumulo.core.spi.crypto.CryptoService; import org.apache.accumulo.core.util.Pair; import org.apache.accumulo.core.util.threads.ThreadPools; +import org.apache.accumulo.server.AbstractServer; import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.server.fs.VolumeManager; import org.apache.accumulo.server.log.SortedLogState; @@ -293,7 +294,8 @@ void writeBuffer(String destPath, List> buffer, in } } - public void startWatchingForRecoveryLogs() throws KeeperException, InterruptedException { + public void startWatchingForRecoveryLogs(AbstractServer server) + throws KeeperException, InterruptedException { @SuppressWarnings("deprecation") int threadPoolSize = this.conf.getCount(this.conf .resolve(Property.TSERV_WAL_SORT_MAX_CONCURRENT, Property.TSERV_RECOVERY_MAX_CONCURRENT)); @@ -301,7 +303,7 @@ public void startWatchingForRecoveryLogs() throws KeeperException, InterruptedEx ThreadPools.getServerThreadPools().getPoolBuilder(TSERVER_WAL_SORT_CONCURRENT_POOL) .numCoreThreads(threadPoolSize).enableThreadPoolMetrics().build(); new DistributedWorkQueue(context.getZooKeeperRoot() + Constants.ZRECOVERY, sortedLogConf, - context).startProcessing(new LogProcessor(), threadPool); + server).startProcessing(new LogProcessor(), threadPool); } public List getLogSorts() { diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationWorker.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationWorker.java index f1d988f7ea2..cd733d4baed 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationWorker.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationWorker.java @@ -24,7 +24,7 @@ import org.apache.accumulo.core.conf.DefaultConfiguration; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.replication.ReplicationConstants; -import org.apache.accumulo.server.ServerContext; +import org.apache.accumulo.server.AbstractServer; import org.apache.accumulo.server.zookeeper.DistributedWorkQueue; import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; @@ -37,11 +37,11 @@ public class ReplicationWorker implements Runnable { private static final Logger log = LoggerFactory.getLogger(ReplicationWorker.class); - private final ServerContext context; + private final AbstractServer server; private ThreadPoolExecutor executor; - public ReplicationWorker(ServerContext context) { - this.context = context; + public ReplicationWorker(AbstractServer server) { + this.server = server; } public void setExecutor(ThreadPoolExecutor executor) { @@ -53,7 +53,7 @@ public void run() { DefaultConfiguration defaultConf = DefaultConfiguration.getInstance(); long defaultDelay = defaultConf.getTimeInMillis(Property.REPLICATION_WORK_PROCESSOR_DELAY); long defaultPeriod = defaultConf.getTimeInMillis(Property.REPLICATION_WORK_PROCESSOR_PERIOD); - AccumuloConfiguration conf = context.getConfiguration(); + AccumuloConfiguration conf = server.getConfiguration(); long delay = conf.getTimeInMillis(Property.REPLICATION_WORK_PROCESSOR_DELAY); long period = conf.getTimeInMillis(Property.REPLICATION_WORK_PROCESSOR_PERIOD); try { @@ -62,15 +62,16 @@ public void run() { log.debug("Configuration DistributedWorkQueue with delay and period of {} and {}", delay, period); workQueue = new DistributedWorkQueue( - context.getZooKeeperRoot() + ReplicationConstants.ZOO_WORK_QUEUE, conf, context, delay, - period); + server.getContext().getZooKeeperRoot() + ReplicationConstants.ZOO_WORK_QUEUE, conf, + server, delay, period); } else { log.debug("Configuring DistributedWorkQueue with default delay and period"); workQueue = new DistributedWorkQueue( - context.getZooKeeperRoot() + ReplicationConstants.ZOO_WORK_QUEUE, conf, context); + server.getContext().getZooKeeperRoot() + ReplicationConstants.ZOO_WORK_QUEUE, conf, + server); } - workQueue.startProcessing(new ReplicationProcessor(context), executor); + workQueue.startProcessing(new ReplicationProcessor(server.getContext()), executor); } catch (KeeperException | InterruptedException e) { throw new RuntimeException(e); } diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/ScanServerTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/ScanServerTest.java index 73a3e0d03bc..5a080408720 100644 --- a/server/tserver/src/test/java/org/apache/accumulo/tserver/ScanServerTest.java +++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/ScanServerTest.java @@ -43,6 +43,7 @@ import org.apache.accumulo.core.dataImpl.thrift.TColumn; import org.apache.accumulo.core.dataImpl.thrift.TKeyExtent; import org.apache.accumulo.core.dataImpl.thrift.TRange; +import org.apache.accumulo.core.process.thrift.ServerProcessService; import org.apache.accumulo.core.securityImpl.thrift.TCredentials; import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException; import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException; @@ -59,7 +60,7 @@ public class ScanServerTest { - public class TestScanServer extends ScanServer { + public class TestScanServer extends ScanServer implements ServerProcessService.Iface { private KeyExtent extent; private TabletResolver resolver; @@ -117,6 +118,11 @@ protected boolean isSystemUser(TCredentials creds) { return systemUser; } + @Override + public boolean isShutdownRequested() { + return false; + } + } private ThriftScanClientHandler handler; diff --git a/test/src/main/java/org/apache/accumulo/test/SelfStoppingScanServer.java b/test/src/main/java/org/apache/accumulo/test/SelfStoppingScanServer.java index 8391190984c..49b40af7f53 100644 --- a/test/src/main/java/org/apache/accumulo/test/SelfStoppingScanServer.java +++ b/test/src/main/java/org/apache/accumulo/test/SelfStoppingScanServer.java @@ -20,6 +20,7 @@ import java.util.concurrent.atomic.AtomicInteger; +import org.apache.accumulo.core.process.thrift.ServerProcessService; import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException; import org.apache.accumulo.core.tabletserver.thrift.TabletScanClientService; import org.apache.accumulo.core.trace.thrift.TInfo; @@ -32,7 +33,7 @@ * */ public class SelfStoppingScanServer extends ScanServer - implements TabletScanClientService.Iface, TabletHostingServer { + implements TabletScanClientService.Iface, TabletHostingServer, ServerProcessService.Iface { private final AtomicInteger scanCount = new AtomicInteger(0); @@ -45,7 +46,7 @@ public void closeMultiScan(TInfo tinfo, long scanID) throws NoSuchScanIDExceptio scanCount.incrementAndGet(); super.closeMultiScan(tinfo, scanID); if (scanCount.get() == 3) { - serverStopRequested = true; + gracefulShutdown(getContext().rpcCreds()); } } diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionTServer.java b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionTServer.java index 8719c8860ed..0edd803d197 100644 --- a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionTServer.java +++ b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionTServer.java @@ -18,13 +18,14 @@ */ package org.apache.accumulo.test.compaction; +import org.apache.accumulo.core.process.thrift.ServerProcessService; import org.apache.accumulo.server.ServerOpts; import org.apache.accumulo.server.zookeeper.TransactionWatcher; import org.apache.accumulo.tserver.TabletClientHandler; import org.apache.accumulo.tserver.TabletServer; import org.apache.accumulo.tserver.WriteTracker; -public class ExternalCompactionTServer extends TabletServer { +public class ExternalCompactionTServer extends TabletServer implements ServerProcessService.Iface { ExternalCompactionTServer(ServerOpts opts, String[] args) { super(opts, args); diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalDoNothingCompactor.java b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalDoNothingCompactor.java index 08fc5fbb8d9..30a51dc343a 100644 --- a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalDoNothingCompactor.java +++ b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalDoNothingCompactor.java @@ -31,6 +31,7 @@ import org.apache.accumulo.core.compaction.thrift.TCompactionState; import org.apache.accumulo.core.compaction.thrift.TCompactionStatusUpdate; import org.apache.accumulo.core.dataImpl.KeyExtent; +import org.apache.accumulo.core.process.thrift.ServerProcessService; import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob; import org.apache.accumulo.core.util.UtilWaitThread; import org.apache.accumulo.server.compaction.FileCompactor; @@ -39,7 +40,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class ExternalDoNothingCompactor extends Compactor implements Iface { +public class ExternalDoNothingCompactor extends Compactor + implements Iface, ServerProcessService.Iface { private static final Logger LOG = LoggerFactory.getLogger(ExternalDoNothingCompactor.class); @@ -60,7 +62,7 @@ protected FileCompactorRunnable createCompactionJob(TExternalCompactionJob job, CountDownLatch stopped, AtomicReference err) { // Set this to true so that only 1 external compaction is run - this.shutdown = true; + gracefulShutdown(getContext().rpcCreds()); return new FileCompactorRunnable() { diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/TestCompactionCoordinatorForOfflineTable.java b/test/src/main/java/org/apache/accumulo/test/compaction/TestCompactionCoordinatorForOfflineTable.java index 7e5242b6fa1..48e86a5b679 100644 --- a/test/src/main/java/org/apache/accumulo/test/compaction/TestCompactionCoordinatorForOfflineTable.java +++ b/test/src/main/java/org/apache/accumulo/test/compaction/TestCompactionCoordinatorForOfflineTable.java @@ -28,13 +28,14 @@ import org.apache.accumulo.core.metadata.schema.ExternalCompactionFinalState; import org.apache.accumulo.core.metadata.schema.ExternalCompactionFinalState.FinalState; import org.apache.accumulo.core.metadata.schema.ExternalCompactionId; +import org.apache.accumulo.core.process.thrift.ServerProcessService; import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.server.ServerOpts; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class TestCompactionCoordinatorForOfflineTable extends CompactionCoordinator - implements CompactionCoordinatorService.Iface { + implements CompactionCoordinatorService.Iface, ServerProcessService.Iface { public static class NonNotifyingCompactionFinalizer extends CompactionFinalizer { diff --git a/test/src/main/java/org/apache/accumulo/test/fate/FlakyFateManager.java b/test/src/main/java/org/apache/accumulo/test/fate/FlakyFateManager.java index 090d4769d97..691a21b8e9a 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/FlakyFateManager.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/FlakyFateManager.java @@ -22,12 +22,13 @@ import org.apache.accumulo.core.fate.Fate; import org.apache.accumulo.core.fate.TStore; +import org.apache.accumulo.core.process.thrift.ServerProcessService; import org.apache.accumulo.manager.Manager; import org.apache.accumulo.manager.tableOps.TraceRepo; import org.apache.accumulo.server.ServerOpts; import org.slf4j.LoggerFactory; -public class FlakyFateManager extends Manager { +public class FlakyFateManager extends Manager implements ServerProcessService.Iface { protected FlakyFateManager(ServerOpts opts, String[] args) throws IOException { super(opts, args); diff --git a/test/src/main/java/org/apache/accumulo/test/functional/GracefulShutdownIT.java b/test/src/main/java/org/apache/accumulo/test/functional/GracefulShutdownIT.java new file mode 100644 index 00000000000..a638459e568 --- /dev/null +++ b/test/src/main/java/org/apache/accumulo/test/functional/GracefulShutdownIT.java @@ -0,0 +1,274 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.accumulo.test.functional; + +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; +import java.util.stream.IntStream; + +import org.apache.accumulo.compactor.Compactor; +import org.apache.accumulo.coordinator.CompactionCoordinator; +import org.apache.accumulo.core.Constants; +import org.apache.accumulo.core.client.Accumulo; +import org.apache.accumulo.core.client.AccumuloClient; +import org.apache.accumulo.core.client.BatchWriter; +import org.apache.accumulo.core.client.IteratorSetting; +import org.apache.accumulo.core.client.Scanner; +import org.apache.accumulo.core.client.ScannerBase.ConsistencyLevel; +import org.apache.accumulo.core.client.admin.CompactionConfig; +import org.apache.accumulo.core.client.admin.NewTableConfiguration; +import org.apache.accumulo.core.conf.ClientProperty; +import org.apache.accumulo.core.conf.Property; +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Mutation; +import org.apache.accumulo.core.data.Range; +import org.apache.accumulo.core.data.TableId; +import org.apache.accumulo.core.data.Value; +import org.apache.accumulo.core.fate.zookeeper.ServiceLock; +import org.apache.accumulo.core.fate.zookeeper.ServiceLock.ServiceLockPath; +import org.apache.accumulo.core.metadata.schema.TabletsMetadata; +import org.apache.accumulo.core.security.Authorizations; +import org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner; +import org.apache.accumulo.core.spi.compaction.SimpleCompactionDispatcher; +import org.apache.accumulo.core.util.HostAndPort; +import org.apache.accumulo.core.util.ServerServices; +import org.apache.accumulo.core.util.ServerServices.Service; +import org.apache.accumulo.harness.MiniClusterConfigurationCallback; +import org.apache.accumulo.harness.SharedMiniClusterBase; +import org.apache.accumulo.minicluster.ServerType; +import org.apache.accumulo.miniclusterImpl.MiniAccumuloClusterControl; +import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl; +import org.apache.accumulo.server.ServerContext; +import org.apache.accumulo.server.util.Admin; +import org.apache.accumulo.test.compaction.ExternalCompactionTestUtils; +import org.apache.accumulo.test.util.Wait; +import org.apache.accumulo.tserver.ScanServer; +import org.apache.hadoop.conf.Configuration; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +public class GracefulShutdownIT extends SharedMiniClusterBase { + + private static final String GROUP_NAME = "graceful"; + + // @formatter:off + private static final String clientConfiguration = + "["+ + " {"+ + " \"isDefault\": true,"+ + " \"maxBusyTimeout\": \"5m\","+ + " \"busyTimeoutMultiplier\": 8,"+ + " \"group\":" + GROUP_NAME + "," + + " \"scanTypeActivations\": [graceful],"+ + " \"attemptPlans\": ["+ + " {"+ + " \"servers\": \"3\","+ + " \"busyTimeout\": \"33ms\","+ + " \"salt\": \"one\""+ + " }"+ + " ]"+ + " }"+ + "]"; + // @formatter:on + + private static class GracefulShutdownITConfig implements MiniClusterConfigurationCallback { + + @Override + public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration coreSite) { + cfg.setNumCompactors(0); + cfg.setNumScanServers(0); + cfg.setNumTservers(2); + cfg.setProperty(Property.COMPACTION_COORDINATOR_FINALIZER_COMPLETION_CHECK_INTERVAL, "5s"); + cfg.setProperty(Property.COMPACTION_COORDINATOR_DEAD_COMPACTOR_CHECK_INTERVAL, "5s"); + cfg.setProperty(Property.COMPACTION_COORDINATOR_TSERVER_COMPACTION_CHECK_INTERVAL, "3s"); + cfg.setProperty(Property.COMPACTION_COORDINATOR_THRIFTCLIENT_PORTSEARCH, "true"); + cfg.setProperty(Property.COMPACTOR_CANCEL_CHECK_INTERVAL, "5s"); + cfg.setProperty(Property.COMPACTOR_PORTSEARCH, "true"); + cfg.setProperty("tserver.compaction.major.service." + GROUP_NAME + ".planner", + DefaultCompactionPlanner.class.getName()); + cfg.setProperty("tserver.compaction.major.service." + GROUP_NAME + ".planner.opts.executors", + "[{'name':'all', 'type': 'external', 'queue': '" + GROUP_NAME + "'}]"); + cfg.setClientProperty(ClientProperty.SCAN_SERVER_SELECTOR_OPTS_PREFIX.getKey() + "profiles", + clientConfiguration); + // Timeout scan sessions after being idle for 3 seconds + cfg.setProperty(Property.TSERV_SESSION_MAXIDLE, "3s"); + } + } + + @BeforeAll + public static void startup() throws Exception { + SharedMiniClusterBase.startMiniClusterWithConfig(new GracefulShutdownITConfig()); + } + + @AfterAll + public static void shutdown() throws Exception { + SharedMiniClusterBase.stopMiniCluster(); + } + + @Test + public void testGracefulShutdown() throws Exception { + + // Start ScanServers and Compactors using named groups + final MiniAccumuloClusterControl control = getCluster().getClusterControl(); + + try (final AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) { + final ServerContext ctx = getCluster().getServerContext(); + final String tableName = getUniqueNames(1)[0]; + + final NewTableConfiguration ntc = new NewTableConfiguration(); + ntc.setProperties(Map.of(Property.TABLE_MAJC_RATIO.getKey(), "10", + "table.compaction.dispatcher", SimpleCompactionDispatcher.class.getName(), + "table.compaction.dispatcher.opts.service", GROUP_NAME)); + + client.tableOperations().create(tableName, ntc); + final TableId tid = ctx.getTableId(tableName); + + // Insert 10 rows, flush after every row to create 10 files + try (BatchWriter writer = client.createBatchWriter(tableName)) { + for (int i : IntStream.rangeClosed(1, 10).toArray()) { + String val = i + ""; + Mutation m = new Mutation(val); + m.put(val, val, val); + writer.addMutation(m); + writer.flush(); + client.tableOperations().flush(tableName, null, null, true); + } + } + long numFiles = getNumFilesForTable(ctx, tid); + assertEquals(10, numFiles); + client.instanceOperations().waitForBalance(); + + // Restart Garbage Collector + final ServiceLockPath gcLockPath = + ServiceLock.path(ctx.getZooKeeperRoot() + Constants.ZGC_LOCK); + byte[] data = ServiceLock.getLockData(ctx.getZooReaderWriter().getZooKeeper(), gcLockPath); + assertNotNull(data); + final HostAndPort gcAddress = + new ServerServices(new String(data, UTF_8)).getAddress(Service.GC_CLIENT); + assertTrue(!control.getProcesses(ServerType.GARBAGE_COLLECTOR).isEmpty()); + // Don't call `new Admin().execute(new String[] {"signalShutdown", "-h ", host, "-p ", + // Integer.toString(port)})` + // because this poisons the SingletonManager and puts it into SERVER mode + Admin.signalGracefulShutdown(ctx, gcAddress.toString()); + Wait.waitFor(() -> { + control.refreshProcesses(ServerType.GARBAGE_COLLECTOR); + return control.getProcesses(ServerType.GARBAGE_COLLECTOR).isEmpty(); + }); + + // Restart Tablet Server + final List tservers = client.instanceOperations().getTabletServers(); + assertEquals(2, tservers.size()); + final HostAndPort tserverAddress = HostAndPort.fromString(tservers.get(0)); + Admin.signalGracefulShutdown(ctx, tserverAddress.toString()); + Wait.waitFor(() -> { + control.refreshProcesses(ServerType.TABLET_SERVER); + return control.getProcesses(ServerType.TABLET_SERVER).size() == 1; + }); + client.instanceOperations().waitForBalance(); + control.start(ServerType.TABLET_SERVER); + Wait.waitFor(() -> control.getProcesses(ServerType.TABLET_SERVER).size() == 2); + client.instanceOperations().waitForBalance(); + + // Restart Manager + final List managers = client.instanceOperations().getManagerLocations(); + assertEquals(1, managers.size()); + final HostAndPort managerAddress = HostAndPort.fromString(managers.get(0)); + Admin.signalGracefulShutdown(ctx, managerAddress.toString()); + Wait.waitFor(() -> { + control.refreshProcesses(ServerType.MANAGER); + return control.getProcesses(ServerType.MANAGER).isEmpty(); + }); + control.start(ServerType.MANAGER); + Wait.waitFor(() -> control.getProcesses(ServerType.MANAGER).size() == 1); + client.instanceOperations().waitForBalance(); + + // Compact table and shutdown compactor + control.startCoordinator(CompactionCoordinator.class); + getCluster().getConfig().setNumCompactors(1); + control.startCompactors(Compactor.class, 1, GROUP_NAME); + Wait.waitFor(() -> client.instanceOperations().getCompactors().size() == 1); + final Set compactors = client.instanceOperations().getCompactors(); + final HostAndPort compactorAddress = HostAndPort.fromString(compactors.iterator().next()); + + final CompactionConfig cc = new CompactionConfig(); + final IteratorSetting is = new IteratorSetting(100, SlowIterator.class); + SlowIterator.setSeekSleepTime(is, 1000); + SlowIterator.setSleepTime(is, 1000); + cc.setIterators(List.of(is)); + cc.setWait(false); + + final long numFiles2 = getNumFilesForTable(ctx, tid); + assertEquals(numFiles2, numFiles); + assertEquals(0, ExternalCompactionTestUtils.getRunningCompactions(ctx).getCompactionsSize()); + client.tableOperations().compact(tableName, cc); + Wait.waitFor( + () -> ExternalCompactionTestUtils.getRunningCompactions(ctx).getCompactionsSize() > 0); + Admin.signalGracefulShutdown(ctx, compactorAddress.toString()); + Wait.waitFor(() -> { + control.refreshProcesses(ServerType.COMPACTOR); + return control.getProcesses(ServerType.COMPACTOR).isEmpty(); + }); + final long numFiles3 = getNumFilesForTable(ctx, tid); + assertTrue(numFiles3 < numFiles2); + assertEquals(1, numFiles3); + + getCluster().getConfig().setNumScanServers(1); + control.startScanServer(ScanServer.class, 1, GROUP_NAME); + Wait.waitFor(() -> client.instanceOperations().getScanServers().size() == 1); + final Set sservers = client.instanceOperations().getScanServers(); + final HostAndPort sserver = HostAndPort.fromString(sservers.iterator().next()); + try (final Scanner scanner = client.createScanner(tableName, Authorizations.EMPTY)) { + scanner.setRange(new Range()); + scanner.setConsistencyLevel(ConsistencyLevel.EVENTUAL); + scanner.setExecutionHints(Map.of("scan_type", "graceful")); + scanner.addScanIterator(is); // add the slow iterator + scanner.setBatchSize(1); + int count = 0; + for (Entry e : scanner) { + count++; + if (count == 2) { + Admin.signalGracefulShutdown(ctx, sserver.toString()); + } + } + assertEquals(10, count); + Wait.waitFor(() -> { + control.refreshProcesses(ServerType.SCAN_SERVER); + return control.getProcesses(ServerType.SCAN_SERVER).isEmpty(); + }); + + } + + } + + } + + long getNumFilesForTable(ServerContext ctx, TableId tid) { + try (TabletsMetadata tablets = ctx.getAmple().readTablets().forTable(tid).build()) { + return tablets.stream().mapToLong(tm -> tm.getFiles().size()).sum(); + } + } +} diff --git a/test/src/main/java/org/apache/accumulo/test/functional/HalfDeadServerWatcherIT.java b/test/src/main/java/org/apache/accumulo/test/functional/HalfDeadServerWatcherIT.java index b0b81623484..355d3265ff6 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/HalfDeadServerWatcherIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/HalfDeadServerWatcherIT.java @@ -37,6 +37,7 @@ import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.dataImpl.KeyExtent; import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeMissingPolicy; +import org.apache.accumulo.core.process.thrift.ServerProcessService; import org.apache.accumulo.core.util.UtilWaitThread; import org.apache.accumulo.harness.AccumuloClusterHarness; import org.apache.accumulo.minicluster.ServerType; @@ -63,7 +64,8 @@ */ public class HalfDeadServerWatcherIT extends AccumuloClusterHarness { - public static class HalfDeadTabletServer extends TabletServer { + public static class HalfDeadTabletServer extends TabletServer + implements ServerProcessService.Iface { private static final Logger LOG = LoggerFactory.getLogger(HalfDeadTabletServer.class);