diff --git a/core/src/main/java/org/apache/accumulo/core/lock/ServiceLockSupport.java b/core/src/main/java/org/apache/accumulo/core/lock/ServiceLockSupport.java index dda14f50c01..d4e0cd0b580 100644 --- a/core/src/main/java/org/apache/accumulo/core/lock/ServiceLockSupport.java +++ b/core/src/main/java/org/apache/accumulo/core/lock/ServiceLockSupport.java @@ -67,16 +67,23 @@ public static class HAServiceLockWatcher implements AccumuloLockWatcher { private static final Logger LOG = LoggerFactory.getLogger(HAServiceLockWatcher.class); private final Type server; + private final Supplier shutdownComplete; private volatile boolean acquiredLock = false; private volatile boolean failedToAcquireLock = false; - public HAServiceLockWatcher(Type server) { + public HAServiceLockWatcher(Type server, Supplier shutdownComplete) { this.server = server; + this.shutdownComplete = shutdownComplete; } @Override public void lostLock(LockLossReason reason) { - Halt.halt(server + " lock in zookeeper lost (reason = " + reason + "), exiting!", -1); + if (shutdownComplete.get()) { + LOG.warn("{} lost lock (reason = {}), not halting because shutdown is complete.", server, + reason); + } else { + Halt.halt(server + " lock in zookeeper lost (reason = " + reason + "), exiting!", -1); + } } @Override @@ -147,24 +154,27 @@ public static class ServiceLockWatcher implements LockWatcher { private static final Logger LOG = LoggerFactory.getLogger(ServiceLockWatcher.class); private final Type server; - private final Supplier shuttingDown; + private final Supplier shutdownComplete; private final Consumer lostLockAction; - public ServiceLockWatcher(Type server, Supplier shuttingDown, + public ServiceLockWatcher(Type server, Supplier shutdownComplete, Consumer lostLockAction) { this.server = server; - 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.", server, + reason); + } else { + Halt.halt(1, () -> { LOG.error("{} lost lock (reason = {}), exiting.", server, reason); - } - lostLockAction.accept(server); - }); + lostLockAction.accept(server); + }); + } } @Override diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/clients/ManagerClient.java b/core/src/main/java/org/apache/accumulo/core/rpc/clients/ManagerClient.java index 29057cf3cb4..5127807e0b2 100644 --- a/core/src/main/java/org/apache/accumulo/core/rpc/clients/ManagerClient.java +++ b/core/src/main/java/org/apache/accumulo/core/rpc/clients/ManagerClient.java @@ -44,7 +44,15 @@ default C getManagerConnection(Logger log, ThriftClientTypes type, ClientCont return null; } - HostAndPort manager = HostAndPort.fromString(managers.iterator().next().toHostPortString()); + final String managerLocation = managers.iterator().next().toHostPortString(); + if (managerLocation.equals("0.0.0.0:0")) { + // The Manager creates the lock with an initial address of 0.0.0.0:0, then + // later updates the lock contents with the actual address after everything + // is started. + log.debug("Manager is up and lock acquired, waiting for address..."); + return null; + } + HostAndPort manager = HostAndPort.fromString(managerLocation); try { // Manager requests can take a long time: don't ever time out return ThriftUtil.getClientNoTimeout(type, manager, context); 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..de21d8d8bc9 --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/core/rpc/clients/ServerProcessServiceThriftClient.java @@ -0,0 +1,55 @@ +/* + * 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.thrift.transport.TTransportException; +import org.slf4j.Logger; + +import com.google.common.net.HostAndPort; + +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 d7f34c4846b..5b9a5c203d1 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 TabletManagementClientServiceThriftClient TABLET_MGMT = new TabletManagementClientServiceThriftClient("tablet"); + 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 6c025a6815b..8e1822893a0 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 @@ -58,6 +58,7 @@ public enum ThreadPoolNames { TSERVER_ASSIGNMENT_POOL("accumulo.pool.tserver.assignment"), 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 0ad5911a33f..1f787d46523 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 manager tabletserver securityImpl clientImpl dataImpl compaction tabletingest tablet tabletscan) +[[ -z $PACKAGES_TO_GENERATE ]] && PACKAGES_TO_GENERATE=(process gc manager tabletserver securityImpl clientImpl dataImpl compaction tabletingest tablet tabletscan) [[ -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 ded70a98dc4..9ecf1f71179 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 621d35dff55..c880120c659 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.clientImpl.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.clientImpl.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.clientImpl.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.clientImpl.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; @@ -95,6 +97,8 @@ public interface AsyncIface { public void shutdownTabletServer(org.apache.accumulo.core.clientImpl.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.clientImpl.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.clientImpl.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.clientImpl.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; @@ -511,6 +515,35 @@ public void recv_shutdownTabletServer() throws org.apache.accumulo.core.clientIm return; } + @Override + public void tabletServerStopping(org.apache.accumulo.core.clientImpl.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.clientImpl.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.clientImpl.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 { @@ -1292,6 +1325,48 @@ public Void getResult() throws org.apache.accumulo.core.clientImpl.thrift.Thrift } } + @Override + public void tabletServerStopping(org.apache.accumulo.core.clientImpl.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.clientImpl.thrift.TInfo tinfo; + private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; + private java.lang.String tabletServer; + public tabletServerStopping_call(org.apache.accumulo.core.clientImpl.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.clientImpl.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(); @@ -1690,6 +1765,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"); @@ -2430,6 +2540,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"); @@ -18787,6 +18972,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.clientImpl.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.clientImpl.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.clientImpl.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.clientImpl.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.clientImpl.thrift.TInfo getTinfo() { + return this.tinfo; + } + + public tabletServerStopping_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.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.clientImpl.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.clientImpl.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.clientImpl.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 08d93dc1852..c27c011a82b 100644 --- a/core/src/main/thrift/manager.thrift +++ b/core/src/main/thrift/manager.thrift @@ -363,6 +363,15 @@ service ManagerClientService { 1:client.ThriftSecurityException sec 2:client.ThriftNotActiveServiceException tnase ) + + void tabletServerStopping( + 1:client.TInfo tinfo + 2:security.TCredentials credentials + 3:string tabletServer + ) throws ( + 1:client.ThriftSecurityException sec + 2:client.ThriftNotActiveServiceException tnase + ) void setSystemProperty( 1:client.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 53ffdf1443f..57164165cfe 100644 --- a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterControl.java +++ b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterControl.java @@ -24,10 +24,12 @@ import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.Set; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -522,4 +524,66 @@ public List getTabletServers(String resourceGroup) { return tabletServerProcesses.get(resourceGroup); } + public void refreshProcesses(ServerType type) { + switch (type) { + case COMPACTOR: + compactorProcesses.forEach((k, v) -> v.removeIf(process -> !process.isAlive())); + break; + case GARBAGE_COLLECTOR: + if (!gcProcess.isAlive()) { + gcProcess = null; + } + break; + case MANAGER: + if (!managerProcess.isAlive()) { + managerProcess = null; + } + break; + case MONITOR: + if (!monitor.isAlive()) { + monitor = null; + } + break; + case SCAN_SERVER: + scanServerProcesses.forEach((k, v) -> v.removeIf(process -> !process.isAlive())); + break; + case TABLET_SERVER: + tabletServerProcesses.forEach((k, v) -> v.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 COMPACTOR: + Set cprocesses = new HashSet<>(); + compactorProcesses.values().forEach(list -> list.forEach(cprocesses::add)); + return cprocesses; + case GARBAGE_COLLECTOR: + return gcProcess == null ? Set.of() : Set.of(gcProcess); + case MANAGER: + return managerProcess == null ? Set.of() : Set.of(managerProcess); + case MONITOR: + return monitor == null ? Set.of() : Set.of(monitor); + case SCAN_SERVER: + Set sprocesses = new HashSet<>(); + scanServerProcesses.values().forEach(list -> list.forEach(sprocesses::add)); + return sprocesses; + case TABLET_SERVER: + Set tprocesses = new HashSet<>(); + tabletServerProcesses.values().forEach(list -> list.forEach(tprocesses::add)); + return tprocesses; + 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 2eb814ac23e..268d8f5dbb2 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 @@ -23,18 +23,22 @@ import java.util.List; import java.util.OptionalInt; import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.classloader.ClassLoaderUtil; import org.apache.accumulo.core.cli.ConfigOpts; +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.conf.SiteConfiguration; import org.apache.accumulo.core.conf.cluster.ClusterConfigParser; import org.apache.accumulo.core.lock.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.Timer; @@ -50,7 +54,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; @@ -62,6 +67,8 @@ public abstract class AbstractServer implements AutoCloseable, MetricsProducer, private volatile Timer idlePeriodTimer = null; 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, ConfigOpts opts, Function serverContextFactory, String[] args) { @@ -128,8 +135,62 @@ public String getResourceGroup() { return resourceGroup; } + @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<>(); @@ -141,6 +202,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 f20aa6dfb55..ab593585b03 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 @@ -29,6 +29,7 @@ import java.util.Map.Entry; import java.util.Optional; import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; import java.util.function.Supplier; @@ -206,6 +207,9 @@ static class TServerInfo { private LiveTServersSnapshot tServersSnapshot = null; + 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<>(); @@ -228,6 +232,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<>(); @@ -274,6 +291,7 @@ private synchronized void checkServer(final Set updates, if (info != null) { doomed.add(info.instance); current.remove(tserverPath.getServer()); + serversShuttingDown.remove(tserverPath.toString()); } Long firstSeen = locklessServers.get(tserverPath); @@ -407,7 +425,9 @@ public synchronized LiveTServersSnapshot getSnapshot() { } public synchronized Set getCurrentServers() { - return getSnapshot().getTservers(); + Set current = new HashSet<>(getSnapshot().getTservers()); + serversShuttingDown.values().forEach(tsi -> current.remove(tsi.instance)); + return 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 6f0a6086eba..0a56b4e32fe 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.rpc.clients.ThriftClientTypes; import org.apache.accumulo.core.tablet.thrift.TabletManagementClientService; import org.apache.accumulo.core.tabletingest.thrift.TabletIngestClientService; @@ -95,28 +96,42 @@ public > TProcessor getTProcessor( public static final ThriftProcessorTypes TABLET_MGMT = new ThriftProcessorTypes<>(ThriftClientTypes.TABLET_MGMT); - public static TMultiplexedProcessor getCompactorTProcessor(ClientServiceHandler clientHandler, + public static final ThriftProcessorTypes SERVER_PROCESS = + new ThriftProcessorTypes<>(ThriftClientTypes.SERVER_PROCESS); + + public static TMultiplexedProcessor getCompactorTProcessor( + ServerProcessService.Iface processHandler, ClientServiceHandler clientHandler, CompactorService.Iface serviceHandler, ServerContext context) { TMultiplexedProcessor muxProcessor = new TMultiplexedProcessor(); muxProcessor.registerProcessor(CLIENT.getServiceName(), CLIENT.getTProcessor( ClientService.Processor.class, ClientService.Iface.class, clientHandler, context)); + 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 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, CompactionCoordinatorService.Iface coordinatorServiceHandler, 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(COORDINATOR.getServiceName(), @@ -128,23 +143,31 @@ public static TMultiplexedProcessor getManagerTProcessor(FateService.Iface fateS return muxProcessor; } - public static TMultiplexedProcessor getScanServerTProcessor(ClientServiceHandler clientHandler, + public static TMultiplexedProcessor getScanServerTProcessor( + ServerProcessService.Iface processHandler, ClientServiceHandler clientHandler, TabletScanClientService.Iface tserverHandler, ServerContext context) { TMultiplexedProcessor muxProcessor = new TMultiplexedProcessor(); muxProcessor.registerProcessor(CLIENT.getServiceName(), CLIENT.getTProcessor( ClientService.Processor.class, ClientService.Iface.class, clientHandler, context)); + muxProcessor.registerProcessor(SERVER_PROCESS.getServiceName(), + SERVER_PROCESS.getTProcessor(ServerProcessService.Processor.class, + ServerProcessService.Iface.class, processHandler, context)); muxProcessor.registerProcessor(TABLET_SCAN.getServiceName(), TABLET_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, TabletServerClientService.Iface tserverHandler, TabletScanClientService.Iface tserverScanHandler, TabletIngestClientService.Iface tserverIngestHandler, TabletManagementClientService.Iface tserverMgmtHandler, 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 7983b4f92ad..f8e8df3c52f 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 @@ -83,6 +83,7 @@ import org.apache.accumulo.core.manager.thrift.TFateId; import org.apache.accumulo.core.metadata.AccumuloTable; import org.apache.accumulo.core.metadata.schema.TabletMetadata; +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; @@ -109,6 +110,7 @@ import org.apache.accumulo.server.util.checkCommand.UserFilesCheckRunner; 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; @@ -135,6 +137,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"}, @@ -450,6 +459,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); @@ -514,6 +526,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")) { @@ -661,6 +675,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.instanceOperations().getServers(ServerId.Type.MANAGER).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 fb374ce2293..a52b63945c3 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 @@ -37,6 +37,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; @@ -60,7 +61,7 @@ public class DistributedWorkQueue { private final ZooReaderWriter zoo; private final String path; - private final ServerContext context; + private final AbstractServer server; private final long timerInitialDelay; private final long timerPeriod; @@ -87,6 +88,11 @@ private void lookForWork(final Processor processor, final 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; } @@ -176,23 +182,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.get().nextInt(toIntExact(MINUTES.toMillis(1))), + this(path, config, server, RANDOM.get().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.getZooSession().asReaderWriter(); + zoo = server.getContext().getZooSession().asReaderWriter(); } public ServerContext getContext() { - return context; + return server.getContext(); + } + + public AbstractServer getServer() { + return server; } public long getCheckInterval() { @@ -254,7 +264,7 @@ public void processExistingAndFuture(final Processor processor, // 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/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java b/server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java index e61ab5a1124..a954b687bc9 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 @@ -167,9 +167,6 @@ public interface FileCompactorRunnable extends Runnable { private ServerAddress compactorAddress = null; private final PausedCompactionMetrics pausedMetrics = new PausedCompactionMetrics(); - // Exposed for tests - protected volatile boolean shutdown = false; - private final AtomicBoolean compactionRunning = new AtomicBoolean(false); protected Compactor(ConfigOpts opts, String[] args) { @@ -279,7 +276,7 @@ protected void announceExistence(HostAndPort clientAddress) getContext().getServerPaths().createCompactorPath(getResourceGroup(), clientAddress); ServiceLockSupport.createNonHaServiceLockPath(Type.COMPACTOR, zoo, path); compactorLock = new ServiceLock(getContext().getZooSession(), path, compactorId); - LockWatcher lw = new ServiceLockWatcher(Type.COMPACTOR, () -> false, + LockWatcher lw = new ServiceLockWatcher(Type.COMPACTOR, () -> getShutdownComplete().get(), (type) -> getContext().getLowMemoryDetector().logGCInfo(getConfiguration())); try { @@ -322,7 +319,7 @@ protected CompactorService.Iface getCompactorThriftHandlerInterface() { protected ServerAddress startCompactorClientService() throws UnknownHostException { ClientServiceHandler clientHandler = new ClientServiceHandler(getContext()); - var processor = ThriftProcessorTypes.getCompactorTProcessor(clientHandler, + var processor = ThriftProcessorTypes.getCompactorTProcessor(this, clientHandler, getCompactorThriftHandlerInterface(), getContext()); ServerAddress sp = TServerUtils.startServer(getContext(), getHostname(), Property.COMPACTOR_CLIENTPORT, processor, this.getClass().getSimpleName(), @@ -697,194 +694,199 @@ public void run() { try { final AtomicReference err = new AtomicReference<>(); - final LogSorter logSorter = new LogSorter(getContext(), getConfiguration()); + final LogSorter logSorter = new LogSorter(this); long nextSortLogsCheckTime = System.currentTimeMillis(); - while (!shutdown) { - - // mark compactor as idle while not in the compaction loop - updateIdleStatus(true); - - currentCompactionId.set(null); - err.set(null); - JOB_HOLDER.reset(); - - if (System.currentTimeMillis() > nextSortLogsCheckTime) { - // Attempt to process all existing log sorting work serially in this thread. - // When no work remains, this call will return so that we can look for compaction - // work. - LOG.debug("Checking to see if any recovery logs need sorting"); - nextSortLogsCheckTime = logSorter.sortLogsIfNeeded(); + while (!isShutdownRequested()) { + if (Thread.currentThread().isInterrupted()) { + LOG.info("Server process thread has been interrupted, shutting down"); + break; } - - TExternalCompactionJob job; try { - TNextCompactionJob next = getNextJob(getNextId()); - job = next.getJob(); - if (!job.isSetExternalCompactionId()) { - LOG.trace("No external compactions in group {}", this.getResourceGroup()); - UtilWaitThread.sleep(getWaitTimeBetweenCompactionChecks(next.getCompactorCount())); - continue; + // mark compactor as idle while not in the compaction loop + updateIdleStatus(true); + + currentCompactionId.set(null); + err.set(null); + JOB_HOLDER.reset(); + + if (System.currentTimeMillis() > nextSortLogsCheckTime) { + // Attempt to process all existing log sorting work serially in this thread. + // When no work remains, this call will return so that we can look for compaction + // work. + LOG.debug("Checking to see if any recovery logs need sorting"); + nextSortLogsCheckTime = logSorter.sortLogsIfNeeded(); } - if (!job.getExternalCompactionId().equals(currentCompactionId.get().toString())) { - throw new IllegalStateException("Returned eci " + job.getExternalCompactionId() - + " does not match supplied eci " + currentCompactionId.get()); + + TExternalCompactionJob job; + try { + TNextCompactionJob next = getNextJob(getNextId()); + job = next.getJob(); + if (!job.isSetExternalCompactionId()) { + LOG.trace("No external compactions in queue {}", this.getResourceGroup()); + 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; } - } 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, paused %d times", - entriesRead, inputEntries, percentComplete, "%", entriesWritten, - info.getTimesPaused()); - 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); + 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 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 { - 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(); } @@ -900,6 +902,7 @@ public void run() { } getContext().getLowMemoryDetector().logGCInfo(getConfiguration()); + getShutdownComplete().set(true); LOG.info("stop requested. exiting ... "); try { if (null != compactorLock) { 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 62449eabb65..4004326622b 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 @@ -195,143 +195,159 @@ 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(); - try { - System.gc(); // make room + 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(); + } - 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 tStop = System.nanoTime(); + log.info(String.format("Collect cycle took %.2f seconds", + (TimeUnit.NANOSECONDS.toMillis(tStop - tStart) / 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); + 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(); + } - log.info("Starting Root table Garbage Collection."); - status.current.bulks += new GarbageCollectionAlgorithm().collect(rootGC); - incrementStatsForRun(rootGC); - logStats(); + // 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(AccumuloTable.METADATA.name(), null, null, + true, true); + accumuloClient.tableOperations().compact(AccumuloTable.ROOT.name(), null, null, + true, true); + break; + case "flush": + accumuloClient.tableOperations().flush(AccumuloTable.METADATA.name(), null, null, + true); + accumuloClient.tableOperations().flush(AccumuloTable.ROOT.name(), null, null, true); + break; + default: + log.trace("'none - no action' or invalid value provided: {}", action); + } - log.info("Starting Metadata table Garbage Collection."); - status.current.bulks += new GarbageCollectionAlgorithm().collect(mdGC); - incrementStatsForRun(mdGC); - logStats(); + final long actionComplete = System.nanoTime(); - log.info("Starting User table Garbage Collection."); - status.current.bulks += new GarbageCollectionAlgorithm().collect(userGC); - incrementStatsForRun(userGC); - logStats(); + gcCycleMetrics.setPostOpDurationNanos(actionComplete - actionStart); - } 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(); - } + log.info("gc post action {} completed in {} seconds", action, String.format("%.2f", + (TimeUnit.NANOSECONDS.toMillis(actionComplete - actionStart) / 1000.0))); - final long tStop = System.nanoTime(); - log.info(String.format("Collect cycle took %.2f seconds", - (TimeUnit.NANOSECONDS.toMillis(tStop - tStart) / 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); - 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(AccumuloTable.METADATA.tableName(), null, - null, true, true); - accumuloClient.tableOperations().compact(AccumuloTable.ROOT.tableName(), null, null, - true, true); - break; - case "flush": - accumuloClient.tableOperations().flush(AccumuloTable.METADATA.tableName(), null, null, - true); - accumuloClient.tableOperations().flush(AccumuloTable.ROOT.tableName(), 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); - log.warn("{}", e.getMessage(), 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); - - if (lastCompactorCheck.hasElapsed(gcDelay * 3, MILLISECONDS)) { - Map> resourceMapping = new HashMap<>(); - for (TableId tid : AccumuloTable.allTableIds()) { - TableConfiguration tconf = getContext().getTableConfiguration(tid); - String resourceGroup = tconf.get(TableLoadBalancer.TABLE_ASSIGNMENT_GROUP_PROPERTY); - resourceGroup = - resourceGroup == null ? Constants.DEFAULT_RESOURCE_GROUP_NAME : resourceGroup; - resourceMapping.getOrDefault(resourceGroup, new HashSet<>()).add(tid); - } - for (Entry> e : resourceMapping.entrySet()) { - if (ExternalCompactionUtil.countCompactors(e.getKey(), getContext()) == 0) { - log.warn("No Compactors exist in resource group {} for system table {}", e.getKey(), - e.getValue()); + gcCycleMetrics.incrementRunCycleCount(); + long gcDelay = getConfiguration().getTimeInMillis(Property.GC_CYCLE_DELAY); + + if (lastCompactorCheck.hasElapsed(gcDelay * 3, MILLISECONDS)) { + Map> resourceMapping = new HashMap<>(); + for (TableId tid : AccumuloTable.allTableIds()) { + TableConfiguration tconf = getContext().getTableConfiguration(tid); + String resourceGroup = tconf.get(TableLoadBalancer.TABLE_ASSIGNMENT_GROUP_PROPERTY); + resourceGroup = + resourceGroup == null ? Constants.DEFAULT_RESOURCE_GROUP_NAME : resourceGroup; + resourceMapping.getOrDefault(resourceGroup, new HashSet<>()).add(tid); + } + for (Entry> e : resourceMapping.entrySet()) { + if (ExternalCompactionUtil.countCompactors(e.getKey(), getContext()) == 0) { + log.warn("No Compactors exist in resource group {} for system table {}", e.getKey(), + e.getValue()); + } } + lastCompactorCheck.restart(); } - lastCompactorCheck.restart(); - } - log.debug("Sleeping for {} milliseconds", gcDelay); - Thread.sleep(gcDelay); + log.debug("Sleeping for {} milliseconds", gcDelay); + Thread.sleep(gcDelay); + } catch (InterruptedException e) { + log.warn("{}", e.getMessage(), e); + throw e; + } } 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) { @@ -371,7 +387,8 @@ private void getZooLock(HostAndPort addr) throws KeeperException, InterruptedExc UUID zooLockUUID = UUID.randomUUID(); gcLock = new ServiceLock(getContext().getZooSession(), path, zooLockUUID); - HAServiceLockWatcher gcLockWatcher = new HAServiceLockWatcher(Type.GARBAGE_COLLECTOR); + HAServiceLockWatcher gcLockWatcher = + new HAServiceLockWatcher(Type.GARBAGE_COLLECTOR, () -> getShutdownComplete().get()); while (true) { gcLock.lock(gcLockWatcher, new ServiceLockData(zooLockUUID, addr.toString(), ThriftService.GC, @@ -398,7 +415,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); long maxMessageSize = getConfiguration().getAsBytes(Property.RPC_MAX_MESSAGE_SIZE); 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 eb22227bb91..48a049fcf99 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 @@ -1136,7 +1136,7 @@ public void run() { HighlyAvailableServiceWrapper.service(managerClientHandler, this); ServerAddress sa; - var processor = ThriftProcessorTypes.getManagerTProcessor(fateServiceHandler, + var processor = ThriftProcessorTypes.getManagerTProcessor(this, fateServiceHandler, compactionCoordinator.getThriftService(), haProxy, getContext()); try { @@ -1335,14 +1335,27 @@ 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("Shutting down fate."); getFateRefs().keySet().forEach(type -> fate(type).shutdown(0, MINUTES)); splitter.stop(); + log.debug("Stopping Thrift Servers"); + sa.server.stop(); + final long deadline = System.currentTimeMillis() + MAX_CLEANUP_WAIT_TIME; try { statusThread.join(remaining(deadline)); @@ -1376,7 +1389,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); + } } protected Fate initializeFateInstance(ServerContext context, FateStore store) { @@ -1502,7 +1521,8 @@ private ServiceLockData getManagerLock(final ServiceLockPath zManagerLoc) managerClientAddress, this.getResourceGroup())); ServiceLockData sld = new ServiceLockData(descriptors); managerLock = new ServiceLock(zooKeeper, zManagerLoc, zooLockUUID); - HAServiceLockWatcher managerLockWatcher = new HAServiceLockWatcher(Type.MANAGER); + HAServiceLockWatcher managerLockWatcher = + new HAServiceLockWatcher(Type.MANAGER, () -> getShutdownComplete().get()); while (true) { 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 2bf22a28418..94ddc573f16 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 @@ -47,6 +47,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; @@ -331,6 +332,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 reportTabletStatus(TInfo info, TCredentials credentials, String serverName, TabletLoadState status, TKeyExtent ttablet) 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 7a4d5db9c37..9c2dd1411d7 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 @@ -80,7 +80,7 @@ public RecoveryManager(Manager manager, long timeToCacheExistsInMillis) { try { List workIDs = new DistributedWorkQueue(manager.getContext().getZooKeeperRoot() + Constants.ZRECOVERY, - manager.getConfiguration(), manager.getContext()).getWorkQueued(); + manager.getConfiguration(), manager).getWorkQueued(); sortsQueued.addAll(workIDs); } catch (Exception e) { log.warn("{}", e.getMessage(), e); @@ -133,7 +133,7 @@ private void initiateSort(String sortId, String source, final String destination throws KeeperException, InterruptedException { String work = source + "|" + destination; new DistributedWorkQueue(manager.getContext().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/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 ac142dc6717..4bcb06a5520 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 @@ -423,6 +423,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() { @@ -718,7 +734,8 @@ private void getMonitorLock(HostAndPort monitorLocation) // Get a ZooLock for the monitor UUID zooLockUUID = UUID.randomUUID(); monitorLock = new ServiceLock(context.getZooSession(), monitorLockPath, zooLockUUID); - HAServiceLockWatcher monitorLockWatcher = new HAServiceLockWatcher(Type.MONITOR); + HAServiceLockWatcher monitorLockWatcher = + new HAServiceLockWatcher(Type.MONITOR, () -> isShutdownRequested()); while (true) { monitorLock.lock(monitorLockWatcher, 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 8a409d566b6..1a00514f7f6 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 @@ -198,7 +198,6 @@ private TabletMetadataLoader(Ample ample) { private final TabletServerResourceManager resourceManager; HostAndPort clientAddress; - protected volatile boolean serverStopRequested = false; private ServiceLock scanServerLock; protected TabletServerScanMetrics scanMetrics; private ScanServerMetrics scanServerMetrics; @@ -305,7 +304,7 @@ protected ServerAddress startScanServerClientService() throws UnknownHostExcepti // to set up the ThriftProcessor using this class, not the delegate. ClientServiceHandler clientHandler = new ClientServiceHandler(context); TProcessor processor = - ThriftProcessorTypes.getScanServerTProcessor(clientHandler, this, getContext()); + ThriftProcessorTypes.getScanServerTProcessor(this, clientHandler, this, getContext()); ServerAddress sp = TServerUtils.startServer(getContext(), getHostname(), Property.SSERV_CLIENTPORT, processor, this.getClass().getSimpleName(), @@ -335,7 +334,7 @@ private ServiceLock announceExistence() { ServiceLockSupport.createNonHaServiceLockPath(Type.SCAN_SERVER, zoo, zLockPath); serverLockUUID = UUID.randomUUID(); scanServerLock = new ServiceLock(getContext().getZooSession(), zLockPath, serverLockUUID); - LockWatcher lw = new ServiceLockWatcher(Type.SCAN_SERVER, () -> serverStopRequested, + LockWatcher lw = new ServiceLockWatcher(Type.SCAN_SERVER, () -> getShutdownComplete().get(), (type) -> context.getLowMemoryDetector().logGCInfo(getConfiguration())); for (int i = 0; i < 120 / 5; i++) { @@ -394,11 +393,11 @@ public void run() { int threadPoolSize = getConfiguration().getCount(Property.SSERV_WAL_SORT_MAX_CONCURRENT); if (threadPoolSize > 0) { - final LogSorter logSorter = new LogSorter(context, getConfiguration()); + final LogSorter logSorter = new LogSorter(this); try { // Attempt to process all existing log sorting work and start a background // thread to look for log sorting work in the future - logSorter.startWatchingForRecoveryLogs(threadPoolSize); + logSorter.startWatchingForRecoveryLogs(); } catch (Exception ex) { LOG.error("Error starting LogSorter"); throw new RuntimeException(ex); @@ -409,18 +408,38 @@ public void run() { } 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(); - LOG.info("Removing server scan references"); - this.getContext().getAmple().scanServerRefs().delete(clientAddress.toString(), - serverLockUUID); + try { + LOG.info("Removing server scan references"); + this.getContext().getAmple().scanServerRefs().delete(clientAddress.toString(), + serverLockUUID); + } catch (Exception e) { + LOG.warn("Failed to remove scan server refs from metadata location", e); + } try { LOG.debug("Closing filesystems"); @@ -438,6 +457,7 @@ public void run() { } context.getLowMemoryDetector().logGCInfo(getConfiguration()); + getShutdownComplete().set(true); LOG.info("stop requested. exiting ... "); try { if (null != lock) { @@ -933,6 +953,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.isSystemTable() && !isSystemUser(credentials)) { @@ -994,6 +1019,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 aea44fcdafc..4686460c726 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 @@ -51,8 +51,10 @@ 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; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; @@ -95,6 +97,7 @@ import org.apache.accumulo.core.metadata.AccumuloTable; import org.apache.accumulo.core.metadata.TServerInstance; import org.apache.accumulo.core.metadata.schema.Ample; +import org.apache.accumulo.core.metadata.schema.Ample.DataLevel; import org.apache.accumulo.core.metadata.schema.TabletMetadata; import org.apache.accumulo.core.metrics.MetricsInfo; import org.apache.accumulo.core.rpc.ThriftUtil; @@ -102,15 +105,20 @@ import org.apache.accumulo.core.spi.fs.VolumeChooserEnvironment; import org.apache.accumulo.core.spi.ondemand.OnDemandTabletUnloader; import org.apache.accumulo.core.spi.ondemand.OnDemandTabletUnloader.UnloaderParams; +import org.apache.accumulo.core.tablet.thrift.TUnloadTabletGoal; import org.apache.accumulo.core.tabletserver.UnloaderParamsImpl; import org.apache.accumulo.core.tabletserver.log.LogEntry; +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.MapCounter; import org.apache.accumulo.core.util.Pair; import org.apache.accumulo.core.util.Retry; import org.apache.accumulo.core.util.Retry.RetryFactory; import org.apache.accumulo.core.util.UtilWaitThread; +import org.apache.accumulo.core.util.threads.ThreadPoolNames; import org.apache.accumulo.core.util.threads.Threads; +import org.apache.accumulo.core.util.time.SteadyTime; import org.apache.accumulo.server.AbstractServer; import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.server.ServiceEnvironmentImpl; @@ -204,9 +212,6 @@ public PausedCompactionMetrics getPausedCompactionMetrics() { volatile HostAndPort clientAddress; - private volatile boolean serverStopRequested = false; - private volatile boolean shutdownComplete = false; - private ServiceLock tabletServerLock; private TServer server; @@ -238,7 +243,7 @@ protected TabletServer(ConfigOpts opts, log.info("Version " + Constants.VERSION); log.info("Instance " + getInstanceID()); this.sessionManager = new SessionManager(context); - this.logSorter = new LogSorter(context, aconf); + this.logSorter = new LogSorter(this); this.statsKeeper = new TabletStatsKeeper(); final int numBusyTabletsToLog = aconf.getCount(Property.TSERV_LOG_BUSY_TABLETS_COUNT); final long logBusyTabletsDelay = @@ -375,7 +380,7 @@ String getLockID() { void requestStop() { log.info("Stop requested."); - serverStopRequested = true; + gracefulShutdown(getContext().rpcCreds()); } public long updateTotalQueuedMutationSize(long additionalMutationSize) { @@ -471,7 +476,7 @@ private HostAndPort startTabletClientService() throws UnknownHostException { scanClientHandler = newThriftScanClientHandler(writeTracker); TProcessor processor = - ThriftProcessorTypes.getTabletServerTProcessor(clientHandler, thriftClientHandler, + ThriftProcessorTypes.getTabletServerTProcessor(this, clientHandler, thriftClientHandler, scanClientHandler, thriftClientHandler, thriftClientHandler, getContext()); HostAndPort address = startServer(clientAddress.getHost(), processor); log.info("address = {}", address); @@ -498,7 +503,7 @@ private void announceExistence() { UUID tabletServerUUID = UUID.randomUUID(); tabletServerLock = new ServiceLock(getContext().getZooSession(), zLockPath, tabletServerUUID); - LockWatcher lw = new ServiceLockWatcher(Type.TABLET_SERVER, () -> serverStopRequested, + LockWatcher lw = new ServiceLockWatcher(Type.TABLET_SERVER, () -> getShutdownComplete().get(), (type) -> context.getLowMemoryDetector().logGCInfo(getConfiguration())); for (int i = 0; i < 120 / 5; i++) { @@ -589,7 +594,7 @@ public void run() { try { // Attempt to process all existing log sorting work and start a background // thread to look for log sorting work in the future - logSorter.startWatchingForRecoveryLogs(threadPoolSize); + logSorter.startWatchingForRecoveryLogs(); } catch (Exception ex) { log.error("Error starting LogSorter"); throw new RuntimeException(ex); @@ -608,7 +613,11 @@ public void run() { }); HostAndPort managerHost; - while (!serverStopRequested) { + while (!isShutdownRequested()) { + if (Thread.currentThread().isInterrupted()) { + log.info("Server process thread has been interrupted, shutting down"); + break; + } updateIdleStatus(getOnlineTablets().isEmpty()); @@ -620,7 +629,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()); } @@ -633,8 +642,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 { @@ -664,7 +673,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 @@ -673,20 +682,77 @@ 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) { + // 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); + } + + // 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, + SteadyTime.from(System.currentTimeMillis(), TimeUnit.MILLISECONDS)))); + } + }); + 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 { - this.wait(1000); - } catch (InterruptedException e) { - log.error(e.toString()); + 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"); @@ -703,8 +769,8 @@ public void run() { context.getLowMemoryDetector().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 1fdae2890e9..74fa6680be0 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 @@ -47,6 +47,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; @@ -222,15 +223,17 @@ synchronized long getBytesCopied() throws IOException { } } + private final AbstractServer server; private final ServerContext context; private final AccumuloConfiguration conf; private final double walBlockSize; private final CryptoService cryptoService; private final AccumuloConfiguration sortedLogConf; - public LogSorter(ServerContext context, AccumuloConfiguration conf) { - this.context = context; - this.conf = conf; + public LogSorter(AbstractServer server) { + this.server = server; + this.context = this.server.getContext(); + this.conf = this.context.getConfiguration(); this.sortedLogConf = extractSortedLogConfig(this.conf); this.walBlockSize = DfsLogger.getWalBlockSize(this.conf); CryptoEnvironment env = new CryptoEnvironmentImpl(CryptoEnvironment.Scope.RECOVERY); @@ -299,7 +302,7 @@ void writeBuffer(String destPath, List> buffer, in */ public long sortLogsIfNeeded() throws KeeperException, InterruptedException { DistributedWorkQueue dwq = new DistributedWorkQueue( - context.getZooKeeperRoot() + Constants.ZRECOVERY, sortedLogConf, context); + context.getZooKeeperRoot() + Constants.ZRECOVERY, sortedLogConf, server); dwq.processExistingWork(new LogProcessor(), MoreExecutors.newDirectExecutorService(), 1, false); return System.currentTimeMillis() + dwq.getCheckInterval(); } @@ -310,13 +313,13 @@ public long sortLogsIfNeeded() throws KeeperException, InterruptedException { * thread to look for log sorting work in the future that will be processed by the * ThreadPoolExecutor */ - public void startWatchingForRecoveryLogs(int threadPoolSize) - throws KeeperException, InterruptedException { + public void startWatchingForRecoveryLogs() throws KeeperException, InterruptedException { + int threadPoolSize = this.conf.getCount(Property.TSERV_WAL_SORT_MAX_CONCURRENT); ThreadPoolExecutor threadPool = ThreadPools.getServerThreadPools().getPoolBuilder(TSERVER_WAL_SORT_CONCURRENT_POOL) .numCoreThreads(threadPoolSize).enableThreadPoolMetrics().build(); new DistributedWorkQueue(context.getZooKeeperRoot() + Constants.ZRECOVERY, sortedLogConf, - context).processExistingAndFuture(new LogProcessor(), threadPool); + server).processExistingAndFuture(new LogProcessor(), threadPool); } public List getLogSorts() { 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 bbe8ffb3fc9..f6b95597031 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 @@ -119,6 +119,11 @@ protected boolean isSystemUser(TCredentials creds) { return systemUser; } + @Override + public boolean isShutdownRequested() { + return false; + } + } private ThriftScanClientHandler handler; diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/log/RecoveryLogsIteratorTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/log/RecoveryLogsIteratorTest.java index dd9b6892299..3d070c983b7 100644 --- a/server/tserver/src/test/java/org/apache/accumulo/tserver/log/RecoveryLogsIteratorTest.java +++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/log/RecoveryLogsIteratorTest.java @@ -23,6 +23,7 @@ import static org.easymock.EasyMock.createMock; import static org.easymock.EasyMock.expect; import static org.easymock.EasyMock.replay; +import static org.easymock.EasyMock.verify; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -48,6 +49,7 @@ import org.apache.accumulo.server.fs.VolumeManager; import org.apache.accumulo.server.fs.VolumeManagerImpl; import org.apache.accumulo.server.log.SortedLogState; +import org.apache.accumulo.tserver.TabletServer; import org.apache.accumulo.tserver.WithTestNames; import org.apache.accumulo.tserver.logger.LogFileKey; import org.apache.accumulo.tserver.logger.LogFileValue; @@ -66,6 +68,7 @@ public class RecoveryLogsIteratorTest extends WithTestNames { private VolumeManager fs; private File workDir; static final KeyExtent extent = new KeyExtent(TableId.of("table"), null, null); + static TabletServer server; static ServerContext context; static LogSorter logSorter; @@ -75,21 +78,23 @@ public class RecoveryLogsIteratorTest extends WithTestNames { @BeforeEach public void setUp() throws Exception { context = createMock(ServerContext.class); - + server = createMock(TabletServer.class); workDir = new File(tempDir, testName()); String path = workDir.getAbsolutePath(); fs = VolumeManagerImpl.getLocalForTesting(path); + expect(server.getContext()).andReturn(context).anyTimes(); expect(context.getCryptoFactory()).andReturn(new GenericCryptoServiceFactory()).anyTimes(); expect(context.getVolumeManager()).andReturn(fs).anyTimes(); expect(context.getConfiguration()).andReturn(DefaultConfiguration.getInstance()).anyTimes(); - replay(context); + replay(server, context); - logSorter = new LogSorter(context, DefaultConfiguration.getInstance()); + logSorter = new LogSorter(server); } @AfterEach public void tearDown() throws Exception { fs.close(); + verify(server, context); } static class KeyValue implements Comparable { diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/log/SortedLogRecoveryTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/log/SortedLogRecoveryTest.java index 5645dea90f3..bffb83f594c 100644 --- a/server/tserver/src/test/java/org/apache/accumulo/tserver/log/SortedLogRecoveryTest.java +++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/log/SortedLogRecoveryTest.java @@ -70,6 +70,7 @@ import org.apache.accumulo.server.fs.VolumeManager; import org.apache.accumulo.server.fs.VolumeManagerImpl; import org.apache.accumulo.server.log.SortedLogState; +import org.apache.accumulo.tserver.TabletServer; import org.apache.accumulo.tserver.WithTestNames; import org.apache.accumulo.tserver.logger.LogEvents; import org.apache.accumulo.tserver.logger.LogFileKey; @@ -96,6 +97,7 @@ public class SortedLogRecoveryTest extends WithTestNames { static final Text cf = new Text("cf"); static final Text cq = new Text("cq"); static final Value value = new Value("value"); + static TabletServer server; static ServerContext context; static LogSorter logSorter; @@ -104,6 +106,7 @@ public class SortedLogRecoveryTest extends WithTestNames { @BeforeEach public void setup() { + server = EasyMock.createMock(TabletServer.class); context = EasyMock.createMock(ServerContext.class); } @@ -186,12 +189,12 @@ private List recover(Map logs, Set files, K final String workdir = new File(tempDir, testName()).getAbsolutePath(); try (var fs = VolumeManagerImpl.getLocalForTesting(workdir)) { CryptoServiceFactory cryptoFactory = new GenericCryptoServiceFactory(); - + expect(server.getContext()).andReturn(context).anyTimes(); expect(context.getVolumeManager()).andReturn(fs).anyTimes(); expect(context.getCryptoFactory()).andReturn(cryptoFactory).anyTimes(); expect(context.getConfiguration()).andReturn(DefaultConfiguration.getInstance()).anyTimes(); - replay(context); - logSorter = new LogSorter(context, DefaultConfiguration.getInstance()); + replay(server, context); + logSorter = new LogSorter(server); final Path workdirPath = new Path("file://" + workdir); fs.deleteRecursively(workdirPath); @@ -223,7 +226,7 @@ private List recover(Map logs, Set files, K SortedLogRecovery recovery = new SortedLogRecovery(context, fileLenCache, cacheProvider); CaptureMutations capture = new CaptureMutations(); recovery.recover(extent, dirs, files, capture); - verify(context); + verify(server, context); return capture.result; } } @@ -809,7 +812,7 @@ public void testMultipleTablets() throws IOException { assertEquals(1, mutations1.size()); assertEquals(m2, mutations1.get(0)); - reset(context); + reset(server, context); List mutations2 = recover(logs, e2); assertEquals(2, mutations2.size()); assertEquals(m3, mutations2.get(0)); @@ -820,7 +823,7 @@ public void testMultipleTablets() throws IOException { Arrays.sort(entries2); logs.put("entries2", entries2); - reset(context); + reset(server, context); mutations2 = recover(logs, e2); assertEquals(1, mutations2.size()); assertEquals(m4, mutations2.get(0)); @@ -860,7 +863,7 @@ public void testPaths() throws IOException { // test having different paths for the same file. This can happen as a result of upgrade or user // changing configuration runPathTest(false, "/t1/f1", "/t1/f0"); - reset(context); + reset(server, context); runPathTest(true, "/t1/f1", "/t1/f0", "/t1/f1"); String[] aliases = {"/t1/f1", "hdfs://nn1/accumulo/tables/8/t1/f1", @@ -871,12 +874,12 @@ public void testPaths() throws IOException { for (String alias1 : aliases) { for (String alias2 : aliases) { - reset(context); + reset(server, context); runPathTest(true, alias1, alias2); for (String other : others) { - reset(context); + reset(server, context); runPathTest(true, alias1, other, alias2); - reset(context); + reset(server, context); runPathTest(true, alias1, alias2, other); } } @@ -884,7 +887,7 @@ public void testPaths() throws IOException { for (String alias1 : aliases) { for (String other : others) { - reset(context); + reset(server, context); runPathTest(false, alias1, other); } } @@ -1035,34 +1038,34 @@ public void testEmptyLogFiles() throws IOException { logs.put("entries2", entries2); - reset(context); + reset(server, context); mutations = recover(logs, extent); assertEquals(1, mutations.size()); assertEquals(m1, mutations.get(0)); logs.put("entries3", entries3); - reset(context); + reset(server, context); mutations = recover(logs, extent); assertEquals(1, mutations.size()); assertEquals(m1, mutations.get(0)); logs.put("entries4", entries4); - reset(context); + reset(server, context); mutations = recover(logs, extent); assertEquals(1, mutations.size()); assertEquals(m1, mutations.get(0)); logs.put("entries5", entries5); - reset(context); + reset(server, context); mutations = recover(logs, extent); assertEquals(0, mutations.size()); logs.put("entries6", entries6); - reset(context); + reset(server, context); mutations = recover(logs, extent); assertEquals(1, mutations.size()); assertEquals(m2, mutations.get(0)); @@ -1098,8 +1101,12 @@ public void testInvalidLogSortedProperties() { // test all the possible properties for tserver.sort.file. prefix String prop = Property.TSERV_WAL_SORT_FILE_PREFIX + "invalid"; testConfig.set(prop, "snappy"); - assertThrows(IllegalArgumentException.class, () -> new LogSorter(context, testConfig), + expect(server.getContext()).andReturn(context).anyTimes(); + expect(context.getConfiguration()).andReturn(testConfig).anyTimes(); + replay(server, context); + assertThrows(IllegalArgumentException.class, () -> new LogSorter(server), "Did not throw IllegalArgumentException for " + prop); + verify(server, context); } @Test @@ -1122,11 +1129,12 @@ public void testLogSortedProperties() throws Exception { try (var vm = VolumeManagerImpl.getLocalForTesting(workdir)) { CryptoServiceFactory cryptoFactory = new GenericCryptoServiceFactory(); + expect(server.getContext()).andReturn(context).anyTimes(); expect(context.getCryptoFactory()).andReturn(cryptoFactory).anyTimes(); expect(context.getVolumeManager()).andReturn(vm).anyTimes(); - expect(context.getConfiguration()).andReturn(DefaultConfiguration.getInstance()).anyTimes(); - replay(context); - LogSorter sorter = new LogSorter(context, testConfig); + expect(context.getConfiguration()).andReturn(testConfig).anyTimes(); + replay(server, context); + LogSorter sorter = new LogSorter(server); final Path workdirPath = new Path("file://" + workdir); vm.deleteRecursively(workdirPath); diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/log/TestUpgradePathForWALogs.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/log/TestUpgradePathForWALogs.java index 8aab03b5c64..cd7d3bf29f7 100644 --- a/server/tserver/src/test/java/org/apache/accumulo/tserver/log/TestUpgradePathForWALogs.java +++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/log/TestUpgradePathForWALogs.java @@ -33,12 +33,12 @@ import java.io.InputStream; import java.io.OutputStream; -import org.apache.accumulo.core.conf.AccumuloConfiguration; import org.apache.accumulo.core.conf.DefaultConfiguration; import org.apache.accumulo.core.spi.crypto.GenericCryptoServiceFactory; import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.server.fs.VolumeManager; import org.apache.accumulo.server.fs.VolumeManagerImpl; +import org.apache.accumulo.tserver.TabletServer; import org.apache.accumulo.tserver.WithTestNames; import org.apache.commons.io.IOUtils; import org.apache.hadoop.fs.Path; @@ -60,8 +60,8 @@ public class TestUpgradePathForWALogs extends WithTestNames { // logs from 2.0 were changed for improved crypto private static final String WALOG_FROM_20 = "/walog-from-20.walog"; - private static final AccumuloConfiguration config = DefaultConfiguration.getInstance(); private ServerContext context; + private TabletServer server; @TempDir private static File tempDir; @@ -71,6 +71,7 @@ public class TestUpgradePathForWALogs extends WithTestNames { @BeforeEach public void setUp() throws Exception { context = createMock(ServerContext.class); + server = createMock(TabletServer.class); // Create a new subdirectory for each test perTestTempSubDir = new File(tempDir, testName()); @@ -81,14 +82,16 @@ public void setUp() throws Exception { VolumeManager fs = VolumeManagerImpl.getLocalForTesting(path); + expect(server.getContext()).andReturn(context).anyTimes(); + expect(context.getConfiguration()).andReturn(DefaultConfiguration.getInstance()).anyTimes(); expect(context.getCryptoFactory()).andReturn(new GenericCryptoServiceFactory()).anyTimes(); expect(context.getVolumeManager()).andReturn(fs).anyTimes(); - replay(context); + replay(server, context); } @AfterEach public void tearDown() { - verify(context); + verify(server, context); } /** @@ -105,7 +108,7 @@ public void testUpgradeOf15WALog() throws IOException { walogInHDFStream.flush(); walogInHDFStream.close(); - LogSorter logSorter = new LogSorter(context, config); + LogSorter logSorter = new LogSorter(server); LogSorter.LogProcessor logProcessor = logSorter.new LogProcessor(); assertThrows(IllegalArgumentException.class, @@ -128,7 +131,7 @@ public void testBasic16WALogRead() throws IOException { assertFalse(context.getVolumeManager().exists(getFinishedMarkerPath(destPath))); - LogSorter logSorter = new LogSorter(context, config); + LogSorter logSorter = new LogSorter(server); LogSorter.LogProcessor logProcessor = logSorter.new LogProcessor(); logProcessor.sort(context.getVolumeManager(), walogToTest, @@ -152,7 +155,7 @@ public void testBasic20WALogRead() throws IOException { assertFalse(context.getVolumeManager().exists(getFinishedMarkerPath(destPath))); - LogSorter logSorter = new LogSorter(context, config); + LogSorter logSorter = new LogSorter(server); LogSorter.LogProcessor logProcessor = logSorter.new LogProcessor(); logProcessor.sort(context.getVolumeManager(), walogToTest, new Path("file://" + testPath + walogToTest), destPath); 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 fe9e15ec04c..8a526220424 100644 --- a/test/src/main/java/org/apache/accumulo/test/SelfStoppingScanServer.java +++ b/test/src/main/java/org/apache/accumulo/test/SelfStoppingScanServer.java @@ -43,7 +43,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/ExternalDoNothingCompactor.java b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalDoNothingCompactor.java index a032aeb1918..c579ea25c20 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 @@ -66,7 +66,7 @@ protected FileCompactorRunnable createCompactionJob(TExternalCompactionJob job, // Set this to true so that only 1 external compaction is run final AtomicReference ref = new AtomicReference<>(); - this.shutdown = true; + gracefulShutdown(getContext().rpcCreds()); return new FileCompactorRunnable() { diff --git a/test/src/main/java/org/apache/accumulo/test/fate/FateExecutionOrderIT.java b/test/src/main/java/org/apache/accumulo/test/fate/FateExecutionOrderIT.java index 9c1234efd42..4b377e09da8 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/FateExecutionOrderIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/FateExecutionOrderIT.java @@ -57,6 +57,7 @@ import org.apache.accumulo.core.fate.Repo; import org.apache.accumulo.harness.SharedMiniClusterBase; import org.apache.accumulo.server.ServerContext; +import org.apache.accumulo.test.fate.FateTestRunner.TestEnv; import org.apache.hadoop.io.Text; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; 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..c2d4f9575c5 --- /dev/null +++ b/test/src/main/java/org/apache/accumulo/test/functional/GracefulShutdownIT.java @@ -0,0 +1,295 @@ +/* + * 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 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.Optional; +import java.util.Set; +import java.util.stream.IntStream; + +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.client.admin.servers.ServerId; +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.lock.ServiceLock; +import org.apache.accumulo.core.lock.ServiceLockData; +import org.apache.accumulo.core.lock.ServiceLockData.ThriftService; +import org.apache.accumulo.core.lock.ServiceLockPaths.AddressSelector; +import org.apache.accumulo.core.lock.ServiceLockPaths.ServiceLockPath; +import org.apache.accumulo.core.metadata.schema.TabletsMetadata; +import org.apache.accumulo.core.security.Authorizations; +import org.apache.accumulo.core.spi.compaction.RatioBasedCompactionPlanner; +import org.apache.accumulo.core.spi.compaction.SimpleCompactionDispatcher; +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.hadoop.conf.Configuration; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import com.google.common.net.HostAndPort; + +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.getClusterServerConfiguration().setNumDefaultCompactors(0); + cfg.getClusterServerConfiguration().setNumDefaultScanServers(0); + cfg.getClusterServerConfiguration().setNumDefaultTabletServers(2); + cfg.setProperty(Property.COMPACTION_COORDINATOR_DEAD_COMPACTOR_CHECK_INTERVAL, "5s"); + cfg.setProperty(Property.COMPACTOR_CANCEL_CHECK_INTERVAL, "5s"); + cfg.setProperty(Property.COMPACTOR_PORTSEARCH, "true"); + cfg.setProperty(Property.COMPACTION_SERVICE_PREFIX.getKey() + GROUP_NAME + ".planner", + RatioBasedCompactionPlanner.class.getName()); + cfg.setProperty( + Property.COMPACTION_SERVICE_PREFIX.getKey() + GROUP_NAME + ".planner.opts.groups", + "[{'group': '" + 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 = + getCluster().getServerContext().getServerPaths().getGarbageCollector(true); + Optional data = ServiceLock.getLockData(ctx.getZooSession(), gcLockPath); + assertTrue(data.isPresent()); + final HostAndPort gcAddress = data.orElseThrow().getAddress(ThriftService.GC); + 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 Set tservers = getCluster().getServerContext().getServerPaths() + .getTabletServer((rg) -> rg.equals(Constants.DEFAULT_RESOURCE_GROUP_NAME), + AddressSelector.all(), true); + assertEquals(2, tservers.size()); + final HostAndPort tserverAddress = + HostAndPort.fromString(tservers.iterator().next().getServer()); + 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 ServiceLockPath manager = + getCluster().getServerContext().getServerPaths().getManager(true); + assertNotNull(manager); + Set managerLocations = + client.instanceOperations().getServers(ServerId.Type.MANAGER); + assertNotNull(managerLocations); + assertEquals(1, managerLocations.size()); + final HostAndPort managerAddress = + HostAndPort.fromString(managerLocations.iterator().next().toHostPortString()); + 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 + getCluster().getConfig().getClusterServerConfiguration().addCompactorResourceGroup(GROUP_NAME, + 1); + getCluster().getClusterControl().start(ServerType.COMPACTOR); + + Wait.waitFor(() -> getCluster().getServerContext().getServerPaths() + .getCompactor((rg) -> rg.equals(GROUP_NAME), AddressSelector.all(), true).size() == 1); + final Set compactors = getCluster().getServerContext().getServerPaths() + .getCompactor((rg) -> rg.equals(GROUP_NAME), AddressSelector.all(), true); + final HostAndPort compactorAddress = + HostAndPort.fromString(compactors.iterator().next().getServer()); + + 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); + Set newManagerLocations = + client.instanceOperations().getServers(ServerId.Type.MANAGER); + assertNotNull(newManagerLocations); + assertEquals(1, newManagerLocations.size()); + final HostAndPort newManagerAddress = + HostAndPort.fromString(newManagerLocations.iterator().next().toHostPortString()); + assertEquals(0, ExternalCompactionTestUtils + .getRunningCompactions(ctx, Optional.of(newManagerAddress)).getCompactionsSize()); + client.tableOperations().compact(tableName, cc); + Wait.waitFor(() -> ExternalCompactionTestUtils + .getRunningCompactions(ctx, Optional.of(newManagerAddress)).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().getClusterServerConfiguration() + .addScanServerResourceGroup(GROUP_NAME, 1); + getCluster().getClusterControl().start(ServerType.SCAN_SERVER); + + Wait.waitFor(() -> getCluster().getServerContext().getServerPaths() + .getScanServer((rg) -> rg.equals(GROUP_NAME), AddressSelector.all(), true).size() == 1); + final Set sservers = getCluster().getServerContext().getServerPaths() + .getScanServer((rg) -> rg.equals(GROUP_NAME), AddressSelector.all(), true); + final HostAndPort sserver = HostAndPort.fromString(sservers.iterator().next().getServer()); + 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(); + } + } +}