From 46f69faee9eef6ed22fbdf264f786b713f65a512 Mon Sep 17 00:00:00 2001 From: Keith Turner Date: Tue, 3 Mar 2026 23:35:43 +0000 Subject: [PATCH 01/13] Adds support for multiple managers running distributed fate MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Lays the foundation for multiple manager with the following changes. The best place to start looking at these changes is in the Manager.run() method which sets everything and ties it all together. * Each manager process acquires two zookeeper locks now, a primary lock and an assistant lock. Only one manager process can obtain the primary lock and when it does it assumes the role of primary manager.  All manager processes acquire an assistant lock, which is similar to a tserver or compactor lock.  The assistant lock advertises the manager process as being available to other Accumulo processes to handle assistant manager operations. * Manager processes have a single thrift server and thrift services hosted on that thrift server are categorized into primary manager and assistant manager services. When an assistant manager receives an RPC for a primary manager thrift service it will not execute the request and will throw an error or ignore the request. * The primary manager process delegates manager responsibility via RPCs to assistant managers. * Any management responsibility not delegated runs on the primary manager. Using the changes above fate is now distributed across all manager processes. In the future the changes above should make it easy to delegate other responsibilities to assistant managers. The following is an outline of the fate changes. * New FateWorker class. This runs in every manager and handles request from the primary manager to adjust what range of the fate table its currently responsible for. FateWorker implements a new thrift service used to assign it ranges. * New FateManager class that is run by the primary manager and is responsible for partitioning fate processing across all assistant managers. As manager processes come and go this will repartition the fate table evenly across all available managers. The FateManager communicates with FateWorkers via thrift. * Some new RPCs for best effort notifications. Before these changes there were in memory notification systems that made the manager more responsive. These would allow a fate operation to signal the Tablet Group Watcher to take action sooner. FateWorkerEnv sends these notifications to the primary manger over a new RPC. Does not matter if they are lost, things will still eventually happen. Other than fate, the primary manager process does everything the current manager does. This change pulls from #3262 and #6139. Co-authored-by: Dave Marion --- .../org/apache/accumulo/core/Constants.java | 1 + .../core/clientImpl/ClientContext.java | 2 +- .../accumulo/core/fate/AbstractFateStore.java | 27 +- .../org/apache/accumulo/core/fate/Fate.java | 48 +- .../apache/accumulo/core/fate/FateClient.java | 29 +- .../accumulo/core/fate/FateExecutor.java | 18 +- .../core/fate/FateExecutorMetrics.java | 4 +- .../accumulo/core/fate/FatePartition.java | 63 + .../apache/accumulo/core/fate/FateStore.java | 12 +- .../accumulo/core/fate/ReadOnlyFateStore.java | 6 +- .../core/fate/user/UserFateStore.java | 24 +- .../core/fate/zookeeper/MetaFateStore.java | 18 +- .../accumulo/core/lock/ServiceLockPaths.java | 28 +- .../accumulo/core/logging/FateLogger.java | 30 +- .../rpc/clients/FateWorkerThriftClient.java | 31 + .../core/rpc/clients/ThriftClientTypes.java | 4 + core/src/main/spotbugs/exclude-filter.xml | 1 + .../core/manager/thrift/FateService.java | 76 +- .../manager/thrift/FateWorkerService.java | 3778 +++++++++++++++++ .../manager/thrift/ManagerClientService.java | 1946 +++++++-- .../accumulo/core/manager/thrift/TEvent.java | 516 +++ .../core/manager/thrift/TFatePartition.java | 511 +++ .../core/manager/thrift/TFatePartitions.java | 561 +++ core/src/main/thrift/manager.thrift | 50 + .../apache/accumulo/core/fate/TestStore.java | 10 +- .../accumulo/server/AbstractServer.java | 2 +- .../server/init/ZooKeeperInitializer.java | 2 + .../server/rpc/ThriftProcessorTypes.java | 10 +- .../util/adminCommand/ServiceStatus.java | 3 + .../accumulo/manager/EventCoordinator.java | 50 +- .../org/apache/accumulo/manager/Manager.java | 215 +- .../manager/ManagerClientServiceHandler.java | 24 +- .../accumulo/manager/TabletGroupWatcher.java | 4 +- .../accumulo/manager/fate/FateManager.java | 453 ++ .../accumulo/manager/fate/FateWorker.java | 188 + .../accumulo/manager/fate/FateWorkerEnv.java | 228 + .../manager/metrics/ManagerMetrics.java | 13 +- .../fate/FateExecutorMetricsProducer.java | 74 + .../manager/metrics/fate/FateMetrics.java | 30 +- .../metrics/fate/meta/MetaFateMetrics.java | 8 +- .../metrics/fate/user/UserFateMetrics.java | 9 +- .../manager/tableOps/split/PreSplit.java | 2 +- .../manager/upgrade/Upgrader11to12.java | 11 + .../test/ComprehensiveMultiManagerIT.java | 58 + .../accumulo/test/MultipleManagerIT.java | 305 ++ .../test/fate/FateExecutionOrderITBase.java | 5 +- .../apache/accumulo/test/fate/FateITBase.java | 133 +- .../test/fate/FateOpsCommandsITBase.java | 6 +- .../test/fate/FatePoolsWatcherITBase.java | 7 + .../accumulo/test/fate/FateStoreITBase.java | 10 +- .../accumulo/test/fate/FlakyFateManager.java | 5 +- .../test/fate/MultipleStoresITBase.java | 22 +- .../test/fate/SlowFateSplitManager.java | 5 +- 53 files changed, 9135 insertions(+), 541 deletions(-) create mode 100644 core/src/main/java/org/apache/accumulo/core/fate/FatePartition.java create mode 100644 core/src/main/java/org/apache/accumulo/core/rpc/clients/FateWorkerThriftClient.java create mode 100644 core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/FateWorkerService.java create mode 100644 core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/TEvent.java create mode 100644 core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/TFatePartition.java create mode 100644 core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/TFatePartitions.java create mode 100644 server/manager/src/main/java/org/apache/accumulo/manager/fate/FateManager.java create mode 100644 server/manager/src/main/java/org/apache/accumulo/manager/fate/FateWorker.java create mode 100644 server/manager/src/main/java/org/apache/accumulo/manager/fate/FateWorkerEnv.java create mode 100644 server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateExecutorMetricsProducer.java create mode 100644 test/src/main/java/org/apache/accumulo/test/ComprehensiveMultiManagerIT.java create mode 100644 test/src/main/java/org/apache/accumulo/test/MultipleManagerIT.java diff --git a/core/src/main/java/org/apache/accumulo/core/Constants.java b/core/src/main/java/org/apache/accumulo/core/Constants.java index ea06bc4ad79..eb8ba1059eb 100644 --- a/core/src/main/java/org/apache/accumulo/core/Constants.java +++ b/core/src/main/java/org/apache/accumulo/core/Constants.java @@ -49,6 +49,7 @@ public class Constants { public static final String ZMANAGERS = "/managers"; public static final String ZMANAGER_LOCK = ZMANAGERS + "/lock"; + public static final String ZMANAGER_ASSISTANT_LOCK = ZMANAGERS + "/assistants"; public static final String ZMANAGER_GOAL_STATE = ZMANAGERS + "/goal_state"; public static final String ZMANAGER_TICK = ZMANAGERS + "/tick"; diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java index 41674ebca71..ac75c7dc65a 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java @@ -1292,7 +1292,7 @@ private static Set createPersistentWatcherPaths() { Constants.ZMANAGER_LOCK, Constants.ZMINI_LOCK, Constants.ZMONITOR_LOCK, Constants.ZNAMESPACES, Constants.ZRECOVERY, Constants.ZSSERVERS, Constants.ZTABLES, Constants.ZTSERVERS, Constants.ZUSERS, RootTable.ZROOT_TABLET, Constants.ZTEST_LOCK, - Constants.ZRESOURCEGROUPS)) { + Constants.ZMANAGER_ASSISTANT_LOCK, Constants.ZRESOURCEGROUPS)) { pathsToWatch.add(path); } return pathsToWatch; diff --git a/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java b/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java index 14755be12bb..f8acdd83063 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java @@ -162,7 +162,12 @@ public FateTxStore reserve(FateId fateId) { EnumSet.of(TStatus.SUBMITTED, TStatus.FAILED_IN_PROGRESS); @Override - public void runnable(BooleanSupplier keepWaiting, Consumer idConsumer) { + public void runnable(Set partitions, BooleanSupplier keepWaiting, + Consumer idConsumer) { + + if (partitions.isEmpty()) { + return; + } AtomicLong seen = new AtomicLong(0); @@ -170,8 +175,8 @@ public void runnable(BooleanSupplier keepWaiting, Consumer idConsu final long beforeCount = unreservedRunnableCount.getCount(); final boolean beforeDeferredOverflow = deferredOverflow.get(); - try (Stream inProgress = getTransactions(IN_PROGRESS_SET); - Stream other = getTransactions(OTHER_RUNNABLE_SET)) { + try (Stream inProgress = getTransactions(partitions, IN_PROGRESS_SET); + Stream other = getTransactions(partitions, OTHER_RUNNABLE_SET)) { // read the in progress transaction first and then everything else in order to process those // first var transactions = Stream.concat(inProgress, other); @@ -200,6 +205,8 @@ public void runnable(BooleanSupplier keepWaiting, Consumer idConsu if (beforeCount == unreservedRunnableCount.getCount()) { long waitTime = 5000; synchronized (deferred) { + deferred.keySet().removeIf( + fateId -> partitions.stream().noneMatch(partition -> partition.contains(fateId))); if (!deferred.isEmpty()) { waitTime = deferred.values().stream() .mapToLong(countDownTimer -> countDownTimer.timeLeft(TimeUnit.MILLISECONDS)).min() @@ -240,9 +247,11 @@ public ReadOnlyFateTxStore read(FateId fateId) { } @Override - public Map getActiveReservations() { - return list().filter(entry -> entry.getFateReservation().isPresent()).collect(Collectors - .toMap(FateIdStatus::getFateId, entry -> entry.getFateReservation().orElseThrow())); + public Map getActiveReservations(Set partitions) { + try (var stream = getTransactions(partitions, EnumSet.allOf(TStatus.class))) { + return stream.filter(entry -> entry.getFateReservation().isPresent()).collect(Collectors + .toMap(FateIdStatus::getFateId, entry -> entry.getFateReservation().orElseThrow())); + } } protected boolean isRunnable(TStatus status) { @@ -289,6 +298,9 @@ protected void verifyLock(ZooUtil.LockID lockID, FateId fateId) { protected abstract Stream getTransactions(EnumSet statuses); + protected abstract Stream getTransactions(Set partitions, + EnumSet statuses); + protected abstract TStatus _getStatus(FateId fateId); protected abstract Optional getKey(FateId fateId); @@ -418,7 +430,8 @@ public interface FateIdGenerator { FateId newRandomId(FateInstanceType instanceType); } - protected void seededTx() { + @Override + public void seeded() { unreservedRunnableCount.increment(); } diff --git a/core/src/main/java/org/apache/accumulo/core/fate/Fate.java b/core/src/main/java/org/apache/accumulo/core/fate/Fate.java index 58bdc5cef66..8f016db6402 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/Fate.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/Fate.java @@ -30,6 +30,7 @@ import java.util.HashMap; import java.util.HashSet; import java.util.Map; +import java.util.Objects; import java.util.Set; import java.util.TreeSet; import java.util.concurrent.ExecutorService; @@ -51,6 +52,8 @@ import org.slf4j.LoggerFactory; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.collect.Sets; import com.google.gson.JsonParser; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; @@ -76,6 +79,7 @@ public class Fate extends FateClient { private final AtomicBoolean keepRunning = new AtomicBoolean(true); // Visible for FlakyFate test object protected final Set> fateExecutors = new HashSet<>(); + private Set currentPartitions = Set.of(); public enum TxInfo { FATE_OP, AUTO_CLEAN, EXCEPTION, TX_AGEOFF, RETURN_VALUE @@ -208,8 +212,10 @@ public void run() { fe -> fe.getFateOps().equals(fateOps) && fe.getName().equals(fateExecutorName))) { log.debug("[{}] Adding FateExecutor for {} with {} threads", store.type(), fateOps, poolSize); - fateExecutors.add( - new FateExecutor<>(Fate.this, environment, fateOps, poolSize, fateExecutorName)); + var fateExecutor = + new FateExecutor<>(Fate.this, environment, fateOps, poolSize, fateExecutorName); + fateExecutors.add(fateExecutor); + fateExecutor.setPartitions(currentPartitions); } } } @@ -233,7 +239,11 @@ private class DeadReservationCleaner implements Runnable { @Override public void run() { if (keepRunning.get()) { - store.deleteDeadReservations(); + Set partitions; + synchronized (fateExecutors) { + partitions = currentPartitions; + } + store.deleteDeadReservations(partitions); } } } @@ -369,6 +379,17 @@ public AtomicInteger getNeedMoreThreadsWarnCount() { return needMoreThreadsWarnCount; } + public void seeded(Set partitions) { + synchronized (fateExecutors) { + if (Sets.intersection(currentPartitions, partitions).isEmpty()) { + return; + } + } + + log.trace("Notified of seeding for {}", partitions); + store.seeded(); + } + /** * Initiates shutdown of background threads that run fate operations and cleanup fate data and * optionally waits on them. Leaves the fate object in a state where it can still update and read @@ -432,6 +453,27 @@ public void close() { store.close(); } + public Set getPartitions() { + synchronized (fateExecutors) { + return currentPartitions; + } + } + + public Set setPartitions(Set partitions) { + Objects.requireNonNull(partitions); + Preconditions.checkArgument( + partitions.stream().allMatch( + fp -> fp.start().getType() == store.type() && fp.end().getType() == store.type()), + "type mismatch type:%s partitions:%s", store.type(), partitions); + + synchronized (fateExecutors) { + var old = currentPartitions; + currentPartitions = Set.copyOf(partitions); + fateExecutors.forEach(fe -> fe.setPartitions(currentPartitions)); + return old; + } + } + private boolean anyFateExecutorIsAlive() { synchronized (fateExecutors) { return fateExecutors.stream().anyMatch(FateExecutor::isAlive); diff --git a/core/src/main/java/org/apache/accumulo/core/fate/FateClient.java b/core/src/main/java/org/apache/accumulo/core/fate/FateClient.java index 2dc472e4bd8..f6f113e81c1 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/FateClient.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/FateClient.java @@ -28,6 +28,9 @@ import java.time.Duration; import java.util.EnumSet; import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; import java.util.function.Function; import java.util.stream.Stream; @@ -46,8 +49,11 @@ public class FateClient { private static final EnumSet FINISHED_STATES = EnumSet.of(FAILED, SUCCESSFUL, UNKNOWN); + private AtomicReference> seedingConsumer = new AtomicReference<>(fid -> {}); + public FateClient(FateStore store, Function,String> toLogStrFunc) { this.store = FateLogger.wrap(store, toLogStrFunc, false); + ; } // get a transaction id back to the requester before doing any work @@ -56,7 +62,23 @@ public FateId startTransaction() { } public FateStore.Seeder beginSeeding() { - return store.beginSeeding(); + var seeder = store.beginSeeding(); + return new FateStore.Seeder() { + @Override + public CompletableFuture> attemptToSeedTransaction(Fate.FateOperation fateOp, + FateKey fateKey, Repo repo, boolean autoCleanUp) { + var cfuture = seeder.attemptToSeedTransaction(fateOp, fateKey, repo, autoCleanUp); + return cfuture.thenApply(optional -> { + optional.ifPresent(seedingConsumer.get()); + return optional; + }); + } + + @Override + public void close() { + seeder.close(); + } + }; } public void seedTransaction(Fate.FateOperation fateOp, FateKey fateKey, Repo repo, @@ -73,6 +95,7 @@ public void seedTransaction(Fate.FateOperation fateOp, FateId fateId, Repo re boolean autoCleanUp, String goalMessage) { Fate.log.info("[{}] Seeding {} {} {}", store.type(), fateOp, fateId, goalMessage); store.seedTransaction(fateOp, fateId, repo, autoCleanUp); + seedingConsumer.get().accept(fateId); } // check on the transaction @@ -176,4 +199,8 @@ public Exception getException(FateId fateId) { public Stream list(FateKey.FateKeyType type) { return store.list(type); } + + public void setSeedingConsumer(Consumer seedingConsumer) { + this.seedingConsumer.set(seedingConsumer); + } } diff --git a/core/src/main/java/org/apache/accumulo/core/fate/FateExecutor.java b/core/src/main/java/org/apache/accumulo/core/fate/FateExecutor.java index 883146eb4e4..2457095209f 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/FateExecutor.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/FateExecutor.java @@ -33,6 +33,7 @@ import java.util.Collections; import java.util.HashSet; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.Set; import java.util.concurrent.ConcurrentLinkedQueue; @@ -43,6 +44,8 @@ import java.util.concurrent.TransferQueue; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BooleanSupplier; import org.apache.accumulo.core.clientImpl.AcceptableThriftTableOperationException; import org.apache.accumulo.core.conf.Property; @@ -81,6 +84,7 @@ public class FateExecutor { private final Set fateOps; private final ConcurrentLinkedQueue idleCountHistory = new ConcurrentLinkedQueue<>(); private final FateExecutorMetrics fateExecutorMetrics; + private final AtomicReference> partitions = new AtomicReference<>(Set.of()); public FateExecutor(Fate fate, T environment, Set fateOps, int poolSize, String name) { @@ -298,6 +302,11 @@ protected ConcurrentLinkedQueue getIdleCountHistory() { return idleCountHistory; } + public void setPartitions(Set partitions) { + Objects.requireNonNull(partitions); + this.partitions.set(Set.copyOf(partitions)); + } + /** * A single thread that finds transactions to work on and queues them up. Do not want each worker * thread going to the store and looking for work as it would place more load on the store. @@ -308,7 +317,12 @@ private class WorkFinder implements Runnable { public void run() { while (fate.getKeepRunning().get() && !isShutdown()) { try { - fate.getStore().runnable(() -> fate.getKeepRunning().get(), fateIdStatus -> { + var localPartitions = partitions.get(); + // if the set of partitions changes, we should stop looking for work w/ the old set of + // partitions + BooleanSupplier keepRunning = + () -> fate.getKeepRunning().get() && localPartitions == partitions.get(); + fate.getStore().runnable(localPartitions, keepRunning, fateIdStatus -> { // The FateId with the fate operation 'fateOp' is workable by this FateExecutor if // 1) This FateExecutor is assigned to work on 'fateOp' ('fateOp' is in 'fateOps') // 2) The transaction was cancelled while NEW. This is an edge case that needs to be @@ -319,7 +333,7 @@ public void run() { var fateOp = fateIdStatus.getFateOperation().orElse(null); if ((fateOp != null && fateOps.contains(fateOp)) || txCancelledWhileNew(status, fateOp)) { - while (fate.getKeepRunning().get() && !isShutdown()) { + while (keepRunning.getAsBoolean() && !isShutdown()) { try { // The reason for calling transfer instead of queueing is avoid rescanning the // storage layer and adding the same thing over and over. For example if all diff --git a/core/src/main/java/org/apache/accumulo/core/fate/FateExecutorMetrics.java b/core/src/main/java/org/apache/accumulo/core/fate/FateExecutorMetrics.java index 4edc70fe7a8..f9753a4ab6e 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/FateExecutorMetrics.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/FateExecutorMetrics.java @@ -22,14 +22,13 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.accumulo.core.metrics.Metric; -import org.apache.accumulo.core.metrics.MetricsProducer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import io.micrometer.core.instrument.Gauge; import io.micrometer.core.instrument.MeterRegistry; -public class FateExecutorMetrics implements MetricsProducer { +public class FateExecutorMetrics { private static final Logger log = LoggerFactory.getLogger(FateExecutorMetrics.class); private final FateInstanceType type; private final String poolName; @@ -49,7 +48,6 @@ protected FateExecutorMetrics(FateInstanceType type, String poolName, this.idleWorkerCount = idleWorkerCount; } - @Override public void registerMetrics(MeterRegistry registry) { // noop if already registered or cleared if (state == State.UNREGISTERED) { diff --git a/core/src/main/java/org/apache/accumulo/core/fate/FatePartition.java b/core/src/main/java/org/apache/accumulo/core/fate/FatePartition.java new file mode 100644 index 00000000000..33cbbc97240 --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/core/fate/FatePartition.java @@ -0,0 +1,63 @@ +/* + * 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.fate; + +import java.util.UUID; + +import org.apache.accumulo.core.manager.thrift.TFatePartition; + +public record FatePartition(FateId start, FateId end) { + + public TFatePartition toThrift() { + return new TFatePartition(start.canonical(), end.canonical()); + } + + public static FatePartition from(TFatePartition tfp) { + return new FatePartition(FateId.from(tfp.start), FateId.from(tfp.stop)); + } + + private static final FatePartition ALL_USER = + new FatePartition(FateId.from(FateInstanceType.USER, new UUID(0, 0)), + FateId.from(FateInstanceType.USER, new UUID(-1, -1))); + private static final FatePartition ALL_META = + new FatePartition(FateId.from(FateInstanceType.META, new UUID(0, 0)), + FateId.from(FateInstanceType.META, new UUID(-1, -1))); + + public static FatePartition all(FateInstanceType type) { + return switch (type) { + case META -> ALL_META; + case USER -> ALL_USER; + }; + } + + private static final UUID LAST_UUID = new UUID(-1, -1); + + public boolean isEndInclusive() { + return end.getTxUUID().equals(LAST_UUID); + } + + public boolean contains(FateId fateId) { + if (isEndInclusive()) { + return fateId.compareTo(start) >= 0 && fateId.compareTo(end) <= 0; + } else { + return fateId.compareTo(start) >= 0 && fateId.compareTo(end) < 0; + } + + } +} diff --git a/core/src/main/java/org/apache/accumulo/core/fate/FateStore.java b/core/src/main/java/org/apache/accumulo/core/fate/FateStore.java index 6756f84f5aa..9436cce05bd 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/FateStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/FateStore.java @@ -27,10 +27,10 @@ import java.util.Arrays; import java.util.Objects; import java.util.Optional; +import java.util.Set; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.function.BooleanSupplier; -import java.util.function.Consumer; import org.apache.accumulo.core.fate.zookeeper.ZooUtil; import org.apache.hadoop.io.DataInputBuffer; @@ -153,7 +153,8 @@ interface FateTxStore extends ReadOnlyFateTxStore { * longer interact with it. * * @param deferTime time to keep this transaction from being returned by - * {@link #runnable(BooleanSupplier, Consumer)}. Must be non-negative. + * {@link #runnable(Set, BooleanSupplier, java.util.function.Consumer)}. Must be + * non-negative. */ void unreserve(Duration deferTime); } @@ -249,7 +250,7 @@ public int hashCode() { * can no longer be worked on so their reservation should be deleted, so they can be picked up and * worked on again. */ - void deleteDeadReservations(); + void deleteDeadReservations(Set partitions); /** * Attempt to reserve the fate transaction. @@ -269,6 +270,11 @@ public int hashCode() { */ FateTxStore reserve(FateId fateId); + /** + * Notification that something in this store was seeded by another process. + */ + void seeded(); + @Override void close(); } diff --git a/core/src/main/java/org/apache/accumulo/core/fate/ReadOnlyFateStore.java b/core/src/main/java/org/apache/accumulo/core/fate/ReadOnlyFateStore.java index 451823ac700..ad7baae9bcc 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/ReadOnlyFateStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/ReadOnlyFateStore.java @@ -23,6 +23,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.function.BooleanSupplier; import java.util.function.Consumer; import java.util.stream.Stream; @@ -155,7 +156,7 @@ interface FateIdStatus { * @return a map of the current active reservations with the keys being the transaction that is * reserved and the value being the value stored to indicate the transaction is reserved. */ - Map getActiveReservations(); + Map getActiveReservations(Set partitions); /** * Finds all fate ops that are (IN_PROGRESS, SUBMITTED, or FAILED_IN_PROGRESS) and unreserved. Ids @@ -163,7 +164,8 @@ interface FateIdStatus { * is found or until the keepWaiting parameter is false. It will return once all runnable ids * found were passed to the consumer. */ - void runnable(BooleanSupplier keepWaiting, Consumer idConsumer); + void runnable(Set partitions, BooleanSupplier keepWaiting, + Consumer idConsumer); /** * Returns true if the deferred map was cleared and if deferred executions are currently disabled diff --git a/core/src/main/java/org/apache/accumulo/core/fate/user/UserFateStore.java b/core/src/main/java/org/apache/accumulo/core/fate/user/UserFateStore.java index 55300470d7b..158c2ce1aa2 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/user/UserFateStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/user/UserFateStore.java @@ -28,6 +28,7 @@ import java.util.Map.Entry; import java.util.Objects; import java.util.Optional; +import java.util.Set; import java.util.SortedMap; import java.util.UUID; import java.util.concurrent.CompletableFuture; @@ -56,6 +57,7 @@ import org.apache.accumulo.core.fate.FateId; import org.apache.accumulo.core.fate.FateInstanceType; import org.apache.accumulo.core.fate.FateKey; +import org.apache.accumulo.core.fate.FatePartition; import org.apache.accumulo.core.fate.ReadOnlyRepo; import org.apache.accumulo.core.fate.Repo; import org.apache.accumulo.core.fate.StackOverflowException; @@ -196,7 +198,7 @@ private boolean seedTransaction(Supplier> mutatorFactory, String var status = mutator.tryMutate(); if (status == FateMutator.Status.ACCEPTED) { // signal to the super class that a new fate transaction was seeded and is ready to run - seededTx(); + seeded(); log.trace("Attempt to seed {} returned {}", logId, status); return true; } else if (status == FateMutator.Status.REJECTED) { @@ -255,8 +257,8 @@ public Optional> tryReserve(FateId fateId) { } @Override - public void deleteDeadReservations() { - for (Entry activeRes : getActiveReservations().entrySet()) { + public void deleteDeadReservations(Set partitions) { + for (Entry activeRes : getActiveReservations(partitions).entrySet()) { FateId fateId = activeRes.getKey(); FateReservation reservation = activeRes.getValue(); if (!isLockHeld.test(reservation.getLockID())) { @@ -281,9 +283,21 @@ public void deleteDeadReservations() { @Override protected Stream getTransactions(EnumSet statuses) { + return getTransactions(FatePartition.all(FateInstanceType.USER), statuses); + } + + @Override + protected Stream getTransactions(Set partitions, + EnumSet statuses) { + return partitions.stream().flatMap(p -> getTransactions(p, statuses)); + } + + private Stream getTransactions(FatePartition partition, EnumSet statuses) { try { Scanner scanner = context.createScanner(tableName, Authorizations.EMPTY); - scanner.setRange(new Range()); + var range = new Range(getRowId(partition.start()), true, getRowId(partition.end()), + partition.isEndInclusive()); + scanner.setRange(range); RowFateStatusFilter.configureScanner(scanner, statuses); // columns fetched here must be in/added to TxAdminColumnFamily for locality group benefits TxAdminColumnFamily.STATUS_COLUMN.fetch(scanner); @@ -453,7 +467,7 @@ public void close() { var future = pending.get(fateId).getSecond(); switch (result.getValue()) { case ACCEPTED: - seededTx(); + seeded(); log.trace("Attempt to seed {} returned {}", fateId.canonical(), status); // Complete the future with the fatId and remove from pending future.complete(Optional.of(fateId)); diff --git a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/MetaFateStore.java b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/MetaFateStore.java index f6f67dfef22..a7cf3236af7 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/MetaFateStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/MetaFateStore.java @@ -55,6 +55,7 @@ import org.apache.accumulo.core.fate.FateId; import org.apache.accumulo.core.fate.FateInstanceType; import org.apache.accumulo.core.fate.FateKey; +import org.apache.accumulo.core.fate.FatePartition; import org.apache.accumulo.core.fate.ReadOnlyRepo; import org.apache.accumulo.core.fate.Repo; import org.apache.accumulo.core.fate.StackOverflowException; @@ -71,6 +72,9 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.base.Suppliers; +import com.google.common.collect.Range; +import com.google.common.collect.RangeSet; +import com.google.common.collect.TreeRangeSet; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; @@ -279,8 +283,8 @@ public Optional> tryReserve(FateId fateId) { } @Override - public void deleteDeadReservations() { - for (Map.Entry entry : getActiveReservations().entrySet()) { + public void deleteDeadReservations(Set partitions) { + for (Map.Entry entry : getActiveReservations(partitions).entrySet()) { FateId fateId = entry.getKey(); FateReservation reservation = entry.getValue(); if (isLockHeld.test(reservation.getLockID())) { @@ -616,6 +620,16 @@ public Optional getFateOperation() { } } + @Override + protected Stream getTransactions(Set partitions, + EnumSet statuses) { + + RangeSet rangeSet = TreeRangeSet.create(); + partitions.forEach(partition -> rangeSet.add(Range.closed(partition.start(), partition.end()))); + + return getTransactions(statuses).filter(fis -> rangeSet.contains(fis.getFateId())); + } + @Override public Stream list(FateKey.FateKeyType type) { return getTransactions(EnumSet.allOf(TStatus.class)) diff --git a/core/src/main/java/org/apache/accumulo/core/lock/ServiceLockPaths.java b/core/src/main/java/org/apache/accumulo/core/lock/ServiceLockPaths.java index 68c29c485f0..5f19a46e921 100644 --- a/core/src/main/java/org/apache/accumulo/core/lock/ServiceLockPaths.java +++ b/core/src/main/java/org/apache/accumulo/core/lock/ServiceLockPaths.java @@ -77,7 +77,8 @@ private ServiceLockPath(String type) { Preconditions.checkArgument(this.type.equals(Constants.ZGC_LOCK) || this.type.equals(Constants.ZMANAGER_LOCK) || this.type.equals(Constants.ZMONITOR_LOCK) || this.type.equals(Constants.ZTABLE_LOCKS) || this.type.equals(Constants.ZADMIN_LOCK) - || this.type.equals(Constants.ZTEST_LOCK), "Unsupported type: " + type); + || this.type.equals(Constants.ZTEST_LOCK) + || this.type.equals(Constants.ZMANAGER_ASSISTANT_LOCK), "Unsupported type: " + type); // These server types support only one active instance, so they use a lock at // a known path, not the server's address. this.resourceGroup = null; @@ -105,7 +106,8 @@ private ServiceLockPath(String type, ResourceGroupId resourceGroup, HostAndPort this.type = requireNonNull(type); Preconditions.checkArgument( this.type.equals(Constants.ZCOMPACTORS) || this.type.equals(Constants.ZSSERVERS) - || this.type.equals(Constants.ZTSERVERS) || this.type.equals(Constants.ZDEADTSERVERS), + || this.type.equals(Constants.ZTSERVERS) || this.type.equals(Constants.ZDEADTSERVERS) + || this.type.equals(Constants.ZMANAGER_ASSISTANT_LOCK), "Unsupported type: " + type); this.resourceGroup = requireNonNull(resourceGroup); this.server = requireNonNull(server).toString(); @@ -170,6 +172,8 @@ private static String determineServerType(final String path) { return Constants.ZGC_LOCK; } else if (pathStartsWith(path, Constants.ZMANAGER_LOCK)) { return Constants.ZMANAGER_LOCK; + } else if (pathStartsWith(path, Constants.ZMANAGER_ASSISTANT_LOCK)) { + return Constants.ZMANAGER_ASSISTANT_LOCK; } else if (pathStartsWith(path, Constants.ZMONITOR_LOCK)) { return Constants.ZMONITOR_LOCK; } else if (pathStartsWith(path, Constants.ZMINI_LOCK)) { @@ -219,7 +223,7 @@ public static ServiceLockPath parse(Optional serverType, String path) { return switch (type) { case Constants.ZMINI_LOCK -> new ServiceLockPath(type, server); case Constants.ZCOMPACTORS, Constants.ZSSERVERS, Constants.ZTSERVERS, - Constants.ZDEADTSERVERS -> + Constants.ZDEADTSERVERS, Constants.ZMANAGER_ASSISTANT_LOCK -> new ServiceLockPath(type, ResourceGroupId.of(resourceGroup), HostAndPort.fromString(server)); default -> @@ -238,6 +242,11 @@ public ServiceLockPath createManagerPath() { return new ServiceLockPath(Constants.ZMANAGER_LOCK); } + public ServiceLockPath createManagerWorkerPath(ResourceGroupId resourceGroup, + HostAndPort advertiseAddress) { + return new ServiceLockPath(Constants.ZMANAGER_ASSISTANT_LOCK, resourceGroup, advertiseAddress); + } + public ServiceLockPath createMiniPath(String miniUUID) { return new ServiceLockPath(Constants.ZMINI_LOCK, miniUUID); } @@ -287,6 +296,11 @@ public Set getCompactor(ResourceGroupPredicate resourceGroupPre return get(Constants.ZCOMPACTORS, resourceGroupPredicate, address, withLock); } + public Set getManagerAssistants(ResourceGroupPredicate resourceGroupPredicate, + AddressSelector address, boolean withLock) { + return get(Constants.ZMANAGER_ASSISTANT_LOCK, resourceGroupPredicate, address, withLock); + } + /** * Note that the ServiceLockPath object returned by this method does not populate the server * attribute. To get the location of the GarbageCollector you will need to parse the lock data at @@ -317,6 +331,11 @@ public ServiceLockPath getManager(boolean withLock) { } } + public Set getAssistantManagers(AddressSelector address, boolean withLock) { + return get(Constants.ZMANAGER_ASSISTANT_LOCK, ResourceGroupPredicate.DEFAULT_RG_ONLY, address, + withLock); + } + /** * Note that the ServiceLockPath object returned by this method does not populate the server * attribute. To get the location of the Monitor you will need to parse the lock data at the @@ -431,7 +450,8 @@ private Set get(final String serverType, } } } else if (serverType.equals(Constants.ZCOMPACTORS) || serverType.equals(Constants.ZSSERVERS) - || serverType.equals(Constants.ZTSERVERS) || serverType.equals(Constants.ZDEADTSERVERS)) { + || serverType.equals(Constants.ZTSERVERS) || serverType.equals(Constants.ZDEADTSERVERS) + || serverType.equals(Constants.ZMANAGER_ASSISTANT_LOCK)) { final List resourceGroups = zooCache.getChildren(typePath); for (final String group : resourceGroups) { if (resourceGroupPredicate.test(ResourceGroupId.of(group))) { diff --git a/core/src/main/java/org/apache/accumulo/core/logging/FateLogger.java b/core/src/main/java/org/apache/accumulo/core/logging/FateLogger.java index 8c7a956c053..0fa5fec2968 100644 --- a/core/src/main/java/org/apache/accumulo/core/logging/FateLogger.java +++ b/core/src/main/java/org/apache/accumulo/core/logging/FateLogger.java @@ -23,6 +23,7 @@ import java.util.Map; import java.util.Objects; import java.util.Optional; +import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.function.BooleanSupplier; import java.util.function.Consumer; @@ -34,6 +35,7 @@ import org.apache.accumulo.core.fate.FateId; import org.apache.accumulo.core.fate.FateInstanceType; import org.apache.accumulo.core.fate.FateKey; +import org.apache.accumulo.core.fate.FatePartition; import org.apache.accumulo.core.fate.FateStore; import org.apache.accumulo.core.fate.FateStore.FateTxStore; import org.apache.accumulo.core.fate.FateStore.Seeder; @@ -113,6 +115,11 @@ public FateTxStore reserve(FateId fateId) { return new LoggingFateTxStore<>(store.reserve(fateId), toLogString, allowForceDel); } + @Override + public void seeded() { + store.seeded(); + } + @Override public Optional> tryReserve(FateId fateId) { return store.tryReserve(fateId) @@ -140,8 +147,9 @@ public Stream list(FateKey.FateKeyType type) { } @Override - public void runnable(BooleanSupplier keepWaiting, Consumer idConsumer) { - store.runnable(keepWaiting, idConsumer); + public void runnable(Set partitions, BooleanSupplier keepWaiting, + Consumer idConsumer) { + store.runnable(partitions, keepWaiting, idConsumer); } @Override @@ -162,8 +170,8 @@ public Seeder beginSeeding() { public boolean seedTransaction(Fate.FateOperation fateOp, FateId fateId, Repo repo, boolean autoCleanUp) { boolean seeded = store.seedTransaction(fateOp, fateId, repo, autoCleanUp); - if (storeLog.isTraceEnabled()) { - storeLog.trace("{} {} {} {}", fateId, seeded ? "seeded" : "unable to seed", + if (storeLog.isDebugEnabled()) { + storeLog.debug("{} {} {} {}", fateId, seeded ? "seeded" : "unable to seed", toLogString.apply(repo), autoCleanUp); } return seeded; @@ -185,13 +193,13 @@ public boolean isDeferredOverflow() { } @Override - public Map getActiveReservations() { - return store.getActiveReservations(); + public Map getActiveReservations(Set partitions) { + return store.getActiveReservations(partitions); } @Override - public void deleteDeadReservations() { - store.deleteDeadReservations(); + public void deleteDeadReservations(Set partitions) { + store.deleteDeadReservations(partitions); } @Override @@ -217,12 +225,12 @@ public CompletableFuture> attemptToSeedTransaction(FateOperatio FateKey fateKey, Repo repo, boolean autoCleanUp) { var future = this.seeder.attemptToSeedTransaction(fateOp, fateKey, repo, autoCleanUp); return future.whenComplete((optional, throwable) -> { - if (storeLog.isTraceEnabled()) { + if (storeLog.isDebugEnabled()) { optional.ifPresentOrElse(fateId -> { - storeLog.trace("{} seeded {} {} {}", fateId, fateKey, toLogString.apply(repo), + storeLog.debug("{} seeded {} {} {}", fateId, fateKey, toLogString.apply(repo), autoCleanUp); }, () -> { - storeLog.trace("Possibly unable to seed {} {} {}", fateKey, toLogString.apply(repo), + storeLog.debug("Possibly unable to seed {} {} {}", fateKey, toLogString.apply(repo), autoCleanUp); }); } diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/clients/FateWorkerThriftClient.java b/core/src/main/java/org/apache/accumulo/core/rpc/clients/FateWorkerThriftClient.java new file mode 100644 index 00000000000..040a0f36c20 --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/core/rpc/clients/FateWorkerThriftClient.java @@ -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. + */ +package org.apache.accumulo.core.rpc.clients; + +import org.apache.accumulo.core.manager.thrift.FateWorkerService; + +/** + * Client side object that can be used to interact with operatoins that are supported by any manager + * process. + */ +public class FateWorkerThriftClient extends ThriftClientTypes { + FateWorkerThriftClient(String serviceName) { + super(serviceName, new FateWorkerService.Client.Factory()); + } +} 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 5b9a5c203d1..1ba5a5c138e 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 @@ -24,6 +24,7 @@ import org.apache.accumulo.core.client.AccumuloException; import org.apache.accumulo.core.client.AccumuloSecurityException; import org.apache.accumulo.core.clientImpl.ClientContext; +import org.apache.accumulo.core.manager.thrift.FateWorkerService; import org.apache.thrift.TException; import org.apache.thrift.TServiceClient; import org.apache.thrift.TServiceClientFactory; @@ -61,6 +62,9 @@ public abstract class ThriftClientTypes { public static final ServerProcessServiceThriftClient SERVER_PROCESS = new ServerProcessServiceThriftClient("process"); + public static final ThriftClientTypes FATE_WORKER = + new FateWorkerThriftClient("fate_worker"); + /** * execute method with supplied client returning object of type R * diff --git a/core/src/main/spotbugs/exclude-filter.xml b/core/src/main/spotbugs/exclude-filter.xml index cfaac3ea6f0..b8e08762165 100644 --- a/core/src/main/spotbugs/exclude-filter.xml +++ b/core/src/main/spotbugs/exclude-filter.xml @@ -30,6 +30,7 @@ + diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/FateService.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/FateService.java index 1670ca4c2d1..5fbeeb4276c 100644 --- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/FateService.java +++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/FateService.java @@ -3182,13 +3182,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, executeFateOperatio case 5: // ARGUMENTS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list70 = iprot.readListBegin(); - struct.arguments = new java.util.ArrayList(_list70.size); - @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem71; - for (int _i72 = 0; _i72 < _list70.size; ++_i72) + org.apache.thrift.protocol.TList _list78 = iprot.readListBegin(); + struct.arguments = new java.util.ArrayList(_list78.size); + @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem79; + for (int _i80 = 0; _i80 < _list78.size; ++_i80) { - _elem71 = iprot.readBinary(); - struct.arguments.add(_elem71); + _elem79 = iprot.readBinary(); + struct.arguments.add(_elem79); } iprot.readListEnd(); } @@ -3200,15 +3200,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, executeFateOperatio case 6: // OPTIONS if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map73 = iprot.readMapBegin(); - struct.options = new java.util.HashMap(2*_map73.size); - @org.apache.thrift.annotation.Nullable java.lang.String _key74; - @org.apache.thrift.annotation.Nullable java.lang.String _val75; - for (int _i76 = 0; _i76 < _map73.size; ++_i76) + org.apache.thrift.protocol.TMap _map81 = iprot.readMapBegin(); + struct.options = new java.util.HashMap(2*_map81.size); + @org.apache.thrift.annotation.Nullable java.lang.String _key82; + @org.apache.thrift.annotation.Nullable java.lang.String _val83; + for (int _i84 = 0; _i84 < _map81.size; ++_i84) { - _key74 = iprot.readString(); - _val75 = iprot.readString(); - struct.options.put(_key74, _val75); + _key82 = iprot.readString(); + _val83 = iprot.readString(); + struct.options.put(_key82, _val83); } iprot.readMapEnd(); } @@ -3265,9 +3265,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, executeFateOperati oprot.writeFieldBegin(ARGUMENTS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.arguments.size())); - for (java.nio.ByteBuffer _iter77 : struct.arguments) + for (java.nio.ByteBuffer _iter85 : struct.arguments) { - oprot.writeBinary(_iter77); + oprot.writeBinary(_iter85); } oprot.writeListEnd(); } @@ -3277,10 +3277,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, executeFateOperati oprot.writeFieldBegin(OPTIONS_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.options.size())); - for (java.util.Map.Entry _iter78 : struct.options.entrySet()) + for (java.util.Map.Entry _iter86 : struct.options.entrySet()) { - oprot.writeString(_iter78.getKey()); - oprot.writeString(_iter78.getValue()); + oprot.writeString(_iter86.getKey()); + oprot.writeString(_iter86.getValue()); } oprot.writeMapEnd(); } @@ -3345,19 +3345,19 @@ public void write(org.apache.thrift.protocol.TProtocol prot, executeFateOperatio if (struct.isSetArguments()) { { oprot.writeI32(struct.arguments.size()); - for (java.nio.ByteBuffer _iter79 : struct.arguments) + for (java.nio.ByteBuffer _iter87 : struct.arguments) { - oprot.writeBinary(_iter79); + oprot.writeBinary(_iter87); } } } if (struct.isSetOptions()) { { oprot.writeI32(struct.options.size()); - for (java.util.Map.Entry _iter80 : struct.options.entrySet()) + for (java.util.Map.Entry _iter88 : struct.options.entrySet()) { - oprot.writeString(_iter80.getKey()); - oprot.writeString(_iter80.getValue()); + oprot.writeString(_iter88.getKey()); + oprot.writeString(_iter88.getValue()); } } } @@ -3391,28 +3391,28 @@ public void read(org.apache.thrift.protocol.TProtocol prot, executeFateOperation } if (incoming.get(4)) { { - org.apache.thrift.protocol.TList _list81 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.arguments = new java.util.ArrayList(_list81.size); - @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem82; - for (int _i83 = 0; _i83 < _list81.size; ++_i83) + org.apache.thrift.protocol.TList _list89 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.arguments = new java.util.ArrayList(_list89.size); + @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem90; + for (int _i91 = 0; _i91 < _list89.size; ++_i91) { - _elem82 = iprot.readBinary(); - struct.arguments.add(_elem82); + _elem90 = iprot.readBinary(); + struct.arguments.add(_elem90); } } struct.setArgumentsIsSet(true); } if (incoming.get(5)) { { - org.apache.thrift.protocol.TMap _map84 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); - struct.options = new java.util.HashMap(2*_map84.size); - @org.apache.thrift.annotation.Nullable java.lang.String _key85; - @org.apache.thrift.annotation.Nullable java.lang.String _val86; - for (int _i87 = 0; _i87 < _map84.size; ++_i87) + org.apache.thrift.protocol.TMap _map92 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); + struct.options = new java.util.HashMap(2*_map92.size); + @org.apache.thrift.annotation.Nullable java.lang.String _key93; + @org.apache.thrift.annotation.Nullable java.lang.String _val94; + for (int _i95 = 0; _i95 < _map92.size; ++_i95) { - _key85 = iprot.readString(); - _val86 = iprot.readString(); - struct.options.put(_key85, _val86); + _key93 = iprot.readString(); + _val94 = iprot.readString(); + struct.options.put(_key93, _val94); } } struct.setOptionsIsSet(true); diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/FateWorkerService.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/FateWorkerService.java new file mode 100644 index 00000000000..7cfbb74d300 --- /dev/null +++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/FateWorkerService.java @@ -0,0 +1,3778 @@ +/* + * 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.manager.thrift; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) +public class FateWorkerService { + + public interface Iface { + + public TFatePartitions getPartitions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException; + + public boolean setPartitions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long updateId, java.util.List desired) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException; + + public void seeded(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.List tpartitions) throws org.apache.thrift.TException; + + } + + public interface AsyncIface { + + public void getPartitions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void setPartitions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long updateId, java.util.List desired, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void seeded(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.List tpartitions, 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 TFatePartitions getPartitions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException + { + send_getPartitions(tinfo, credentials); + return recv_getPartitions(); + } + + public void send_getPartitions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException + { + getPartitions_args args = new getPartitions_args(); + args.setTinfo(tinfo); + args.setCredentials(credentials); + sendBase("getPartitions", args); + } + + public TFatePartitions recv_getPartitions() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException + { + getPartitions_result result = new getPartitions_result(); + receiveBase(result, "getPartitions"); + if (result.isSetSuccess()) { + return result.success; + } + if (result.sec != null) { + throw result.sec; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getPartitions failed: unknown result"); + } + + @Override + public boolean setPartitions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long updateId, java.util.List desired) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException + { + send_setPartitions(tinfo, credentials, updateId, desired); + return recv_setPartitions(); + } + + public void send_setPartitions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long updateId, java.util.List desired) throws org.apache.thrift.TException + { + setPartitions_args args = new setPartitions_args(); + args.setTinfo(tinfo); + args.setCredentials(credentials); + args.setUpdateId(updateId); + args.setDesired(desired); + sendBase("setPartitions", args); + } + + public boolean recv_setPartitions() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException + { + setPartitions_result result = new setPartitions_result(); + receiveBase(result, "setPartitions"); + if (result.isSetSuccess()) { + return result.success; + } + if (result.sec != null) { + throw result.sec; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "setPartitions failed: unknown result"); + } + + @Override + public void seeded(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.List tpartitions) throws org.apache.thrift.TException + { + send_seeded(tinfo, credentials, tpartitions); + recv_seeded(); + } + + public void send_seeded(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.List tpartitions) throws org.apache.thrift.TException + { + seeded_args args = new seeded_args(); + args.setTinfo(tinfo); + args.setCredentials(credentials); + args.setTpartitions(tpartitions); + sendBase("seeded", args); + } + + public void recv_seeded() throws org.apache.thrift.TException + { + seeded_result result = new seeded_result(); + receiveBase(result, "seeded"); + return; + } + + } + 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 getPartitions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + getPartitions_call method_call = new getPartitions_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class getPartitions_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; + public getPartitions_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, 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, false); + this.tinfo = tinfo; + 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("getPartitions", org.apache.thrift.protocol.TMessageType.CALL, 0)); + getPartitions_args args = new getPartitions_args(); + args.setTinfo(tinfo); + args.setCredentials(credentials); + args.write(prot); + prot.writeMessageEnd(); + } + + @Override + public TFatePartitions getResult() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, 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 (new Client(prot)).recv_getPartitions(); + } + } + + @Override + public void setPartitions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long updateId, java.util.List desired, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + setPartitions_call method_call = new setPartitions_call(tinfo, credentials, updateId, desired, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class setPartitions_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 long updateId; + private java.util.List desired; + public setPartitions_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long updateId, java.util.List desired, 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.updateId = updateId; + this.desired = desired; + } + + @Override + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("setPartitions", org.apache.thrift.protocol.TMessageType.CALL, 0)); + setPartitions_args args = new setPartitions_args(); + args.setTinfo(tinfo); + args.setCredentials(credentials); + args.setUpdateId(updateId); + args.setDesired(desired); + args.write(prot); + prot.writeMessageEnd(); + } + + @Override + public java.lang.Boolean getResult() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, 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 (new Client(prot)).recv_setPartitions(); + } + } + + @Override + public void seeded(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.List tpartitions, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + seeded_call method_call = new seeded_call(tinfo, credentials, tpartitions, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class seeded_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.util.List tpartitions; + public seeded_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.List tpartitions, 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.tpartitions = tpartitions; + } + + @Override + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("seeded", org.apache.thrift.protocol.TMessageType.CALL, 0)); + seeded_args args = new seeded_args(); + args.setTinfo(tinfo); + args.setCredentials(credentials); + args.setTpartitions(tpartitions); + 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); + (new Client(prot)).recv_seeded(); + 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("getPartitions", new getPartitions()); + processMap.put("setPartitions", new setPartitions()); + processMap.put("seeded", new seeded()); + return processMap; + } + + public static class getPartitions extends org.apache.thrift.ProcessFunction { + public getPartitions() { + super("getPartitions"); + } + + @Override + public getPartitions_args getEmptyArgsInstance() { + return new getPartitions_args(); + } + + @Override + protected boolean isOneway() { + return false; + } + + @Override + protected boolean rethrowUnhandledExceptions() { + return false; + } + + @Override + public getPartitions_result getResult(I iface, getPartitions_args args) throws org.apache.thrift.TException { + getPartitions_result result = new getPartitions_result(); + try { + result.success = iface.getPartitions(args.tinfo, args.credentials); + } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) { + result.sec = sec; + } + return result; + } + } + + public static class setPartitions extends org.apache.thrift.ProcessFunction { + public setPartitions() { + super("setPartitions"); + } + + @Override + public setPartitions_args getEmptyArgsInstance() { + return new setPartitions_args(); + } + + @Override + protected boolean isOneway() { + return false; + } + + @Override + protected boolean rethrowUnhandledExceptions() { + return false; + } + + @Override + public setPartitions_result getResult(I iface, setPartitions_args args) throws org.apache.thrift.TException { + setPartitions_result result = new setPartitions_result(); + try { + result.success = iface.setPartitions(args.tinfo, args.credentials, args.updateId, args.desired); + result.setSuccessIsSet(true); + } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) { + result.sec = sec; + } + return result; + } + } + + public static class seeded extends org.apache.thrift.ProcessFunction { + public seeded() { + super("seeded"); + } + + @Override + public seeded_args getEmptyArgsInstance() { + return new seeded_args(); + } + + @Override + protected boolean isOneway() { + return false; + } + + @Override + protected boolean rethrowUnhandledExceptions() { + return false; + } + + @Override + public seeded_result getResult(I iface, seeded_args args) throws org.apache.thrift.TException { + seeded_result result = new seeded_result(); + iface.seeded(args.tinfo, args.credentials, args.tpartitions); + return result; + } + } + + } + + 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("getPartitions", new getPartitions()); + processMap.put("setPartitions", new setPartitions()); + processMap.put("seeded", new seeded()); + return processMap; + } + + public static class getPartitions extends org.apache.thrift.AsyncProcessFunction { + public getPartitions() { + super("getPartitions"); + } + + @Override + public getPartitions_args getEmptyArgsInstance() { + return new getPartitions_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(TFatePartitions o) { + getPartitions_result result = new getPartitions_result(); + result.success = o; + 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; + getPartitions_result result = new getPartitions_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.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, getPartitions_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + iface.getPartitions(args.tinfo, args.credentials,resultHandler); + } + } + + public static class setPartitions extends org.apache.thrift.AsyncProcessFunction { + public setPartitions() { + super("setPartitions"); + } + + @Override + public setPartitions_args getEmptyArgsInstance() { + return new setPartitions_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(java.lang.Boolean o) { + setPartitions_result result = new setPartitions_result(); + result.success = o; + result.setSuccessIsSet(true); + 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; + setPartitions_result result = new setPartitions_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.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, setPartitions_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + iface.setPartitions(args.tinfo, args.credentials, args.updateId, args.desired,resultHandler); + } + } + + public static class seeded extends org.apache.thrift.AsyncProcessFunction { + public seeded() { + super("seeded"); + } + + @Override + public seeded_args getEmptyArgsInstance() { + return new seeded_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) { + seeded_result result = new seeded_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; + seeded_result result = new seeded_result(); + 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, seeded_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + iface.seeded(args.tinfo, args.credentials, args.tpartitions,resultHandler); + } + } + + } + + @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) + public static class getPartitions_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("getPartitions_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.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getPartitions_argsStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getPartitions_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 + + /** 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"); + + 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; + 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))); + metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getPartitions_args.class, metaDataMap); + } + + public getPartitions_args() { + } + + public getPartitions_args( + org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, + org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) + { + this(); + this.tinfo = tinfo; + this.credentials = credentials; + } + + /** + * Performs a deep copy on other. + */ + public getPartitions_args(getPartitions_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); + } + } + + @Override + public getPartitions_args deepCopy() { + return new getPartitions_args(this); + } + + @Override + public void clear() { + this.tinfo = null; + this.credentials = null; + } + + @org.apache.thrift.annotation.Nullable + public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() { + return this.tinfo; + } + + public getPartitions_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 getPartitions_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 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; + + } + } + + @org.apache.thrift.annotation.Nullable + @Override + public java.lang.Object getFieldValue(_Fields field) { + switch (field) { + case TINFO: + return getTinfo(); + + 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 TINFO: + return isSetTinfo(); + case CREDENTIALS: + return isSetCredentials(); + } + throw new java.lang.IllegalStateException(); + } + + @Override + public boolean equals(java.lang.Object that) { + if (that instanceof getPartitions_args) + return this.equals((getPartitions_args)that); + return false; + } + + public boolean equals(getPartitions_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; + } + + 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(); + + return hashCode; + } + + @Override + public int compareTo(getPartitions_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; + } + } + 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("getPartitions_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; + 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 getPartitions_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + @Override + public getPartitions_argsStandardScheme getScheme() { + return new getPartitions_argsStandardScheme(); + } + } + + private static class getPartitions_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { + + @Override + public void read(org.apache.thrift.protocol.TProtocol iprot, getPartitions_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; + 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, getPartitions_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(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getPartitions_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + @Override + public getPartitions_argsTupleScheme getScheme() { + return new getPartitions_argsTupleScheme(); + } + } + + private static class getPartitions_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getPartitions_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); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetTinfo()) { + struct.tinfo.write(oprot); + } + if (struct.isSetCredentials()) { + struct.credentials.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, getPartitions_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(2); + 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); + } + } + } + + 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 getPartitions_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("getPartitions_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + 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.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getPartitions_resultStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getPartitions_resultTupleSchemeFactory(); + + public @org.apache.thrift.annotation.Nullable TFatePartitions success; // required + public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec; // 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 { + SUCCESS((short)0, "success"), + SEC((short)1, "sec"); + + 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 0: // SUCCESS + return SUCCESS; + case 1: // SEC + return SEC; + 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.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TFatePartitions.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))); + metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getPartitions_result.class, metaDataMap); + } + + public getPartitions_result() { + } + + public getPartitions_result( + TFatePartitions success, + org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) + { + this(); + this.success = success; + this.sec = sec; + } + + /** + * Performs a deep copy on other. + */ + public getPartitions_result(getPartitions_result other) { + if (other.isSetSuccess()) { + this.success = new TFatePartitions(other.success); + } + if (other.isSetSec()) { + this.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(other.sec); + } + } + + @Override + public getPartitions_result deepCopy() { + return new getPartitions_result(this); + } + + @Override + public void clear() { + this.success = null; + this.sec = null; + } + + @org.apache.thrift.annotation.Nullable + public TFatePartitions getSuccess() { + return this.success; + } + + public getPartitions_result setSuccess(@org.apache.thrift.annotation.Nullable TFatePartitions success) { + this.success = success; + return this; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + @org.apache.thrift.annotation.Nullable + public org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException getSec() { + return this.sec; + } + + public getPartitions_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; + } + } + + @Override + public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((TFatePartitions)value); + } + break; + + case SEC: + if (value == null) { + unsetSec(); + } else { + setSec((org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException)value); + } + break; + + } + } + + @org.apache.thrift.annotation.Nullable + @Override + public java.lang.Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + case SEC: + return getSec(); + + } + 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 SUCCESS: + return isSetSuccess(); + case SEC: + return isSetSec(); + } + throw new java.lang.IllegalStateException(); + } + + @Override + public boolean equals(java.lang.Object that) { + if (that instanceof getPartitions_result) + return this.equals((getPartitions_result)that); + return false; + } + + public boolean equals(getPartitions_result that) { + if (that == null) + return false; + if (this == that) + return true; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + 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; + } + + return true; + } + + @Override + public int hashCode() { + int hashCode = 1; + + hashCode = hashCode * 8191 + ((isSetSuccess()) ? 131071 : 524287); + if (isSetSuccess()) + hashCode = hashCode * 8191 + success.hashCode(); + + hashCode = hashCode * 8191 + ((isSetSec()) ? 131071 : 524287); + if (isSetSec()) + hashCode = hashCode * 8191 + sec.hashCode(); + + return hashCode; + } + + @Override + public int compareTo(getPartitions_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = java.lang.Boolean.compare(isSetSuccess(), other.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + 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; + } + } + 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("getPartitions_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + if (!first) sb.append(", "); + sb.append("sec:"); + if (this.sec == null) { + sb.append("null"); + } else { + sb.append(this.sec); + } + 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 (success != null) { + success.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 getPartitions_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + @Override + public getPartitions_resultStandardScheme getScheme() { + return new getPartitions_resultStandardScheme(); + } + } + + private static class getPartitions_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { + + @Override + public void read(org.apache.thrift.protocol.TProtocol iprot, getPartitions_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 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TFatePartitions(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + 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; + 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, getPartitions_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.sec != null) { + oprot.writeFieldBegin(SEC_FIELD_DESC); + struct.sec.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getPartitions_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + @Override + public getPartitions_resultTupleScheme getScheme() { + return new getPartitions_resultTupleScheme(); + } + } + + private static class getPartitions_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getPartitions_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.isSetSuccess()) { + optionals.set(0); + } + if (struct.isSetSec()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + if (struct.isSetSec()) { + struct.sec.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, getPartitions_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.success = new TFatePartitions(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + if (incoming.get(1)) { + struct.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(); + struct.sec.read(iprot); + struct.setSecIsSet(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 setPartitions_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("setPartitions_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 UPDATE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("updateId", org.apache.thrift.protocol.TType.I64, (short)3); + private static final org.apache.thrift.protocol.TField DESIRED_FIELD_DESC = new org.apache.thrift.protocol.TField("desired", org.apache.thrift.protocol.TType.LIST, (short)4); + + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new setPartitions_argsStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new setPartitions_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 long updateId; // required + public @org.apache.thrift.annotation.Nullable java.util.List desired; // 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"), + UPDATE_ID((short)3, "updateId"), + DESIRED((short)4, "desired"); + + 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: // UPDATE_ID + return UPDATE_ID; + case 4: // DESIRED + return DESIRED; + 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 + private static final int __UPDATEID_ISSET_ID = 0; + private byte __isset_bitfield = 0; + 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.UPDATE_ID, new org.apache.thrift.meta_data.FieldMetaData("updateId", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + tmpMap.put(_Fields.DESIRED, new org.apache.thrift.meta_data.FieldMetaData("desired", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TFatePartition.class)))); + metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(setPartitions_args.class, metaDataMap); + } + + public setPartitions_args() { + } + + public setPartitions_args( + org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, + org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, + long updateId, + java.util.List desired) + { + this(); + this.tinfo = tinfo; + this.credentials = credentials; + this.updateId = updateId; + setUpdateIdIsSet(true); + this.desired = desired; + } + + /** + * Performs a deep copy on other. + */ + public setPartitions_args(setPartitions_args other) { + __isset_bitfield = other.__isset_bitfield; + 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); + } + this.updateId = other.updateId; + if (other.isSetDesired()) { + java.util.List __this__desired = new java.util.ArrayList(other.desired.size()); + for (TFatePartition other_element : other.desired) { + __this__desired.add(new TFatePartition(other_element)); + } + this.desired = __this__desired; + } + } + + @Override + public setPartitions_args deepCopy() { + return new setPartitions_args(this); + } + + @Override + public void clear() { + this.tinfo = null; + this.credentials = null; + setUpdateIdIsSet(false); + this.updateId = 0; + this.desired = null; + } + + @org.apache.thrift.annotation.Nullable + public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() { + return this.tinfo; + } + + public setPartitions_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 setPartitions_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; + } + } + + public long getUpdateId() { + return this.updateId; + } + + public setPartitions_args setUpdateId(long updateId) { + this.updateId = updateId; + setUpdateIdIsSet(true); + return this; + } + + public void unsetUpdateId() { + __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __UPDATEID_ISSET_ID); + } + + /** Returns true if field updateId is set (has been assigned a value) and false otherwise */ + public boolean isSetUpdateId() { + return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __UPDATEID_ISSET_ID); + } + + public void setUpdateIdIsSet(boolean value) { + __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __UPDATEID_ISSET_ID, value); + } + + public int getDesiredSize() { + return (this.desired == null) ? 0 : this.desired.size(); + } + + @org.apache.thrift.annotation.Nullable + public java.util.Iterator getDesiredIterator() { + return (this.desired == null) ? null : this.desired.iterator(); + } + + public void addToDesired(TFatePartition elem) { + if (this.desired == null) { + this.desired = new java.util.ArrayList(); + } + this.desired.add(elem); + } + + @org.apache.thrift.annotation.Nullable + public java.util.List getDesired() { + return this.desired; + } + + public setPartitions_args setDesired(@org.apache.thrift.annotation.Nullable java.util.List desired) { + this.desired = desired; + return this; + } + + public void unsetDesired() { + this.desired = null; + } + + /** Returns true if field desired is set (has been assigned a value) and false otherwise */ + public boolean isSetDesired() { + return this.desired != null; + } + + public void setDesiredIsSet(boolean value) { + if (!value) { + this.desired = 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 UPDATE_ID: + if (value == null) { + unsetUpdateId(); + } else { + setUpdateId((java.lang.Long)value); + } + break; + + case DESIRED: + if (value == null) { + unsetDesired(); + } else { + setDesired((java.util.List)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 UPDATE_ID: + return getUpdateId(); + + case DESIRED: + return getDesired(); + + } + 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 UPDATE_ID: + return isSetUpdateId(); + case DESIRED: + return isSetDesired(); + } + throw new java.lang.IllegalStateException(); + } + + @Override + public boolean equals(java.lang.Object that) { + if (that instanceof setPartitions_args) + return this.equals((setPartitions_args)that); + return false; + } + + public boolean equals(setPartitions_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_updateId = true; + boolean that_present_updateId = true; + if (this_present_updateId || that_present_updateId) { + if (!(this_present_updateId && that_present_updateId)) + return false; + if (this.updateId != that.updateId) + return false; + } + + boolean this_present_desired = true && this.isSetDesired(); + boolean that_present_desired = true && that.isSetDesired(); + if (this_present_desired || that_present_desired) { + if (!(this_present_desired && that_present_desired)) + return false; + if (!this.desired.equals(that.desired)) + 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 + org.apache.thrift.TBaseHelper.hashCode(updateId); + + hashCode = hashCode * 8191 + ((isSetDesired()) ? 131071 : 524287); + if (isSetDesired()) + hashCode = hashCode * 8191 + desired.hashCode(); + + return hashCode; + } + + @Override + public int compareTo(setPartitions_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(isSetUpdateId(), other.isSetUpdateId()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetUpdateId()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.updateId, other.updateId); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = java.lang.Boolean.compare(isSetDesired(), other.isSetDesired()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetDesired()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.desired, other.desired); + 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("setPartitions_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("updateId:"); + sb.append(this.updateId); + first = false; + if (!first) sb.append(", "); + sb.append("desired:"); + if (this.desired == null) { + sb.append("null"); + } else { + sb.append(this.desired); + } + 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 { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + 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 setPartitions_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + @Override + public setPartitions_argsStandardScheme getScheme() { + return new setPartitions_argsStandardScheme(); + } + } + + private static class setPartitions_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { + + @Override + public void read(org.apache.thrift.protocol.TProtocol iprot, setPartitions_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: // UPDATE_ID + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.updateId = iprot.readI64(); + struct.setUpdateIdIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // DESIRED + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list138 = iprot.readListBegin(); + struct.desired = new java.util.ArrayList(_list138.size); + @org.apache.thrift.annotation.Nullable TFatePartition _elem139; + for (int _i140 = 0; _i140 < _list138.size; ++_i140) + { + _elem139 = new TFatePartition(); + _elem139.read(iprot); + struct.desired.add(_elem139); + } + iprot.readListEnd(); + } + struct.setDesiredIsSet(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, setPartitions_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(); + } + oprot.writeFieldBegin(UPDATE_ID_FIELD_DESC); + oprot.writeI64(struct.updateId); + oprot.writeFieldEnd(); + if (struct.desired != null) { + oprot.writeFieldBegin(DESIRED_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.desired.size())); + for (TFatePartition _iter141 : struct.desired) + { + _iter141.write(oprot); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class setPartitions_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + @Override + public setPartitions_argsTupleScheme getScheme() { + return new setPartitions_argsTupleScheme(); + } + } + + private static class setPartitions_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, setPartitions_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.isSetUpdateId()) { + optionals.set(2); + } + if (struct.isSetDesired()) { + optionals.set(3); + } + oprot.writeBitSet(optionals, 4); + if (struct.isSetTinfo()) { + struct.tinfo.write(oprot); + } + if (struct.isSetCredentials()) { + struct.credentials.write(oprot); + } + if (struct.isSetUpdateId()) { + oprot.writeI64(struct.updateId); + } + if (struct.isSetDesired()) { + { + oprot.writeI32(struct.desired.size()); + for (TFatePartition _iter142 : struct.desired) + { + _iter142.write(oprot); + } + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, setPartitions_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(4); + 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.updateId = iprot.readI64(); + struct.setUpdateIdIsSet(true); + } + if (incoming.get(3)) { + { + org.apache.thrift.protocol.TList _list143 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.desired = new java.util.ArrayList(_list143.size); + @org.apache.thrift.annotation.Nullable TFatePartition _elem144; + for (int _i145 = 0; _i145 < _list143.size; ++_i145) + { + _elem144 = new TFatePartition(); + _elem144.read(iprot); + struct.desired.add(_elem144); + } + } + struct.setDesiredIsSet(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 setPartitions_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("setPartitions_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.BOOL, (short)0); + 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.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new setPartitions_resultStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new setPartitions_resultTupleSchemeFactory(); + + public boolean success; // required + public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec; // 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 { + SUCCESS((short)0, "success"), + SEC((short)1, "sec"); + + 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 0: // SUCCESS + return SUCCESS; + case 1: // SEC + return SEC; + 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 + private static final int __SUCCESS_ISSET_ID = 0; + private byte __isset_bitfield = 0; + 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.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL))); + 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))); + metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(setPartitions_result.class, metaDataMap); + } + + public setPartitions_result() { + } + + public setPartitions_result( + boolean success, + org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) + { + this(); + this.success = success; + setSuccessIsSet(true); + this.sec = sec; + } + + /** + * Performs a deep copy on other. + */ + public setPartitions_result(setPartitions_result other) { + __isset_bitfield = other.__isset_bitfield; + this.success = other.success; + if (other.isSetSec()) { + this.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(other.sec); + } + } + + @Override + public setPartitions_result deepCopy() { + return new setPartitions_result(this); + } + + @Override + public void clear() { + setSuccessIsSet(false); + this.success = false; + this.sec = null; + } + + public boolean isSuccess() { + return this.success; + } + + public setPartitions_result setSuccess(boolean success) { + this.success = success; + setSuccessIsSet(true); + return this; + } + + public void unsetSuccess() { + __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __SUCCESS_ISSET_ID); + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __SUCCESS_ISSET_ID); + } + + public void setSuccessIsSet(boolean value) { + __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __SUCCESS_ISSET_ID, value); + } + + @org.apache.thrift.annotation.Nullable + public org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException getSec() { + return this.sec; + } + + public setPartitions_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; + } + } + + @Override + public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((java.lang.Boolean)value); + } + break; + + case SEC: + if (value == null) { + unsetSec(); + } else { + setSec((org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException)value); + } + break; + + } + } + + @org.apache.thrift.annotation.Nullable + @Override + public java.lang.Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return isSuccess(); + + case SEC: + return getSec(); + + } + 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 SUCCESS: + return isSetSuccess(); + case SEC: + return isSetSec(); + } + throw new java.lang.IllegalStateException(); + } + + @Override + public boolean equals(java.lang.Object that) { + if (that instanceof setPartitions_result) + return this.equals((setPartitions_result)that); + return false; + } + + public boolean equals(setPartitions_result that) { + if (that == null) + return false; + if (this == that) + return true; + + boolean this_present_success = true; + boolean that_present_success = true; + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (this.success != that.success) + return false; + } + + 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; + } + + return true; + } + + @Override + public int hashCode() { + int hashCode = 1; + + hashCode = hashCode * 8191 + ((success) ? 131071 : 524287); + + hashCode = hashCode * 8191 + ((isSetSec()) ? 131071 : 524287); + if (isSetSec()) + hashCode = hashCode * 8191 + sec.hashCode(); + + return hashCode; + } + + @Override + public int compareTo(setPartitions_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = java.lang.Boolean.compare(isSetSuccess(), other.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + 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; + } + } + 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("setPartitions_result("); + boolean first = true; + + sb.append("success:"); + sb.append(this.success); + first = false; + if (!first) sb.append(", "); + sb.append("sec:"); + if (this.sec == null) { + sb.append("null"); + } else { + sb.append(this.sec); + } + 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 { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + 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 setPartitions_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + @Override + public setPartitions_resultStandardScheme getScheme() { + return new setPartitions_resultStandardScheme(); + } + } + + private static class setPartitions_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { + + @Override + public void read(org.apache.thrift.protocol.TProtocol iprot, setPartitions_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 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) { + struct.success = iprot.readBool(); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + 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; + 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, setPartitions_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.isSetSuccess()) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + oprot.writeBool(struct.success); + oprot.writeFieldEnd(); + } + if (struct.sec != null) { + oprot.writeFieldBegin(SEC_FIELD_DESC); + struct.sec.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class setPartitions_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + @Override + public setPartitions_resultTupleScheme getScheme() { + return new setPartitions_resultTupleScheme(); + } + } + + private static class setPartitions_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, setPartitions_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.isSetSuccess()) { + optionals.set(0); + } + if (struct.isSetSec()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetSuccess()) { + oprot.writeBool(struct.success); + } + if (struct.isSetSec()) { + struct.sec.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, setPartitions_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.success = iprot.readBool(); + struct.setSuccessIsSet(true); + } + if (incoming.get(1)) { + struct.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(); + struct.sec.read(iprot); + struct.setSecIsSet(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 seeded_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("seeded_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 TPARTITIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("tpartitions", org.apache.thrift.protocol.TType.LIST, (short)3); + + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new seeded_argsStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new seeded_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.util.List tpartitions; // 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"), + TPARTITIONS((short)3, "tpartitions"); + + 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: // TPARTITIONS + return TPARTITIONS; + 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.TPARTITIONS, new org.apache.thrift.meta_data.FieldMetaData("tpartitions", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TFatePartition.class)))); + metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(seeded_args.class, metaDataMap); + } + + public seeded_args() { + } + + public seeded_args( + org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, + org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, + java.util.List tpartitions) + { + this(); + this.tinfo = tinfo; + this.credentials = credentials; + this.tpartitions = tpartitions; + } + + /** + * Performs a deep copy on other. + */ + public seeded_args(seeded_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.isSetTpartitions()) { + java.util.List __this__tpartitions = new java.util.ArrayList(other.tpartitions.size()); + for (TFatePartition other_element : other.tpartitions) { + __this__tpartitions.add(new TFatePartition(other_element)); + } + this.tpartitions = __this__tpartitions; + } + } + + @Override + public seeded_args deepCopy() { + return new seeded_args(this); + } + + @Override + public void clear() { + this.tinfo = null; + this.credentials = null; + this.tpartitions = null; + } + + @org.apache.thrift.annotation.Nullable + public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() { + return this.tinfo; + } + + public seeded_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 seeded_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; + } + } + + public int getTpartitionsSize() { + return (this.tpartitions == null) ? 0 : this.tpartitions.size(); + } + + @org.apache.thrift.annotation.Nullable + public java.util.Iterator getTpartitionsIterator() { + return (this.tpartitions == null) ? null : this.tpartitions.iterator(); + } + + public void addToTpartitions(TFatePartition elem) { + if (this.tpartitions == null) { + this.tpartitions = new java.util.ArrayList(); + } + this.tpartitions.add(elem); + } + + @org.apache.thrift.annotation.Nullable + public java.util.List getTpartitions() { + return this.tpartitions; + } + + public seeded_args setTpartitions(@org.apache.thrift.annotation.Nullable java.util.List tpartitions) { + this.tpartitions = tpartitions; + return this; + } + + public void unsetTpartitions() { + this.tpartitions = null; + } + + /** Returns true if field tpartitions is set (has been assigned a value) and false otherwise */ + public boolean isSetTpartitions() { + return this.tpartitions != null; + } + + public void setTpartitionsIsSet(boolean value) { + if (!value) { + this.tpartitions = 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 TPARTITIONS: + if (value == null) { + unsetTpartitions(); + } else { + setTpartitions((java.util.List)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 TPARTITIONS: + return getTpartitions(); + + } + 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 TPARTITIONS: + return isSetTpartitions(); + } + throw new java.lang.IllegalStateException(); + } + + @Override + public boolean equals(java.lang.Object that) { + if (that instanceof seeded_args) + return this.equals((seeded_args)that); + return false; + } + + public boolean equals(seeded_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_tpartitions = true && this.isSetTpartitions(); + boolean that_present_tpartitions = true && that.isSetTpartitions(); + if (this_present_tpartitions || that_present_tpartitions) { + if (!(this_present_tpartitions && that_present_tpartitions)) + return false; + if (!this.tpartitions.equals(that.tpartitions)) + 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 + ((isSetTpartitions()) ? 131071 : 524287); + if (isSetTpartitions()) + hashCode = hashCode * 8191 + tpartitions.hashCode(); + + return hashCode; + } + + @Override + public int compareTo(seeded_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(isSetTpartitions(), other.isSetTpartitions()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTpartitions()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tpartitions, other.tpartitions); + 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("seeded_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("tpartitions:"); + if (this.tpartitions == null) { + sb.append("null"); + } else { + sb.append(this.tpartitions); + } + 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 seeded_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + @Override + public seeded_argsStandardScheme getScheme() { + return new seeded_argsStandardScheme(); + } + } + + private static class seeded_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { + + @Override + public void read(org.apache.thrift.protocol.TProtocol iprot, seeded_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: // TPARTITIONS + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list146 = iprot.readListBegin(); + struct.tpartitions = new java.util.ArrayList(_list146.size); + @org.apache.thrift.annotation.Nullable TFatePartition _elem147; + for (int _i148 = 0; _i148 < _list146.size; ++_i148) + { + _elem147 = new TFatePartition(); + _elem147.read(iprot); + struct.tpartitions.add(_elem147); + } + iprot.readListEnd(); + } + struct.setTpartitionsIsSet(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, seeded_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.tpartitions != null) { + oprot.writeFieldBegin(TPARTITIONS_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.tpartitions.size())); + for (TFatePartition _iter149 : struct.tpartitions) + { + _iter149.write(oprot); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class seeded_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + @Override + public seeded_argsTupleScheme getScheme() { + return new seeded_argsTupleScheme(); + } + } + + private static class seeded_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, seeded_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.isSetTpartitions()) { + optionals.set(2); + } + oprot.writeBitSet(optionals, 3); + if (struct.isSetTinfo()) { + struct.tinfo.write(oprot); + } + if (struct.isSetCredentials()) { + struct.credentials.write(oprot); + } + if (struct.isSetTpartitions()) { + { + oprot.writeI32(struct.tpartitions.size()); + for (TFatePartition _iter150 : struct.tpartitions) + { + _iter150.write(oprot); + } + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, seeded_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)) { + { + org.apache.thrift.protocol.TList _list151 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.tpartitions = new java.util.ArrayList(_list151.size); + @org.apache.thrift.annotation.Nullable TFatePartition _elem152; + for (int _i153 = 0; _i153 < _list151.size; ++_i153) + { + _elem152 = new TFatePartition(); + _elem152.read(iprot); + struct.tpartitions.add(_elem152); + } + } + struct.setTpartitionsIsSet(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 seeded_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("seeded_result"); + + + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new seeded_resultStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new seeded_resultTupleSchemeFactory(); + + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { +; + + 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) { + 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; + } + } + 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); + metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(seeded_result.class, metaDataMap); + } + + public seeded_result() { + } + + /** + * Performs a deep copy on other. + */ + public seeded_result(seeded_result other) { + } + + @Override + public seeded_result deepCopy() { + return new seeded_result(this); + } + + @Override + public void clear() { + } + + @Override + public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) { + switch (field) { + } + } + + @org.apache.thrift.annotation.Nullable + @Override + public java.lang.Object getFieldValue(_Fields field) { + switch (field) { + } + 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) { + } + throw new java.lang.IllegalStateException(); + } + + @Override + public boolean equals(java.lang.Object that) { + if (that instanceof seeded_result) + return this.equals((seeded_result)that); + return false; + } + + public boolean equals(seeded_result that) { + if (that == null) + return false; + if (this == that) + return true; + + return true; + } + + @Override + public int hashCode() { + int hashCode = 1; + + return hashCode; + } + + @Override + public int compareTo(seeded_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + 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("seeded_result("); + boolean first = true; + + 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 seeded_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + @Override + public seeded_resultStandardScheme getScheme() { + return new seeded_resultStandardScheme(); + } + } + + private static class seeded_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { + + @Override + public void read(org.apache.thrift.protocol.TProtocol iprot, seeded_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) { + 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, seeded_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class seeded_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + @Override + public seeded_resultTupleScheme getScheme() { + return new seeded_resultTupleScheme(); + } + } + + private static class seeded_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, seeded_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot; + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, seeded_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; + } + } + + 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-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 d34eef78b07..0455f9be21a 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 @@ -85,6 +85,8 @@ public interface Iface { public long getManagerTimeNanos(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException; + public void processEvents(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.List events) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException; + } public interface AsyncIface { @@ -145,6 +147,8 @@ public interface AsyncIface { public void getManagerTimeNanos(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void processEvents(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.List events, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + } public static class Client extends org.apache.thrift.TServiceClient implements Iface { @@ -1078,6 +1082,35 @@ public long recv_getManagerTimeNanos() throws org.apache.accumulo.core.clientImp throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getManagerTimeNanos failed: unknown result"); } + @Override + public void processEvents(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.List events) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException + { + send_processEvents(tinfo, credentials, events); + recv_processEvents(); + } + + public void send_processEvents(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.List events) throws org.apache.thrift.TException + { + processEvents_args args = new processEvents_args(); + args.setTinfo(tinfo); + args.setCredentials(credentials); + args.setEvents(events); + sendBase("processEvents", args); + } + + public void recv_processEvents() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException + { + processEvents_result result = new processEvents_result(); + receiveBase(result, "processEvents"); + if (result.sec != null) { + throw result.sec; + } + if (result.tnase != null) { + throw result.tnase; + } + return; + } + } public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface { public static class Factory implements org.apache.thrift.async.TAsyncClientFactory { @@ -2320,6 +2353,48 @@ public java.lang.Long getResult() throws org.apache.accumulo.core.clientImpl.thr } } + @Override + public void processEvents(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.List events, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + processEvents_call method_call = new processEvents_call(tinfo, credentials, events, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class processEvents_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.util.List events; + public processEvents_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.List events, 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.events = events; + } + + @Override + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("processEvents", org.apache.thrift.protocol.TMessageType.CALL, 0)); + processEvents_args args = new processEvents_args(); + args.setTinfo(tinfo); + args.setCredentials(credentials); + args.setEvents(events); + 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_processEvents(); + return null; + } + } + } public static class Processor extends org.apache.thrift.TBaseProcessor implements org.apache.thrift.TProcessor { @@ -2361,6 +2436,7 @@ protected Processor(I iface, java.util.Map extends org.apache.thrift.ProcessFunction { + public processEvents() { + super("processEvents"); + } + + @Override + public processEvents_args getEmptyArgsInstance() { + return new processEvents_args(); + } + + @Override + protected boolean isOneway() { + return false; + } + + @Override + protected boolean rethrowUnhandledExceptions() { + return false; + } + + @Override + public processEvents_result getResult(I iface, processEvents_args args) throws org.apache.thrift.TException { + processEvents_result result = new processEvents_result(); + try { + iface.processEvents(args.tinfo, args.credentials, args.events); + } 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 AsyncProcessor extends org.apache.thrift.TBaseAsyncProcessor { @@ -3402,6 +3512,7 @@ protected AsyncProcessor(I iface, java.util.Map extends org.apache.thrift.AsyncProcessFunction { + public processEvents() { + super("processEvents"); + } + + @Override + public processEvents_args getEmptyArgsInstance() { + return new processEvents_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) { + processEvents_result result = new processEvents_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; + processEvents_result result = new processEvents_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, processEvents_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + iface.processEvents(args.tinfo, args.credentials, args.events,resultHandler); + } + } + } @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) @@ -35645,13 +35830,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getActiveTservers_r case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list88 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list88.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem89; - for (int _i90 = 0; _i90 < _list88.size; ++_i90) + org.apache.thrift.protocol.TList _list96 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list96.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem97; + for (int _i98 = 0; _i98 < _list96.size; ++_i98) { - _elem89 = iprot.readString(); - struct.success.add(_elem89); + _elem97 = iprot.readString(); + struct.success.add(_elem97); } iprot.readListEnd(); } @@ -35698,9 +35883,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, getActiveTservers_ oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (java.lang.String _iter91 : struct.success) + for (java.lang.String _iter99 : struct.success) { - oprot.writeString(_iter91); + oprot.writeString(_iter99); } oprot.writeListEnd(); } @@ -35748,9 +35933,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, getActiveTservers_r if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (java.lang.String _iter92 : struct.success) + for (java.lang.String _iter100 : struct.success) { - oprot.writeString(_iter92); + oprot.writeString(_iter100); } } } @@ -35768,13 +35953,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, getActiveTservers_re java.util.BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list93 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.success = new java.util.ArrayList(_list93.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem94; - for (int _i95 = 0; _i95 < _list93.size; ++_i95) + org.apache.thrift.protocol.TList _list101 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.success = new java.util.ArrayList(_list101.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem102; + for (int _i103 = 0; _i103 < _list101.size; ++_i103) { - _elem94 = iprot.readString(); - struct.success.add(_elem94); + _elem102 = iprot.readString(); + struct.success.add(_elem102); } } struct.setSuccessIsSet(true); @@ -37606,14 +37791,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, requestTabletHostin case 4: // EXTENTS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list96 = iprot.readListBegin(); - struct.extents = new java.util.ArrayList(_list96.size); - @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _elem97; - for (int _i98 = 0; _i98 < _list96.size; ++_i98) + org.apache.thrift.protocol.TList _list104 = iprot.readListBegin(); + struct.extents = new java.util.ArrayList(_list104.size); + @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _elem105; + for (int _i106 = 0; _i106 < _list104.size; ++_i106) { - _elem97 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(); - _elem97.read(iprot); - struct.extents.add(_elem97); + _elem105 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(); + _elem105.read(iprot); + struct.extents.add(_elem105); } iprot.readListEnd(); } @@ -37657,9 +37842,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, requestTabletHosti oprot.writeFieldBegin(EXTENTS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.extents.size())); - for (org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _iter99 : struct.extents) + for (org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _iter107 : struct.extents) { - _iter99.write(oprot); + _iter107.write(oprot); } oprot.writeListEnd(); } @@ -37709,9 +37894,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, requestTabletHostin if (struct.isSetExtents()) { { oprot.writeI32(struct.extents.size()); - for (org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _iter100 : struct.extents) + for (org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _iter108 : struct.extents) { - _iter100.write(oprot); + _iter108.write(oprot); } } } @@ -37737,14 +37922,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, requestTabletHosting } if (incoming.get(3)) { { - org.apache.thrift.protocol.TList _list101 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.extents = new java.util.ArrayList(_list101.size); - @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _elem102; - for (int _i103 = 0; _i103 < _list101.size; ++_i103) + org.apache.thrift.protocol.TList _list109 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.extents = new java.util.ArrayList(_list109.size); + @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _elem110; + for (int _i111 = 0; _i111 < _list109.size; ++_i111) { - _elem102 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(); - _elem102.read(iprot); - struct.extents.add(_elem102); + _elem110 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(); + _elem110.read(iprot); + struct.extents.add(_elem110); } } struct.setExtentsIsSet(true); @@ -38963,17 +39148,17 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, updateTabletMergeab case 4: // SPLITS if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map104 = iprot.readMapBegin(); - struct.splits = new java.util.HashMap(2*_map104.size); - @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _key105; - @org.apache.thrift.annotation.Nullable TTabletMergeability _val106; - for (int _i107 = 0; _i107 < _map104.size; ++_i107) + org.apache.thrift.protocol.TMap _map112 = iprot.readMapBegin(); + struct.splits = new java.util.HashMap(2*_map112.size); + @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _key113; + @org.apache.thrift.annotation.Nullable TTabletMergeability _val114; + for (int _i115 = 0; _i115 < _map112.size; ++_i115) { - _key105 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(); - _key105.read(iprot); - _val106 = new TTabletMergeability(); - _val106.read(iprot); - struct.splits.put(_key105, _val106); + _key113 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(); + _key113.read(iprot); + _val114 = new TTabletMergeability(); + _val114.read(iprot); + struct.splits.put(_key113, _val114); } iprot.readMapEnd(); } @@ -39017,10 +39202,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, updateTabletMergea oprot.writeFieldBegin(SPLITS_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT, struct.splits.size())); - for (java.util.Map.Entry _iter108 : struct.splits.entrySet()) + for (java.util.Map.Entry _iter116 : struct.splits.entrySet()) { - _iter108.getKey().write(oprot); - _iter108.getValue().write(oprot); + _iter116.getKey().write(oprot); + _iter116.getValue().write(oprot); } oprot.writeMapEnd(); } @@ -39070,10 +39255,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, updateTabletMergeab if (struct.isSetSplits()) { { oprot.writeI32(struct.splits.size()); - for (java.util.Map.Entry _iter109 : struct.splits.entrySet()) + for (java.util.Map.Entry _iter117 : struct.splits.entrySet()) { - _iter109.getKey().write(oprot); - _iter109.getValue().write(oprot); + _iter117.getKey().write(oprot); + _iter117.getValue().write(oprot); } } } @@ -39099,17 +39284,17 @@ public void read(org.apache.thrift.protocol.TProtocol prot, updateTabletMergeabi } if (incoming.get(3)) { { - org.apache.thrift.protocol.TMap _map110 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT); - struct.splits = new java.util.HashMap(2*_map110.size); - @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _key111; - @org.apache.thrift.annotation.Nullable TTabletMergeability _val112; - for (int _i113 = 0; _i113 < _map110.size; ++_i113) + org.apache.thrift.protocol.TMap _map118 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT); + struct.splits = new java.util.HashMap(2*_map118.size); + @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _key119; + @org.apache.thrift.annotation.Nullable TTabletMergeability _val120; + for (int _i121 = 0; _i121 < _map118.size; ++_i121) { - _key111 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(); - _key111.read(iprot); - _val112 = new TTabletMergeability(); - _val112.read(iprot); - struct.splits.put(_key111, _val112); + _key119 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(); + _key119.read(iprot); + _val120 = new TTabletMergeability(); + _val120.read(iprot); + struct.splits.put(_key119, _val120); } } struct.setSplitsIsSet(true); @@ -39674,17 +39859,1159 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class updateTabletMergeability_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class updateTabletMergeability_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + @Override + public updateTabletMergeability_resultStandardScheme getScheme() { + return new updateTabletMergeability_resultStandardScheme(); + } + } + + private static class updateTabletMergeability_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { + + @Override + public void read(org.apache.thrift.protocol.TProtocol iprot, updateTabletMergeability_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 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list122 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list122.size); + @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _elem123; + for (int _i124 = 0; _i124 < _list122.size; ++_i124) + { + _elem123 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(); + _elem123.read(iprot); + struct.success.add(_elem123); + } + iprot.readListEnd(); + } + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + 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: // TOE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.toe = new org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException(); + struct.toe.read(iprot); + struct.setToeIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // 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, updateTabletMergeability_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); + for (org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _iter125 : struct.success) + { + _iter125.write(oprot); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.sec != null) { + oprot.writeFieldBegin(SEC_FIELD_DESC); + struct.sec.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.toe != null) { + oprot.writeFieldBegin(TOE_FIELD_DESC); + struct.toe.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 updateTabletMergeability_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + @Override + public updateTabletMergeability_resultTupleScheme getScheme() { + return new updateTabletMergeability_resultTupleScheme(); + } + } + + private static class updateTabletMergeability_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, updateTabletMergeability_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.isSetSuccess()) { + optionals.set(0); + } + if (struct.isSetSec()) { + optionals.set(1); + } + if (struct.isSetToe()) { + optionals.set(2); + } + if (struct.isSetTnase()) { + optionals.set(3); + } + oprot.writeBitSet(optionals, 4); + if (struct.isSetSuccess()) { + { + oprot.writeI32(struct.success.size()); + for (org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _iter126 : struct.success) + { + _iter126.write(oprot); + } + } + } + if (struct.isSetSec()) { + struct.sec.write(oprot); + } + if (struct.isSetToe()) { + struct.toe.write(oprot); + } + if (struct.isSetTnase()) { + struct.tnase.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, updateTabletMergeability_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(4); + if (incoming.get(0)) { + { + org.apache.thrift.protocol.TList _list127 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list127.size); + @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _elem128; + for (int _i129 = 0; _i129 < _list127.size; ++_i129) + { + _elem128 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(); + _elem128.read(iprot); + struct.success.add(_elem128); + } + } + struct.setSuccessIsSet(true); + } + if (incoming.get(1)) { + struct.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(); + struct.sec.read(iprot); + struct.setSecIsSet(true); + } + if (incoming.get(2)) { + struct.toe = new org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException(); + struct.toe.read(iprot); + struct.setToeIsSet(true); + } + if (incoming.get(3)) { + 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 getManagerTimeNanos_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("getManagerTimeNanos_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.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getManagerTimeNanos_argsStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getManagerTimeNanos_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 + + /** 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"); + + 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; + 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))); + metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getManagerTimeNanos_args.class, metaDataMap); + } + + public getManagerTimeNanos_args() { + } + + public getManagerTimeNanos_args( + org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, + org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) + { + this(); + this.tinfo = tinfo; + this.credentials = credentials; + } + + /** + * Performs a deep copy on other. + */ + public getManagerTimeNanos_args(getManagerTimeNanos_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); + } + } + + @Override + public getManagerTimeNanos_args deepCopy() { + return new getManagerTimeNanos_args(this); + } + + @Override + public void clear() { + this.tinfo = null; + this.credentials = null; + } + + @org.apache.thrift.annotation.Nullable + public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() { + return this.tinfo; + } + + public getManagerTimeNanos_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 getManagerTimeNanos_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 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; + + } + } + + @org.apache.thrift.annotation.Nullable + @Override + public java.lang.Object getFieldValue(_Fields field) { + switch (field) { + case TINFO: + return getTinfo(); + + 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 TINFO: + return isSetTinfo(); + case CREDENTIALS: + return isSetCredentials(); + } + throw new java.lang.IllegalStateException(); + } + + @Override + public boolean equals(java.lang.Object that) { + if (that instanceof getManagerTimeNanos_args) + return this.equals((getManagerTimeNanos_args)that); + return false; + } + + public boolean equals(getManagerTimeNanos_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; + } + + 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(); + + return hashCode; + } + + @Override + public int compareTo(getManagerTimeNanos_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; + } + } + 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("getManagerTimeNanos_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; + 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 getManagerTimeNanos_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + @Override + public getManagerTimeNanos_argsStandardScheme getScheme() { + return new getManagerTimeNanos_argsStandardScheme(); + } + } + + private static class getManagerTimeNanos_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { + + @Override + public void read(org.apache.thrift.protocol.TProtocol iprot, getManagerTimeNanos_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; + 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, getManagerTimeNanos_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(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getManagerTimeNanos_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + @Override + public getManagerTimeNanos_argsTupleScheme getScheme() { + return new getManagerTimeNanos_argsTupleScheme(); + } + } + + private static class getManagerTimeNanos_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getManagerTimeNanos_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); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetTinfo()) { + struct.tinfo.write(oprot); + } + if (struct.isSetCredentials()) { + struct.credentials.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, getManagerTimeNanos_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(2); + 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); + } + } + } + + 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 getManagerTimeNanos_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("getManagerTimeNanos_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.I64, (short)0); + 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 getManagerTimeNanos_resultStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getManagerTimeNanos_resultTupleSchemeFactory(); + + public long success; // required + 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 { + SUCCESS((short)0, "success"), + 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 0: // SUCCESS + return SUCCESS; + 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 + private static final int __SUCCESS_ISSET_ID = 0; + private byte __isset_bitfield = 0; + 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.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + 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(getManagerTimeNanos_result.class, metaDataMap); + } + + public getManagerTimeNanos_result() { + } + + public getManagerTimeNanos_result( + long success, + org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec, + org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException tnase) + { + this(); + this.success = success; + setSuccessIsSet(true); + this.sec = sec; + this.tnase = tnase; + } + + /** + * Performs a deep copy on other. + */ + public getManagerTimeNanos_result(getManagerTimeNanos_result other) { + __isset_bitfield = other.__isset_bitfield; + this.success = other.success; + 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 getManagerTimeNanos_result deepCopy() { + return new getManagerTimeNanos_result(this); + } + + @Override + public void clear() { + setSuccessIsSet(false); + this.success = 0; + this.sec = null; + this.tnase = null; + } + + public long getSuccess() { + return this.success; + } + + public getManagerTimeNanos_result setSuccess(long success) { + this.success = success; + setSuccessIsSet(true); + return this; + } + + public void unsetSuccess() { + __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __SUCCESS_ISSET_ID); + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __SUCCESS_ISSET_ID); + } + + public void setSuccessIsSet(boolean value) { + __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __SUCCESS_ISSET_ID, value); + } + + @org.apache.thrift.annotation.Nullable + public org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException getSec() { + return this.sec; + } + + public getManagerTimeNanos_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 getManagerTimeNanos_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 SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((java.lang.Long)value); + } + break; + + 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 SUCCESS: + return getSuccess(); + + 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 SUCCESS: + return isSetSuccess(); + case SEC: + return isSetSec(); + case TNASE: + return isSetTnase(); + } + throw new java.lang.IllegalStateException(); + } + + @Override + public boolean equals(java.lang.Object that) { + if (that instanceof getManagerTimeNanos_result) + return this.equals((getManagerTimeNanos_result)that); + return false; + } + + public boolean equals(getManagerTimeNanos_result that) { + if (that == null) + return false; + if (this == that) + return true; + + boolean this_present_success = true; + boolean that_present_success = true; + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (this.success != that.success) + return false; + } + + 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 + org.apache.thrift.TBaseHelper.hashCode(success); + + 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(getManagerTimeNanos_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = java.lang.Boolean.compare(isSetSuccess(), other.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + 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("getManagerTimeNanos_result("); + boolean first = true; + + sb.append("success:"); + sb.append(this.success); + first = false; + if (!first) sb.append(", "); + 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 { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + 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 getManagerTimeNanos_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public updateTabletMergeability_resultStandardScheme getScheme() { - return new updateTabletMergeability_resultStandardScheme(); + public getManagerTimeNanos_resultStandardScheme getScheme() { + return new getManagerTimeNanos_resultStandardScheme(); } } - private static class updateTabletMergeability_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class getManagerTimeNanos_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { @Override - public void read(org.apache.thrift.protocol.TProtocol iprot, updateTabletMergeability_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, getManagerTimeNanos_result struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -39695,19 +41022,8 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, updateTabletMergeab } switch (schemeField.id) { case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { - { - org.apache.thrift.protocol.TList _list114 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list114.size); - @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _elem115; - for (int _i116 = 0; _i116 < _list114.size; ++_i116) - { - _elem115 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(); - _elem115.read(iprot); - struct.success.add(_elem115); - } - iprot.readListEnd(); - } + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.success = iprot.readI64(); struct.setSuccessIsSet(true); } else { org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); @@ -39722,16 +41038,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, updateTabletMergeab org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; - case 2: // TOE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.toe = new org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException(); - struct.toe.read(iprot); - struct.setToeIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 3: // TNASE + 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); @@ -39752,20 +41059,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, updateTabletMergeab } @Override - public void write(org.apache.thrift.protocol.TProtocol oprot, updateTabletMergeability_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, getManagerTimeNanos_result struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); - if (struct.success != null) { + if (struct.isSetSuccess()) { oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _iter117 : struct.success) - { - _iter117.write(oprot); - } - oprot.writeListEnd(); - } + oprot.writeI64(struct.success); oprot.writeFieldEnd(); } if (struct.sec != null) { @@ -39773,11 +41073,6 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, updateTabletMergea struct.sec.write(oprot); oprot.writeFieldEnd(); } - if (struct.toe != null) { - oprot.writeFieldBegin(TOE_FIELD_DESC); - struct.toe.write(oprot); - oprot.writeFieldEnd(); - } if (struct.tnase != null) { oprot.writeFieldBegin(TNASE_FIELD_DESC); struct.tnase.write(oprot); @@ -39789,17 +41084,17 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, updateTabletMergea } - private static class updateTabletMergeability_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class getManagerTimeNanos_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public updateTabletMergeability_resultTupleScheme getScheme() { - return new updateTabletMergeability_resultTupleScheme(); + public getManagerTimeNanos_resultTupleScheme getScheme() { + return new getManagerTimeNanos_resultTupleScheme(); } } - private static class updateTabletMergeability_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class getManagerTimeNanos_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, updateTabletMergeability_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, getManagerTimeNanos_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.isSetSuccess()) { @@ -39808,49 +41103,27 @@ public void write(org.apache.thrift.protocol.TProtocol prot, updateTabletMergeab if (struct.isSetSec()) { optionals.set(1); } - if (struct.isSetToe()) { - optionals.set(2); - } if (struct.isSetTnase()) { - optionals.set(3); + optionals.set(2); } - oprot.writeBitSet(optionals, 4); + oprot.writeBitSet(optionals, 3); if (struct.isSetSuccess()) { - { - oprot.writeI32(struct.success.size()); - for (org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _iter118 : struct.success) - { - _iter118.write(oprot); - } - } + oprot.writeI64(struct.success); } if (struct.isSetSec()) { struct.sec.write(oprot); } - if (struct.isSetToe()) { - struct.toe.write(oprot); - } if (struct.isSetTnase()) { struct.tnase.write(oprot); } } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, updateTabletMergeability_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, getManagerTimeNanos_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(4); + java.util.BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { - { - org.apache.thrift.protocol.TList _list119 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList(_list119.size); - @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _elem120; - for (int _i121 = 0; _i121 < _list119.size; ++_i121) - { - _elem120 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(); - _elem120.read(iprot); - struct.success.add(_elem120); - } - } + struct.success = iprot.readI64(); struct.setSuccessIsSet(true); } if (incoming.get(1)) { @@ -39859,11 +41132,6 @@ public void read(org.apache.thrift.protocol.TProtocol prot, updateTabletMergeabi struct.setSecIsSet(true); } if (incoming.get(2)) { - struct.toe = new org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException(); - struct.toe.read(iprot); - struct.setToeIsSet(true); - } - if (incoming.get(3)) { struct.tnase = new org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException(); struct.tnase.read(iprot); struct.setTnaseIsSet(true); @@ -39877,22 +41145,25 @@ private static S scheme(org.apache. } @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) - public static class getManagerTimeNanos_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("getManagerTimeNanos_args"); + public static class processEvents_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("processEvents_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 EVENTS_FIELD_DESC = new org.apache.thrift.protocol.TField("events", org.apache.thrift.protocol.TType.LIST, (short)3); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getManagerTimeNanos_argsStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getManagerTimeNanos_argsTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new processEvents_argsStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new processEvents_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.util.List events; // 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"); + CREDENTIALS((short)2, "credentials"), + EVENTS((short)3, "events"); private static final java.util.Map byName = new java.util.HashMap(); @@ -39912,6 +41183,8 @@ public static _Fields findByThriftId(int fieldId) { return TINFO; case 2: // CREDENTIALS return CREDENTIALS; + case 3: // EVENTS + return EVENTS; default: return null; } @@ -39962,43 +41235,56 @@ public java.lang.String getFieldName() { 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.EVENTS, new org.apache.thrift.meta_data.FieldMetaData("events", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TEvent.class)))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getManagerTimeNanos_args.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(processEvents_args.class, metaDataMap); } - public getManagerTimeNanos_args() { + public processEvents_args() { } - public getManagerTimeNanos_args( + public processEvents_args( org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, - org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) + org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, + java.util.List events) { this(); this.tinfo = tinfo; this.credentials = credentials; + this.events = events; } /** * Performs a deep copy on other. */ - public getManagerTimeNanos_args(getManagerTimeNanos_args other) { + public processEvents_args(processEvents_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.isSetEvents()) { + java.util.List __this__events = new java.util.ArrayList(other.events.size()); + for (TEvent other_element : other.events) { + __this__events.add(new TEvent(other_element)); + } + this.events = __this__events; + } } @Override - public getManagerTimeNanos_args deepCopy() { - return new getManagerTimeNanos_args(this); + public processEvents_args deepCopy() { + return new processEvents_args(this); } @Override public void clear() { this.tinfo = null; this.credentials = null; + this.events = null; } @org.apache.thrift.annotation.Nullable @@ -40006,7 +41292,7 @@ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() { return this.tinfo; } - public getManagerTimeNanos_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) { + public processEvents_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) { this.tinfo = tinfo; return this; } @@ -40031,7 +41317,7 @@ public org.apache.accumulo.core.securityImpl.thrift.TCredentials getCredentials( return this.credentials; } - public getManagerTimeNanos_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) { + public processEvents_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) { this.credentials = credentials; return this; } @@ -40051,6 +41337,47 @@ public void setCredentialsIsSet(boolean value) { } } + public int getEventsSize() { + return (this.events == null) ? 0 : this.events.size(); + } + + @org.apache.thrift.annotation.Nullable + public java.util.Iterator getEventsIterator() { + return (this.events == null) ? null : this.events.iterator(); + } + + public void addToEvents(TEvent elem) { + if (this.events == null) { + this.events = new java.util.ArrayList(); + } + this.events.add(elem); + } + + @org.apache.thrift.annotation.Nullable + public java.util.List getEvents() { + return this.events; + } + + public processEvents_args setEvents(@org.apache.thrift.annotation.Nullable java.util.List events) { + this.events = events; + return this; + } + + public void unsetEvents() { + this.events = null; + } + + /** Returns true if field events is set (has been assigned a value) and false otherwise */ + public boolean isSetEvents() { + return this.events != null; + } + + public void setEventsIsSet(boolean value) { + if (!value) { + this.events = null; + } + } + @Override public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) { switch (field) { @@ -40070,6 +41397,14 @@ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable } break; + case EVENTS: + if (value == null) { + unsetEvents(); + } else { + setEvents((java.util.List)value); + } + break; + } } @@ -40083,6 +41418,9 @@ public java.lang.Object getFieldValue(_Fields field) { case CREDENTIALS: return getCredentials(); + case EVENTS: + return getEvents(); + } throw new java.lang.IllegalStateException(); } @@ -40099,18 +41437,20 @@ public boolean isSet(_Fields field) { return isSetTinfo(); case CREDENTIALS: return isSetCredentials(); + case EVENTS: + return isSetEvents(); } throw new java.lang.IllegalStateException(); } @Override public boolean equals(java.lang.Object that) { - if (that instanceof getManagerTimeNanos_args) - return this.equals((getManagerTimeNanos_args)that); + if (that instanceof processEvents_args) + return this.equals((processEvents_args)that); return false; } - public boolean equals(getManagerTimeNanos_args that) { + public boolean equals(processEvents_args that) { if (that == null) return false; if (this == that) @@ -40134,6 +41474,15 @@ public boolean equals(getManagerTimeNanos_args that) { return false; } + boolean this_present_events = true && this.isSetEvents(); + boolean that_present_events = true && that.isSetEvents(); + if (this_present_events || that_present_events) { + if (!(this_present_events && that_present_events)) + return false; + if (!this.events.equals(that.events)) + return false; + } + return true; } @@ -40149,11 +41498,15 @@ public int hashCode() { if (isSetCredentials()) hashCode = hashCode * 8191 + credentials.hashCode(); + hashCode = hashCode * 8191 + ((isSetEvents()) ? 131071 : 524287); + if (isSetEvents()) + hashCode = hashCode * 8191 + events.hashCode(); + return hashCode; } @Override - public int compareTo(getManagerTimeNanos_args other) { + public int compareTo(processEvents_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -40180,6 +41533,16 @@ public int compareTo(getManagerTimeNanos_args other) { return lastComparison; } } + lastComparison = java.lang.Boolean.compare(isSetEvents(), other.isSetEvents()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetEvents()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.events, other.events); + if (lastComparison != 0) { + return lastComparison; + } + } return 0; } @@ -40201,7 +41564,7 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public java.lang.String toString() { - java.lang.StringBuilder sb = new java.lang.StringBuilder("getManagerTimeNanos_args("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("processEvents_args("); boolean first = true; sb.append("tinfo:"); @@ -40219,6 +41582,14 @@ public java.lang.String toString() { sb.append(this.credentials); } first = false; + if (!first) sb.append(", "); + sb.append("events:"); + if (this.events == null) { + sb.append("null"); + } else { + sb.append(this.events); + } + first = false; sb.append(")"); return sb.toString(); } @@ -40250,17 +41621,17 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class getManagerTimeNanos_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class processEvents_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public getManagerTimeNanos_argsStandardScheme getScheme() { - return new getManagerTimeNanos_argsStandardScheme(); + public processEvents_argsStandardScheme getScheme() { + return new processEvents_argsStandardScheme(); } } - private static class getManagerTimeNanos_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class processEvents_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { @Override - public void read(org.apache.thrift.protocol.TProtocol iprot, getManagerTimeNanos_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, processEvents_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -40288,6 +41659,25 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getManagerTimeNanos org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; + case 3: // EVENTS + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list130 = iprot.readListBegin(); + struct.events = new java.util.ArrayList(_list130.size); + @org.apache.thrift.annotation.Nullable TEvent _elem131; + for (int _i132 = 0; _i132 < _list130.size; ++_i132) + { + _elem131 = new TEvent(); + _elem131.read(iprot); + struct.events.add(_elem131); + } + iprot.readListEnd(); + } + struct.setEventsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -40300,7 +41690,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getManagerTimeNanos } @Override - public void write(org.apache.thrift.protocol.TProtocol oprot, getManagerTimeNanos_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, processEvents_args struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -40314,23 +41704,35 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, getManagerTimeNano struct.credentials.write(oprot); oprot.writeFieldEnd(); } + if (struct.events != null) { + oprot.writeFieldBegin(EVENTS_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.events.size())); + for (TEvent _iter133 : struct.events) + { + _iter133.write(oprot); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } oprot.writeFieldStop(); oprot.writeStructEnd(); } } - private static class getManagerTimeNanos_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class processEvents_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public getManagerTimeNanos_argsTupleScheme getScheme() { - return new getManagerTimeNanos_argsTupleScheme(); + public processEvents_argsTupleScheme getScheme() { + return new processEvents_argsTupleScheme(); } } - private static class getManagerTimeNanos_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class processEvents_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, getManagerTimeNanos_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, processEvents_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()) { @@ -40339,19 +41741,31 @@ public void write(org.apache.thrift.protocol.TProtocol prot, getManagerTimeNanos if (struct.isSetCredentials()) { optionals.set(1); } - oprot.writeBitSet(optionals, 2); + if (struct.isSetEvents()) { + optionals.set(2); + } + oprot.writeBitSet(optionals, 3); if (struct.isSetTinfo()) { struct.tinfo.write(oprot); } if (struct.isSetCredentials()) { struct.credentials.write(oprot); } + if (struct.isSetEvents()) { + { + oprot.writeI32(struct.events.size()); + for (TEvent _iter134 : struct.events) + { + _iter134.write(oprot); + } + } + } } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, getManagerTimeNanos_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, processEvents_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(2); + 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); @@ -40362,6 +41776,20 @@ public void read(org.apache.thrift.protocol.TProtocol prot, getManagerTimeNanos_ struct.credentials.read(iprot); struct.setCredentialsIsSet(true); } + if (incoming.get(2)) { + { + org.apache.thrift.protocol.TList _list135 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.events = new java.util.ArrayList(_list135.size); + @org.apache.thrift.annotation.Nullable TEvent _elem136; + for (int _i137 = 0; _i137 < _list135.size; ++_i137) + { + _elem136 = new TEvent(); + _elem136.read(iprot); + struct.events.add(_elem136); + } + } + struct.setEventsIsSet(true); + } } } @@ -40371,23 +41799,20 @@ private static S scheme(org.apache. } @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) - public static class getManagerTimeNanos_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("getManagerTimeNanos_result"); + public static class processEvents_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("processEvents_result"); - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.I64, (short)0); 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 getManagerTimeNanos_resultStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getManagerTimeNanos_resultTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new processEvents_resultStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new processEvents_resultTupleSchemeFactory(); - public long success; // required 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 { - SUCCESS((short)0, "success"), SEC((short)1, "sec"), TNASE((short)2, "tnase"); @@ -40405,8 +41830,6 @@ public enum _Fields implements org.apache.thrift.TFieldIdEnum { @org.apache.thrift.annotation.Nullable public static _Fields findByThriftId(int fieldId) { switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; case 1: // SEC return SEC; case 2: // TNASE @@ -40454,32 +41877,25 @@ public java.lang.String getFieldName() { } // isset id assignments - private static final int __SUCCESS_ISSET_ID = 0; - private byte __isset_bitfield = 0; 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.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); 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(getManagerTimeNanos_result.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(processEvents_result.class, metaDataMap); } - public getManagerTimeNanos_result() { + public processEvents_result() { } - public getManagerTimeNanos_result( - long success, + public processEvents_result( org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException tnase) { this(); - this.success = success; - setSuccessIsSet(true); this.sec = sec; this.tnase = tnase; } @@ -40487,9 +41903,7 @@ public getManagerTimeNanos_result( /** * Performs a deep copy on other. */ - public getManagerTimeNanos_result(getManagerTimeNanos_result other) { - __isset_bitfield = other.__isset_bitfield; - this.success = other.success; + public processEvents_result(processEvents_result other) { if (other.isSetSec()) { this.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(other.sec); } @@ -40499,47 +41913,22 @@ public getManagerTimeNanos_result(getManagerTimeNanos_result other) { } @Override - public getManagerTimeNanos_result deepCopy() { - return new getManagerTimeNanos_result(this); + public processEvents_result deepCopy() { + return new processEvents_result(this); } @Override public void clear() { - setSuccessIsSet(false); - this.success = 0; this.sec = null; this.tnase = null; } - public long getSuccess() { - return this.success; - } - - public getManagerTimeNanos_result setSuccess(long success) { - this.success = success; - setSuccessIsSet(true); - return this; - } - - public void unsetSuccess() { - __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __SUCCESS_ISSET_ID); - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean isSetSuccess() { - return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __SUCCESS_ISSET_ID); - } - - public void setSuccessIsSet(boolean value) { - __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __SUCCESS_ISSET_ID, value); - } - @org.apache.thrift.annotation.Nullable public org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException getSec() { return this.sec; } - public getManagerTimeNanos_result setSec(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) { + public processEvents_result setSec(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) { this.sec = sec; return this; } @@ -40564,7 +41953,7 @@ public org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceExceptio return this.tnase; } - public getManagerTimeNanos_result setTnase(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException tnase) { + public processEvents_result setTnase(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException tnase) { this.tnase = tnase; return this; } @@ -40587,14 +41976,6 @@ public void setTnaseIsSet(boolean value) { @Override public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) { switch (field) { - case SUCCESS: - if (value == null) { - unsetSuccess(); - } else { - setSuccess((java.lang.Long)value); - } - break; - case SEC: if (value == null) { unsetSec(); @@ -40618,9 +41999,6 @@ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable @Override public java.lang.Object getFieldValue(_Fields field) { switch (field) { - case SUCCESS: - return getSuccess(); - case SEC: return getSec(); @@ -40639,8 +42017,6 @@ public boolean isSet(_Fields field) { } switch (field) { - case SUCCESS: - return isSetSuccess(); case SEC: return isSetSec(); case TNASE: @@ -40651,26 +42027,17 @@ public boolean isSet(_Fields field) { @Override public boolean equals(java.lang.Object that) { - if (that instanceof getManagerTimeNanos_result) - return this.equals((getManagerTimeNanos_result)that); + if (that instanceof processEvents_result) + return this.equals((processEvents_result)that); return false; } - public boolean equals(getManagerTimeNanos_result that) { + public boolean equals(processEvents_result that) { if (that == null) return false; if (this == that) return true; - boolean this_present_success = true; - boolean that_present_success = true; - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (this.success != that.success) - return false; - } - boolean this_present_sec = true && this.isSetSec(); boolean that_present_sec = true && that.isSetSec(); if (this_present_sec || that_present_sec) { @@ -40696,8 +42063,6 @@ public boolean equals(getManagerTimeNanos_result that) { public int hashCode() { int hashCode = 1; - hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(success); - hashCode = hashCode * 8191 + ((isSetSec()) ? 131071 : 524287); if (isSetSec()) hashCode = hashCode * 8191 + sec.hashCode(); @@ -40710,23 +42075,13 @@ public int hashCode() { } @Override - public int compareTo(getManagerTimeNanos_result other) { + public int compareTo(processEvents_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - lastComparison = java.lang.Boolean.compare(isSetSuccess(), other.isSetSuccess()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); - if (lastComparison != 0) { - return lastComparison; - } - } lastComparison = java.lang.Boolean.compare(isSetSec(), other.isSetSec()); if (lastComparison != 0) { return lastComparison; @@ -40767,13 +42122,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public java.lang.String toString() { - java.lang.StringBuilder sb = new java.lang.StringBuilder("getManagerTimeNanos_result("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("processEvents_result("); boolean first = true; - sb.append("success:"); - sb.append(this.success); - first = false; - if (!first) sb.append(", "); sb.append("sec:"); if (this.sec == null) { sb.append("null"); @@ -40808,25 +42159,23 @@ private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOExcept private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException { try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bitfield = 0; 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 getManagerTimeNanos_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class processEvents_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public getManagerTimeNanos_resultStandardScheme getScheme() { - return new getManagerTimeNanos_resultStandardScheme(); + public processEvents_resultStandardScheme getScheme() { + return new processEvents_resultStandardScheme(); } } - private static class getManagerTimeNanos_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class processEvents_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { @Override - public void read(org.apache.thrift.protocol.TProtocol iprot, getManagerTimeNanos_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, processEvents_result struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -40836,14 +42185,6 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getManagerTimeNanos break; } switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.I64) { - struct.success = iprot.readI64(); - struct.setSuccessIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; case 1: // SEC if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { struct.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(); @@ -40874,15 +42215,10 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getManagerTimeNanos } @Override - public void write(org.apache.thrift.protocol.TProtocol oprot, getManagerTimeNanos_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, processEvents_result struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); - if (struct.isSetSuccess()) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - oprot.writeI64(struct.success); - oprot.writeFieldEnd(); - } if (struct.sec != null) { oprot.writeFieldBegin(SEC_FIELD_DESC); struct.sec.write(oprot); @@ -40899,32 +42235,26 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, getManagerTimeNano } - private static class getManagerTimeNanos_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class processEvents_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public getManagerTimeNanos_resultTupleScheme getScheme() { - return new getManagerTimeNanos_resultTupleScheme(); + public processEvents_resultTupleScheme getScheme() { + return new processEvents_resultTupleScheme(); } } - private static class getManagerTimeNanos_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class processEvents_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, getManagerTimeNanos_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, processEvents_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.isSetSuccess()) { - optionals.set(0); - } if (struct.isSetSec()) { - optionals.set(1); + optionals.set(0); } if (struct.isSetTnase()) { - optionals.set(2); - } - oprot.writeBitSet(optionals, 3); - if (struct.isSetSuccess()) { - oprot.writeI64(struct.success); + optionals.set(1); } + oprot.writeBitSet(optionals, 2); if (struct.isSetSec()) { struct.sec.write(oprot); } @@ -40934,19 +42264,15 @@ public void write(org.apache.thrift.protocol.TProtocol prot, getManagerTimeNanos } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, getManagerTimeNanos_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, processEvents_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(3); + java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { - struct.success = iprot.readI64(); - struct.setSuccessIsSet(true); - } - if (incoming.get(1)) { struct.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(); struct.sec.read(iprot); struct.setSecIsSet(true); } - if (incoming.get(2)) { + if (incoming.get(1)) { struct.tnase = new org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException(); struct.tnase.read(iprot); struct.setTnaseIsSet(true); diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/TEvent.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/TEvent.java new file mode 100644 index 00000000000..a4ff9d6281e --- /dev/null +++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/TEvent.java @@ -0,0 +1,516 @@ +/* + * 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.manager.thrift; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) +public class TEvent 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("TEvent"); + + private static final org.apache.thrift.protocol.TField LEVEL_FIELD_DESC = new org.apache.thrift.protocol.TField("level", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField EXTENT_FIELD_DESC = new org.apache.thrift.protocol.TField("extent", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new TEventStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new TEventTupleSchemeFactory(); + + public @org.apache.thrift.annotation.Nullable java.lang.String level; // required + public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent; // 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 { + LEVEL((short)1, "level"), + EXTENT((short)2, "extent"); + + 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: // LEVEL + return LEVEL; + case 2: // EXTENT + return EXTENT; + 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.LEVEL, new org.apache.thrift.meta_data.FieldMetaData("level", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.EXTENT, new org.apache.thrift.meta_data.FieldMetaData("extent", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent.class))); + metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TEvent.class, metaDataMap); + } + + public TEvent() { + } + + public TEvent( + java.lang.String level, + org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) + { + this(); + this.level = level; + this.extent = extent; + } + + /** + * Performs a deep copy on other. + */ + public TEvent(TEvent other) { + if (other.isSetLevel()) { + this.level = other.level; + } + if (other.isSetExtent()) { + this.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(other.extent); + } + } + + @Override + public TEvent deepCopy() { + return new TEvent(this); + } + + @Override + public void clear() { + this.level = null; + this.extent = null; + } + + @org.apache.thrift.annotation.Nullable + public java.lang.String getLevel() { + return this.level; + } + + public TEvent setLevel(@org.apache.thrift.annotation.Nullable java.lang.String level) { + this.level = level; + return this; + } + + public void unsetLevel() { + this.level = null; + } + + /** Returns true if field level is set (has been assigned a value) and false otherwise */ + public boolean isSetLevel() { + return this.level != null; + } + + public void setLevelIsSet(boolean value) { + if (!value) { + this.level = null; + } + } + + @org.apache.thrift.annotation.Nullable + public org.apache.accumulo.core.dataImpl.thrift.TKeyExtent getExtent() { + return this.extent; + } + + public TEvent setExtent(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) { + this.extent = extent; + return this; + } + + public void unsetExtent() { + this.extent = null; + } + + /** Returns true if field extent is set (has been assigned a value) and false otherwise */ + public boolean isSetExtent() { + return this.extent != null; + } + + public void setExtentIsSet(boolean value) { + if (!value) { + this.extent = null; + } + } + + @Override + public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) { + switch (field) { + case LEVEL: + if (value == null) { + unsetLevel(); + } else { + setLevel((java.lang.String)value); + } + break; + + case EXTENT: + if (value == null) { + unsetExtent(); + } else { + setExtent((org.apache.accumulo.core.dataImpl.thrift.TKeyExtent)value); + } + break; + + } + } + + @org.apache.thrift.annotation.Nullable + @Override + public java.lang.Object getFieldValue(_Fields field) { + switch (field) { + case LEVEL: + return getLevel(); + + case EXTENT: + return getExtent(); + + } + 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 LEVEL: + return isSetLevel(); + case EXTENT: + return isSetExtent(); + } + throw new java.lang.IllegalStateException(); + } + + @Override + public boolean equals(java.lang.Object that) { + if (that instanceof TEvent) + return this.equals((TEvent)that); + return false; + } + + public boolean equals(TEvent that) { + if (that == null) + return false; + if (this == that) + return true; + + boolean this_present_level = true && this.isSetLevel(); + boolean that_present_level = true && that.isSetLevel(); + if (this_present_level || that_present_level) { + if (!(this_present_level && that_present_level)) + return false; + if (!this.level.equals(that.level)) + return false; + } + + boolean this_present_extent = true && this.isSetExtent(); + boolean that_present_extent = true && that.isSetExtent(); + if (this_present_extent || that_present_extent) { + if (!(this_present_extent && that_present_extent)) + return false; + if (!this.extent.equals(that.extent)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + int hashCode = 1; + + hashCode = hashCode * 8191 + ((isSetLevel()) ? 131071 : 524287); + if (isSetLevel()) + hashCode = hashCode * 8191 + level.hashCode(); + + hashCode = hashCode * 8191 + ((isSetExtent()) ? 131071 : 524287); + if (isSetExtent()) + hashCode = hashCode * 8191 + extent.hashCode(); + + return hashCode; + } + + @Override + public int compareTo(TEvent other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = java.lang.Boolean.compare(isSetLevel(), other.isSetLevel()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetLevel()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.level, other.level); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = java.lang.Boolean.compare(isSetExtent(), other.isSetExtent()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetExtent()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.extent, other.extent); + 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("TEvent("); + boolean first = true; + + sb.append("level:"); + if (this.level == null) { + sb.append("null"); + } else { + sb.append(this.level); + } + first = false; + if (!first) sb.append(", "); + sb.append("extent:"); + if (this.extent == null) { + sb.append("null"); + } else { + sb.append(this.extent); + } + 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 (extent != null) { + extent.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 TEventStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + @Override + public TEventStandardScheme getScheme() { + return new TEventStandardScheme(); + } + } + + private static class TEventStandardScheme extends org.apache.thrift.scheme.StandardScheme { + + @Override + public void read(org.apache.thrift.protocol.TProtocol iprot, TEvent 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: // LEVEL + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.level = iprot.readString(); + struct.setLevelIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // EXTENT + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(); + struct.extent.read(iprot); + struct.setExtentIsSet(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, TEvent struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.level != null) { + oprot.writeFieldBegin(LEVEL_FIELD_DESC); + oprot.writeString(struct.level); + oprot.writeFieldEnd(); + } + if (struct.extent != null) { + oprot.writeFieldBegin(EXTENT_FIELD_DESC); + struct.extent.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TEventTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + @Override + public TEventTupleScheme getScheme() { + return new TEventTupleScheme(); + } + } + + private static class TEventTupleScheme extends org.apache.thrift.scheme.TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TEvent 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.isSetLevel()) { + optionals.set(0); + } + if (struct.isSetExtent()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetLevel()) { + oprot.writeString(struct.level); + } + if (struct.isSetExtent()) { + struct.extent.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TEvent 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.level = iprot.readString(); + struct.setLevelIsSet(true); + } + if (incoming.get(1)) { + struct.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(); + struct.extent.read(iprot); + struct.setExtentIsSet(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-gen-java/org/apache/accumulo/core/manager/thrift/TFatePartition.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/TFatePartition.java new file mode 100644 index 00000000000..349d67a1982 --- /dev/null +++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/TFatePartition.java @@ -0,0 +1,511 @@ +/* + * 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.manager.thrift; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) +public class TFatePartition 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("TFatePartition"); + + private static final org.apache.thrift.protocol.TField START_FIELD_DESC = new org.apache.thrift.protocol.TField("start", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField STOP_FIELD_DESC = new org.apache.thrift.protocol.TField("stop", org.apache.thrift.protocol.TType.STRING, (short)2); + + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new TFatePartitionStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new TFatePartitionTupleSchemeFactory(); + + public @org.apache.thrift.annotation.Nullable java.lang.String start; // required + public @org.apache.thrift.annotation.Nullable java.lang.String stop; // 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 { + START((short)1, "start"), + STOP((short)2, "stop"); + + 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: // START + return START; + case 2: // STOP + return STOP; + 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.START, new org.apache.thrift.meta_data.FieldMetaData("start", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.STOP, new org.apache.thrift.meta_data.FieldMetaData("stop", 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(TFatePartition.class, metaDataMap); + } + + public TFatePartition() { + } + + public TFatePartition( + java.lang.String start, + java.lang.String stop) + { + this(); + this.start = start; + this.stop = stop; + } + + /** + * Performs a deep copy on other. + */ + public TFatePartition(TFatePartition other) { + if (other.isSetStart()) { + this.start = other.start; + } + if (other.isSetStop()) { + this.stop = other.stop; + } + } + + @Override + public TFatePartition deepCopy() { + return new TFatePartition(this); + } + + @Override + public void clear() { + this.start = null; + this.stop = null; + } + + @org.apache.thrift.annotation.Nullable + public java.lang.String getStart() { + return this.start; + } + + public TFatePartition setStart(@org.apache.thrift.annotation.Nullable java.lang.String start) { + this.start = start; + return this; + } + + public void unsetStart() { + this.start = null; + } + + /** Returns true if field start is set (has been assigned a value) and false otherwise */ + public boolean isSetStart() { + return this.start != null; + } + + public void setStartIsSet(boolean value) { + if (!value) { + this.start = null; + } + } + + @org.apache.thrift.annotation.Nullable + public java.lang.String getStop() { + return this.stop; + } + + public TFatePartition setStop(@org.apache.thrift.annotation.Nullable java.lang.String stop) { + this.stop = stop; + return this; + } + + public void unsetStop() { + this.stop = null; + } + + /** Returns true if field stop is set (has been assigned a value) and false otherwise */ + public boolean isSetStop() { + return this.stop != null; + } + + public void setStopIsSet(boolean value) { + if (!value) { + this.stop = null; + } + } + + @Override + public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) { + switch (field) { + case START: + if (value == null) { + unsetStart(); + } else { + setStart((java.lang.String)value); + } + break; + + case STOP: + if (value == null) { + unsetStop(); + } else { + setStop((java.lang.String)value); + } + break; + + } + } + + @org.apache.thrift.annotation.Nullable + @Override + public java.lang.Object getFieldValue(_Fields field) { + switch (field) { + case START: + return getStart(); + + case STOP: + return getStop(); + + } + 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 START: + return isSetStart(); + case STOP: + return isSetStop(); + } + throw new java.lang.IllegalStateException(); + } + + @Override + public boolean equals(java.lang.Object that) { + if (that instanceof TFatePartition) + return this.equals((TFatePartition)that); + return false; + } + + public boolean equals(TFatePartition that) { + if (that == null) + return false; + if (this == that) + return true; + + boolean this_present_start = true && this.isSetStart(); + boolean that_present_start = true && that.isSetStart(); + if (this_present_start || that_present_start) { + if (!(this_present_start && that_present_start)) + return false; + if (!this.start.equals(that.start)) + return false; + } + + boolean this_present_stop = true && this.isSetStop(); + boolean that_present_stop = true && that.isSetStop(); + if (this_present_stop || that_present_stop) { + if (!(this_present_stop && that_present_stop)) + return false; + if (!this.stop.equals(that.stop)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + int hashCode = 1; + + hashCode = hashCode * 8191 + ((isSetStart()) ? 131071 : 524287); + if (isSetStart()) + hashCode = hashCode * 8191 + start.hashCode(); + + hashCode = hashCode * 8191 + ((isSetStop()) ? 131071 : 524287); + if (isSetStop()) + hashCode = hashCode * 8191 + stop.hashCode(); + + return hashCode; + } + + @Override + public int compareTo(TFatePartition other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = java.lang.Boolean.compare(isSetStart(), other.isSetStart()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStart()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.start, other.start); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = java.lang.Boolean.compare(isSetStop(), other.isSetStop()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStop()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.stop, other.stop); + 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("TFatePartition("); + boolean first = true; + + sb.append("start:"); + if (this.start == null) { + sb.append("null"); + } else { + sb.append(this.start); + } + first = false; + if (!first) sb.append(", "); + sb.append("stop:"); + if (this.stop == null) { + sb.append("null"); + } else { + sb.append(this.stop); + } + 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 TFatePartitionStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + @Override + public TFatePartitionStandardScheme getScheme() { + return new TFatePartitionStandardScheme(); + } + } + + private static class TFatePartitionStandardScheme extends org.apache.thrift.scheme.StandardScheme { + + @Override + public void read(org.apache.thrift.protocol.TProtocol iprot, TFatePartition 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: // START + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.start = iprot.readString(); + struct.setStartIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // STOP + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.stop = iprot.readString(); + struct.setStopIsSet(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, TFatePartition struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.start != null) { + oprot.writeFieldBegin(START_FIELD_DESC); + oprot.writeString(struct.start); + oprot.writeFieldEnd(); + } + if (struct.stop != null) { + oprot.writeFieldBegin(STOP_FIELD_DESC); + oprot.writeString(struct.stop); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TFatePartitionTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + @Override + public TFatePartitionTupleScheme getScheme() { + return new TFatePartitionTupleScheme(); + } + } + + private static class TFatePartitionTupleScheme extends org.apache.thrift.scheme.TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TFatePartition 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.isSetStart()) { + optionals.set(0); + } + if (struct.isSetStop()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetStart()) { + oprot.writeString(struct.start); + } + if (struct.isSetStop()) { + oprot.writeString(struct.stop); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TFatePartition 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.start = iprot.readString(); + struct.setStartIsSet(true); + } + if (incoming.get(1)) { + struct.stop = iprot.readString(); + struct.setStopIsSet(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-gen-java/org/apache/accumulo/core/manager/thrift/TFatePartitions.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/TFatePartitions.java new file mode 100644 index 00000000000..477b52ad855 --- /dev/null +++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/TFatePartitions.java @@ -0,0 +1,561 @@ +/* + * 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.manager.thrift; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) +public class TFatePartitions 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("TFatePartitions"); + + private static final org.apache.thrift.protocol.TField UPDATE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("updateId", org.apache.thrift.protocol.TType.I64, (short)1); + private static final org.apache.thrift.protocol.TField PARTITIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("partitions", org.apache.thrift.protocol.TType.LIST, (short)2); + + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new TFatePartitionsStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new TFatePartitionsTupleSchemeFactory(); + + public long updateId; // required + public @org.apache.thrift.annotation.Nullable java.util.List partitions; // 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 { + UPDATE_ID((short)1, "updateId"), + PARTITIONS((short)2, "partitions"); + + 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: // UPDATE_ID + return UPDATE_ID; + case 2: // PARTITIONS + return PARTITIONS; + 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 + private static final int __UPDATEID_ISSET_ID = 0; + private byte __isset_bitfield = 0; + 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.UPDATE_ID, new org.apache.thrift.meta_data.FieldMetaData("updateId", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + tmpMap.put(_Fields.PARTITIONS, new org.apache.thrift.meta_data.FieldMetaData("partitions", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TFatePartition.class)))); + metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TFatePartitions.class, metaDataMap); + } + + public TFatePartitions() { + } + + public TFatePartitions( + long updateId, + java.util.List partitions) + { + this(); + this.updateId = updateId; + setUpdateIdIsSet(true); + this.partitions = partitions; + } + + /** + * Performs a deep copy on other. + */ + public TFatePartitions(TFatePartitions other) { + __isset_bitfield = other.__isset_bitfield; + this.updateId = other.updateId; + if (other.isSetPartitions()) { + java.util.List __this__partitions = new java.util.ArrayList(other.partitions.size()); + for (TFatePartition other_element : other.partitions) { + __this__partitions.add(new TFatePartition(other_element)); + } + this.partitions = __this__partitions; + } + } + + @Override + public TFatePartitions deepCopy() { + return new TFatePartitions(this); + } + + @Override + public void clear() { + setUpdateIdIsSet(false); + this.updateId = 0; + this.partitions = null; + } + + public long getUpdateId() { + return this.updateId; + } + + public TFatePartitions setUpdateId(long updateId) { + this.updateId = updateId; + setUpdateIdIsSet(true); + return this; + } + + public void unsetUpdateId() { + __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __UPDATEID_ISSET_ID); + } + + /** Returns true if field updateId is set (has been assigned a value) and false otherwise */ + public boolean isSetUpdateId() { + return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __UPDATEID_ISSET_ID); + } + + public void setUpdateIdIsSet(boolean value) { + __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __UPDATEID_ISSET_ID, value); + } + + public int getPartitionsSize() { + return (this.partitions == null) ? 0 : this.partitions.size(); + } + + @org.apache.thrift.annotation.Nullable + public java.util.Iterator getPartitionsIterator() { + return (this.partitions == null) ? null : this.partitions.iterator(); + } + + public void addToPartitions(TFatePartition elem) { + if (this.partitions == null) { + this.partitions = new java.util.ArrayList(); + } + this.partitions.add(elem); + } + + @org.apache.thrift.annotation.Nullable + public java.util.List getPartitions() { + return this.partitions; + } + + public TFatePartitions setPartitions(@org.apache.thrift.annotation.Nullable java.util.List partitions) { + this.partitions = partitions; + return this; + } + + public void unsetPartitions() { + this.partitions = null; + } + + /** Returns true if field partitions is set (has been assigned a value) and false otherwise */ + public boolean isSetPartitions() { + return this.partitions != null; + } + + public void setPartitionsIsSet(boolean value) { + if (!value) { + this.partitions = null; + } + } + + @Override + public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) { + switch (field) { + case UPDATE_ID: + if (value == null) { + unsetUpdateId(); + } else { + setUpdateId((java.lang.Long)value); + } + break; + + case PARTITIONS: + if (value == null) { + unsetPartitions(); + } else { + setPartitions((java.util.List)value); + } + break; + + } + } + + @org.apache.thrift.annotation.Nullable + @Override + public java.lang.Object getFieldValue(_Fields field) { + switch (field) { + case UPDATE_ID: + return getUpdateId(); + + case PARTITIONS: + return getPartitions(); + + } + 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 UPDATE_ID: + return isSetUpdateId(); + case PARTITIONS: + return isSetPartitions(); + } + throw new java.lang.IllegalStateException(); + } + + @Override + public boolean equals(java.lang.Object that) { + if (that instanceof TFatePartitions) + return this.equals((TFatePartitions)that); + return false; + } + + public boolean equals(TFatePartitions that) { + if (that == null) + return false; + if (this == that) + return true; + + boolean this_present_updateId = true; + boolean that_present_updateId = true; + if (this_present_updateId || that_present_updateId) { + if (!(this_present_updateId && that_present_updateId)) + return false; + if (this.updateId != that.updateId) + return false; + } + + boolean this_present_partitions = true && this.isSetPartitions(); + boolean that_present_partitions = true && that.isSetPartitions(); + if (this_present_partitions || that_present_partitions) { + if (!(this_present_partitions && that_present_partitions)) + return false; + if (!this.partitions.equals(that.partitions)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + int hashCode = 1; + + hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(updateId); + + hashCode = hashCode * 8191 + ((isSetPartitions()) ? 131071 : 524287); + if (isSetPartitions()) + hashCode = hashCode * 8191 + partitions.hashCode(); + + return hashCode; + } + + @Override + public int compareTo(TFatePartitions other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = java.lang.Boolean.compare(isSetUpdateId(), other.isSetUpdateId()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetUpdateId()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.updateId, other.updateId); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = java.lang.Boolean.compare(isSetPartitions(), other.isSetPartitions()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetPartitions()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.partitions, other.partitions); + 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("TFatePartitions("); + boolean first = true; + + sb.append("updateId:"); + sb.append(this.updateId); + first = false; + if (!first) sb.append(", "); + sb.append("partitions:"); + if (this.partitions == null) { + sb.append("null"); + } else { + sb.append(this.partitions); + } + 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 { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + 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 TFatePartitionsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + @Override + public TFatePartitionsStandardScheme getScheme() { + return new TFatePartitionsStandardScheme(); + } + } + + private static class TFatePartitionsStandardScheme extends org.apache.thrift.scheme.StandardScheme { + + @Override + public void read(org.apache.thrift.protocol.TProtocol iprot, TFatePartitions 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: // UPDATE_ID + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.updateId = iprot.readI64(); + struct.setUpdateIdIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // PARTITIONS + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list70 = iprot.readListBegin(); + struct.partitions = new java.util.ArrayList(_list70.size); + @org.apache.thrift.annotation.Nullable TFatePartition _elem71; + for (int _i72 = 0; _i72 < _list70.size; ++_i72) + { + _elem71 = new TFatePartition(); + _elem71.read(iprot); + struct.partitions.add(_elem71); + } + iprot.readListEnd(); + } + struct.setPartitionsIsSet(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, TFatePartitions struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldBegin(UPDATE_ID_FIELD_DESC); + oprot.writeI64(struct.updateId); + oprot.writeFieldEnd(); + if (struct.partitions != null) { + oprot.writeFieldBegin(PARTITIONS_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.partitions.size())); + for (TFatePartition _iter73 : struct.partitions) + { + _iter73.write(oprot); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TFatePartitionsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + @Override + public TFatePartitionsTupleScheme getScheme() { + return new TFatePartitionsTupleScheme(); + } + } + + private static class TFatePartitionsTupleScheme extends org.apache.thrift.scheme.TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TFatePartitions 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.isSetUpdateId()) { + optionals.set(0); + } + if (struct.isSetPartitions()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetUpdateId()) { + oprot.writeI64(struct.updateId); + } + if (struct.isSetPartitions()) { + { + oprot.writeI32(struct.partitions.size()); + for (TFatePartition _iter74 : struct.partitions) + { + _iter74.write(oprot); + } + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TFatePartitions 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.updateId = iprot.readI64(); + struct.setUpdateIdIsSet(true); + } + if (incoming.get(1)) { + { + org.apache.thrift.protocol.TList _list75 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.partitions = new java.util.ArrayList(_list75.size); + @org.apache.thrift.annotation.Nullable TFatePartition _elem76; + for (int _i77 = 0; _i77 < _list75.size; ++_i77) + { + _elem76 = new TFatePartition(); + _elem76.read(iprot); + struct.partitions.add(_elem76); + } + } + struct.setPartitionsIsSet(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 b20b7debf32..7b9cbfd87aa 100644 --- a/core/src/main/thrift/manager.thrift +++ b/core/src/main/thrift/manager.thrift @@ -159,6 +159,11 @@ struct TTabletMergeability { 2:i64 delay } +struct TEvent { + 1:string level + 2:data.TKeyExtent extent +} + service FateService { // register a fate operation by reserving an opid @@ -517,4 +522,49 @@ service ManagerClientService { 1:client.ThriftSecurityException sec 2:client.ThriftNotActiveServiceException tnase ) + + void processEvents( + 1:client.TInfo tinfo + 2:security.TCredentials credentials + 3:list events + ) throws ( + 1:client.ThriftSecurityException sec + 2:client.ThriftNotActiveServiceException tnase + ) + +} + +struct TFatePartitions { + 1:i64 updateId + 2:list partitions +} + +struct TFatePartition { + 1:string start + 2:string stop +} + +service FateWorkerService { + + TFatePartitions getPartitions( + 1:client.TInfo tinfo, + 2:security.TCredentials credentials + ) throws ( + 1:client.ThriftSecurityException sec + ) + + bool setPartitions( + 1:client.TInfo tinfo, + 2:security.TCredentials credentials, + 3:i64 updateId, + 4:list desired + ) throws ( + 1:client.ThriftSecurityException sec + ) + + void seeded( + 1:client.TInfo tinfo, + 2:security.TCredentials credentials, + 3:list tpartitions + ) } diff --git a/core/src/test/java/org/apache/accumulo/core/fate/TestStore.java b/core/src/test/java/org/apache/accumulo/core/fate/TestStore.java index 86a4106c614..ff8594d133d 100644 --- a/core/src/test/java/org/apache/accumulo/core/fate/TestStore.java +++ b/core/src/test/java/org/apache/accumulo/core/fate/TestStore.java @@ -85,6 +85,9 @@ public FateTxStore reserve(FateId fateId) { return new TestFateTxStore(fateId); } + @Override + public void seeded() {} + @Override public Optional> tryReserve(FateId fateId) { synchronized (this) { @@ -97,13 +100,13 @@ public Optional> tryReserve(FateId fateId) { } @Override - public Map getActiveReservations() { + public Map getActiveReservations(Set partitions) { // This method only makes sense for the FateStores that don't store their reservations in memory throw new UnsupportedOperationException(); } @Override - public void deleteDeadReservations() { + public void deleteDeadReservations(Set partitions) { // This method only makes sense for the FateStores that don't store their reservations in memory throw new UnsupportedOperationException(); } @@ -273,7 +276,8 @@ public Stream list(FateKey.FateKeyType type) { } @Override - public void runnable(BooleanSupplier keepWaiting, Consumer idConsumer) { + public void runnable(Set partitions, BooleanSupplier keepWaiting, + Consumer idConsumer) { throw new UnsupportedOperationException(); } 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 4adc0afe489..72526feb0f9 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 @@ -285,7 +285,7 @@ public void runServer() throws Exception { verificationThread.interrupt(); verificationThread.join(); } - log.info(getClass().getSimpleName() + " process shut down."); + log.info(getClass().getSimpleName() + " process shut down. "); Throwable thrown = err.get(); if (thrown != null) { System.err.println("Uncaught execption in AbstractServer.runServer"); diff --git a/server/base/src/main/java/org/apache/accumulo/server/init/ZooKeeperInitializer.java b/server/base/src/main/java/org/apache/accumulo/server/init/ZooKeeperInitializer.java index 3dde58446c3..464a6dae5a6 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/init/ZooKeeperInitializer.java +++ b/server/base/src/main/java/org/apache/accumulo/server/init/ZooKeeperInitializer.java @@ -176,6 +176,8 @@ void initialize(final ServerContext context, final String rootTabletDirName, ZooUtil.NodeExistsPolicy.FAIL); zrwChroot.putPersistentData(Constants.ZCOMPACTIONS, EMPTY_BYTE_ARRAY, ZooUtil.NodeExistsPolicy.FAIL); + zrwChroot.putPersistentData(Constants.ZMANAGER_ASSISTANT_LOCK, EMPTY_BYTE_ARRAY, + ZooUtil.NodeExistsPolicy.FAIL); } /** 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 0a56b4e32fe..0509735b22d 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 @@ -23,6 +23,7 @@ import org.apache.accumulo.core.compaction.thrift.CompactorService; import org.apache.accumulo.core.gc.thrift.GCMonitorService; import org.apache.accumulo.core.manager.thrift.FateService; +import org.apache.accumulo.core.manager.thrift.FateWorkerService; import org.apache.accumulo.core.manager.thrift.ManagerClientService; import org.apache.accumulo.core.process.thrift.ServerProcessService; import org.apache.accumulo.core.rpc.clients.ThriftClientTypes; @@ -82,6 +83,9 @@ public > TProcessor getTProcessor( private static final ThriftProcessorTypes MANAGER = new ThriftProcessorTypes<>(ThriftClientTypes.MANAGER); + private static final ThriftProcessorTypes FATE_WORKER = + new ThriftProcessorTypes<>(ThriftClientTypes.FATE_WORKER); + @VisibleForTesting public static final ThriftProcessorTypes TABLET_SERVER = new ThriftProcessorTypes<>(ThriftClientTypes.TABLET_SERVER); @@ -127,7 +131,8 @@ public static TMultiplexedProcessor getGcTProcessor(ServerProcessService.Iface p public static TMultiplexedProcessor getManagerTProcessor( ServerProcessService.Iface processHandler, FateService.Iface fateServiceHandler, CompactionCoordinatorService.Iface coordinatorServiceHandler, - ManagerClientService.Iface managerServiceHandler, ServerContext context) { + ManagerClientService.Iface managerServiceHandler, FateWorkerService.Iface fateWorkerService, + ServerContext context) { TMultiplexedProcessor muxProcessor = new TMultiplexedProcessor(); muxProcessor.registerProcessor(SERVER_PROCESS.getServiceName(), SERVER_PROCESS.getTProcessor(ServerProcessService.Processor.class, @@ -140,6 +145,9 @@ public static TMultiplexedProcessor getManagerTProcessor( muxProcessor.registerProcessor(MANAGER.getServiceName(), MANAGER.getTProcessor(ManagerClientService.Processor.class, ManagerClientService.Iface.class, managerServiceHandler, context)); + muxProcessor.registerProcessor(FATE_WORKER.getServiceName(), + FATE_WORKER.getTProcessor(FateWorkerService.Processor.class, FateWorkerService.Iface.class, + fateWorkerService, context)); return muxProcessor; } diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/adminCommand/ServiceStatus.java b/server/base/src/main/java/org/apache/accumulo/server/util/adminCommand/ServiceStatus.java index 4b6b519d7d7..0164c68a740 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/adminCommand/ServiceStatus.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/adminCommand/ServiceStatus.java @@ -96,6 +96,9 @@ public void execute(JCommander cl, ServiceStatusCmdOpts options) throws Exceptio final Map services = new TreeMap<>(); + // FOLLOW_ON display information about multiple managers. Could display which is primary. Also, + // could potentially display the additional port that is being listened on by the assistant + // manager. services.put(ServiceStatusReport.ReportKey.MANAGER, getManagerStatus(context)); services.put(ServiceStatusReport.ReportKey.MONITOR, getMonitorStatus(context)); services.put(ServiceStatusReport.ReportKey.T_SERVER, getTServerStatus(context)); diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/EventCoordinator.java b/server/manager/src/main/java/org/apache/accumulo/manager/EventCoordinator.java index c1ada4b2371..da4f7114e23 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/EventCoordinator.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/EventCoordinator.java @@ -20,10 +20,12 @@ import java.util.Collection; import java.util.EnumMap; +import java.util.Iterator; import java.util.Map; import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.dataImpl.KeyExtent; +import org.apache.accumulo.core.manager.thrift.TEvent; import org.apache.accumulo.core.metadata.schema.Ample; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -64,30 +66,61 @@ public static class Event { private final Ample.DataLevel level; private final KeyExtent extent; - Event(KeyExtent extent) { + public Event(KeyExtent extent) { this.scope = EventScope.TABLE_RANGE; this.level = Ample.DataLevel.of(extent.tableId()); this.extent = extent; } - Event(TableId tableId) { + public Event(TableId tableId) { this.scope = EventScope.TABLE; this.level = Ample.DataLevel.of(tableId); this.extent = new KeyExtent(tableId, null, null); } - Event(Ample.DataLevel level) { + public Event(Ample.DataLevel level) { this.scope = EventScope.DATA_LEVEL; this.level = level; this.extent = null; } - Event() { + public Event() { this.scope = EventScope.ALL; this.level = null; this.extent = null; } + public TEvent toThrift() { + switch (scope) { + case ALL: + return new TEvent(null, null); + case DATA_LEVEL: + return new TEvent(getLevel().toString(), null); + case TABLE: + case TABLE_RANGE: + return new TEvent(null, getExtent().toThrift()); + default: + throw new IllegalStateException("scope : " + scope); + } + } + + public static Event fromThrift(TEvent tEvent) { + if (tEvent.getLevel() == null && tEvent.getExtent() == null) { + return new Event(); + } else if (tEvent.getLevel() != null && tEvent.getExtent() == null) { + return new Event(Ample.DataLevel.valueOf(tEvent.getLevel())); + } else if (tEvent.getLevel() == null && tEvent.getExtent() != null) { + var extent = KeyExtent.fromThrift(tEvent.getExtent()); + if (extent.endRow() == null && extent.prevEndRow() == null) { + return new Event(extent.tableId()); + } else { + return new Event(extent); + } + } else { + throw new IllegalArgumentException("Illegal TEvent " + tEvent); + } + } + public EventScope getScope() { return scope; } @@ -106,6 +139,11 @@ public KeyExtent getExtent() { Preconditions.checkState(scope == EventScope.TABLE || scope == EventScope.TABLE_RANGE); return extent; } + + @Override + public String toString() { + return "{ scope:" + scope + ", level:" + level + ", extent:" + extent + " }"; + } } @Override @@ -132,6 +170,10 @@ public void event(KeyExtent extent, String msg, Object... args) { publish(new Event(extent)); } + public void events(Iterator events) { + events.forEachRemaining(this::publish); + } + @Override public void event(Collection extents, String msg, Object... args) { if (!extents.isEmpty()) { 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 245ef6a21a6..c04bae47ef7 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 @@ -50,6 +50,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; @@ -73,6 +74,7 @@ import org.apache.accumulo.core.fate.FateClient; import org.apache.accumulo.core.fate.FateId; import org.apache.accumulo.core.fate.FateInstanceType; +import org.apache.accumulo.core.fate.FatePartition; import org.apache.accumulo.core.fate.FateStore; import org.apache.accumulo.core.fate.TraceRepo; import org.apache.accumulo.core.fate.user.UserFateStore; @@ -85,9 +87,11 @@ import org.apache.accumulo.core.lock.ServiceLockData.ServiceDescriptor; import org.apache.accumulo.core.lock.ServiceLockData.ServiceDescriptors; import org.apache.accumulo.core.lock.ServiceLockData.ThriftService; +import org.apache.accumulo.core.lock.ServiceLockPaths; import org.apache.accumulo.core.lock.ServiceLockPaths.AddressSelector; import org.apache.accumulo.core.lock.ServiceLockPaths.ResourceGroupPredicate; import org.apache.accumulo.core.lock.ServiceLockPaths.ServiceLockPath; +import org.apache.accumulo.core.lock.ServiceLockSupport; import org.apache.accumulo.core.lock.ServiceLockSupport.HAServiceLockWatcher; import org.apache.accumulo.core.logging.ConditionalLogger.DeduplicatingLogger; import org.apache.accumulo.core.manager.state.tables.TableState; @@ -113,8 +117,11 @@ import org.apache.accumulo.core.util.time.SteadyTime; import org.apache.accumulo.core.zookeeper.ZcStat; import org.apache.accumulo.manager.compaction.coordinator.CompactionCoordinator; +import org.apache.accumulo.manager.fate.FateManager; +import org.apache.accumulo.manager.fate.FateWorker; import org.apache.accumulo.manager.merge.FindMergeableRangeTask; import org.apache.accumulo.manager.metrics.ManagerMetrics; +import org.apache.accumulo.manager.metrics.fate.FateExecutorMetricsProducer; import org.apache.accumulo.manager.recovery.RecoveryManager; import org.apache.accumulo.manager.split.FileRangeCache; import org.apache.accumulo.manager.split.Splitter; @@ -149,6 +156,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.collect.Comparators; import com.google.common.collect.ImmutableSortedMap; @@ -166,6 +174,8 @@ *

* The manager will also coordinate log recoveries and reports general status. */ +// TODO create standalone PrimaryFateEnv class and pull everything into there relatated to +// FateEnv... this will make it much more clear the env is for metadata ops only public class Manager extends AbstractServer implements LiveTServerSet.Listener, FateEnv, HighlyAvailableService { @@ -203,6 +213,7 @@ public class Manager extends AbstractServer private AuthenticationTokenKeyManager authenticationTokenKeyManager; ServiceLock managerLock = null; + ServiceLock primaryManagerLock = null; private final BalanceManager balanceManager; private ManagerState state = ManagerState.INITIAL; @@ -211,8 +222,11 @@ public class Manager extends AbstractServer // should already have been set; ConcurrentHashMap will guarantee that all threads will see // the initialized fate references after the latch is ready private final CountDownLatch fateReadyLatch = new CountDownLatch(1); + private final AtomicReference>> fateClients = + new AtomicReference<>(); private final AtomicReference>> fateRefs = new AtomicReference<>(); + private volatile FateManager fateManager; private final ManagerMetrics managerMetrics = new ManagerMetrics(); @@ -341,7 +355,9 @@ public Fate fate(FateInstanceType type) { } public FateClient fateClient(FateInstanceType type) { - return fate(type); + waitForFate(); + var client = Objects.requireNonNull(fateClients.get(), "fateClients is not set yet").get(type); + return Objects.requireNonNull(client, () -> "fate client type " + type + " is not present"); } static final boolean X = true; @@ -708,9 +724,7 @@ public void run() { case CLEAN_STOP: switch (getManagerState()) { case NORMAL: - // USER fate stores its data in a user table and its operations may interact with - // all tables, need to completely shut it down before unloading user tablets - fate(FateInstanceType.USER).shutdown(1, MINUTES); + fateManager.stop(Duration.ofMinutes(1)); setManagerState(ManagerState.SAFE_MODE); break; case SAFE_MODE: { @@ -751,7 +765,7 @@ public void run() { for (TServerInstance server : currentServers) { try { serversToShutdown.add(server); - tserverSet.getConnection(server).fastHalt(managerLock); + tserverSet.getConnection(server).fastHalt(primaryManagerLock); } catch (TException e) { // its probably down, and we don't care } finally { @@ -816,7 +830,7 @@ private void checkForHeldServer(SortedMap ts try { TServerConnection connection = tserverSet.getConnection(instance); if (connection != null) { - connection.fastHalt(managerLock); + connection.fastHalt(primaryManagerLock); } } catch (TException e) { log.error("{}", e.getMessage(), e); @@ -877,7 +891,7 @@ private void checkForHeldServer(SortedMap ts try { TServerConnection connection2 = tserverSet.getConnection(server); if (connection2 != null) { - connection2.halt(managerLock); + connection2.halt(primaryManagerLock); } } catch (TTransportException e1) { // ignore: it's probably down @@ -946,8 +960,11 @@ public void run() { CompactionCoordinatorService.Iface.class, CompactionCoordinatorService.Processor::new, compactionCoordinator.getThriftService(), this); + // This is not wrapped w/ HighlyAvailableServiceWrapper because it can be run by any manager. + FateWorker fateWorker = new FateWorker(context, tserverSet, this::createFateInstance); + var processor = ThriftProcessorTypes.getManagerTProcessor(this, fateServiceHandler, - wrappedCoordinator, managerClientHandler, getContext()); + wrappedCoordinator, managerClientHandler, fateWorker, getContext()); try { updateThriftServer(() -> { return TServerUtils.createThriftServer(context, getBindAddress(), @@ -958,6 +975,26 @@ public void run() { throw new IllegalStateException("Unable to start server on host " + getBindAddress(), e); } + tserverSet.startListeningForTabletServerChanges(this); + + MetricsInfo metricsInfo = getContext().getMetricsInfo(); + + try { + // Acquire the lock that all managers get before the primary lock, this allows non primary + // manager processes to work on stuff. + getManagerLock(); + } catch (KeeperException | InterruptedException e) { + throw new IllegalStateException("Unable to get manager lock ", e); + } + + fateWorker.setLock(managerLock); + + metricsInfo + .addMetricsProducers(fateWorker.getMetricsProducers().toArray(new MetricsProducer[0])); + + metricsInfo.init(MetricsInfo.serviceTags(getContext().getInstanceName(), getApplicationName(), + getAdvertiseAddress(), getResourceGroup())); + // block until we can obtain the ZK lock for the manager. Create the // initial lock using ThriftService.NONE. This will allow the lock // allocation to occur, but prevent any services from getting the @@ -966,7 +1003,7 @@ public void run() { // for each of these services. ServiceLockData sld; try { - sld = getManagerLock(context.getServerPaths().createManagerPath()); + sld = getPrimaryManagerLock(context.getServerPaths().createManagerPath()); } catch (KeeperException | InterruptedException e) { throw new IllegalStateException("Exception getting manager lock", e); } @@ -977,7 +1014,7 @@ public void run() { recoveryManager = new RecoveryManager(this, timeToCacheRecoveryWalExistence); context.getZooCache().addZooCacheWatcher(new TableStateWatcher((tableId, event) -> { - TableState state = getTableManager().getTableState(tableId); + TableState state = context.getTableManager().getTableState(tableId); log.debug("Table state transition to {} @ {}", state, event); nextEvent.event(tableId, "Table state in zookeeper changed for %s to %s", tableId, state); })); @@ -993,7 +1030,6 @@ public void run() { Thread statusThread = Threads.createCriticalThread("Status Thread", new StatusThread()); statusThread.start(); - tserverSet.startListeningForTabletServerChanges(this); try { blockForTservers(); } catch (InterruptedException ex) { @@ -1018,9 +1054,7 @@ public void process(WatchedEvent event) { throw new IllegalStateException("Unable to read " + Constants.ZRECOVERY, e); } - MetricsInfo metricsInfo = getContext().getMetricsInfo(); - List producers = new ArrayList<>(); - producers.add(balanceManager.getMetrics()); + metricsInfo.addMetricsProducers(balanceManager.getMetrics()); final TabletGroupWatcher userTableTGW = new TabletGroupWatcher(this, this.userTabletStore, null, managerMetrics) { @@ -1142,29 +1176,14 @@ boolean canSuspendTablets() { this.splitter.start(); this.fileRangeCache = new FileRangeCache(context); - try { - Predicate isLockHeld = - lock -> ServiceLock.isLockHeld(context.getZooCache(), lock); - var metaInstance = initializeFateInstance(context, - new MetaFateStore<>(context.getZooSession(), managerLock.getLockID(), isLockHeld)); - var userInstance = initializeFateInstance(context, new UserFateStore<>(context, - SystemTables.FATE.tableName(), managerLock.getLockID(), isLockHeld)); + setupFate(context, metricsInfo); - if (!fateRefs.compareAndSet(null, - Map.of(FateInstanceType.META, metaInstance, FateInstanceType.USER, userInstance))) { - throw new IllegalStateException( - "Unexpected previous fate reference map already initialized"); - } - managerMetrics.configureFateMetrics(getConfiguration(), this, fateRefs.get()); - fateReadyLatch.countDown(); - } catch (KeeperException | InterruptedException e) { - throw new IllegalStateException("Exception setting up FaTE cleanup thread", e); - } + fateManager = new FateManager(getContext()); + fateManager.start(); + fateClient(FateInstanceType.USER).setSeedingConsumer(fateManager::notifySeeded); - producers.addAll(managerMetrics.getProducers(this)); - metricsInfo.addMetricsProducers(producers.toArray(new MetricsProducer[0])); - metricsInfo.init(MetricsInfo.serviceTags(getContext().getInstanceName(), getApplicationName(), - getAdvertiseAddress(), getResourceGroup())); + metricsInfo + .addMetricsProducers(managerMetrics.getProducers(this).toArray(new MetricsProducer[0])); ThreadPools.watchCriticalScheduledTask(context.getScheduledExecutor() .scheduleWithFixedDelay(() -> ScanServerMetadataEntries.clean(context), 10, 10, MINUTES)); @@ -1214,9 +1233,9 @@ boolean canSuspendTablets() { } sld = new ServiceLockData(descriptors); - log.info("Setting manager lock data to {}", sld); + log.info("Setting primary manager lock data to {}", sld); try { - managerLock.replaceLockData(sld); + primaryManagerLock.replaceLockData(sld); } catch (KeeperException | InterruptedException e) { throw new IllegalStateException("Exception updating manager lock", e); } @@ -1244,7 +1263,8 @@ boolean canSuspendTablets() { } log.debug("Shutting down fate."); - getFateRefs().keySet().forEach(type -> fate(type).close()); + fate(FateInstanceType.META).close(); + fateManager.stop(Duration.ZERO); splitter.stop(); @@ -1288,16 +1308,56 @@ public void mainWait() throws InterruptedException { Thread.sleep(500); } - protected Fate initializeFateInstance(ServerContext context, FateStore store) { + /** + * This method exist so test can hook creating a fate instance. + */ + @VisibleForTesting + protected Fate createFateInstance(FateEnv env, FateStore store, + ServerContext context) { + return new Fate<>(env, store, true, TraceRepo::toLogString, getConfiguration(), + context.getScheduledExecutor()); + } - final Fate fateInstance = new Fate<>(this, store, true, TraceRepo::toLogString, - getConfiguration(), context.getScheduledExecutor()); + private void setupFate(ServerContext context, MetricsInfo metricsInfo) { + try { + Predicate isLockHeld = + lock -> ServiceLock.isLockHeld(context.getZooCache(), lock); + var metaStore = new MetaFateStore(context.getZooSession(), + primaryManagerLock.getLockID(), isLockHeld); + var metaInstance = createFateInstance(this, metaStore, context); + // configure this instance to process all data + metaInstance.setPartitions(Set.of(FatePartition.all(FateInstanceType.META))); + var userStore = new UserFateStore(context, SystemTables.FATE.tableName(), + managerLock.getLockID(), isLockHeld); + var userFateClient = new FateClient(userStore, TraceRepo::toLogString); + + var metaCleaner = new FateCleaner<>(metaStore, Duration.ofHours(8), this::getSteadyTime); + ThreadPools.watchCriticalScheduledTask(context.getScheduledExecutor() + .scheduleWithFixedDelay(metaCleaner::ageOff, 10, 4 * 60, MINUTES)); + var userCleaner = new FateCleaner<>(userStore, Duration.ofHours(8), this::getSteadyTime); + ThreadPools.watchCriticalScheduledTask(context.getScheduledExecutor() + .scheduleWithFixedDelay(userCleaner::ageOff, 10, 4 * 60, MINUTES)); + + if (!fateClients.compareAndSet(null, + Map.of(FateInstanceType.META, metaInstance, FateInstanceType.USER, userFateClient))) { + throw new IllegalStateException( + "Unexpected previous fateClient reference map already initialized"); + } + if (!fateRefs.compareAndSet(null, Map.of(FateInstanceType.META, metaInstance))) { + throw new IllegalStateException( + "Unexpected previous fate reference map already initialized"); + } - var fateCleaner = new FateCleaner<>(store, Duration.ofHours(8), this::getSteadyTime); - ThreadPools.watchCriticalScheduledTask(context.getScheduledExecutor() - .scheduleWithFixedDelay(fateCleaner::ageOff, 10, 4 * 60, MINUTES)); + managerMetrics.configureFateMetrics(getConfiguration(), this); + fateReadyLatch.countDown(); - return fateInstance; + var metaFateExecutorMetrics = new FateExecutorMetricsProducer(context, + fate(FateInstanceType.META).getFateExecutors(), + getConfiguration().getTimeInMillis(Property.MANAGER_FATE_METRICS_MIN_UPDATE_INTERVAL)); + metricsInfo.addMetricsProducers(metaFateExecutorMetrics); + } catch (KeeperException | InterruptedException e) { + throw new IllegalStateException("Exception setting up FaTE cleanup thread", e); + } } /** @@ -1391,15 +1451,60 @@ private long remaining(long deadline) { return Math.max(1, deadline - System.currentTimeMillis()); } + private void getManagerLock() throws KeeperException, InterruptedException { + log.info("trying to get assistant manager lock"); + + final ZooReaderWriter zoo = getContext().getZooSession().asReaderWriter(); + try { + + var advertiseAddress = getAdvertiseAddress(); + + final ServiceLockPaths.ServiceLockPath zLockPath = getContext().getServerPaths() + .createManagerWorkerPath(getResourceGroup(), advertiseAddress); + ServiceLockSupport.createNonHaServiceLockPath(Type.MANAGER, zoo, zLockPath); + + var serverLockUUID = UUID.randomUUID(); + managerLock = new ServiceLock(getContext().getZooSession(), zLockPath, serverLockUUID); + ServiceLock.LockWatcher lw = new ServiceLockSupport.ServiceLockWatcher(Type.MANAGER, + () -> getShutdownComplete().get(), + (type) -> getContext().getLowMemoryDetector().logGCInfo(getContext().getConfiguration())); + + for (int i = 0; i < 120 / 5; i++) { + zoo.putPersistentData(zLockPath.toString(), new byte[0], ZooUtil.NodeExistsPolicy.SKIP); + + ServiceLockData.ServiceDescriptors descriptors = new ServiceLockData.ServiceDescriptors(); + for (ServiceLockData.ThriftService svc : new ServiceLockData.ThriftService[] { + ThriftService.MANAGER}) { + descriptors.addService(new ServiceLockData.ServiceDescriptor(serverLockUUID, svc, + advertiseAddress.toString(), this.getResourceGroup())); + } + + if (managerLock.tryLock(lw, new ServiceLockData(descriptors))) { + log.info("Obtained manager assistant lock {}", managerLock.getLockPath()); + this.getContext().setServiceLock(managerLock); + return; + } + log.info("Waiting for manager assistant lock"); + sleepUninterruptibly(5, TimeUnit.SECONDS); + } + String msg = "Too many retries, exiting."; + log.info(msg); + throw new RuntimeException(msg); + } catch (Exception e) { + log.info("Could not obtain manager assistant lock, exiting.", e); + throw new RuntimeException(e); + } + } + @Override public ServiceLock getServiceLock() { - return managerLock; + return primaryManagerLock; } - private ServiceLockData getManagerLock(final ServiceLockPath zManagerLoc) + private ServiceLockData getPrimaryManagerLock(final ServiceLockPath zManagerLoc) throws KeeperException, InterruptedException { var zooKeeper = getContext().getZooSession(); - log.info("trying to get manager lock"); + log.info("trying to get primary manager lock"); UUID zooLockUUID = UUID.randomUUID(); @@ -1411,13 +1516,13 @@ private ServiceLockData getManagerLock(final ServiceLockPath zManagerLoc) descriptors.addService(new ServiceDescriptor(zooLockUUID, ThriftService.NONE, ServerOpts.BIND_ALL_ADDRESSES, this.getResourceGroup())); ServiceLockData sld = new ServiceLockData(descriptors); - managerLock = new ServiceLock(zooKeeper, zManagerLoc, zooLockUUID); + primaryManagerLock = new ServiceLock(zooKeeper, zManagerLoc, zooLockUUID); HAServiceLockWatcher managerLockWatcher = new HAServiceLockWatcher(Type.MANAGER, () -> getShutdownComplete().get()); while (true) { - managerLock.lock(managerLockWatcher, sld); + primaryManagerLock.lock(managerLockWatcher, sld); managerLockWatcher.waitForChange(); @@ -1430,12 +1535,10 @@ private ServiceLockData getManagerLock(final ServiceLockPath zManagerLoc) throw new IllegalStateException("manager lock in unknown state"); } - managerLock.tryToCancelAsyncLockOrUnlock(); + primaryManagerLock.tryToCancelAsyncLockOrUnlock(); sleepUninterruptibly(TIME_TO_WAIT_BETWEEN_LOCK_CHECKS, MILLISECONDS); } - - this.getContext().setServiceLock(getServiceLock()); return sld; } @@ -1636,12 +1739,6 @@ public void registerMetrics(MeterRegistry registry) { compactionCoordinator.registerMetrics(registry); } - private Map> getFateRefs() { - var fateRefs = this.fateRefs.get(); - Preconditions.checkState(fateRefs != null, "Unexpected null fate references map"); - return fateRefs; - } - @Override public ServiceLock getLock() { return managerLock; 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 1713dca5921..457ff8252a6 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 @@ -59,6 +59,7 @@ import org.apache.accumulo.core.dataImpl.KeyExtent; import org.apache.accumulo.core.dataImpl.thrift.TKeyExtent; import org.apache.accumulo.core.fate.Fate; +import org.apache.accumulo.core.fate.FateClient; import org.apache.accumulo.core.fate.FateId; import org.apache.accumulo.core.fate.FateInstanceType; import org.apache.accumulo.core.fate.TraceRepo; @@ -68,6 +69,7 @@ import org.apache.accumulo.core.manager.thrift.ManagerGoalState; import org.apache.accumulo.core.manager.thrift.ManagerMonitorInfo; import org.apache.accumulo.core.manager.thrift.ManagerState; +import org.apache.accumulo.core.manager.thrift.TEvent; import org.apache.accumulo.core.manager.thrift.TTabletMergeability; import org.apache.accumulo.core.manager.thrift.TabletLoadState; import org.apache.accumulo.core.manager.thrift.ThriftPropertyException; @@ -104,6 +106,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Lists; +// FOLLOW_ON move some of these thrift calls to the assistant manager thrift service public class ManagerClientServiceHandler implements ManagerClientService.Iface { private static final Logger log = Manager.log; @@ -172,7 +175,7 @@ public void waitForFlush(TInfo tinfo, TCredentials c, String tableIdStr, ByteBuf try { final TServerConnection server = manager.tserverSet.getConnection(instance); if (server != null) { - server.flush(manager.managerLock, tableId, ByteBufferUtil.toBytes(startRowBB), + server.flush(manager.primaryManagerLock, tableId, ByteBufferUtil.toBytes(startRowBB), ByteBufferUtil.toBytes(endRowBB)); } } catch (TException ex) { @@ -333,7 +336,7 @@ public void shutdownTabletServer(TInfo info, TCredentials c, String tabletServer } } - Fate fate = manager.fate(FateInstanceType.META); + FateClient fate = manager.fateClient(FateInstanceType.META); FateId fateId = fate.startTransaction(); String msg = "Shutdown tserver " + tabletServer; @@ -361,7 +364,7 @@ public void tabletServerStopping(TInfo tinfo, TCredentials credentials, String t if (manager.shutdownTServer(tserver)) { // If there is an exception seeding the fate tx this should cause the RPC to fail which should // cause the tserver to halt. Because of that not making an attempt to handle failure here. - Fate fate = manager.fate(FateInstanceType.META); + FateClient fate = manager.fateClient(FateInstanceType.META); var tid = fate.startTransaction(); String msg = "Shutdown tserver " + tabletServer; @@ -403,6 +406,9 @@ public void reportTabletStatus(TInfo info, TCredentials credentials, String serv } } + // FOLLOW_ON need to ensure SAFE_MODE and CLEAN_STOP work w/ multiple managers. Probably need + // tests for + // this. @Override public void setManagerGoalState(TInfo info, TCredentials c, ManagerGoalState state) throws ThriftSecurityException { @@ -805,6 +811,18 @@ public long getManagerTimeNanos(TInfo tinfo, TCredentials credentials) return manager.getSteadyTime().getNanos(); } + @Override + public void processEvents(TInfo tinfo, TCredentials credentials, List tEvents) + throws TException { + if (!security.canPerformSystemActions(credentials)) { + throw new ThriftSecurityException(credentials.getPrincipal(), + SecurityErrorCode.PERMISSION_DENIED); + } + + manager.getEventCoordinator().events(tEvents.stream().map(EventCoordinator.Event::fromThrift) + .peek(event -> log.trace("remote event : {}", event)).iterator()); + } + protected TableId getTableId(ClientContext context, String tableName) throws ThriftTableOperationException { return ClientServiceHandler.checkTableId(context, tableName, null); diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/TabletGroupWatcher.java b/server/manager/src/main/java/org/apache/accumulo/manager/TabletGroupWatcher.java index 5c6a3931262..ed9d4ed40ee 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/TabletGroupWatcher.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/TabletGroupWatcher.java @@ -689,7 +689,7 @@ private TableMgmtStats manageTablets(Iterator iter, if (client != null) { TABLET_UNLOAD_LOGGER.trace("[{}] Requesting TabletServer {} unload {} {}", store.name(), location.getServerInstance(), tm.getExtent(), goal.howUnload()); - client.unloadTablet(manager.managerLock, tm.getExtent(), goal.howUnload(), + client.unloadTablet(manager.primaryManagerLock, tm.getExtent(), goal.howUnload(), manager.getSteadyTime().getMillis()); tableMgmtStats.totalUnloaded++; unloaded++; @@ -1065,7 +1065,7 @@ private void flushChanges(TabletLists tLists) try { TServerConnection client = manager.tserverSet.getConnection(a.server); if (client != null) { - client.assignTablet(manager.managerLock, a.tablet); + client.assignTablet(manager.primaryManagerLock, a.tablet); manager.assignedTablet(a.tablet); } else { Manager.log.warn("Could not connect to server {} for assignment of {}", a.server, diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/fate/FateManager.java b/server/manager/src/main/java/org/apache/accumulo/manager/fate/FateManager.java new file mode 100644 index 00000000000..fb552f66181 --- /dev/null +++ b/server/manager/src/main/java/org/apache/accumulo/manager/fate/FateManager.java @@ -0,0 +1,453 @@ +/* + * 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.manager.fate; + +import static org.apache.accumulo.core.lock.ServiceLockPaths.ResourceGroupPredicate.DEFAULT_RG_ONLY; + +import java.time.Duration; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; + +import org.apache.accumulo.core.fate.FateId; +import org.apache.accumulo.core.fate.FateInstanceType; +import org.apache.accumulo.core.fate.FatePartition; +import org.apache.accumulo.core.fate.user.UserFateStore; +import org.apache.accumulo.core.lock.ServiceLockPaths.AddressSelector; +import org.apache.accumulo.core.manager.thrift.FateWorkerService; +import org.apache.accumulo.core.metadata.SystemTables; +import org.apache.accumulo.core.rpc.ThriftUtil; +import org.apache.accumulo.core.rpc.clients.ThriftClientTypes; +import org.apache.accumulo.core.trace.TraceUtil; +import org.apache.accumulo.core.util.CountDownTimer; +import org.apache.accumulo.core.util.threads.Threads; +import org.apache.accumulo.manager.tableOps.FateEnv; +import org.apache.accumulo.server.ServerContext; +import org.apache.thrift.TException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Range; +import com.google.common.collect.RangeMap; +import com.google.common.collect.Sets; +import com.google.common.collect.TreeRangeMap; +import com.google.common.net.HostAndPort; + +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; + +/** + * Partitions {@link FateInstanceType#USER} fate across manager assistant processes. This is done by + * assigning ranges of the fate uuid key space to different processes. The partitions are logical + * and do not correspond to the physical partitioning of the fate table. + * + *

+ * Does not currently manage {@link FateInstanceType#META} + *

+ */ +public class FateManager { + + private static final Logger log = LoggerFactory.getLogger(FateManager.class); + + private final ServerContext context; + + public FateManager(ServerContext context) { + this.context = context; + } + + private final AtomicBoolean stop = new AtomicBoolean(false); + + record FateHostPartition(HostAndPort hostPort, FatePartition partition) { + } + + private final AtomicReference> stableAssignments = + new AtomicReference<>(TreeRangeMap.create()); + + private final Map> pendingNotifications = new HashMap<>(); + + private void manageAssistants() throws TException, InterruptedException { + log.debug("Started Fate Manager"); + long stableCount = 0; + outer: while (!stop.get()) { + + long sleepTime = Math.min(stableCount * 100, 5_000); + Thread.sleep(sleepTime); + + // This map will contain all current workers even if their partitions are empty + Map currentPartitions; + try { + currentPartitions = getCurrentAssignments(); + } catch (TException e) { + log.warn("Failed to get current partitions ", e); + continue; + } + Map> currentAssignments = new HashMap<>(); + currentPartitions.forEach((k, v) -> currentAssignments.put(k, v.partitions())); + Set desiredParititions = getDesiredPartitions(currentAssignments.size()); + + Map> desired = + computeDesiredAssignments(currentAssignments, desiredParititions); + + if (desired.equals(currentAssignments)) { + RangeMap rangeMap = TreeRangeMap.create(); + currentAssignments.forEach((hostAndPort, partitions) -> { + partitions.forEach(partition -> { + rangeMap.put(Range.closed(partition.start(), partition.end()), + new FateHostPartition(hostAndPort, partition)); + }); + }); + stableAssignments.set(rangeMap); + stableCount++; + } else { + stableAssignments.set(TreeRangeMap.create()); + stableCount = 0; + } + + // are there any workers with extra partitions? If so need to unload those first. + int unloads = 0; + for (Map.Entry> entry : desired.entrySet()) { + HostAndPort worker = entry.getKey(); + Set partitions = entry.getValue(); + var curr = currentAssignments.getOrDefault(worker, Set.of()); + if (!Sets.difference(curr, partitions).isEmpty()) { + // This worker has extra partitions that are not desired + var intersection = Sets.intersection(curr, partitions); + if (!setPartitions(worker, currentPartitions.get(worker).updateId(), intersection)) { + log.debug("Failed to set partitions for {} to {}", worker, intersection); + // could not set, so start completely over + continue outer; + } else { + log.debug("Set partitions for {} to {} from {}", worker, intersection, curr); + unloads++; + } + } + } + + if (unloads > 0) { + // some tablets were unloaded, so start over and get new update ids and the current + // partitions + continue outer; + } + + // Load all partitions on all workers.. + for (Map.Entry> entry : desired.entrySet()) { + HostAndPort worker = entry.getKey(); + Set partitions = entry.getValue(); + var curr = currentAssignments.getOrDefault(worker, Set.of()); + if (!curr.equals(partitions)) { + if (!setPartitions(worker, currentPartitions.get(worker).updateId(), partitions)) { + log.debug("Failed to set partitions for {} to {}", worker, partitions); + // could not set, so start completely over + continue outer; + } else { + log.debug("Set partitions for {} to {} from {}", worker, partitions, curr); + } + } + } + } + } + + private Thread assignmentThread = null; + private Thread ntfyThread = null; + + public synchronized void start() { + Preconditions.checkState(assignmentThread == null); + Preconditions.checkState(ntfyThread == null); + Preconditions.checkState(!stop.get()); + + assignmentThread = Threads.createCriticalThread("Fate Manager", () -> { + try { + manageAssistants(); + } catch (Exception e) { + throw new IllegalStateException(e); + } + }); + assignmentThread.start(); + + ntfyThread = Threads.createCriticalThread("Fate Notify", new NotifyTask()); + ntfyThread.start(); + } + + @SuppressFBWarnings(value = "SWL_SLEEP_WITH_LOCK_HELD", + justification = "Sleep is okay. Can hold the lock as long as needed, as we are shutting down." + + " Don't need or want other operations to run.") + public synchronized void stop(Duration timeout) { + if (!stop.compareAndSet(false, true)) { + return; + } + + var timer = CountDownTimer.startNew(timeout); + + try { + if (assignmentThread != null) { + assignmentThread.join(); + } + if (ntfyThread != null) { + ntfyThread.join(); + } + } catch (InterruptedException e) { + throw new IllegalStateException(e); + } + // Try to set every assistant manager to an empty set of partitions. This will cause them all to + // stop looking for work. + Map currentAssignments = null; + try { + currentAssignments = getCurrentAssignments(); + } catch (TException e) { + log.warn("Failed to get current assignments", e); + currentAssignments = Map.of(); + } + for (var entry : currentAssignments.entrySet()) { + var hostPort = entry.getKey(); + var currentPartitions = entry.getValue(); + if (!currentPartitions.partitions.isEmpty()) { + try { + setPartitions(hostPort, currentPartitions.updateId(), Set.of()); + } catch (TException e) { + log.warn("Failed to unassign fate partitions {}", hostPort, e); + } + } + } + + stableAssignments.set(TreeRangeMap.create()); + + if (!timer.isExpired()) { + var store = new UserFateStore(context, SystemTables.FATE.tableName(), null, null); + + var reserved = store.getActiveReservations(Set.of(FatePartition.all(FateInstanceType.USER))); + while (!reserved.isEmpty() && !timer.isExpired()) { + if (log.isTraceEnabled()) { + reserved.forEach((fateId, reservation) -> { + log.trace("In stop(), waiting on {} {} ", fateId, reservation); + }); + } + try { + Thread.sleep(Math.min(100, timer.timeLeft(TimeUnit.MILLISECONDS))); + } catch (InterruptedException e) { + throw new IllegalStateException(e); + } + } + } + } + + /** + * Makes a best effort to notify this fate operation was seeded. + */ + public void notifySeeded(FateId fateId) { + var hostPartition = stableAssignments.get().get(fateId); + if (hostPartition != null) { + synchronized (pendingNotifications) { + pendingNotifications.computeIfAbsent(hostPartition.hostPort(), k -> new HashSet<>()) + .add(hostPartition.partition()); + pendingNotifications.notify(); + } + } + } + + private class NotifyTask implements Runnable { + + @Override + public void run() { + while (!stop.get()) { + try { + Map> copy; + synchronized (pendingNotifications) { + if (pendingNotifications.isEmpty()) { + pendingNotifications.wait(100); + } + copy = Map.copyOf(pendingNotifications); + pendingNotifications.clear(); + } + + for (var entry : copy.entrySet()) { + HostAndPort address = entry.getKey(); + Set partitions = entry.getValue(); + FateWorkerService.Client client = + ThriftUtil.getClient(ThriftClientTypes.FATE_WORKER, address, context); + try { + log.trace("Notifying about seeding {} {}", address, partitions); + client.seeded(TraceUtil.traceInfo(), context.rpcCreds(), + partitions.stream().map(FatePartition::toThrift).toList()); + } finally { + ThriftUtil.returnClient(client, context); + } + } + + } catch (InterruptedException e) { + throw new IllegalStateException(e); + } catch (TException e) { + log.warn("Failed to send notification that fate was seeded", e); + } + } + } + } + + /** + * Sets the complete set of partitions an assistant manager should work on. It will only succeed + * if the update id is valid. The update id avoids race conditions w/ previously queued network + * messages, it's a distributed compare and set mechanism that can detect changes. + * + * @param address The assistant manager to set partitions on + * @param updateId The update id returned when asking the assistant manager what its current + * partitions were. + * @param desired The new set of fate partitions this server should start working. It should only + * work on these and nothing else. + * @return true if the partitions were set false if they were not set. + */ + private boolean setPartitions(HostAndPort address, long updateId, Set desired) + throws TException { + FateWorkerService.Client client = + ThriftUtil.getClient(ThriftClientTypes.FATE_WORKER, address, context); + try { + log.trace("Setting partitions {} {} {}", address, updateId, desired); + return client.setPartitions(TraceUtil.traceInfo(), context.rpcCreds(), updateId, + desired.stream().map(FatePartition::toThrift).toList()); + } finally { + ThriftUtil.returnClient(client, context); + } + } + + /** + * Compute the desired distribution of partitions across workers. Favors leaving partitions in + * place if possible. + */ + private Map> computeDesiredAssignments( + Map> currentAssignments, + Set desiredParititions) { + + Preconditions.checkArgument(currentAssignments.size() == desiredParititions.size()); + Map> desiredAssignments = new HashMap<>(); + + var copy = new HashSet<>(desiredParititions); + + currentAssignments.forEach((hp, partitions) -> { + if (!partitions.isEmpty()) { + var firstPart = partitions.iterator().next(); + if (copy.contains(firstPart)) { + desiredAssignments.put(hp, Set.of(firstPart)); + copy.remove(firstPart); + } + } + }); + + var iter = copy.iterator(); + currentAssignments.forEach((hp, partitions) -> { + if (!desiredAssignments.containsKey(hp)) { + desiredAssignments.put(hp, Set.of(iter.next())); + } + }); + + if (log.isTraceEnabled()) { + log.trace("Logging desired partitions"); + desiredAssignments.forEach((hp, parts) -> { + log.trace(" desired {} {} {}", hp, parts.size(), parts); + }); + } + + return desiredAssignments; + } + + /** + * Computes a single partition for each worker such that the partition cover all possible UUIDs + * and evenly divide the UUIDs. + */ + private Set getDesiredPartitions(int numWorkers) { + Preconditions.checkArgument(numWorkers >= 0); + + if (numWorkers == 0) { + return Set.of(); + } + + // create a single partition per worker that equally divides the space + HashSet desired = new HashSet<>(); + // All the shifting is because java does not have unsigned integers. Want to evenly partition + // [0,2^64) into numWorker ranges, but can not directly do that. Work w/ 60 bit unsigned + // integers to partition the space and then shift over by 4. Used 60 bits instead of 63 so it + // nicely aligns w/ hex in the uuid. + long jump = ((1L << 60)) / numWorkers; + for (int i = 0; i < numWorkers - 1; i++) { + long start = (i * jump) << 4; + long end = ((i + 1) * jump) << 4; + + UUID startUuid = new UUID(start, 0); + UUID endUuid = new UUID(end, 0); + + desired.add(new FatePartition(FateId.from(FateInstanceType.USER, startUuid), + FateId.from(FateInstanceType.USER, endUuid))); + } + + long start = ((numWorkers - 1) * jump) << 4; + UUID startUuid = new UUID(start, 0); + // last partition has a special end uuid that is all f nibbles. + UUID endUuid = new UUID(-1, -1); + desired.add(new FatePartition(FateId.from(FateInstanceType.USER, startUuid), + FateId.from(FateInstanceType.USER, endUuid))); + + return desired; + } + + // The updateId accomplishes two things. First it ensures that setting partition RPC can only + // execute once on the server side. Second when a new update id is requested it cancels any + // outstanding RPCs to set partitions that have not executed yet. + record CurrentPartitions(long updateId, Set partitions) { + } + + /** + * @return the fate partitions that assistant managers are currently assigned + */ + private Map getCurrentAssignments() throws TException { + var workers = + context.getServerPaths().getManagerAssistants(DEFAULT_RG_ONLY, AddressSelector.all(), true); + + log.trace("getting current assignments from {}", workers); + + Map currentAssignments = new HashMap<>(); + + for (var worker : workers) { + var address = HostAndPort.fromString(worker.getServer()); + + FateWorkerService.Client client = + ThriftUtil.getClient(ThriftClientTypes.FATE_WORKER, address, context); + try { + + var tparitions = client.getPartitions(TraceUtil.traceInfo(), context.rpcCreds()); + var partitions = + tparitions.partitions.stream().map(FatePartition::from).collect(Collectors.toSet()); + currentAssignments.put(address, new CurrentPartitions(tparitions.updateId, partitions)); + } finally { + ThriftUtil.returnClient(client, context); + } + } + + if (log.isTraceEnabled()) { + log.trace("Logging current assignments"); + currentAssignments.forEach((hostPort, partitions) -> { + log.trace("current assignment {} {}", hostPort, partitions); + }); + } + + return currentAssignments; + } +} diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/fate/FateWorker.java b/server/manager/src/main/java/org/apache/accumulo/manager/fate/FateWorker.java new file mode 100644 index 00000000000..f0b66a6404f --- /dev/null +++ b/server/manager/src/main/java/org/apache/accumulo/manager/fate/FateWorker.java @@ -0,0 +1,188 @@ +/* + * 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.manager.fate; + +import static org.apache.accumulo.core.util.LazySingletons.RANDOM; + +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.function.Predicate; +import java.util.stream.Collectors; + +import org.apache.accumulo.core.client.AccumuloSecurityException; +import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode; +import org.apache.accumulo.core.clientImpl.thrift.TInfo; +import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException; +import org.apache.accumulo.core.conf.Property; +import org.apache.accumulo.core.fate.Fate; +import org.apache.accumulo.core.fate.FatePartition; +import org.apache.accumulo.core.fate.FateStore; +import org.apache.accumulo.core.fate.user.UserFateStore; +import org.apache.accumulo.core.fate.zookeeper.ZooUtil; +import org.apache.accumulo.core.lock.ServiceLock; +import org.apache.accumulo.core.manager.thrift.FateWorkerService; +import org.apache.accumulo.core.manager.thrift.TFatePartition; +import org.apache.accumulo.core.manager.thrift.TFatePartitions; +import org.apache.accumulo.core.metadata.SystemTables; +import org.apache.accumulo.core.metrics.MetricsProducer; +import org.apache.accumulo.core.securityImpl.thrift.TCredentials; +import org.apache.accumulo.manager.metrics.fate.FateExecutorMetricsProducer; +import org.apache.accumulo.manager.tableOps.FateEnv; +import org.apache.accumulo.server.AccumuloDataVersion; +import org.apache.accumulo.server.ServerContext; +import org.apache.accumulo.server.manager.LiveTServerSet; +import org.apache.accumulo.server.security.AuditedSecurityOperation; +import org.apache.thrift.TException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.base.Preconditions; + +public class FateWorker implements FateWorkerService.Iface { + + private static final Logger log = LoggerFactory.getLogger(FateWorker.class); + private final ServerContext context; + private final AuditedSecurityOperation security; + private final LiveTServerSet liveTserverSet; + private final FateFactory fateFactory; + private Fate fate; + private FateWorkerEnv fateWorkerEnv; + + public interface FateFactory { + Fate create(FateEnv env, FateStore store, ServerContext context); + } + + public FateWorker(ServerContext ctx, LiveTServerSet liveTServerSet, FateFactory fateFactory) { + this.context = ctx; + this.security = ctx.getSecurityOperation(); + this.fate = null; + this.liveTserverSet = liveTServerSet; + this.fateFactory = fateFactory; + } + + public synchronized void setLock(ServiceLock lock) { + fateWorkerEnv = new FateWorkerEnv(context, lock, liveTserverSet); + Predicate isLockHeld = l -> ServiceLock.isLockHeld(context.getZooCache(), l); + UserFateStore store = + new UserFateStore<>(context, SystemTables.FATE.tableName(), lock.getLockID(), isLockHeld); + this.fate = fateFactory.create(fateWorkerEnv, store, context); + } + + private Long expectedUpdateId = null; + + @Override + public TFatePartitions getPartitions(TInfo tinfo, TCredentials credentials) + throws ThriftSecurityException { + if (!security.canPerformSystemActions(credentials)) { + throw new AccumuloSecurityException(credentials.getPrincipal(), + SecurityErrorCode.PERMISSION_DENIED).asThriftException(); + } + + // generate a new one time use update id + long updateId = RANDOM.get().nextLong(); + + // Getting the partitions and setting the new update id must be mutually exclusive with any + // updates of the partitions concurrently executing. This ensures the new update id goes with + // the current partitions returned. + synchronized (this) { + // invalidate any queued partitions update that have not executed yet and set the new update + // id + expectedUpdateId = updateId; + + if (fate == null) { + return new TFatePartitions(updateId, List.of()); + } else { + return new TFatePartitions(updateId, + fate.getPartitions().stream().map(FatePartition::toThrift).toList()); + } + } + } + + private boolean upgradeComplete = false; + + // Checks in persistent storage if upgrade is complete. Once it sees its complete remembers this + // and stops checking. + private synchronized boolean isUpgradeComplete() { + if (!upgradeComplete) { + upgradeComplete = AccumuloDataVersion.getCurrentVersion(context) >= AccumuloDataVersion.get(); + } + + return upgradeComplete; + } + + @Override + public boolean setPartitions(TInfo tinfo, TCredentials credentials, long updateId, + List desired) throws ThriftSecurityException { + if (!security.canPerformSystemActions(credentials)) { + throw new AccumuloSecurityException(credentials.getPrincipal(), + SecurityErrorCode.PERMISSION_DENIED).asThriftException(); + } + + synchronized (this) { + // The primary manager should not assign any fate partitions until after upgrade is complete. + Preconditions.checkState(isUpgradeComplete()); + if (fate != null && expectedUpdateId != null && updateId == expectedUpdateId) { + // Set to null which makes it so that an update id can only be used once. + expectedUpdateId = null; + var desiredSet = desired.stream().map(FatePartition::from).collect(Collectors.toSet()); + var oldPartitions = fate.setPartitions(desiredSet); + log.info("Changed partitions from {} to {}", oldPartitions, desiredSet); + return true; + } else { + log.debug("Did not change partitions to {} expectedUpdateId:{} updateId:{} fate==null:{}", + desired, expectedUpdateId, updateId, fate == null); + return false; + } + } + } + + @Override + public void seeded(TInfo tinfo, TCredentials credentials, List tpartitions) + throws TException { + + if (!security.canPerformSystemActions(credentials)) { + throw new AccumuloSecurityException(credentials.getPrincipal(), + SecurityErrorCode.PERMISSION_DENIED).asThriftException(); + } + + Fate localFate; + synchronized (this) { + localFate = fate; + } + + if (localFate != null) { + localFate.seeded(tpartitions.stream().map(FatePartition::from).collect(Collectors.toSet())); + } + } + + public synchronized void stop() { + fate.shutdown(1, TimeUnit.MINUTES); + fate.close(); + fateWorkerEnv.stop(); + fate = null; + fateWorkerEnv = null; + } + + public synchronized List getMetricsProducers() { + Preconditions.checkState(fate != null, "Not started yet"); + return List.of(new FateExecutorMetricsProducer(context, fate.getFateExecutors(), context + .getConfiguration().getTimeInMillis(Property.MANAGER_FATE_METRICS_MIN_UPDATE_INTERVAL))); + + } +} diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/fate/FateWorkerEnv.java b/server/manager/src/main/java/org/apache/accumulo/manager/fate/FateWorkerEnv.java new file mode 100644 index 00000000000..2b826540014 --- /dev/null +++ b/server/manager/src/main/java/org/apache/accumulo/manager/fate/FateWorkerEnv.java @@ -0,0 +1,228 @@ +/* + * 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.manager.fate; + +import static org.apache.accumulo.core.util.threads.ThreadPoolNames.FILE_RENAME_POOL; + +import java.util.Collection; +import java.util.Set; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.apache.accumulo.core.client.AccumuloException; +import org.apache.accumulo.core.client.AccumuloSecurityException; +import org.apache.accumulo.core.conf.Property; +import org.apache.accumulo.core.data.TableId; +import org.apache.accumulo.core.dataImpl.KeyExtent; +import org.apache.accumulo.core.lock.ServiceLock; +import org.apache.accumulo.core.metadata.TServerInstance; +import org.apache.accumulo.core.metadata.schema.Ample; +import org.apache.accumulo.core.metadata.schema.ExternalCompactionId; +import org.apache.accumulo.core.rpc.ThriftUtil; +import org.apache.accumulo.core.rpc.clients.ThriftClientTypes; +import org.apache.accumulo.core.trace.TraceUtil; +import org.apache.accumulo.core.util.threads.ThreadPools; +import org.apache.accumulo.core.util.threads.Threads; +import org.apache.accumulo.core.util.time.SteadyTime; +import org.apache.accumulo.manager.EventCoordinator; +import org.apache.accumulo.manager.EventPublisher; +import org.apache.accumulo.manager.EventQueue; +import org.apache.accumulo.manager.split.FileRangeCache; +import org.apache.accumulo.manager.tableOps.FateEnv; +import org.apache.accumulo.server.ServerContext; +import org.apache.accumulo.server.fs.VolumeManager; +import org.apache.accumulo.server.manager.LiveTServerSet; +import org.apache.accumulo.server.tables.TableManager; +import org.apache.thrift.TException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class FateWorkerEnv implements FateEnv { + + private static final Logger log = LoggerFactory.getLogger(FateWorkerEnv.class); + + private final ServerContext ctx; + private final ExecutorService refreshPool; + private final ExecutorService renamePool; + private final ServiceLock serviceLock; + private final FileRangeCache fileRangeCache; + private final EventHandler eventHandler; + private final LiveTServerSet liveTServerSet; + + private final EventQueue queue = new EventQueue(); + private final AtomicBoolean stopped = new AtomicBoolean(false); + private final Thread eventSendThread; + + public void stop() { + stopped.set(true); + try { + eventSendThread.join(); + } catch (InterruptedException e) { + throw new IllegalStateException(e); + } + } + + private class EventSender implements Runnable { + @Override + public void run() { + while (!stopped.get()) { + try { + var events = queue.poll(100, TimeUnit.MILLISECONDS); + if (events.isEmpty()) { + continue; + } + + var tEvents = events.stream().map(EventCoordinator.Event::toThrift).toList(); + + var client = ThriftClientTypes.MANAGER.getConnection(ctx); + try { + if (client != null) { + client.processEvents(TraceUtil.traceInfo(), ctx.rpcCreds(), tEvents); + } + } catch (TException e) { + log.warn("Failed to send events to manager", e); + } finally { + if (client != null) { + ThriftUtil.close(client, ctx); + } + } + + } catch (InterruptedException e) { + throw new IllegalStateException(e); + } + } + } + } + + private class EventHandler implements EventPublisher { + + @Override + public void event(String msg, Object... args) { + log.info(String.format(msg, args)); + queue.add(new EventCoordinator.Event()); + } + + @Override + public void event(Ample.DataLevel level, String msg, Object... args) { + log.info(String.format(msg, args)); + queue.add(new EventCoordinator.Event(level)); + } + + @Override + public void event(TableId tableId, String msg, Object... args) { + log.info(String.format(msg, args)); + queue.add(new EventCoordinator.Event(tableId)); + } + + @Override + public void event(KeyExtent extent, String msg, Object... args) { + log.debug(String.format(msg, args)); + queue.add(new EventCoordinator.Event(extent)); + } + + @Override + public void event(Collection extents, String msg, Object... args) { + if (!extents.isEmpty()) { + log.debug(String.format(msg, args)); + extents.forEach(extent -> queue.add(new EventCoordinator.Event(extent))); + } + } + } + + FateWorkerEnv(ServerContext ctx, ServiceLock lock, LiveTServerSet liveTserverSet) { + this.ctx = ctx; + this.refreshPool = ThreadPools.getServerThreadPools().getPoolBuilder("Tablet refresh ") + .numCoreThreads(ctx.getConfiguration().getCount(Property.MANAGER_TABLET_REFRESH_MINTHREADS)) + .numMaxThreads(ctx.getConfiguration().getCount(Property.MANAGER_TABLET_REFRESH_MAXTHREADS)) + .build(); + int poolSize = ctx.getConfiguration().getCount(Property.MANAGER_RENAME_THREADS); + this.renamePool = ThreadPools.getServerThreadPools().getPoolBuilder(FILE_RENAME_POOL.poolName) + .numCoreThreads(poolSize).build(); + this.serviceLock = lock; + this.fileRangeCache = new FileRangeCache(ctx); + this.eventHandler = new EventHandler(); + this.liveTServerSet = liveTserverSet; + + eventSendThread = Threads.createCriticalThread("Fate Worker Event Sender", new EventSender()); + eventSendThread.start(); + } + + @Override + public ServerContext getContext() { + return ctx; + } + + @Override + public EventPublisher getEventPublisher() { + return eventHandler; + } + + @Override + public void recordCompactionCompletion(ExternalCompactionId ecid) { + // FOLLOW_ON This data is stored in memory on the manager. This entire feature needs to be + // examined and potentially reworked. One solution would be to send an RPC to the manager to + // update it's in memory state. A better solution would be to move away from in memory state + // that is lost when the manager restarts. + } + + @Override + public Set onlineTabletServers() { + return liveTServerSet.getSnapshot().getTservers(); + } + + @Override + public TableManager getTableManager() { + return ctx.getTableManager(); + } + + @Override + public VolumeManager getVolumeManager() { + return ctx.getVolumeManager(); + } + + @Override + public ServiceLock getServiceLock() { + return serviceLock; + } + + @Override + public SteadyTime getSteadyTime() { + try { + return SteadyTime.from(ctx.instanceOperations().getManagerTime()); + } catch (AccumuloException | AccumuloSecurityException e) { + throw new IllegalStateException(e); + } + } + + @Override + public ExecutorService getTabletRefreshThreadPool() { + return refreshPool; + } + + @Override + public FileRangeCache getFileRangeCache() { + return fileRangeCache; + } + + @Override + public ExecutorService getRenamePool() { + return renamePool; + } +} diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/ManagerMetrics.java b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/ManagerMetrics.java index 77689cea028..9e6f9c4f2dd 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/ManagerMetrics.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/ManagerMetrics.java @@ -27,14 +27,11 @@ import java.util.ArrayList; import java.util.List; -import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import org.apache.accumulo.core.conf.AccumuloConfiguration; import org.apache.accumulo.core.conf.Property; -import org.apache.accumulo.core.fate.Fate; -import org.apache.accumulo.core.fate.FateInstanceType; import org.apache.accumulo.core.manager.thrift.ManagerGoalState; import org.apache.accumulo.core.metadata.schema.Ample.DataLevel; import org.apache.accumulo.core.metrics.MetricsProducer; @@ -42,7 +39,6 @@ import org.apache.accumulo.manager.metrics.fate.FateMetrics; import org.apache.accumulo.manager.metrics.fate.meta.MetaFateMetrics; import org.apache.accumulo.manager.metrics.fate.user.UserFateMetrics; -import org.apache.accumulo.manager.tableOps.FateEnv; import io.micrometer.core.instrument.Gauge; import io.micrometer.core.instrument.MeterRegistry; @@ -66,17 +62,14 @@ public void updateManagerGoalState(ManagerGoalState goal) { goalState.set(newValue); } - public void configureFateMetrics(final AccumuloConfiguration conf, final Manager manager, - Map> fateRefs) { + public void configureFateMetrics(final AccumuloConfiguration conf, final Manager manager) { requireNonNull(conf, "AccumuloConfiguration must not be null"); requireNonNull(conf, "Manager must not be null"); fateMetrics = List.of( new MetaFateMetrics(manager.getContext(), - conf.getTimeInMillis(Property.MANAGER_FATE_METRICS_MIN_UPDATE_INTERVAL), - fateRefs.get(FateInstanceType.META).getFateExecutors()), + conf.getTimeInMillis(Property.MANAGER_FATE_METRICS_MIN_UPDATE_INTERVAL)), new UserFateMetrics(manager.getContext(), - conf.getTimeInMillis(Property.MANAGER_FATE_METRICS_MIN_UPDATE_INTERVAL), - fateRefs.get(FateInstanceType.USER).getFateExecutors())); + conf.getTimeInMillis(Property.MANAGER_FATE_METRICS_MIN_UPDATE_INTERVAL))); } public void incrementTabletGroupWatcherError(DataLevel level) { diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateExecutorMetricsProducer.java b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateExecutorMetricsProducer.java new file mode 100644 index 00000000000..43b8508c410 --- /dev/null +++ b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateExecutorMetricsProducer.java @@ -0,0 +1,74 @@ +/* + * 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.manager.metrics.fate; + +import static org.apache.accumulo.manager.metrics.fate.FateMetrics.DEFAULT_MIN_REFRESH_DELAY; + +import java.util.Set; +import java.util.concurrent.TimeUnit; + +import org.apache.accumulo.core.fate.FateExecutor; +import org.apache.accumulo.core.metrics.MetricsProducer; +import org.apache.accumulo.core.util.threads.ThreadPools; +import org.apache.accumulo.manager.tableOps.FateEnv; +import org.apache.accumulo.server.ServerContext; + +import io.micrometer.core.instrument.MeterRegistry; + +public class FateExecutorMetricsProducer implements MetricsProducer { + private final Set> fateExecutors; + private final ServerContext context; + private final long refreshDelay; + private MeterRegistry registry; + + public FateExecutorMetricsProducer(ServerContext context, + Set> fateExecutors, long minimumRefreshDelay) { + this.context = context; + this.fateExecutors = fateExecutors; + this.refreshDelay = Math.max(DEFAULT_MIN_REFRESH_DELAY, minimumRefreshDelay); + } + + protected void update() { + // there may have been new fate executors added, so these need to be registered. + // fate executors removed will have their metrics removed from the registry before they are + // removed from the set. + if (registry != null) { + synchronized (fateExecutors) { + fateExecutors.forEach(fe -> { + var feMetrics = fe.getFateExecutorMetrics(); + if (!feMetrics.isRegistered()) { + feMetrics.registerMetrics(registry); + } + }); + } + } + } + + @Override + public void registerMetrics(final MeterRegistry registry) { + this.registry = registry; + synchronized (fateExecutors) { + fateExecutors.forEach(fe -> fe.getFateExecutorMetrics().registerMetrics(registry)); + } + + var future = context.getScheduledExecutor().scheduleAtFixedRate(this::update, refreshDelay, + refreshDelay, TimeUnit.MILLISECONDS); + ThreadPools.watchCriticalScheduledTask(future); + } +} diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetrics.java b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetrics.java index bf69885b9a1..16af60b4723 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetrics.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetrics.java @@ -25,18 +25,15 @@ import java.util.EnumMap; import java.util.Map.Entry; import java.util.Objects; -import java.util.Set; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; -import org.apache.accumulo.core.fate.FateExecutor; import org.apache.accumulo.core.fate.ReadOnlyFateStore; import org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus; import org.apache.accumulo.core.metrics.MetricsProducer; import org.apache.accumulo.core.util.threads.ThreadPools; -import org.apache.accumulo.manager.tableOps.FateEnv; import org.apache.accumulo.server.ServerContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -51,25 +48,21 @@ public abstract class FateMetrics implements Metrics private static final Logger log = LoggerFactory.getLogger(FateMetrics.class); // limit calls to update fate counters to guard against hammering zookeeper. - private static final long DEFAULT_MIN_REFRESH_DELAY = TimeUnit.SECONDS.toMillis(5); + static final long DEFAULT_MIN_REFRESH_DELAY = TimeUnit.SECONDS.toMillis(5); private static final String OP_TYPE_TAG = "op.type"; protected final ServerContext context; protected final ReadOnlyFateStore> readOnlyFateStore; protected final long refreshDelay; - private final Set> fateExecutors; - private MeterRegistry registry; protected final AtomicLong totalCurrentOpsCount = new AtomicLong(0); private final EnumMap txStatusCounters = new EnumMap<>(TStatus.class); - public FateMetrics(final ServerContext context, final long minimumRefreshDelay, - Set> fateExecutors) { + public FateMetrics(final ServerContext context, final long minimumRefreshDelay) { this.context = context; this.refreshDelay = Math.max(DEFAULT_MIN_REFRESH_DELAY, minimumRefreshDelay); this.readOnlyFateStore = Objects.requireNonNull(buildReadOnlyStore(context)); - this.fateExecutors = fateExecutors; for (TStatus status : TStatus.values()) { txStatusCounters.put(status, new AtomicLong(0)); @@ -98,25 +91,10 @@ protected void update(T metricValues) { metricValues.getOpTypeCounters().forEach((name, count) -> Metrics .gauge(FATE_TYPE_IN_PROGRESS.getName(), Tags.of(OP_TYPE_TAG, name), count)); - - // there may have been new fate executors added, so these need to be registered. - // fate executors removed will have their metrics removed from the registry before they are - // removed from the set. - if (registry != null) { - synchronized (fateExecutors) { - fateExecutors.forEach(fe -> { - var feMetrics = fe.getFateExecutorMetrics(); - if (!feMetrics.isRegistered()) { - feMetrics.registerMetrics(registry); - } - }); - } - } } @Override public void registerMetrics(final MeterRegistry registry) { - this.registry = registry; String type = readOnlyFateStore.type().name().toLowerCase(); Gauge.builder(FATE_OPS.getName(), totalCurrentOpsCount, AtomicLong::get) @@ -126,10 +104,6 @@ public void registerMetrics(final MeterRegistry registry) { .builder(FATE_TX.getName(), counter, AtomicLong::get).description(FATE_TX.getDescription()) .tags("state", status.name().toLowerCase(), "instanceType", type).register(registry)); - synchronized (fateExecutors) { - fateExecutors.forEach(fe -> fe.getFateExecutorMetrics().registerMetrics(registry)); - } - // get fate status is read only operation - no reason to be nice on shutdown. ScheduledExecutorService scheduler = ThreadPools.getServerThreadPools() .createScheduledExecutorService(1, type + "FateMetricsPoller"); diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/meta/MetaFateMetrics.java b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/meta/MetaFateMetrics.java index 7e19d70e847..9d87f9a9cc2 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/meta/MetaFateMetrics.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/meta/MetaFateMetrics.java @@ -21,15 +21,12 @@ import static org.apache.accumulo.core.metrics.Metric.FATE_ERRORS; import static org.apache.accumulo.core.metrics.Metric.FATE_OPS_ACTIVITY; -import java.util.Set; import java.util.concurrent.atomic.AtomicLong; import org.apache.accumulo.core.Constants; -import org.apache.accumulo.core.fate.FateExecutor; import org.apache.accumulo.core.fate.ReadOnlyFateStore; import org.apache.accumulo.core.fate.zookeeper.MetaFateStore; import org.apache.accumulo.manager.metrics.fate.FateMetrics; -import org.apache.accumulo.manager.tableOps.FateEnv; import org.apache.accumulo.server.ServerContext; import org.apache.zookeeper.KeeperException; @@ -41,9 +38,8 @@ public class MetaFateMetrics extends FateMetrics { private final AtomicLong totalOpsGauge = new AtomicLong(0); private final AtomicLong fateErrorsGauge = new AtomicLong(0); - public MetaFateMetrics(ServerContext context, long minimumRefreshDelay, - Set> fateExecutors) { - super(context, minimumRefreshDelay, fateExecutors); + public MetaFateMetrics(ServerContext context, long minimumRefreshDelay) { + super(context, minimumRefreshDelay); } @Override diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/user/UserFateMetrics.java b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/user/UserFateMetrics.java index 7fab73944d8..4f1df05762a 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/user/UserFateMetrics.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/user/UserFateMetrics.java @@ -18,21 +18,16 @@ */ package org.apache.accumulo.manager.metrics.fate.user; -import java.util.Set; - -import org.apache.accumulo.core.fate.FateExecutor; import org.apache.accumulo.core.fate.ReadOnlyFateStore; import org.apache.accumulo.core.fate.user.UserFateStore; import org.apache.accumulo.core.metadata.SystemTables; import org.apache.accumulo.manager.metrics.fate.FateMetrics; -import org.apache.accumulo.manager.tableOps.FateEnv; import org.apache.accumulo.server.ServerContext; public class UserFateMetrics extends FateMetrics { - public UserFateMetrics(ServerContext context, long minimumRefreshDelay, - Set> fateExecutors) { - super(context, minimumRefreshDelay, fateExecutors); + public UserFateMetrics(ServerContext context, long minimumRefreshDelay) { + super(context, minimumRefreshDelay); } @Override diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/split/PreSplit.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/split/PreSplit.java index 9b1eaf6d682..ed6879eaffd 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/split/PreSplit.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/split/PreSplit.java @@ -107,7 +107,7 @@ public long isReady(FateId fateId, FateEnv env) throws Exception { // now that the operation id set, generate an event to unload the tablet or recover the // logs env.getEventPublisher().event(splitInfo.getOriginal(), - "Set operation id %s on tablet for split", fateId); + "Set operation id %s on tablet %s for split", fateId, splitInfo.getOriginal()); // the operation id was set, but a location is also set wait for it be unset return 1000; } diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader11to12.java b/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader11to12.java index 2464c9476bf..a230dc7149a 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader11to12.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader11to12.java @@ -253,6 +253,8 @@ public void upgradeZookeeper(ServerContext context) { addDefaultResourceGroupConfigNode(context); LOG.info("Moving table properties from system to namespaces"); moveTableProperties(context); + LOG.info("Add assistant manager node"); + addAssistantManager(context); } @Override @@ -297,6 +299,15 @@ public void upgradeMetadata(ServerContext context) { removeBulkFileColumnsFromTable(context, SystemTables.METADATA.tableName()); } + private static void addAssistantManager(ServerContext context) { + try { + context.getZooSession().asReaderWriter().putPersistentData(Constants.ZMANAGER_ASSISTANT_LOCK, + new byte[0], ZooUtil.NodeExistsPolicy.SKIP); + } catch (KeeperException | InterruptedException e) { + throw new IllegalStateException(e); + } + } + private static void addCompactionsNode(ServerContext context) { try { context.getZooSession().asReaderWriter().putPersistentData(Constants.ZCOMPACTIONS, diff --git a/test/src/main/java/org/apache/accumulo/test/ComprehensiveMultiManagerIT.java b/test/src/main/java/org/apache/accumulo/test/ComprehensiveMultiManagerIT.java new file mode 100644 index 00000000000..278a051ec8a --- /dev/null +++ b/test/src/main/java/org/apache/accumulo/test/ComprehensiveMultiManagerIT.java @@ -0,0 +1,58 @@ +/* + * 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; + +import org.apache.accumulo.core.client.Accumulo; +import org.apache.accumulo.core.client.AccumuloClient; +import org.apache.accumulo.core.conf.Property; +import org.apache.accumulo.harness.MiniClusterConfigurationCallback; +import org.apache.accumulo.harness.SharedMiniClusterBase; +import org.apache.accumulo.manager.Manager; +import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; + +public class ComprehensiveMultiManagerIT extends ComprehensiveITBase { + + private static class ComprehensiveITConfiguration implements MiniClusterConfigurationCallback { + @Override + public void configureMiniCluster(MiniAccumuloConfigImpl cfg, + org.apache.hadoop.conf.Configuration coreSite) { + cfg.setProperty(Property.SSERV_CACHED_TABLET_METADATA_EXPIRATION, "5s"); + } + } + + @BeforeAll + public static void setup() throws Exception { + ComprehensiveITConfiguration c = new ComprehensiveITConfiguration(); + SharedMiniClusterBase.startMiniClusterWithConfig(c); + try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) { + client.securityOperations().changeUserAuthorizations("root", AUTHORIZATIONS); + } + + // Start two more managers + getCluster().exec(Manager.class); + getCluster().exec(Manager.class); + } + + @AfterAll + public static void teardown() { + SharedMiniClusterBase.stopMiniCluster(); + } +} diff --git a/test/src/main/java/org/apache/accumulo/test/MultipleManagerIT.java b/test/src/main/java/org/apache/accumulo/test/MultipleManagerIT.java new file mode 100644 index 00000000000..ca073b551fc --- /dev/null +++ b/test/src/main/java/org/apache/accumulo/test/MultipleManagerIT.java @@ -0,0 +1,305 @@ +/* + * 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; + +import static java.util.stream.Collectors.toSet; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.TreeSet; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import org.apache.accumulo.core.Constants; +import org.apache.accumulo.core.cli.ServerOpts; +import org.apache.accumulo.core.client.Accumulo; +import org.apache.accumulo.core.client.admin.CompactionConfig; +import org.apache.accumulo.core.clientImpl.ClientContext; +import org.apache.accumulo.core.conf.Property; +import org.apache.accumulo.core.data.Mutation; +import org.apache.accumulo.core.fate.Fate; +import org.apache.accumulo.core.fate.FateInstanceType; +import org.apache.accumulo.core.fate.FatePartition; +import org.apache.accumulo.core.fate.FateStore; +import org.apache.accumulo.core.fate.TraceRepo; +import org.apache.accumulo.core.fate.user.UserFateStore; +import org.apache.accumulo.core.lock.ServiceLock; +import org.apache.accumulo.core.lock.ServiceLockPaths; +import org.apache.accumulo.core.lock.ServiceLockPaths.ServiceLockPath; +import org.apache.accumulo.core.metadata.SystemTables; +import org.apache.accumulo.core.util.UtilWaitThread; +import org.apache.accumulo.manager.Manager; +import org.apache.accumulo.manager.tableOps.FateEnv; +import org.apache.accumulo.minicluster.ServerType; +import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl; +import org.apache.accumulo.server.ServerContext; +import org.apache.accumulo.test.fate.FastFate; +import org.apache.accumulo.test.functional.ConfigurableMacBase; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.Text; +import org.junit.jupiter.api.Test; +import org.slf4j.LoggerFactory; + +import com.google.common.collect.Sets; +import com.google.common.net.HostAndPort; + +/** + * {@link ComprehensiveMultiManagerIT} runs multiple managers with lots of Accumulo APIs, however + * that does not actually verify that fate operations actually run on multiple managers. This test + * runs a smaller set of Accumulo API operations and does the following. + * + *
    + *
  • Starts new manager processes and verifies fate operations start running on them
  • + *
  • Kills assistant/non-primary manager processes and verifies the system recovers
  • + *
  • Kills primary manager process and verifies the system recovers
  • + *
  • Verifies that Accumulo API calls are not impacted by managers starting/stopping
  • + *
+ * + */ +public class MultipleManagerIT extends ConfigurableMacBase { + + // A manager that will quickly clean up fate reservations held by dead managers + public static class FastFateCleanupManager extends Manager { + protected FastFateCleanupManager(ServerOpts opts, String[] args) throws IOException { + super(opts, ServerContext::new, args); + } + + @Override + protected Fate createFateInstance(FateEnv env, FateStore store, + ServerContext context) { + LoggerFactory.getLogger(FastFateCleanupManager.class) + .info("Creating Fast fate cleanup manager for {}", store.type()); + return new FastFate<>(env, store, true, TraceRepo::toLogString, getConfiguration()); + } + + public static void main(String[] args) throws Exception { + try (FastFateCleanupManager manager = new FastFateCleanupManager(new ServerOpts(), args)) { + manager.runServer(); + } + } + } + + @Override + protected void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) { + // FOLLOW_ON add a way to start multiple managers to mini + cfg.getClusterServerConfiguration().setNumDefaultCompactors(8); + // Set this lower so that locks timeout faster + cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "5s"); + cfg.setServerClass(ServerType.MANAGER, r -> FastFateCleanupManager.class); + super.configure(cfg, hadoopCoreSite); + } + + @Test + public void testFate() throws Exception { + + List managerWorkers = new ArrayList<>(); + var executor = Executors.newCachedThreadPool(); + + // Start a lot of background threads that should cause fate operations to run. + try (var client = Accumulo.newClient().from(getClientProperties()).build()) { + // Create a table in order to wait for the single manager to become the primary manager + client.tableOperations().create("waitTable"); + + // start more manager processes, should be assigned fate work + managerWorkers.add(exec(FastFateCleanupManager.class)); + managerWorkers.add(exec(FastFateCleanupManager.class)); + + AtomicBoolean stop = new AtomicBoolean(false); + + var splits = IntStream.range(1, 10).mapToObj(i -> String.format("%03d", i)).map(Text::new) + .collect(Collectors.toCollection(TreeSet::new)); + var tableOpFutures = new ArrayList>(); + for (int i = 0; i < 10; i++) { + var table = "t" + i; + + // FOLLOW_ON its hard to find everything related to a table id in the logs across processes, + // especially when the + // table id is like "b". Was trying to follow a single table across multiple manager workers + // processes. + var tableOpsFuture = executor.submit(() -> { + int loops = 0; + while (!stop.get() || loops == 0) { + client.tableOperations().create(table); + log.info("Created table {}", table); + if (stop.get() && loops > 0) { + break; + } + var expectedRows = new HashSet(); + try (var writer = client.createBatchWriter(table)) { + for (int r = 0; r < 10; r++) { + var row = String.format("%03d", r); + expectedRows.add(row); + Mutation m = new Mutation(row); + m.put("f", "q", "v"); + writer.addMutation(m); + } + } + log.info("Wrote data to table {}", table); + if (stop.get() && loops > 0) { + break; + } + client.tableOperations().addSplits(table, splits); + log.info("Split table {}", table); + if (stop.get() && loops > 0) { + break; + } + client.tableOperations().compact(table, new CompactionConfig().setWait(true)); + log.info("Compacted table {}", table); + if (stop.get() && loops > 0) { + break; + } + client.tableOperations().merge(table, null, null); + log.info("Merged table {}", table); + if (stop.get() && loops > 0) { + break; + } + try (var scanner = client.createScanner(table)) { + var rowsSeen = + scanner.stream().map(e -> e.getKey().getRowData().toString()).collect(toSet()); + assertEquals(expectedRows, rowsSeen); + log.info("verified table {}", table); + } + client.tableOperations().delete(table); + log.info("Deleted table {}", table); + loops++; + } + return loops; + }); + tableOpFutures.add(tableOpsFuture); + } + + var ctx = getServerContext(); + + var store = new UserFateStore(ctx, SystemTables.FATE.tableName(), null, null); + + // Wait until three different manager are seen running fate operations. + waitToSeeManagers(ctx, 3, store, false); + + // Start two new manager processes and wait until 5 managers are seen running fate operations + managerWorkers.add(exec(FastFateCleanupManager.class)); + managerWorkers.add(exec(FastFateCleanupManager.class)); + waitToSeeManagers(ctx, 5, store, false); + + // Kill two assistant manager processes. Any fate operations that were running should resume + // elsewhere. Should see three manager running operations after that. + managerWorkers.get(2).destroy(); + managerWorkers.get(3).destroy(); + log.debug("Killed 2 managers"); + waitToSeeManagers(ctx, 3, store, true); + + // Delete the lock of the primary manager which should cause it to halt. Then wait to see two + // assistant managers. + var primaryManager = ctx.getServerPaths().getManager(true); + ServiceLock.deleteLock(ctx.getZooSession().asReaderWriter(), primaryManager); + log.debug("Deleted lock of primary manager"); + waitToSeeManagers(ctx, 2, store, true); + + stop.set(true); + // Wait for the background operations to complete and ensure that none had errors. Managers + // stoppping/starting should not cause any problems for Accumulo API operations. + for (var tof : tableOpFutures) { + int loops = tof.get(); + log.debug("Background thread loops {}", loops); + // Check that each background thread made a least one loop over all its table operations. + assertTrue(loops > 0); + } + } + + executor.shutdown(); + + managerWorkers.forEach(Process::destroy); + } + + private static void waitToSeeManagers(ClientContext context, int expectedManagers, + UserFateStore store, boolean managersKilled) { + + // Track what reservations exist when entering, want to see new reservations created during this + // function call. + var existingReservationUUIDs = + store.getActiveReservations(Set.of(FatePartition.all(FateInstanceType.USER))).values() + .stream().map(FateStore.FateReservation::getReservationUUID).collect(toSet()); + log.debug("existingReservationUUIDs {}", existingReservationUUIDs); + + var assistants = + context.getServerPaths().getAssistantManagers(ServiceLockPaths.AddressSelector.all(), true); + // Wait for there to be the expected number of managers in zookeeper. After manager processes + // are kill these entries in zookeeper may persist for a bit. + while (assistants.size() != expectedManagers) { + UtilWaitThread.sleep(1); + assistants = context.getServerPaths() + .getAssistantManagers(ServiceLockPaths.AddressSelector.all(), true); + } + + var expectedServers = assistants.stream().map(ServiceLockPath::getServer) + .map(HostAndPort::fromString).collect(toSet()); + log.debug("managers seen in zookeeper :{}", expectedServers); + + Set reservationsSeen = new HashSet<>(); + Set extraSeen = new HashSet<>(); + Set expectedPrefixes = + Set.of('0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'a', 'b', 'c', 'd', 'e', 'f'); + // Track fate uuid prefixes seen. This is done because fate is partitioned across managers by + // uuid ranges. If all uuid prefixes are seen then it is an indication that fate ids are being + // processed. After new manager processes are started or stopped the partitions should be + // reassigned. + Set seenPrefixes = new HashSet<>(); + + while (reservationsSeen.size() < expectedManagers || !seenPrefixes.equals(expectedPrefixes)) { + var reservations = + store.getActiveReservations(Set.of(FatePartition.all(FateInstanceType.USER))); + reservations.forEach((fateId, reservation) -> { + var slp = ServiceLockPaths.parse(Optional.empty(), reservation.getLockID().path); + if (slp.getType().equals(Constants.ZMANAGER_ASSISTANT_LOCK)) { + var hostPort = HostAndPort.fromString(slp.getServer()); + if (expectedServers.contains(hostPort)) { + if (!existingReservationUUIDs.contains(reservation.getReservationUUID())) { + reservationsSeen.add(hostPort); + Character prefix = fateId.getTxUUIDStr().charAt(0); + if (seenPrefixes.add(prefix)) { + log.debug("Saw fate uuid prefix {} in id {} still waiting for {}", prefix, fateId, + Sets.difference(expectedPrefixes, seenPrefixes)); + } + } + } else if (!managersKilled) { + fail("Saw unexpected extra manager " + slp); + } else { + extraSeen.add(hostPort); + } + } + }); + UtilWaitThread.sleep(1); + } + + log.debug("managers seen in fate reservations :{}", reservationsSeen); + if (managersKilled) { + log.debug("killed managers seen in fate reservations : {}", extraSeen); + } + assertEquals(expectedManagers, reservationsSeen.size()); + } +} diff --git a/test/src/main/java/org/apache/accumulo/test/fate/FateExecutionOrderITBase.java b/test/src/main/java/org/apache/accumulo/test/fate/FateExecutionOrderITBase.java index ee3427fc2c2..4bbf06e0a1a 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/FateExecutionOrderITBase.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/FateExecutionOrderITBase.java @@ -54,6 +54,7 @@ import org.apache.accumulo.core.fate.Fate; import org.apache.accumulo.core.fate.Fate.TxInfo; import org.apache.accumulo.core.fate.FateId; +import org.apache.accumulo.core.fate.FatePartition; import org.apache.accumulo.core.fate.FateStore; import org.apache.accumulo.core.fate.Repo; import org.apache.accumulo.harness.SharedMiniClusterBase; @@ -194,9 +195,11 @@ private void waitFor(FateStore store, FateId txid) throws Exception } protected Fate initializeFate(AccumuloClient client, FateStore store) { - return new Fate<>(new FeoTestEnv(client), store, false, r -> r + "", + var fate = new Fate<>(new FeoTestEnv(client), store, false, r -> r + "", FateTestUtil.updateFateConfig(new ConfigurationCopy(), 1, "AllFateOps"), new ScheduledThreadPoolExecutor(2)); + fate.setPartitions(Set.of(FatePartition.all(store.type()))); + return fate; } private static Entry toIdStep(Entry e) { diff --git a/test/src/main/java/org/apache/accumulo/test/fate/FateITBase.java b/test/src/main/java/org/apache/accumulo/test/fate/FateITBase.java index 278f65d33a2..1411e258ccb 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/FateITBase.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/FateITBase.java @@ -24,6 +24,7 @@ import static org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus.IN_PROGRESS; import static org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus.NEW; import static org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus.SUBMITTED; +import static org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus.SUCCESSFUL; import static org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus.UNKNOWN; import static org.apache.accumulo.test.fate.FateTestUtil.TEST_FATE_OP; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; @@ -47,10 +48,11 @@ import java.util.concurrent.atomic.AtomicReference; import org.apache.accumulo.core.conf.ConfigurationCopy; -import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.fate.AbstractFateStore; import org.apache.accumulo.core.fate.Fate; import org.apache.accumulo.core.fate.FateId; +import org.apache.accumulo.core.fate.FateInstanceType; +import org.apache.accumulo.core.fate.FatePartition; import org.apache.accumulo.core.fate.FateStore; import org.apache.accumulo.core.fate.ReadOnlyFateStore; import org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus; @@ -559,9 +561,11 @@ private void submitDeferred(Fate fate, ServerContext sctx, Set } protected Fate initializeFate(FateStore store) { - return new Fate<>(new TestEnv(), store, false, r -> r + "", + var fate = new Fate<>(new TestEnv(), store, false, r -> r + "", FateTestUtil.updateFateConfig(new ConfigurationCopy(), 1, "AllFateOps"), new ScheduledThreadPoolExecutor(2)); + fate.setPartitions(Set.of(FatePartition.all(store.type()))); + return fate; } protected abstract TStatus getTxStatus(ServerContext sctx, FateId fateId); @@ -574,9 +578,6 @@ public void testShutdownDoesNotFailTx() throws Exception { protected void testShutdownDoesNotFailTx(FateStore store, ServerContext sctx) throws Exception { - ConfigurationCopy config = new ConfigurationCopy(); - config.set(Property.GENERAL_THREADPOOL_SIZE, "2"); - Fate fate = initializeFate(store); // Wait for the transaction runner to be scheduled. @@ -633,6 +634,128 @@ protected void testShutdownDoesNotFailTx(FateStore store, ServerContext assertNull(interruptedException.get()); } + public static class DoNothingRepo implements Repo { + private static final long serialVersionUID = 1L; + + @Override + public Repo call(FateId fateId, TestEnv environment) throws Exception { + return null; + } + + @Override + public void undo(FateId fateId, TestEnv environment) throws Exception { + + } + + @Override + public String getReturn() { + return ""; + } + + @Override + public long isReady(FateId fateId, TestEnv environment) throws Exception { + return 0; + } + + @Override + public String getName() { + return "none"; + } + } + + @Test + @Timeout(60) + public void testPartitions() throws Exception { + executeTest(this::testPartitions); + } + + protected void testPartitions(FateStore store, ServerContext sctx) { + // This test ensures that fate only processes fateids that fall within its assigned partitions + // of fateids. + Fate fate = initializeFate(store); + fate.setPartitions(Set.of()); + + Set fateIds = new HashSet<>(); + + for (int i = 0; i < 100; i++) { + var txid = fate.startTransaction(); + fateIds.add(txid); + + fate.seedTransaction(TEST_FATE_OP, txid, new DoNothingRepo(), false, "no goal"); + } + + for (var fateId : fateIds) { + assertEquals(SUBMITTED, getTxStatus(sctx, fateId)); + } + + // start processing all uuids that start with 1 or 5, but no other ids + fate.setPartitions(Set.of(newPartition(store.type(), "1"), newPartition(store.type(), "5"))); + + Wait.waitFor(() -> fateIds.stream().filter( + fateId -> fateId.getTxUUIDStr().startsWith("1") || fateId.getTxUUIDStr().startsWith("5")) + .map(fateId -> getTxStatus(sctx, fateId)).allMatch(status -> status == SUCCESSFUL)); + + for (var fateId : fateIds) { + var uuid = fateId.getTxUUIDStr(); + if (uuid.startsWith("1") || uuid.startsWith("5")) { + assertEquals(SUCCESSFUL, getTxStatus(sctx, fateId)); + } else { + assertEquals(SUBMITTED, getTxStatus(sctx, fateId)); + } + } + + // start processing uuids that start with e + fate.setPartitions(Set.of(newPartition(store.type(), "e"))); + Wait.waitFor(() -> fateIds.stream().filter(fateId -> fateId.getTxUUIDStr().startsWith("e")) + .map(fateId -> getTxStatus(sctx, fateId)).allMatch(status -> status == SUCCESSFUL)); + + for (var fateId : fateIds) { + var uuid = fateId.getTxUUIDStr(); + if (uuid.startsWith("1") || uuid.startsWith("5") || uuid.startsWith("e")) { + assertEquals(SUCCESSFUL, getTxStatus(sctx, fateId)); + } else { + assertEquals(SUBMITTED, getTxStatus(sctx, fateId)); + } + } + + // add new ids to ensure that uuid prefixes 1 and 5 are no longer processed + Set fateIds2 = new HashSet<>(); + + for (int i = 0; i < 100; i++) { + var txid = fate.startTransaction(); + fateIds2.add(txid); + fate.seedTransaction(TEST_FATE_OP, txid, new DoNothingRepo(), false, "no goal"); + } + Wait.waitFor(() -> fateIds2.stream().filter(fateId -> fateId.getTxUUIDStr().startsWith("e")) + .map(fateId -> getTxStatus(sctx, fateId)).allMatch(status -> status == SUCCESSFUL)); + for (var fateId : fateIds2) { + var uuid = fateId.getTxUUIDStr(); + if (uuid.startsWith("e")) { + assertEquals(SUCCESSFUL, getTxStatus(sctx, fateId)); + } else { + assertEquals(SUBMITTED, getTxStatus(sctx, fateId)); + } + } + + // nothing should have changed with the first set of ids + for (var fateId : fateIds) { + var uuid = fateId.getTxUUIDStr(); + if (uuid.startsWith("1") || uuid.startsWith("5") || uuid.startsWith("e")) { + assertEquals(SUCCESSFUL, getTxStatus(sctx, fateId)); + } else { + assertEquals(SUBMITTED, getTxStatus(sctx, fateId)); + } + } + } + + private FatePartition newPartition(FateInstanceType type, String firstNibble) { + // these suffixes have all uuid chars except for the first nibble/4-bits + String zeroSuffix = "0000000-0000-0000-0000-000000000000"; + String ffSuffix = "fffffff-ffff-ffff-ffff-ffffffffffff"; + return new FatePartition(FateId.from(type, firstNibble + zeroSuffix), + FateId.from(type, firstNibble + ffSuffix)); + } + private static void inCall() throws InterruptedException { // signal that call started callStarted.countDown(); diff --git a/test/src/main/java/org/apache/accumulo/test/fate/FateOpsCommandsITBase.java b/test/src/main/java/org/apache/accumulo/test/fate/FateOpsCommandsITBase.java index e0ff93e4a4f..280efe6312b 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/FateOpsCommandsITBase.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/FateOpsCommandsITBase.java @@ -59,6 +59,7 @@ import org.apache.accumulo.core.fate.Fate; import org.apache.accumulo.core.fate.FateId; import org.apache.accumulo.core.fate.FateInstanceType; +import org.apache.accumulo.core.fate.FatePartition; import org.apache.accumulo.core.fate.FateStore; import org.apache.accumulo.core.fate.ReadOnlyFateStore; import org.apache.accumulo.core.fate.user.UserFateStore; @@ -935,7 +936,10 @@ protected FastFate initFateWithDeadResCleaner(FateStore(env, store, true, Object::toString, DefaultConfiguration.getInstance()); + var fate = + new FastFate<>(env, store, true, Object::toString, DefaultConfiguration.getInstance()); + fate.setPartitions(Set.of(FatePartition.all(store.type()))); + return fate; } protected Fate initFateNoDeadResCleaner(FateStore store) { diff --git a/test/src/main/java/org/apache/accumulo/test/fate/FatePoolsWatcherITBase.java b/test/src/main/java/org/apache/accumulo/test/fate/FatePoolsWatcherITBase.java index ae24acbfa02..c09a1ee737f 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/FatePoolsWatcherITBase.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/FatePoolsWatcherITBase.java @@ -32,6 +32,7 @@ import org.apache.accumulo.core.fate.Fate; import org.apache.accumulo.core.fate.FateId; import org.apache.accumulo.core.fate.FateInstanceType; +import org.apache.accumulo.core.fate.FatePartition; import org.apache.accumulo.core.fate.FateStore; import org.apache.accumulo.core.fate.Repo; import org.apache.accumulo.harness.SharedMiniClusterBase; @@ -92,6 +93,7 @@ protected void testIncrease1(FateStore store, ServerContext s final ConfigurationCopy config = initConfigIncTest1(); final var env = new PoolResizeTestEnv(); final Fate fate = new FastFate<>(env, store, false, r -> r + "", config); + fate.setPartitions(Set.of(FatePartition.all(store.type()))); boolean isUserStore = store.type() == FateInstanceType.USER; final Set set1 = isUserStore ? USER_FATE_OPS_SET1 : META_FATE_OPS_SET1; final Set set2 = isUserStore ? USER_FATE_OPS_SET2 : META_FATE_OPS_SET2; @@ -224,6 +226,7 @@ protected void testIncrease2(FateStore store, ServerContext s FateTestUtil.updateFateConfig(new ConfigurationCopy(), 2, fateExecName); final var env = new PoolResizeTestEnv(); final Fate fate = new FastFate<>(env, store, false, r -> r + "", config); + fate.setPartitions(Set.of(FatePartition.all(store.type()))); final int numWorkers = 2; final int newNumWorkers = 3; final Set allFateOps = @@ -305,6 +308,7 @@ protected void testDecrease(FateStore store, ServerContext sc final ConfigurationCopy config = initConfigDecTest(); final var env = new PoolResizeTestEnv(); final Fate fate = new FastFate<>(env, store, false, r -> r + "", config); + fate.setPartitions(Set.of(FatePartition.all(store.type()))); boolean isUserStore = store.type() == FateInstanceType.USER; final Set set1 = isUserStore ? USER_FATE_OPS_SET1 : META_FATE_OPS_SET1; final Set set2 = isUserStore ? USER_FATE_OPS_SET2 : META_FATE_OPS_SET2; @@ -435,6 +439,7 @@ protected void testIdleCountHistory(FateStore store, ServerCo final var env = new PoolResizeTestEnv(); final Fate fate = new FastFate<>(env, store, false, r -> r + "", config); + fate.setPartitions(Set.of(FatePartition.all(store.type()))); try { // We have two worker threads. Submit 3 transactions that won't complete yet so we can check // for a warning @@ -548,6 +553,7 @@ protected void testFatePoolsPartitioning(FateStore store, Ser final var env = new PoolResizeTestEnv(); final Fate fate = new FastFate<>(env, store, false, r -> r + "", config); + fate.setPartitions(Set.of(FatePartition.all(store.type()))); try { // seeding pool1/FateExecutor1 @@ -650,6 +656,7 @@ protected void testFateExecutorRename(FateStore store, Server final var config = FateTestUtil.updateFateConfig(new ConfigurationCopy(), poolSize, "AllFateOps"); final Fate fate = new FastFate<>(env, store, false, r -> r + "", config); + fate.setPartitions(Set.of(FatePartition.all(store.type()))); try { // start a single transaction diff --git a/test/src/main/java/org/apache/accumulo/test/fate/FateStoreITBase.java b/test/src/main/java/org/apache/accumulo/test/fate/FateStoreITBase.java index 9fd3ba769d9..36e0c0f2474 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/FateStoreITBase.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/FateStoreITBase.java @@ -55,6 +55,7 @@ import org.apache.accumulo.core.fate.FateInstanceType; import org.apache.accumulo.core.fate.FateKey; import org.apache.accumulo.core.fate.FateKey.FateKeyType; +import org.apache.accumulo.core.fate.FatePartition; import org.apache.accumulo.core.fate.FateStore; import org.apache.accumulo.core.fate.FateStore.FateTxStore; import org.apache.accumulo.core.fate.ReadOnlyFateStore.FateIdStatus; @@ -199,8 +200,8 @@ protected void testDeferredOverflow(FateStore store, ServerContext sctx try { // Run and verify all 10 transactions still exist and were not // run because of the deferral time of all the transactions - future = executor.submit(() -> store.runnable(keepRunning::get, - fateIdStatus -> transactions.remove(fateIdStatus.getFateId()))); + future = executor.submit(() -> store.runnable(Set.of(FatePartition.all(store.type())), + keepRunning::get, fateIdStatus -> transactions.remove(fateIdStatus.getFateId()))); Thread.sleep(2000); assertEquals(10, transactions.size()); // Setting this flag to false should terminate the task if sleeping @@ -225,8 +226,8 @@ protected void testDeferredOverflow(FateStore store, ServerContext sctx // Run and verify all 11 transactions were processed // and removed from the store keepRunning.set(true); - future = executor.submit(() -> store.runnable(keepRunning::get, - fateIdStatus -> transactions.remove(fateIdStatus.getFateId()))); + future = executor.submit(() -> store.runnable(Set.of(FatePartition.all(store.type())), + keepRunning::get, fateIdStatus -> transactions.remove(fateIdStatus.getFateId()))); Wait.waitFor(transactions::isEmpty); // Setting this flag to false should terminate the task if sleeping keepRunning.set(false); @@ -769,5 +770,4 @@ public TestOperation2() { super("testOperation2"); } } - } diff --git a/test/src/main/java/org/apache/accumulo/test/fate/FlakyFateManager.java b/test/src/main/java/org/apache/accumulo/test/fate/FlakyFateManager.java index be9a21c7356..8a0470d1dc4 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/FlakyFateManager.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/FlakyFateManager.java @@ -35,10 +35,11 @@ protected FlakyFateManager(ServerOpts opts, String[] args) throws IOException { } @Override - protected Fate initializeFateInstance(ServerContext context, FateStore store) { + protected Fate createFateInstance(FateEnv env, FateStore store, + ServerContext context) { LoggerFactory.getLogger(FlakyFateManager.class).info("Creating Flaky Fate for {}", store.type()); - return new FlakyFate<>(this, store, TraceRepo::toLogString, getConfiguration()); + return new FlakyFate<>(env, store, TraceRepo::toLogString, getConfiguration()); } public static void main(String[] args) throws Exception { diff --git a/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresITBase.java b/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresITBase.java index d2c79855f4c..5a5a7734508 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresITBase.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresITBase.java @@ -43,6 +43,7 @@ import org.apache.accumulo.core.conf.DefaultConfiguration; import org.apache.accumulo.core.fate.Fate; import org.apache.accumulo.core.fate.FateId; +import org.apache.accumulo.core.fate.FatePartition; import org.apache.accumulo.core.fate.FateStore; import org.apache.accumulo.core.fate.ReadOnlyFateStore; import org.apache.accumulo.core.fate.Repo; @@ -103,9 +104,9 @@ private void testReserveUnreserve(TestStoreFactory testStoreFac assertTrue(store1.tryReserve(fakeFateId).isEmpty()); assertTrue(store2.tryReserve(fakeFateId).isEmpty()); // Both stores should return the same reserved transactions - activeReservations = store1.getActiveReservations(); + activeReservations = store1.getActiveReservations(Set.of(FatePartition.all(store1.type()))); assertEquals(allIds, activeReservations.keySet()); - activeReservations = store2.getActiveReservations(); + activeReservations = store2.getActiveReservations(Set.of(FatePartition.all(store2.type()))); assertEquals(allIds, activeReservations.keySet()); // Test setting/getting the TStatus and unreserving the transactions @@ -120,8 +121,8 @@ private void testReserveUnreserve(TestStoreFactory testStoreFac assertThrows(IllegalStateException.class, () -> reservation.setStatus(ReadOnlyFateStore.TStatus.NEW)); } - assertTrue(store1.getActiveReservations().isEmpty()); - assertTrue(store2.getActiveReservations().isEmpty()); + assertTrue(store1.getActiveReservations(Set.of(FatePartition.all(store1.type()))).isEmpty()); + assertTrue(store2.getActiveReservations(Set.of(FatePartition.all(store2.type()))).isEmpty()); } } @@ -254,8 +255,10 @@ private void testMultipleFateInstances(TestStoreFactory testSto Fate fate1 = new Fate<>(testEnv1, store1, true, Object::toString, DefaultConfiguration.getInstance(), new ScheduledThreadPoolExecutor(2)); + fate1.setPartitions(Set.of(FatePartition.all(store1.type()))); Fate fate2 = new Fate<>(testEnv2, store2, false, Object::toString, DefaultConfiguration.getInstance(), new ScheduledThreadPoolExecutor(2)); + fate2.setPartitions(Set.of(FatePartition.all(store2.type()))); try { for (int i = 0; i < numFateIds; i++) { @@ -320,8 +323,10 @@ private void testDeadReservationsCleanup(TestStoreFactory testStor try { fate1 = new FastFate<>(testEnv1, store1, true, Object::toString, config); + fate1.setPartitions(Set.of(FatePartition.all(store1.type()))); // Ensure nothing is reserved yet - assertTrue(store1.getActiveReservations().isEmpty()); + assertTrue( + store1.getActiveReservations(Set.of(FatePartition.all(store1.type()))).isEmpty()); // Create transactions for (int i = 0; i < numFateIds; i++) { @@ -337,7 +342,7 @@ private void testDeadReservationsCleanup(TestStoreFactory testStor // Each fate worker will be hung up working (IN_PROGRESS) on a single transaction // Verify store1 has the transactions reserved and that they were reserved with lock1 - reservations = store1.getActiveReservations(); + reservations = store1.getActiveReservations(Set.of(FatePartition.all(store1.type()))); assertEquals(allIds, reservations.keySet()); reservations.values().forEach(res -> assertEquals(lock1, res.getLockID())); @@ -345,7 +350,7 @@ private void testDeadReservationsCleanup(TestStoreFactory testStor // Verify store2 can see the reserved transactions even though they were reserved using // store1 - reservations = store2.getActiveReservations(); + reservations = store2.getActiveReservations(Set.of(FatePartition.all(store2.type()))); assertEquals(allIds, reservations.keySet()); reservations.values().forEach(res -> assertEquals(lock1, res.getLockID())); @@ -361,6 +366,7 @@ private void testDeadReservationsCleanup(TestStoreFactory testStor // fate1. fate2 = new Fate<>(testEnv2, store2, false, Object::toString, config, new ScheduledThreadPoolExecutor(2)); + fate2.setPartitions(Set.of(FatePartition.all(store2.type()))); // Wait for the "dead" reservations to be deleted and picked up again (reserved using // fate2/store2/lock2 now). @@ -370,7 +376,7 @@ private void testDeadReservationsCleanup(TestStoreFactory testStor // the workers for fate1 are hung up Wait.waitFor(() -> { Map store2Reservations = - store2.getActiveReservations(); + store2.getActiveReservations(Set.of(FatePartition.all(store2.type()))); boolean allReservedWithLock2 = store2Reservations.values().stream() .allMatch(entry -> entry.getLockID().equals(lock2)); return store2Reservations.keySet().equals(allIds) && allReservedWithLock2; diff --git a/test/src/main/java/org/apache/accumulo/test/fate/SlowFateSplitManager.java b/test/src/main/java/org/apache/accumulo/test/fate/SlowFateSplitManager.java index b33a47044a9..3261b037e1c 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/SlowFateSplitManager.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/SlowFateSplitManager.java @@ -45,9 +45,10 @@ protected SlowFateSplitManager(ServerOpts opts, String[] args) throws IOExceptio } @Override - protected Fate initializeFateInstance(ServerContext context, FateStore store) { + protected Fate createFateInstance(FateEnv env, FateStore store, + ServerContext context) { log.info("Creating Slow Split Fate for {}", store.type()); - return new SlowFateSplit<>(this, store, TraceRepo::toLogString, getConfiguration()); + return new SlowFateSplit<>(env, store, TraceRepo::toLogString, getConfiguration()); } public static void main(String[] args) throws Exception { From b92fe8686f060420d4ed6738f03936516e57ec52 Mon Sep 17 00:00:00 2001 From: Keith Turner Date: Wed, 4 Mar 2026 09:29:14 -0800 Subject: [PATCH 02/13] Update core/src/main/java/org/apache/accumulo/core/rpc/clients/FateWorkerThriftClient.java Co-authored-by: Dave Marion --- .../accumulo/core/rpc/clients/FateWorkerThriftClient.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/clients/FateWorkerThriftClient.java b/core/src/main/java/org/apache/accumulo/core/rpc/clients/FateWorkerThriftClient.java index 040a0f36c20..1171efe8765 100644 --- a/core/src/main/java/org/apache/accumulo/core/rpc/clients/FateWorkerThriftClient.java +++ b/core/src/main/java/org/apache/accumulo/core/rpc/clients/FateWorkerThriftClient.java @@ -21,7 +21,7 @@ import org.apache.accumulo.core.manager.thrift.FateWorkerService; /** - * Client side object that can be used to interact with operatoins that are supported by any manager + * Client side object that can be used to interact with fate operations, which are supported by any manager * process. */ public class FateWorkerThriftClient extends ThriftClientTypes { From 00f5c149020bd8966a367d2eaf86d461af03f267 Mon Sep 17 00:00:00 2001 From: Keith Turner Date: Wed, 4 Mar 2026 18:59:26 +0000 Subject: [PATCH 03/13] code review update --- .../org/apache/accumulo/core/lock/ServiceLockData.java | 3 ++- .../org/apache/accumulo/core/lock/ServiceLockPaths.java | 6 +++--- .../java/org/apache/accumulo/server/AbstractServer.java | 2 +- .../main/java/org/apache/accumulo/manager/Manager.java | 8 ++++---- 4 files changed, 10 insertions(+), 9 deletions(-) diff --git a/core/src/main/java/org/apache/accumulo/core/lock/ServiceLockData.java b/core/src/main/java/org/apache/accumulo/core/lock/ServiceLockData.java index c29879f05c7..3e48b20243f 100644 --- a/core/src/main/java/org/apache/accumulo/core/lock/ServiceLockData.java +++ b/core/src/main/java/org/apache/accumulo/core/lock/ServiceLockData.java @@ -45,7 +45,8 @@ public static enum ThriftService { CLIENT, COORDINATOR, COMPACTOR, - FATE, + FATE_CLIENT, + FATE_WORKER, GC, MANAGER, NONE, diff --git a/core/src/main/java/org/apache/accumulo/core/lock/ServiceLockPaths.java b/core/src/main/java/org/apache/accumulo/core/lock/ServiceLockPaths.java index 5f19a46e921..54fded9c79d 100644 --- a/core/src/main/java/org/apache/accumulo/core/lock/ServiceLockPaths.java +++ b/core/src/main/java/org/apache/accumulo/core/lock/ServiceLockPaths.java @@ -242,9 +242,9 @@ public ServiceLockPath createManagerPath() { return new ServiceLockPath(Constants.ZMANAGER_LOCK); } - public ServiceLockPath createManagerWorkerPath(ResourceGroupId resourceGroup, - HostAndPort advertiseAddress) { - return new ServiceLockPath(Constants.ZMANAGER_ASSISTANT_LOCK, resourceGroup, advertiseAddress); + public ServiceLockPath createAssistantManagerPath(HostAndPort advertiseAddress) { + return new ServiceLockPath(Constants.ZMANAGER_ASSISTANT_LOCK, ResourceGroupId.DEFAULT, + advertiseAddress); } public ServiceLockPath createMiniPath(String miniUUID) { 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 72526feb0f9..4adc0afe489 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 @@ -285,7 +285,7 @@ public void runServer() throws Exception { verificationThread.interrupt(); verificationThread.join(); } - log.info(getClass().getSimpleName() + " process shut down. "); + log.info(getClass().getSimpleName() + " process shut down."); Throwable thrown = err.get(); if (thrown != null) { System.err.println("Uncaught execption in AbstractServer.runServer"); 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 c04bae47ef7..b5433a1b007 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 @@ -1227,7 +1227,7 @@ boolean canSuspendTablets() { UUID uuid = sld.getServerUUID(ThriftService.NONE); ServiceDescriptors descriptors = new ServiceDescriptors(); for (ThriftService svc : new ThriftService[] {ThriftService.MANAGER, ThriftService.COORDINATOR, - ThriftService.FATE}) { + ThriftService.FATE_CLIENT}) { descriptors.addService(new ServiceDescriptor(uuid, svc, getAdvertiseAddress().toString(), this.getResourceGroup())); } @@ -1459,8 +1459,8 @@ private void getManagerLock() throws KeeperException, InterruptedException { var advertiseAddress = getAdvertiseAddress(); - final ServiceLockPaths.ServiceLockPath zLockPath = getContext().getServerPaths() - .createManagerWorkerPath(getResourceGroup(), advertiseAddress); + final ServiceLockPaths.ServiceLockPath zLockPath = + getContext().getServerPaths().createAssistantManagerPath(advertiseAddress); ServiceLockSupport.createNonHaServiceLockPath(Type.MANAGER, zoo, zLockPath); var serverLockUUID = UUID.randomUUID(); @@ -1474,7 +1474,7 @@ private void getManagerLock() throws KeeperException, InterruptedException { ServiceLockData.ServiceDescriptors descriptors = new ServiceLockData.ServiceDescriptors(); for (ServiceLockData.ThriftService svc : new ServiceLockData.ThriftService[] { - ThriftService.MANAGER}) { + ThriftService.FATE_WORKER}) { descriptors.addService(new ServiceLockData.ServiceDescriptor(serverLockUUID, svc, advertiseAddress.toString(), this.getResourceGroup())); } From 32db7bd1bfee8672de50052a3af874e3bf61bd6d Mon Sep 17 00:00:00 2001 From: Keith Turner Date: Wed, 4 Mar 2026 19:13:02 +0000 Subject: [PATCH 04/13] format code --- .../accumulo/core/rpc/clients/FateWorkerThriftClient.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/clients/FateWorkerThriftClient.java b/core/src/main/java/org/apache/accumulo/core/rpc/clients/FateWorkerThriftClient.java index 1171efe8765..abd9748def2 100644 --- a/core/src/main/java/org/apache/accumulo/core/rpc/clients/FateWorkerThriftClient.java +++ b/core/src/main/java/org/apache/accumulo/core/rpc/clients/FateWorkerThriftClient.java @@ -21,8 +21,8 @@ import org.apache.accumulo.core.manager.thrift.FateWorkerService; /** - * Client side object that can be used to interact with fate operations, which are supported by any manager - * process. + * Client side object that can be used to interact with fate operations, which are supported by any + * manager process. */ public class FateWorkerThriftClient extends ThriftClientTypes { FateWorkerThriftClient(String serviceName) { From 365b83313520995914107219a3af623d27a1e392 Mon Sep 17 00:00:00 2001 From: Keith Turner Date: Wed, 4 Mar 2026 21:20:09 +0000 Subject: [PATCH 05/13] fix compaction seeding notifications --- .../java/org/apache/accumulo/core/fate/FateClient.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/accumulo/core/fate/FateClient.java b/core/src/main/java/org/apache/accumulo/core/fate/FateClient.java index f6f113e81c1..bf3bd96bce7 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/FateClient.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/FateClient.java @@ -84,8 +84,11 @@ public void close() { public void seedTransaction(Fate.FateOperation fateOp, FateKey fateKey, Repo repo, boolean autoCleanUp) { try (var seeder = store.beginSeeding()) { - @SuppressWarnings("unused") - var unused = seeder.attemptToSeedTransaction(fateOp, fateKey, repo, autoCleanUp); + var cfuture = seeder.attemptToSeedTransaction(fateOp, fateKey, repo, autoCleanUp); + cfuture.thenApply(optional -> { + optional.ifPresent(seedingConsumer.get()); + return optional; + }); } } From 20f290d2f0c41e701984b125a35b666176f24315 Mon Sep 17 00:00:00 2001 From: Keith Turner Date: Thu, 5 Mar 2026 00:08:53 +0000 Subject: [PATCH 06/13] use future return value --- .../java/org/apache/accumulo/core/fate/FateClient.java | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/core/src/main/java/org/apache/accumulo/core/fate/FateClient.java b/core/src/main/java/org/apache/accumulo/core/fate/FateClient.java index bf3bd96bce7..31e576269e8 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/FateClient.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/FateClient.java @@ -83,13 +83,12 @@ public void close() { public void seedTransaction(Fate.FateOperation fateOp, FateKey fateKey, Repo repo, boolean autoCleanUp) { + CompletableFuture> cfuture; try (var seeder = store.beginSeeding()) { - var cfuture = seeder.attemptToSeedTransaction(fateOp, fateKey, repo, autoCleanUp); - cfuture.thenApply(optional -> { - optional.ifPresent(seedingConsumer.get()); - return optional; - }); + cfuture = seeder.attemptToSeedTransaction(fateOp, fateKey, repo, autoCleanUp); } + var optional = cfuture.join(); + optional.ifPresent(seedingConsumer.get()); } // start work in the transaction.. it is safe to call this From 32f481382a6cc691c44b01afcfa3debde1d95a3d Mon Sep 17 00:00:00 2001 From: Keith Turner Date: Thu, 5 Mar 2026 07:48:36 -0800 Subject: [PATCH 07/13] Update core/src/main/java/org/apache/accumulo/core/fate/FateClient.java Co-authored-by: Dave Marion --- core/src/main/java/org/apache/accumulo/core/fate/FateClient.java | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/main/java/org/apache/accumulo/core/fate/FateClient.java b/core/src/main/java/org/apache/accumulo/core/fate/FateClient.java index 31e576269e8..1bc42639864 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/FateClient.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/FateClient.java @@ -53,7 +53,6 @@ public class FateClient { public FateClient(FateStore store, Function,String> toLogStrFunc) { this.store = FateLogger.wrap(store, toLogStrFunc, false); - ; } // get a transaction id back to the requester before doing any work From 78cc891c17270c7494eb80a613f19ba8d52b4715 Mon Sep 17 00:00:00 2001 From: Keith Turner Date: Thu, 5 Mar 2026 07:50:32 -0800 Subject: [PATCH 08/13] Update server/manager/src/main/java/org/apache/accumulo/manager/fate/FateWorkerEnv.java Co-authored-by: Dave Marion --- .../java/org/apache/accumulo/manager/fate/FateWorkerEnv.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/fate/FateWorkerEnv.java b/server/manager/src/main/java/org/apache/accumulo/manager/fate/FateWorkerEnv.java index 2b826540014..57c0e9d2b2a 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/fate/FateWorkerEnv.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/fate/FateWorkerEnv.java @@ -97,7 +97,7 @@ public void run() { client.processEvents(TraceUtil.traceInfo(), ctx.rpcCreds(), tEvents); } } catch (TException e) { - log.warn("Failed to send events to manager", e); + log.warn("Failed to send events to primary manager", e); } finally { if (client != null) { ThriftUtil.close(client, ctx); From 0556afaa8e0c3190f22cc7c5e4aeec02f4689ee9 Mon Sep 17 00:00:00 2001 From: Keith Turner Date: Thu, 5 Mar 2026 16:27:20 +0000 Subject: [PATCH 09/13] code review update --- .../accumulo/core/lock/ServiceLockPaths.java | 5 ---- .../org/apache/accumulo/manager/Manager.java | 4 +-- .../accumulo/manager/fate/FateManager.java | 14 +++++----- .../test/ComprehensiveMultiManagerIT.java | 26 +++++++++++++++++++ 4 files changed, 34 insertions(+), 15 deletions(-) diff --git a/core/src/main/java/org/apache/accumulo/core/lock/ServiceLockPaths.java b/core/src/main/java/org/apache/accumulo/core/lock/ServiceLockPaths.java index 54fded9c79d..363f3cc4022 100644 --- a/core/src/main/java/org/apache/accumulo/core/lock/ServiceLockPaths.java +++ b/core/src/main/java/org/apache/accumulo/core/lock/ServiceLockPaths.java @@ -296,11 +296,6 @@ public Set getCompactor(ResourceGroupPredicate resourceGroupPre return get(Constants.ZCOMPACTORS, resourceGroupPredicate, address, withLock); } - public Set getManagerAssistants(ResourceGroupPredicate resourceGroupPredicate, - AddressSelector address, boolean withLock) { - return get(Constants.ZMANAGER_ASSISTANT_LOCK, resourceGroupPredicate, address, withLock); - } - /** * Note that the ServiceLockPath object returned by this method does not populate the server * attribute. To get the location of the GarbageCollector you will need to parse the lock data at 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 b5433a1b007..bc211d8aa53 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 @@ -982,7 +982,7 @@ public void run() { try { // Acquire the lock that all managers get before the primary lock, this allows non primary // manager processes to work on stuff. - getManagerLock(); + getAssistantManagerLock(); } catch (KeeperException | InterruptedException e) { throw new IllegalStateException("Unable to get manager lock ", e); } @@ -1451,7 +1451,7 @@ private long remaining(long deadline) { return Math.max(1, deadline - System.currentTimeMillis()); } - private void getManagerLock() throws KeeperException, InterruptedException { + private void getAssistantManagerLock() throws KeeperException, InterruptedException { log.info("trying to get assistant manager lock"); final ZooReaderWriter zoo = getContext().getZooSession().asReaderWriter(); diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/fate/FateManager.java b/server/manager/src/main/java/org/apache/accumulo/manager/fate/FateManager.java index fb552f66181..6b294e0b8ad 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/fate/FateManager.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/fate/FateManager.java @@ -18,8 +18,6 @@ */ package org.apache.accumulo.manager.fate; -import static org.apache.accumulo.core.lock.ServiceLockPaths.ResourceGroupPredicate.DEFAULT_RG_ONLY; - import java.time.Duration; import java.util.HashMap; import java.util.HashSet; @@ -98,7 +96,7 @@ private void manageAssistants() throws TException, InterruptedException { // This map will contain all current workers even if their partitions are empty Map currentPartitions; try { - currentPartitions = getCurrentAssignments(); + currentPartitions = getCurrentAssignments(context); } catch (TException e) { log.warn("Failed to get current partitions ", e); continue; @@ -214,7 +212,7 @@ public synchronized void stop(Duration timeout) { // stop looking for work. Map currentAssignments = null; try { - currentAssignments = getCurrentAssignments(); + currentAssignments = getCurrentAssignments(context); } catch (TException e) { log.warn("Failed to get current assignments", e); currentAssignments = Map.of(); @@ -411,15 +409,15 @@ private Set getDesiredPartitions(int numWorkers) { // The updateId accomplishes two things. First it ensures that setting partition RPC can only // execute once on the server side. Second when a new update id is requested it cancels any // outstanding RPCs to set partitions that have not executed yet. - record CurrentPartitions(long updateId, Set partitions) { + public record CurrentPartitions(long updateId, Set partitions) { } /** * @return the fate partitions that assistant managers are currently assigned */ - private Map getCurrentAssignments() throws TException { - var workers = - context.getServerPaths().getManagerAssistants(DEFAULT_RG_ONLY, AddressSelector.all(), true); + public static Map getCurrentAssignments(ServerContext context) + throws TException { + var workers = context.getServerPaths().getAssistantManagers(AddressSelector.all(), true); log.trace("getting current assignments from {}", workers); diff --git a/test/src/main/java/org/apache/accumulo/test/ComprehensiveMultiManagerIT.java b/test/src/main/java/org/apache/accumulo/test/ComprehensiveMultiManagerIT.java index 278a051ec8a..1c5f8d10a49 100644 --- a/test/src/main/java/org/apache/accumulo/test/ComprehensiveMultiManagerIT.java +++ b/test/src/main/java/org/apache/accumulo/test/ComprehensiveMultiManagerIT.java @@ -18,18 +18,28 @@ */ package org.apache.accumulo.test; +import java.util.Map; + import org.apache.accumulo.core.client.Accumulo; import org.apache.accumulo.core.client.AccumuloClient; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.harness.MiniClusterConfigurationCallback; import org.apache.accumulo.harness.SharedMiniClusterBase; import org.apache.accumulo.manager.Manager; +import org.apache.accumulo.manager.fate.FateManager; import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl; +import org.apache.accumulo.test.util.Wait; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.net.HostAndPort; public class ComprehensiveMultiManagerIT extends ComprehensiveITBase { + private static final Logger log = LoggerFactory.getLogger(ComprehensiveMultiManagerIT.class); + private static class ComprehensiveITConfiguration implements MiniClusterConfigurationCallback { @Override public void configureMiniCluster(MiniAccumuloConfigImpl cfg, @@ -49,6 +59,22 @@ public static void setup() throws Exception { // Start two more managers getCluster().exec(Manager.class); getCluster().exec(Manager.class); + + // Wait for 3 managers to have a fate partition assigned to them + var srvCtx = getCluster().getServerContext(); + Wait.waitFor(() -> { + Map fateAssignments = + FateManager.getCurrentAssignments(srvCtx); + boolean allAssigned = fateAssignments.size() == 3 && fateAssignments.values().stream() + .noneMatch(currentPartitions -> currentPartitions.partitions().isEmpty()); + if (allAssigned) { + fateAssignments.forEach((hostPort, partitions) -> { + log.debug("Fate assignment {} {}", hostPort, partitions.partitions()); + }); + } + return allAssigned; + }); + } @AfterAll From 80986e232c2737ce1268cd0bed543f1059973c29 Mon Sep 17 00:00:00 2001 From: Keith Turner Date: Thu, 5 Mar 2026 16:46:25 +0000 Subject: [PATCH 10/13] renamed HighlyAvailableService to PrimaryManagerThriftService --- ....java => PrimaryManagerThriftService.java} | 8 ++++---- ...rimaryManagerThriftInvocationHandler.java} | 18 +++++++++--------- ...> PrimaryManagerThriftServiceWrapper.java} | 14 ++++++-------- .../org/apache/accumulo/manager/Manager.java | 19 ++++++++++--------- 4 files changed, 29 insertions(+), 30 deletions(-) rename server/base/src/main/java/org/apache/accumulo/server/{HighlyAvailableService.java => PrimaryManagerThriftService.java} (84%) rename server/base/src/main/java/org/apache/accumulo/server/rpc/{HighlyAvailableServiceInvocationHandler.java => PrimaryManagerThriftInvocationHandler.java} (84%) rename server/base/src/main/java/org/apache/accumulo/server/rpc/{HighlyAvailableServiceWrapper.java => PrimaryManagerThriftServiceWrapper.java} (90%) diff --git a/server/base/src/main/java/org/apache/accumulo/server/HighlyAvailableService.java b/server/base/src/main/java/org/apache/accumulo/server/PrimaryManagerThriftService.java similarity index 84% rename from server/base/src/main/java/org/apache/accumulo/server/HighlyAvailableService.java rename to server/base/src/main/java/org/apache/accumulo/server/PrimaryManagerThriftService.java index 4d529369a0a..f8e845a89b3 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/HighlyAvailableService.java +++ b/server/base/src/main/java/org/apache/accumulo/server/PrimaryManagerThriftService.java @@ -19,17 +19,17 @@ package org.apache.accumulo.server; /** - * This interface allows service implementations which support running multiple instances - * concurrently with only one active instance to report whether or not they are the active service. + * This interface allows manager processes which support running multiple instances concurrently + * with only one active instance to report whether it is the primary manager. */ -public interface HighlyAvailableService { +public interface PrimaryManagerThriftService { /** * Is this service instance currently the active instance for the Accumulo cluster. * * @return True if the service is the active service, false otherwise. */ - boolean isActiveService(); + boolean isPrimaryManager(); /** * Is this service instance currently in the process of upgrading. diff --git a/server/base/src/main/java/org/apache/accumulo/server/rpc/HighlyAvailableServiceInvocationHandler.java b/server/base/src/main/java/org/apache/accumulo/server/rpc/PrimaryManagerThriftInvocationHandler.java similarity index 84% rename from server/base/src/main/java/org/apache/accumulo/server/rpc/HighlyAvailableServiceInvocationHandler.java rename to server/base/src/main/java/org/apache/accumulo/server/rpc/PrimaryManagerThriftInvocationHandler.java index e0e527e329e..bbd703d4a4e 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/rpc/HighlyAvailableServiceInvocationHandler.java +++ b/server/base/src/main/java/org/apache/accumulo/server/rpc/PrimaryManagerThriftInvocationHandler.java @@ -26,24 +26,24 @@ import java.util.Set; import org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException; -import org.apache.accumulo.server.HighlyAvailableService; +import org.apache.accumulo.server.PrimaryManagerThriftService; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** - * An {@link InvocationHandler} which checks to see if a {@link HighlyAvailableService} is the - * current active instance of that service, throwing {@link ThriftNotActiveServiceException} when it - * is not the current active instance. + * An {@link InvocationHandler} which checks to see if a {@link PrimaryManagerThriftService} is the + * current active primary manager, throwing {@link ThriftNotActiveServiceException} when it is not + * the current active instance. */ -public class HighlyAvailableServiceInvocationHandler implements InvocationHandler { +public class PrimaryManagerThriftInvocationHandler implements InvocationHandler { private static final Logger LOG = - LoggerFactory.getLogger(HighlyAvailableServiceInvocationHandler.class); + LoggerFactory.getLogger(PrimaryManagerThriftInvocationHandler.class); private final I instance; - private final HighlyAvailableService service; + private final PrimaryManagerThriftService service; private final Set onewayMethods; - public HighlyAvailableServiceInvocationHandler(I instance, HighlyAvailableService service, + public PrimaryManagerThriftInvocationHandler(I instance, PrimaryManagerThriftService service, Set onewayMethods) { this.instance = Objects.requireNonNull(instance); this.service = Objects.requireNonNull(service); @@ -68,7 +68,7 @@ public Object invoke(Object proxy, Method method, Object[] args) throws Throwabl } // If the service is not active, throw an exception - if (!service.isActiveService()) { + if (!service.isPrimaryManager()) { if (onewayMethods.contains(method.getName())) { // if thrift one way method throws an exception it will just log an error LOG.debug("Ignoring one way thrift call because not active : {} {}", method.getName(), diff --git a/server/base/src/main/java/org/apache/accumulo/server/rpc/HighlyAvailableServiceWrapper.java b/server/base/src/main/java/org/apache/accumulo/server/rpc/PrimaryManagerThriftServiceWrapper.java similarity index 90% rename from server/base/src/main/java/org/apache/accumulo/server/rpc/HighlyAvailableServiceWrapper.java rename to server/base/src/main/java/org/apache/accumulo/server/rpc/PrimaryManagerThriftServiceWrapper.java index b6a7d54d675..d23a59bf13b 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/rpc/HighlyAvailableServiceWrapper.java +++ b/server/base/src/main/java/org/apache/accumulo/server/rpc/PrimaryManagerThriftServiceWrapper.java @@ -28,23 +28,21 @@ import java.util.stream.Collectors; import org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException; -import org.apache.accumulo.server.HighlyAvailableService; +import org.apache.accumulo.server.PrimaryManagerThriftService; import org.apache.thrift.ProcessFunction; import org.apache.thrift.TBaseProcessor; /** * A class to wrap invocations to the Thrift handler to prevent these invocations from succeeding - * when the Accumulo service that this Thrift service is for has not yet obtained its ZooKeeper - * lock. + * when the manager that this Thrift service is for has not yet obtained the primary manager lock. * *

* Its expected that all methods in the wrapped thrift service declare they throw * {@link org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException}. The methods * should declare they throw in the thrift IDL. * - * @since 2.0 */ -public class HighlyAvailableServiceWrapper { +public class PrimaryManagerThriftServiceWrapper { /** * Returns all thrift methods on a processor along w/ an indication if they are oneway or not. @@ -100,10 +98,10 @@ private static void validateHAServerExceptions(Class thriftInterface, } // Not for public use. - private HighlyAvailableServiceWrapper() {} + private PrimaryManagerThriftServiceWrapper() {} public static I service(Class iface, Function> processorFactory, - final I handler, HighlyAvailableService service) { + final I handler, PrimaryManagerThriftService service) { var processor = processorFactory.apply(handler); var thriftMethods = getThriftMethods(processor); validateHAServerExceptions(iface, thriftMethods); @@ -112,7 +110,7 @@ public static I service(Class iface, Function> proces .map(Map.Entry::getKey).collect(Collectors.toSet()); InvocationHandler proxyHandler = - new HighlyAvailableServiceInvocationHandler<>(handler, service, onewayMethods); + new PrimaryManagerThriftInvocationHandler<>(handler, service, onewayMethods); @SuppressWarnings("unchecked") I proxiedInstance = (I) Proxy.newProxyInstance(handler.getClass().getClassLoader(), new Class[] {iface}, proxyHandler); 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 bc211d8aa53..07f1022f636 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 @@ -130,7 +130,7 @@ import org.apache.accumulo.manager.upgrade.UpgradeCoordinator; import org.apache.accumulo.manager.upgrade.UpgradeCoordinator.UpgradeStatus; import org.apache.accumulo.server.AbstractServer; -import org.apache.accumulo.server.HighlyAvailableService; +import org.apache.accumulo.server.PrimaryManagerThriftService; import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.server.compaction.CompactionConfigStorage; import org.apache.accumulo.server.fs.VolumeManager; @@ -140,7 +140,7 @@ import org.apache.accumulo.server.manager.state.DeadServerList; import org.apache.accumulo.server.manager.state.TabletServerState; import org.apache.accumulo.server.manager.state.TabletStateStore; -import org.apache.accumulo.server.rpc.HighlyAvailableServiceWrapper; +import org.apache.accumulo.server.rpc.PrimaryManagerThriftServiceWrapper; import org.apache.accumulo.server.rpc.TServerUtils; import org.apache.accumulo.server.rpc.ThriftProcessorTypes; import org.apache.accumulo.server.security.delegation.AuthenticationTokenKeyManager; @@ -177,7 +177,7 @@ // TODO create standalone PrimaryFateEnv class and pull everything into there relatated to // FateEnv... this will make it much more clear the env is for metadata ops only public class Manager extends AbstractServer - implements LiveTServerSet.Listener, FateEnv, HighlyAvailableService { + implements LiveTServerSet.Listener, FateEnv, PrimaryManagerThriftService { static final Logger log = LoggerFactory.getLogger(Manager.class); @@ -950,15 +950,16 @@ public void run() { throw new RuntimeException(e); } - FateService.Iface fateServiceHandler = HighlyAvailableServiceWrapper.service( + FateService.Iface fateServiceHandler = PrimaryManagerThriftServiceWrapper.service( FateService.Iface.class, FateService.Processor::new, new FateServiceHandler(this), this); ManagerClientService.Iface managerClientHandler = - HighlyAvailableServiceWrapper.service(ManagerClientService.Iface.class, + PrimaryManagerThriftServiceWrapper.service(ManagerClientService.Iface.class, ManagerClientService.Processor::new, new ManagerClientServiceHandler(this), this); compactionCoordinator = new CompactionCoordinator(this, this::fateClient); - CompactionCoordinatorService.Iface wrappedCoordinator = HighlyAvailableServiceWrapper.service( - CompactionCoordinatorService.Iface.class, CompactionCoordinatorService.Processor::new, - compactionCoordinator.getThriftService(), this); + CompactionCoordinatorService.Iface wrappedCoordinator = + PrimaryManagerThriftServiceWrapper.service(CompactionCoordinatorService.Iface.class, + CompactionCoordinatorService.Processor::new, compactionCoordinator.getThriftService(), + this); // This is not wrapped w/ HighlyAvailableServiceWrapper because it can be run by any manager. FateWorker fateWorker = new FateWorker(context, tserverSet, this::createFateInstance); @@ -1724,7 +1725,7 @@ public SteadyTime getSteadyTime() { } @Override - public boolean isActiveService() { + public boolean isPrimaryManager() { return managerInitialized.get(); } From d74a34c4a07f62f6aef2830ed9c693850be24d82 Mon Sep 17 00:00:00 2001 From: Keith Turner Date: Thu, 5 Mar 2026 17:09:29 +0000 Subject: [PATCH 11/13] code review update --- .../org/apache/accumulo/manager/metrics/ManagerMetrics.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/ManagerMetrics.java b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/ManagerMetrics.java index 9e6f9c4f2dd..56bbe766f20 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/ManagerMetrics.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/ManagerMetrics.java @@ -45,7 +45,7 @@ public class ManagerMetrics implements MetricsProducer { - private List> fateMetrics; + private List> fateMetrics = null; private final AtomicLong rootTGWErrorsGauge = new AtomicLong(0); private final AtomicLong metadataTGWErrorsGauge = new AtomicLong(0); @@ -98,6 +98,7 @@ public void clearCompactionServiceConfigurationError() { @Override public void registerMetrics(MeterRegistry registry) { + requireNonNull(fateMetrics, "Must register fate metrics first"); fateMetrics.forEach(fm -> fm.registerMetrics(registry)); Gauge.builder(MANAGER_ROOT_TGW_ERRORS.getName(), rootTGWErrorsGauge, AtomicLong::get) .description(MANAGER_ROOT_TGW_ERRORS.getDescription()).register(registry); From b17391b00b7859818fb044b0aa44259c56a67c79 Mon Sep 17 00:00:00 2001 From: Keith Turner Date: Thu, 5 Mar 2026 22:54:50 +0000 Subject: [PATCH 12/13] removed follow on comments --- .../thrift/CompactionCoordinatorService.java | 728 ++++++++++++++++++ .../main/thrift/compaction-coordinator.thrift | 6 + .../util/adminCommand/ServiceStatus.java | 3 - .../manager/ManagerClientServiceHandler.java | 4 - .../coordinator/CompactionCoordinator.java | 8 + .../accumulo/manager/fate/FateWorkerEnv.java | 22 +- .../accumulo/test/MultipleManagerIT.java | 5 - 7 files changed, 760 insertions(+), 16 deletions(-) diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/compaction/thrift/CompactionCoordinatorService.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/compaction/thrift/CompactionCoordinatorService.java index 3b12a9ccc37..c9d2708c86b 100644 --- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/compaction/thrift/CompactionCoordinatorService.java +++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/compaction/thrift/CompactionCoordinatorService.java @@ -45,6 +45,8 @@ public interface Iface { public void cancel(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException; + public void recordCompletion(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId) throws org.apache.thrift.TException; + } public interface AsyncIface { @@ -65,6 +67,8 @@ public interface AsyncIface { public void cancel(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void recordCompletion(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + } public static class Client extends org.apache.thrift.TServiceClient implements Iface { @@ -339,6 +343,21 @@ public void recv_cancel() throws org.apache.accumulo.core.clientImpl.thrift.Thri return; } + @Override + public void recordCompletion(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId) throws org.apache.thrift.TException + { + send_recordCompletion(tinfo, credentials, externalCompactionId); + } + + public void send_recordCompletion(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId) throws org.apache.thrift.TException + { + recordCompletion_args args = new recordCompletion_args(); + args.setTinfo(tinfo); + args.setCredentials(credentials); + args.setExternalCompactionId(externalCompactionId); + sendBaseOneway("recordCompletion", args); + } + } public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface { public static class Factory implements org.apache.thrift.async.TAsyncClientFactory { @@ -708,6 +727,47 @@ public Void getResult() throws org.apache.accumulo.core.clientImpl.thrift.Thrift } } + @Override + public void recordCompletion(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + recordCompletion_call method_call = new recordCompletion_call(tinfo, credentials, externalCompactionId, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class recordCompletion_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 externalCompactionId; + public recordCompletion_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, 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.tinfo = tinfo; + this.credentials = credentials; + this.externalCompactionId = externalCompactionId; + } + + @Override + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("recordCompletion", org.apache.thrift.protocol.TMessageType.ONEWAY, 0)); + recordCompletion_args args = new recordCompletion_args(); + args.setTinfo(tinfo); + args.setCredentials(credentials); + args.setExternalCompactionId(externalCompactionId); + 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 { @@ -729,6 +789,7 @@ protected Processor(I iface, java.util.Map extends org.apache.thrift.ProcessFunction { + public recordCompletion() { + super("recordCompletion"); + } + + @Override + public recordCompletion_args getEmptyArgsInstance() { + return new recordCompletion_args(); + } + + @Override + protected boolean isOneway() { + return true; + } + + @Override + protected boolean rethrowUnhandledExceptions() { + return false; + } + + @Override + public org.apache.thrift.TBase getResult(I iface, recordCompletion_args args) throws org.apache.thrift.TException { + iface.recordCompletion(args.tinfo, args.credentials, args.externalCompactionId); + return null; + } + } + } public static class AsyncProcessor extends org.apache.thrift.TBaseAsyncProcessor { @@ -1025,6 +1113,7 @@ protected AsyncProcessor(I iface, java.util.Map extends org.apache.thrift.AsyncProcessFunction { + public recordCompletion() { + super("recordCompletion"); + } + + @Override + public recordCompletion_args getEmptyArgsInstance() { + return new recordCompletion_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, recordCompletion_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + iface.recordCompletion(args.tinfo, args.credentials, args.externalCompactionId,resultHandler); + } + } + } @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) @@ -11475,5 +11604,604 @@ private static S scheme(org.apache. } } + @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) + public static class recordCompletion_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("recordCompletion_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 EXTERNAL_COMPACTION_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("externalCompactionId", org.apache.thrift.protocol.TType.STRING, (short)3); + + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new recordCompletion_argsStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new recordCompletion_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 externalCompactionId; // 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"), + EXTERNAL_COMPACTION_ID((short)3, "externalCompactionId"); + + 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: // EXTERNAL_COMPACTION_ID + return EXTERNAL_COMPACTION_ID; + 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.EXTERNAL_COMPACTION_ID, new org.apache.thrift.meta_data.FieldMetaData("externalCompactionId", 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(recordCompletion_args.class, metaDataMap); + } + + public recordCompletion_args() { + } + + public recordCompletion_args( + org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, + org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, + java.lang.String externalCompactionId) + { + this(); + this.tinfo = tinfo; + this.credentials = credentials; + this.externalCompactionId = externalCompactionId; + } + + /** + * Performs a deep copy on other. + */ + public recordCompletion_args(recordCompletion_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.isSetExternalCompactionId()) { + this.externalCompactionId = other.externalCompactionId; + } + } + + @Override + public recordCompletion_args deepCopy() { + return new recordCompletion_args(this); + } + + @Override + public void clear() { + this.tinfo = null; + this.credentials = null; + this.externalCompactionId = null; + } + + @org.apache.thrift.annotation.Nullable + public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() { + return this.tinfo; + } + + public recordCompletion_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 recordCompletion_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 getExternalCompactionId() { + return this.externalCompactionId; + } + + public recordCompletion_args setExternalCompactionId(@org.apache.thrift.annotation.Nullable java.lang.String externalCompactionId) { + this.externalCompactionId = externalCompactionId; + return this; + } + + public void unsetExternalCompactionId() { + this.externalCompactionId = null; + } + + /** Returns true if field externalCompactionId is set (has been assigned a value) and false otherwise */ + public boolean isSetExternalCompactionId() { + return this.externalCompactionId != null; + } + + public void setExternalCompactionIdIsSet(boolean value) { + if (!value) { + this.externalCompactionId = 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 EXTERNAL_COMPACTION_ID: + if (value == null) { + unsetExternalCompactionId(); + } else { + setExternalCompactionId((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 EXTERNAL_COMPACTION_ID: + return getExternalCompactionId(); + + } + 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 EXTERNAL_COMPACTION_ID: + return isSetExternalCompactionId(); + } + throw new java.lang.IllegalStateException(); + } + + @Override + public boolean equals(java.lang.Object that) { + if (that instanceof recordCompletion_args) + return this.equals((recordCompletion_args)that); + return false; + } + + public boolean equals(recordCompletion_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_externalCompactionId = true && this.isSetExternalCompactionId(); + boolean that_present_externalCompactionId = true && that.isSetExternalCompactionId(); + if (this_present_externalCompactionId || that_present_externalCompactionId) { + if (!(this_present_externalCompactionId && that_present_externalCompactionId)) + return false; + if (!this.externalCompactionId.equals(that.externalCompactionId)) + 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 + ((isSetExternalCompactionId()) ? 131071 : 524287); + if (isSetExternalCompactionId()) + hashCode = hashCode * 8191 + externalCompactionId.hashCode(); + + return hashCode; + } + + @Override + public int compareTo(recordCompletion_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(isSetExternalCompactionId(), other.isSetExternalCompactionId()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetExternalCompactionId()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.externalCompactionId, other.externalCompactionId); + 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("recordCompletion_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("externalCompactionId:"); + if (this.externalCompactionId == null) { + sb.append("null"); + } else { + sb.append(this.externalCompactionId); + } + 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 recordCompletion_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + @Override + public recordCompletion_argsStandardScheme getScheme() { + return new recordCompletion_argsStandardScheme(); + } + } + + private static class recordCompletion_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { + + @Override + public void read(org.apache.thrift.protocol.TProtocol iprot, recordCompletion_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: // EXTERNAL_COMPACTION_ID + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.externalCompactionId = iprot.readString(); + struct.setExternalCompactionIdIsSet(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, recordCompletion_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.externalCompactionId != null) { + oprot.writeFieldBegin(EXTERNAL_COMPACTION_ID_FIELD_DESC); + oprot.writeString(struct.externalCompactionId); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class recordCompletion_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + @Override + public recordCompletion_argsTupleScheme getScheme() { + return new recordCompletion_argsTupleScheme(); + } + } + + private static class recordCompletion_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, recordCompletion_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.isSetExternalCompactionId()) { + optionals.set(2); + } + oprot.writeBitSet(optionals, 3); + if (struct.isSetTinfo()) { + struct.tinfo.write(oprot); + } + if (struct.isSetCredentials()) { + struct.credentials.write(oprot); + } + if (struct.isSetExternalCompactionId()) { + oprot.writeString(struct.externalCompactionId); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, recordCompletion_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.externalCompactionId = iprot.readString(); + struct.setExternalCompactionIdIsSet(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/compaction-coordinator.thrift b/core/src/main/thrift/compaction-coordinator.thrift index d5b0d3c4ac4..99d5df10ee9 100644 --- a/core/src/main/thrift/compaction-coordinator.thrift +++ b/core/src/main/thrift/compaction-coordinator.thrift @@ -173,6 +173,12 @@ service CompactionCoordinatorService { 1:client.ThriftSecurityException sec 2:client.ThriftNotActiveServiceException tnase ) + + oneway void recordCompletion( + 1:client.TInfo tinfo + 2:security.TCredentials credentials + 3:string externalCompactionId + ) } service CompactorService { diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/adminCommand/ServiceStatus.java b/server/base/src/main/java/org/apache/accumulo/server/util/adminCommand/ServiceStatus.java index 0164c68a740..4b6b519d7d7 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/adminCommand/ServiceStatus.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/adminCommand/ServiceStatus.java @@ -96,9 +96,6 @@ public void execute(JCommander cl, ServiceStatusCmdOpts options) throws Exceptio final Map services = new TreeMap<>(); - // FOLLOW_ON display information about multiple managers. Could display which is primary. Also, - // could potentially display the additional port that is being listened on by the assistant - // manager. services.put(ServiceStatusReport.ReportKey.MANAGER, getManagerStatus(context)); services.put(ServiceStatusReport.ReportKey.MONITOR, getMonitorStatus(context)); services.put(ServiceStatusReport.ReportKey.T_SERVER, getTServerStatus(context)); 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 457ff8252a6..f96332dc9d6 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 @@ -106,7 +106,6 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Lists; -// FOLLOW_ON move some of these thrift calls to the assistant manager thrift service public class ManagerClientServiceHandler implements ManagerClientService.Iface { private static final Logger log = Manager.log; @@ -406,9 +405,6 @@ public void reportTabletStatus(TInfo info, TCredentials credentials, String serv } } - // FOLLOW_ON need to ensure SAFE_MODE and CLEAN_STOP work w/ multiple managers. Probably need - // tests for - // this. @Override public void setManagerGoalState(TInfo info, TCredentials c, ManagerGoalState state) throws ThriftSecurityException { diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/CompactionCoordinator.java b/server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/CompactionCoordinator.java index 57d339485bb..19397286222 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/CompactionCoordinator.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/CompactionCoordinator.java @@ -1074,6 +1074,14 @@ public void updateCompactionStatus(TInfo tinfo, TCredentials credentials, } } + @Override + public void recordCompletion(TInfo tinfo, TCredentials credentials, String externalCompactionId) + throws TException { + if (security.canPerformSystemActions(credentials)) { + recordCompletion(ExternalCompactionId.of(externalCompactionId)); + } + } + public void recordCompletion(ExternalCompactionId ecid) { var rc = RUNNING_CACHE.remove(ecid); if (rc != null) { diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/fate/FateWorkerEnv.java b/server/manager/src/main/java/org/apache/accumulo/manager/fate/FateWorkerEnv.java index 57c0e9d2b2a..c1842e25efc 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/fate/FateWorkerEnv.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/fate/FateWorkerEnv.java @@ -28,6 +28,7 @@ import org.apache.accumulo.core.client.AccumuloException; import org.apache.accumulo.core.client.AccumuloSecurityException; +import org.apache.accumulo.core.compaction.thrift.CompactionCoordinatorService; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.dataImpl.KeyExtent; @@ -38,6 +39,7 @@ import org.apache.accumulo.core.rpc.ThriftUtil; import org.apache.accumulo.core.rpc.clients.ThriftClientTypes; import org.apache.accumulo.core.trace.TraceUtil; +import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil; import org.apache.accumulo.core.util.threads.ThreadPools; import org.apache.accumulo.core.util.threads.Threads; import org.apache.accumulo.core.util.time.SteadyTime; @@ -176,10 +178,22 @@ public EventPublisher getEventPublisher() { @Override public void recordCompactionCompletion(ExternalCompactionId ecid) { - // FOLLOW_ON This data is stored in memory on the manager. This entire feature needs to be - // examined and potentially reworked. One solution would be to send an RPC to the manager to - // update it's in memory state. A better solution would be to move away from in memory state - // that is lost when the manager restarts. + var coordinatorHost = ExternalCompactionUtil.findCompactionCoordinator(getContext()); + if (coordinatorHost.isPresent()) { + CompactionCoordinatorService.Client client = null; + try { + client = ThriftUtil.getClient(ThriftClientTypes.COORDINATOR, coordinatorHost.orElseThrow(), + getContext()); + client.recordCompletion(TraceUtil.traceInfo(), getContext().rpcCreds(), ecid.canonical()); + log.trace("Sent compaction completion {} {}", coordinatorHost, ecid); + } catch (Exception te) { + log.trace("Failed to send compaction completion {} {}", coordinatorHost, ecid, te); + } finally { + ThriftUtil.returnClient(client, getContext()); + } + } else { + log.trace("No coordinator found, dropping compaction completion for {}", ecid); + } } @Override diff --git a/test/src/main/java/org/apache/accumulo/test/MultipleManagerIT.java b/test/src/main/java/org/apache/accumulo/test/MultipleManagerIT.java index ca073b551fc..8430ab527b1 100644 --- a/test/src/main/java/org/apache/accumulo/test/MultipleManagerIT.java +++ b/test/src/main/java/org/apache/accumulo/test/MultipleManagerIT.java @@ -107,7 +107,6 @@ public static void main(String[] args) throws Exception { @Override protected void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) { - // FOLLOW_ON add a way to start multiple managers to mini cfg.getClusterServerConfiguration().setNumDefaultCompactors(8); // Set this lower so that locks timeout faster cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "5s"); @@ -138,10 +137,6 @@ public void testFate() throws Exception { for (int i = 0; i < 10; i++) { var table = "t" + i; - // FOLLOW_ON its hard to find everything related to a table id in the logs across processes, - // especially when the - // table id is like "b". Was trying to follow a single table across multiple manager workers - // processes. var tableOpsFuture = executor.submit(() -> { int loops = 0; while (!stop.get() || loops == 0) { From 916dfda53f850f582c9b6d0be6f30764103c132a Mon Sep 17 00:00:00 2001 From: Keith Turner Date: Thu, 5 Mar 2026 22:59:22 +0000 Subject: [PATCH 13/13] narrow exception caught --- .../java/org/apache/accumulo/manager/fate/FateWorkerEnv.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/fate/FateWorkerEnv.java b/server/manager/src/main/java/org/apache/accumulo/manager/fate/FateWorkerEnv.java index c1842e25efc..e7909dd0a71 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/fate/FateWorkerEnv.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/fate/FateWorkerEnv.java @@ -186,7 +186,7 @@ public void recordCompactionCompletion(ExternalCompactionId ecid) { getContext()); client.recordCompletion(TraceUtil.traceInfo(), getContext().rpcCreds(), ecid.canonical()); log.trace("Sent compaction completion {} {}", coordinatorHost, ecid); - } catch (Exception te) { + } catch (TException te) { log.trace("Failed to send compaction completion {} {}", coordinatorHost, ecid, te); } finally { ThriftUtil.returnClient(client, getContext());