diff --git a/assemble/bin/accumulo-cluster b/assemble/bin/accumulo-cluster index 69879d20613..80fae52524c 100755 --- a/assemble/bin/accumulo-cluster +++ b/assemble/bin/accumulo-cluster @@ -228,7 +228,7 @@ function start_all() { echo "Starting compactors for group $group" Q="COMPACTOR_HOSTS_${group}" for compactor in ${!Q}; do - start_service "$compactor" compactor "-o" "compactor.group=$group" + start_service "$compactor" compactor "-o" "compactor.group=$group" "-o" "task.runner.worker.type=COMPACTION" done done @@ -295,7 +295,7 @@ function start_here() { Q="COMPACTOR_HOSTS_${group}" for compactor in ${!Q}; do if echo "$compactor" | grep -q "^${host}\$"; then - start_service "$compactor" compactor "-o" "compactor.group=$group" + start_service "$compactor" compactor "-o" "compactor.group=$group" "-o" "task.runner.worker.type=COMPACTION" fi done done @@ -525,6 +525,16 @@ tserver: default: - localhost +# ELASTICITY_TODO: Add these new TaskRunner types to MAC, ClusterConfigParser, etc. +log_sorters: + default: + - localhost + +# ELASTICITY_TODO: Add these new TaskRunner types to MAC, ClusterConfigParser, etc. +split_calculators: + default: + - localhost + compactor: accumulo_meta: - localhost diff --git a/assemble/pom.xml b/assemble/pom.xml index 17bba8eeac3..94128947068 100644 --- a/assemble/pom.xml +++ b/assemble/pom.xml @@ -186,11 +186,6 @@ jakarta.xml.bind-api true - - org.apache.accumulo - accumulo-compactor - true - org.apache.accumulo accumulo-core @@ -242,6 +237,11 @@ accumulo-start true + + org.apache.accumulo + accumulo-task-runner + true + org.apache.accumulo accumulo-tserver diff --git a/core/src/main/java/org/apache/accumulo/core/conf/Property.java b/core/src/main/java/org/apache/accumulo/core/conf/Property.java index 82bfd72a8a3..6cf1fd1dec6 100644 --- a/core/src/main/java/org/apache/accumulo/core/conf/Property.java +++ b/core/src/main/java/org/apache/accumulo/core/conf/Property.java @@ -1104,65 +1104,48 @@ public enum Property { + "also consider configuring the `" + NoDeleteConstraint.class.getName() + "` " + "constraint.", "2.0.0"), - // Compactor properties - @Experimental - COMPACTOR_PREFIX("compactor.", null, PropertyType.PREFIX, - "Properties in this category affect the behavior of the accumulo compactor server.", "2.1.0"), - @Experimental - COMPACTOR_PORTSEARCH("compactor.port.search", "true", PropertyType.BOOLEAN, - "If the compactor.port.client is in use, search higher ports until one is available", - "2.1.0"), - @Experimental - COMPACTOR_CLIENTPORT("compactor.port.client", "9133", PropertyType.PORT, - "The port used for handling client connections on the compactor servers", "2.1.0"), - COMPACTOR_MIN_JOB_WAIT_TIME("compactor.wait.time.job.min", "1s", PropertyType.TIMEDURATION, - "The minimum amount of time to wait between checks for the next compaction job, backing off" - + "exponentially until COMPACTOR_MAX_JOB_WAIT_TIME is reached.", + // ELASTICITY_TODO: Deprecate CompactionCoordinator properties in a 3.x release + TASK_MANAGER_PREFIX("task.manager.", null, PropertyType.PREFIX, + "Properties in this category affect the behavior of the Accumulo TaskManager component.", + "4.0.0"), + TASK_MANAGER_DEAD_COMPACTOR_CHECK_INTERVAL("task.manager.compactor.dead.check.interval", "5m", + PropertyType.TIMEDURATION, "The interval at which to check for dead compactors.", "4.0.0"), + TASK_MANAGER_TSERVER_COMPACTION_CHECK_INTERVAL("task.manager.tserver.check.interval", "1m", + PropertyType.TIMEDURATION, + "The interval at which to check the tservers for external compactions.", "4.0.0"), + // ELASTICITY_TODO: Deprecate Compactor properties in a 3.x release + TASK_RUNNER_PREFIX("task.runner.", null, PropertyType.PREFIX, + "Properties in this category affect the behavior of the accumulo TaskRunner server.", + "4.0.0"), + TASK_RUNNER_CLIENTPORT("task.runner.port.client", "9133", PropertyType.PORT, + "The port used for handling client connections on the TaskRunner servers", "4.0.0"), + TASK_RUNNER_GROUP_NAME("task.runner.group", Constants.DEFAULT_RESOURCE_GROUP_NAME, + PropertyType.STRING, "Resource group name for this Compactor.", "4.0.0"), + TASK_RUNNER_MAX_MESSAGE_SIZE("task.runner.message.size.max", "10M", PropertyType.BYTES, + "The maximum size of a message that can be sent to a tablet server.", "4.0.0"), + TASK_RUNNER_MIN_JOB_WAIT_TIME("task.runner.wait.time.job.min", "1s", PropertyType.TIMEDURATION, + "The minimum amount of time to wait between checks for the next job, backing off" + + "exponentially until TASK_RUNNER_MAX_JOB_WAIT_TIME is reached.", "4.0.0"), - COMPACTOR_MAX_JOB_WAIT_TIME("compactor.wait.time.job.max", "5m", PropertyType.TIMEDURATION, - "Compactors do exponential backoff when their request for work repeatedly come back empty. " + TASK_RUNNER_MAX_JOB_WAIT_TIME("task.runner.wait.time.job.max", "5m", PropertyType.TIMEDURATION, + "TaskRunners do exponential backoff when their request for work repeatedly come back empty. " + "This is the maximum amount of time to wait between checks for the next compaction job.", "4.0.0"), - @Experimental - COMPACTOR_MINTHREADS("compactor.threads.minimum", "1", PropertyType.COUNT, - "The minimum number of threads to use to handle incoming requests.", "2.1.0"), - @Experimental - COMPACTOR_MINTHREADS_TIMEOUT("compactor.threads.timeout", "0s", PropertyType.TIMEDURATION, + TASK_RUNNER_MINTHREADS("task.runner.threads.minimum", "1", PropertyType.COUNT, + "The minimum number of threads to use to handle incoming requests.", "4.0.0"), + TASK_RUNNER_MINTHREADS_TIMEOUT("task.runner.threads.timeout", "0s", PropertyType.TIMEDURATION, "The time after which incoming request threads terminate with no work available. Zero (0) will keep the threads alive indefinitely.", - "2.1.0"), - @Experimental - COMPACTOR_THREADCHECK("compactor.threadcheck.time", "1s", PropertyType.TIMEDURATION, - "The time between adjustments of the server thread pool.", "2.1.0"), - @Experimental - COMPACTOR_MAX_MESSAGE_SIZE("compactor.message.size.max", "10M", PropertyType.BYTES, - "The maximum size of a message that can be sent to a tablet server.", "2.1.0"), - @Experimental - COMPACTOR_GROUP_NAME("compactor.group", Constants.DEFAULT_RESOURCE_GROUP_NAME, - PropertyType.STRING, "Resource group name for this Compactor.", "3.0.0"), - // CompactionCoordinator properties - @Experimental - COMPACTION_COORDINATOR_PREFIX("compaction.coordinator.", null, PropertyType.PREFIX, - "Properties in this category affect the behavior of the accumulo compaction coordinator server.", - "2.1.0"), - @Experimental - COMPACTION_COORDINATOR_DEAD_COMPACTOR_CHECK_INTERVAL( - "compaction.coordinator.compactor.dead.check.interval", "5m", PropertyType.TIMEDURATION, - "The interval at which to check for dead compactors.", "2.1.0"), - @Experimental - COMPACTION_COORDINATOR_FINALIZER_TSERVER_NOTIFIER_MAXTHREADS( - "compaction.coordinator.compaction.finalizer.threads.maximum", "5", PropertyType.COUNT, - "The maximum number of threads to use for notifying tablet servers that an external compaction has completed.", - "2.1.0"), - @Experimental - COMPACTION_COORDINATOR_FINALIZER_COMPLETION_CHECK_INTERVAL( - "compaction.coordinator.compaction.finalizer.check.interval", "60s", - PropertyType.TIMEDURATION, - "The interval at which to check for external compaction final state markers in the metadata table.", - "2.1.0"), - @Experimental - COMPACTION_COORDINATOR_TSERVER_COMPACTION_CHECK_INTERVAL( - "compaction.coordinator.tserver.check.interval", "1m", PropertyType.TIMEDURATION, - "The interval at which to check the tservers for external compactions.", "2.1.0"); + "4.0.0"), + TASK_RUNNER_PORTSEARCH("task.runner.port.search", "true", PropertyType.BOOLEAN, + "If the task.runner.port.client is in use, search higher ports until one is available", + "4.0.0"), + TASK_RUNNER_THREADCHECK("task.runner.threadcheck.time", "1s", PropertyType.TIMEDURATION, + "The time between adjustments of the server thread pool.", "4.0.0"), + TASK_RUNNER_WORKER_TYPE("task.runner.worker.type", "", PropertyType.STRING, + "Property used by the TaskWorker " + + "processes that determines which type of tasks it will perform. Valid values are COMPACTION," + + "LOG_SORTING, SPLIT_POINT_CALCULATION", + "4.0.0"); private final String key; private final String defaultValue; 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 050c2b4cace..17729897ce0 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 @@ -41,8 +41,6 @@ public class ServiceLockData implements Comparable { */ public static enum ThriftService { CLIENT, - COORDINATOR, - COMPACTOR, FATE, GC, MANAGER, @@ -50,6 +48,8 @@ public static enum ThriftService { TABLET_INGEST, TABLET_MANAGEMENT, TABLET_SCAN, + TASK_RUNNER, + TASK_MANAGER, TSERV } diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/clients/TaskManagerThriftClient.java b/core/src/main/java/org/apache/accumulo/core/rpc/clients/TaskManagerThriftClient.java new file mode 100644 index 00000000000..aae575dfb07 --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/core/rpc/clients/TaskManagerThriftClient.java @@ -0,0 +1,40 @@ +/* + * 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.clientImpl.ClientContext; +import org.apache.accumulo.core.tasks.thrift.TaskManager.Client; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TaskManagerThriftClient extends ThriftClientTypes + implements ManagerClient { + + private static Logger LOG = LoggerFactory.getLogger(TaskManagerThriftClient.class); + + public TaskManagerThriftClient(String serviceName) { + super(serviceName, new Client.Factory()); + } + + @Override + public Client getConnection(ClientContext context) { + return getManagerConnection(LOG, this, context); + } + +} diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/clients/CompactorServiceThriftClient.java b/core/src/main/java/org/apache/accumulo/core/rpc/clients/TaskRunnerThriftClient.java similarity index 81% rename from core/src/main/java/org/apache/accumulo/core/rpc/clients/CompactorServiceThriftClient.java rename to core/src/main/java/org/apache/accumulo/core/rpc/clients/TaskRunnerThriftClient.java index c8c00aa3265..329a144018f 100644 --- a/core/src/main/java/org/apache/accumulo/core/rpc/clients/CompactorServiceThriftClient.java +++ b/core/src/main/java/org/apache/accumulo/core/rpc/clients/TaskRunnerThriftClient.java @@ -18,11 +18,11 @@ */ package org.apache.accumulo.core.rpc.clients; -import org.apache.accumulo.core.compaction.thrift.CompactorService.Client; +import org.apache.accumulo.core.tasks.thrift.TaskRunner.Client; -public class CompactorServiceThriftClient extends ThriftClientTypes { +public class TaskRunnerThriftClient extends ThriftClientTypes { - CompactorServiceThriftClient(String serviceName) { + TaskRunnerThriftClient(String serviceName) { super(serviceName, new 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 9499e7880a0..dee79bdd219 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 @@ -34,12 +34,6 @@ public abstract class ThriftClientTypes { public static final ClientServiceThriftClient CLIENT = new ClientServiceThriftClient("client"); - public static final CompactorServiceThriftClient COMPACTOR = - new CompactorServiceThriftClient("compactor"); - - public static final CompactionCoordinatorServiceThriftClient COORDINATOR = - new CompactionCoordinatorServiceThriftClient("coordinator"); - public static final FateThriftClient FATE = new FateThriftClient("fate"); public static final GCMonitorServiceThriftClient GC = new GCMonitorServiceThriftClient("gc"); @@ -58,6 +52,12 @@ public abstract class ThriftClientTypes { public static final TabletManagementClientServiceThriftClient TABLET_MGMT = new TabletManagementClientServiceThriftClient("tablet"); + public static final TaskRunnerThriftClient TASK_RUNNER = + new TaskRunnerThriftClient("task_runner"); + + public static final TaskManagerThriftClient TASK_MANAGER = + new TaskManagerThriftClient("task_mgr"); + /** * execute method with supplied client returning object of type R * diff --git a/core/src/main/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlanner.java b/core/src/main/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlanner.java index 0f84f8ed05a..48d51423b58 100644 --- a/core/src/main/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlanner.java +++ b/core/src/main/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlanner.java @@ -99,8 +99,7 @@ * ]} * * - * Note that the use of 'external' requires that the CompactionCoordinator and at least one - * Compactor for Queue1 is running. + * Note that the use of 'external' at least one Compactor for Queue1 is running. *
  • {@code tserver.compaction.major.service..opts.maxOpen} This determines the maximum * number of files that will be included in a single compaction. * diff --git a/core/src/main/java/org/apache/accumulo/core/tasks/TaskMessage.java b/core/src/main/java/org/apache/accumulo/core/tasks/TaskMessage.java new file mode 100644 index 00000000000..9a4339f6985 --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/core/tasks/TaskMessage.java @@ -0,0 +1,84 @@ +/* + * 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.tasks; + +import org.apache.accumulo.core.tasks.thrift.Task; +import org.apache.accumulo.core.util.json.ByteArrayToBase64TypeAdapter; +import org.apache.accumulo.core.util.json.GsonIgnoreExclusionStrategy; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.base.Preconditions; +import com.google.gson.Gson; +import com.google.gson.GsonBuilder; + +/** + * Generic object that can be used to transport TaskMessage subclasses over Thrift using the Thrift + * Task type. Implementations of this class are serialized to JSON then transported via Task and + * deserialized on the other side. + */ +public abstract class TaskMessage { + + private static final Logger LOG = LoggerFactory.getLogger(TaskMessage.class); + + @SuppressWarnings("unchecked") + public static T fromThiftTask(Task task, TaskMessageType expectedType) { + TaskMessageType type = TaskMessageType.valueOf(task.getMessageType()); + Preconditions.checkState(type == expectedType, + "Task is of type: " + type + ", expected: " + expectedType); + T decodedMsg = (T) TaskMessage.GSON_FOR_TASKS.fromJson(task.getMessage(), type.getTaskClass()); + LOG.trace("Received {}", TaskMessage.GSON_FOR_TASKS.toJson(decodedMsg)); + return decodedMsg; + } + + private static final Gson GSON_FOR_TASKS = + new GsonBuilder().setExclusionStrategies(new GsonIgnoreExclusionStrategy()) + .registerTypeHierarchyAdapter(byte[].class, new ByteArrayToBase64TypeAdapter()).create(); + + private String taskId; + private TaskMessageType type; + + public TaskMessage() {} + + public String getTaskId() { + return taskId; + } + + public void setTaskId(String taskId) { + this.taskId = taskId; + } + + void setMessageType(TaskMessageType type) { + this.type = type; + } + + public TaskMessageType getMessageType() { + return type; + } + + public Task toThriftTask() { + Task t = new Task(); + t.setTaskId(getTaskId()); + t.setMessageType(getMessageType().name()); + t.setMessage(TaskMessage.GSON_FOR_TASKS.toJson(this)); + LOG.trace("Sending {}", TaskMessage.GSON_FOR_TASKS.toJson(this)); + return t; + } + +} diff --git a/core/src/main/java/org/apache/accumulo/core/tasks/TaskMessageType.java b/core/src/main/java/org/apache/accumulo/core/tasks/TaskMessageType.java new file mode 100644 index 00000000000..9683a73c12a --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/core/tasks/TaskMessageType.java @@ -0,0 +1,68 @@ +/* + * 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.tasks; + +import java.lang.reflect.InvocationTargetException; + +import org.apache.accumulo.core.tasks.compaction.ActiveCompactionTasks; +import org.apache.accumulo.core.tasks.compaction.CompactionTask; +import org.apache.accumulo.core.tasks.compaction.CompactionTaskCompleted; +import org.apache.accumulo.core.tasks.compaction.CompactionTaskFailed; +import org.apache.accumulo.core.tasks.compaction.CompactionTaskStatus; +import org.apache.accumulo.core.tasks.compaction.CompactionTasksCompleted; +import org.apache.accumulo.core.tasks.compaction.CompactionTasksRunning; + +public enum TaskMessageType { + + // Compaction Related Tasks + COMPACTION_TASK(CompactionTask.class), + COMPACTION_TASK_COMPLETED(CompactionTaskCompleted.class), + COMPACTION_TASK_FAILED(CompactionTaskFailed.class), + COMPACTION_TASK_LIST(ActiveCompactionTasks.class), + COMPACTION_TASK_STATUS(CompactionTaskStatus.class), + COMPACTION_TASKS_COMPLETED(CompactionTasksCompleted.class), + COMPACTION_TASKS_RUNNING(CompactionTasksRunning.class); + + // TODO: Tasks for calculating split points + + // TODO: Tasks for log sorting and recovery + + private Class taskClass; + + TaskMessageType(Class taskClass) { + this.taskClass = taskClass; + } + + public Class getTaskClass() { + return this.taskClass; + } + + @SuppressWarnings("unchecked") + public T getTaskMessage() { + try { + T msg = (T) this.getTaskClass().getConstructor().newInstance(); + msg.setMessageType(this); + return msg; + } catch (InstantiationException | IllegalAccessException | IllegalArgumentException + | InvocationTargetException | NoSuchMethodException | SecurityException e) { + throw new RuntimeException("Error creating instance of " + taskClass); + } + } + +} diff --git a/core/src/main/java/org/apache/accumulo/core/tasks/ThriftSerializers.java b/core/src/main/java/org/apache/accumulo/core/tasks/ThriftSerializers.java new file mode 100644 index 00000000000..c08a0d0f790 --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/core/tasks/ThriftSerializers.java @@ -0,0 +1,95 @@ +/* + * 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.tasks; + +import java.util.function.Supplier; + +import org.apache.accumulo.core.compaction.thrift.TCompactionStatusUpdate; +import org.apache.accumulo.core.compaction.thrift.TExternalCompactionList; +import org.apache.accumulo.core.tabletserver.thrift.ActiveCompactionList; +import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats; +import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob; +import org.apache.thrift.TBase; +import org.apache.thrift.TDeserializer; +import org.apache.thrift.TException; +import org.apache.thrift.TSerializer; +import org.apache.thrift.protocol.TBinaryProtocol; +import org.apache.thrift.transport.TTransportException; + +public class ThriftSerializers { + + public static class ThriftSerializer> { + + private static final TBinaryProtocol.Factory PROTOCOL_FACTORY = new TBinaryProtocol.Factory(); + + private static final ThreadLocal SERIALIZER = new ThreadLocal<>() { + @Override + protected TSerializer initialValue() { + try { + return new TSerializer(PROTOCOL_FACTORY); + } catch (TTransportException e) { + throw new RuntimeException("Error creating serializer", e); + } + } + }; + + private static final ThreadLocal DESERIALIZER = new ThreadLocal<>() { + @Override + protected TDeserializer initialValue() { + try { + return new TDeserializer(PROTOCOL_FACTORY); + } catch (TTransportException e) { + throw new RuntimeException("Error creating serializer", e); + } + } + }; + + // prevent construction outside of this class + private ThriftSerializer() {} + + public byte[] serialize(T object) throws TException { + return SERIALIZER.get().serialize(object); + } + + public void deserialize(T object, byte[] bytes) throws TException { + DESERIALIZER.get().deserialize(object, bytes); + } + + } + + public static final Supplier< + ThriftSerializer> EXTERNAL_COMPACTION_ACTIVE_COMPACTION_LIST = + () -> new ThriftSerializer(); + + public static final Supplier< + ThriftSerializer> EXTERNAL_COMPACTION_JOB_SERIALIZER = + () -> new ThriftSerializer(); + + public static final Supplier> EXTERNAL_COMPACTION_LIST = + () -> new ThriftSerializer(); + + public static final Supplier< + ThriftSerializer> EXTERNAL_COMPACTION_STATUS_SERIALIZER = + () -> new ThriftSerializer(); + + public static final Supplier< + ThriftSerializer> EXTERNAL_COMPACTION_STATS_SERIALIZER = + () -> new ThriftSerializer(); + +} diff --git a/core/src/main/java/org/apache/accumulo/core/tasks/compaction/ActiveCompactionTasks.java b/core/src/main/java/org/apache/accumulo/core/tasks/compaction/ActiveCompactionTasks.java new file mode 100644 index 00000000000..46d6ed4b824 --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/core/tasks/compaction/ActiveCompactionTasks.java @@ -0,0 +1,51 @@ +/* + * 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.tasks.compaction; + +import org.apache.accumulo.core.tabletserver.thrift.ActiveCompactionList; +import org.apache.accumulo.core.tasks.ThriftSerializers; +import org.apache.accumulo.core.util.json.GsonIgnore; +import org.apache.thrift.TException; + +public class ActiveCompactionTasks extends CompactionTask { + + private byte[] activeCompactions; + + @GsonIgnore + private ActiveCompactionList thriftCompactions; + + public ActiveCompactionTasks() {} + + public void setActiveCompactions(ActiveCompactionList compactions) throws TException { + this.activeCompactions = + ThriftSerializers.EXTERNAL_COMPACTION_ACTIVE_COMPACTION_LIST.get().serialize(compactions); + this.thriftCompactions = compactions; + } + + public ActiveCompactionList getActiveCompactions() throws TException { + if (this.activeCompactions == null && this.thriftCompactions != null) { + ActiveCompactionList obj = new ActiveCompactionList(); + ThriftSerializers.EXTERNAL_COMPACTION_ACTIVE_COMPACTION_LIST.get().deserialize(obj, + activeCompactions); + this.thriftCompactions = obj; + } + return thriftCompactions; + } + +} diff --git a/core/src/main/java/org/apache/accumulo/core/tasks/compaction/CompactionTask.java b/core/src/main/java/org/apache/accumulo/core/tasks/compaction/CompactionTask.java new file mode 100644 index 00000000000..296a5297005 --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/core/tasks/compaction/CompactionTask.java @@ -0,0 +1,50 @@ +/* + * 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.tasks.compaction; + +import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob; +import org.apache.accumulo.core.tasks.TaskMessage; +import org.apache.accumulo.core.tasks.ThriftSerializers; +import org.apache.accumulo.core.util.json.GsonIgnore; +import org.apache.thrift.TException; + +public class CompactionTask extends TaskMessage { + + private byte[] job; + + @GsonIgnore + private TExternalCompactionJob thriftJob; + + public CompactionTask() {} + + public void setCompactionJob(TExternalCompactionJob job) throws TException { + this.job = ThriftSerializers.EXTERNAL_COMPACTION_JOB_SERIALIZER.get().serialize(job); + this.thriftJob = job; + } + + public TExternalCompactionJob getCompactionJob() throws TException { + if (this.thriftJob == null && this.job != null) { + TExternalCompactionJob obj = new TExternalCompactionJob(); + ThriftSerializers.EXTERNAL_COMPACTION_JOB_SERIALIZER.get().deserialize(obj, job); + this.thriftJob = obj; + } + return this.thriftJob; + } + +} diff --git a/core/src/main/java/org/apache/accumulo/core/tasks/compaction/CompactionTaskCompleted.java b/core/src/main/java/org/apache/accumulo/core/tasks/compaction/CompactionTaskCompleted.java new file mode 100644 index 00000000000..fa29fb8f241 --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/core/tasks/compaction/CompactionTaskCompleted.java @@ -0,0 +1,49 @@ +/* + * 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.tasks.compaction; + +import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats; +import org.apache.accumulo.core.tasks.ThriftSerializers; +import org.apache.accumulo.core.util.json.GsonIgnore; +import org.apache.thrift.TException; + +public class CompactionTaskCompleted extends CompactionTask { + + private byte[] stats; + + @GsonIgnore + private TCompactionStats thriftStats; + + public CompactionTaskCompleted() {} + + public void setCompactionStats(TCompactionStats stats) throws TException { + this.stats = ThriftSerializers.EXTERNAL_COMPACTION_STATS_SERIALIZER.get().serialize(stats); + this.thriftStats = stats; + } + + public TCompactionStats getCompactionStats() throws TException { + if (this.thriftStats == null && this.stats != null) { + TCompactionStats obj = new TCompactionStats(); + ThriftSerializers.EXTERNAL_COMPACTION_STATS_SERIALIZER.get().deserialize(obj, stats); + this.thriftStats = obj; + } + return thriftStats; + } + +} diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/clients/CompactionCoordinatorServiceThriftClient.java b/core/src/main/java/org/apache/accumulo/core/tasks/compaction/CompactionTaskFailed.java similarity index 70% rename from core/src/main/java/org/apache/accumulo/core/rpc/clients/CompactionCoordinatorServiceThriftClient.java rename to core/src/main/java/org/apache/accumulo/core/tasks/compaction/CompactionTaskFailed.java index faefc8b7588..7dee4099ceb 100644 --- a/core/src/main/java/org/apache/accumulo/core/rpc/clients/CompactionCoordinatorServiceThriftClient.java +++ b/core/src/main/java/org/apache/accumulo/core/tasks/compaction/CompactionTaskFailed.java @@ -16,14 +16,10 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.accumulo.core.rpc.clients; +package org.apache.accumulo.core.tasks.compaction; -import org.apache.accumulo.core.compaction.thrift.CompactionCoordinatorService.Client; +public class CompactionTaskFailed extends CompactionTask { -public class CompactionCoordinatorServiceThriftClient extends ThriftClientTypes { - - CompactionCoordinatorServiceThriftClient(String serviceName) { - super(serviceName, new Client.Factory()); - } + public CompactionTaskFailed() {} } diff --git a/core/src/main/java/org/apache/accumulo/core/tasks/compaction/CompactionTaskStatus.java b/core/src/main/java/org/apache/accumulo/core/tasks/compaction/CompactionTaskStatus.java new file mode 100644 index 00000000000..046b701fcac --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/core/tasks/compaction/CompactionTaskStatus.java @@ -0,0 +1,50 @@ +/* + * 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.tasks.compaction; + +import org.apache.accumulo.core.compaction.thrift.TCompactionStatusUpdate; +import org.apache.accumulo.core.tasks.TaskMessage; +import org.apache.accumulo.core.tasks.ThriftSerializers; +import org.apache.accumulo.core.util.json.GsonIgnore; +import org.apache.thrift.TException; + +public class CompactionTaskStatus extends TaskMessage { + + private byte[] status; + + @GsonIgnore + private TCompactionStatusUpdate statusThriftObj; + + public CompactionTaskStatus() {} + + public void setCompactionStatus(TCompactionStatusUpdate status) throws TException { + this.status = ThriftSerializers.EXTERNAL_COMPACTION_STATUS_SERIALIZER.get().serialize(status); + this.statusThriftObj = status; + } + + public TCompactionStatusUpdate getCompactionStatus() throws TException { + if (this.statusThriftObj == null && this.status != null) { + TCompactionStatusUpdate obj = new TCompactionStatusUpdate(); + ThriftSerializers.EXTERNAL_COMPACTION_STATUS_SERIALIZER.get().deserialize(obj, status); + this.statusThriftObj = obj; + } + return this.statusThriftObj; + } + +} diff --git a/core/src/main/java/org/apache/accumulo/core/tasks/compaction/CompactionTasksCompleted.java b/core/src/main/java/org/apache/accumulo/core/tasks/compaction/CompactionTasksCompleted.java new file mode 100644 index 00000000000..7d650072068 --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/core/tasks/compaction/CompactionTasksCompleted.java @@ -0,0 +1,50 @@ +/* + * 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.tasks.compaction; + +import org.apache.accumulo.core.compaction.thrift.TExternalCompactionList; +import org.apache.accumulo.core.tasks.TaskMessage; +import org.apache.accumulo.core.tasks.ThriftSerializers; +import org.apache.accumulo.core.util.json.GsonIgnore; +import org.apache.thrift.TException; + +public class CompactionTasksCompleted extends TaskMessage { + + private byte[] completed; + + @GsonIgnore + private TExternalCompactionList list; + + public CompactionTasksCompleted() {} + + public void setCompleted(TExternalCompactionList list) throws TException { + this.completed = ThriftSerializers.EXTERNAL_COMPACTION_LIST.get().serialize(list); + this.list = list; + } + + public TExternalCompactionList getCompleted() throws TException { + if (this.list == null && this.completed != null) { + TExternalCompactionList obj = new TExternalCompactionList(); + ThriftSerializers.EXTERNAL_COMPACTION_LIST.get().deserialize(obj, completed); + this.list = obj; + } + return list; + } + +} diff --git a/core/src/main/java/org/apache/accumulo/core/tasks/compaction/CompactionTasksRunning.java b/core/src/main/java/org/apache/accumulo/core/tasks/compaction/CompactionTasksRunning.java new file mode 100644 index 00000000000..ae66d2fee5f --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/core/tasks/compaction/CompactionTasksRunning.java @@ -0,0 +1,50 @@ +/* + * 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.tasks.compaction; + +import org.apache.accumulo.core.compaction.thrift.TExternalCompactionList; +import org.apache.accumulo.core.tasks.TaskMessage; +import org.apache.accumulo.core.tasks.ThriftSerializers; +import org.apache.accumulo.core.util.json.GsonIgnore; +import org.apache.thrift.TException; + +public class CompactionTasksRunning extends TaskMessage { + + private byte[] running; + + @GsonIgnore + private TExternalCompactionList list; + + public CompactionTasksRunning() {} + + public void setRunning(TExternalCompactionList list) throws TException { + this.running = ThriftSerializers.EXTERNAL_COMPACTION_LIST.get().serialize(list); + this.list = list; + } + + public TExternalCompactionList getRunning() throws TException { + if (this.list == null && this.running != null) { + TExternalCompactionList obj = new TExternalCompactionList(); + ThriftSerializers.EXTERNAL_COMPACTION_LIST.get().deserialize(obj, running); + this.list = obj; + } + return list; + } + +} diff --git a/core/src/main/java/org/apache/accumulo/core/util/compaction/ExternalCompactionUtil.java b/core/src/main/java/org/apache/accumulo/core/util/compaction/ExternalCompactionUtil.java index e4eda5713da..ae023b9420d 100644 --- a/core/src/main/java/org/apache/accumulo/core/util/compaction/ExternalCompactionUtil.java +++ b/core/src/main/java/org/apache/accumulo/core/util/compaction/ExternalCompactionUtil.java @@ -32,7 +32,6 @@ import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.clientImpl.ClientContext; import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException; -import org.apache.accumulo.core.compaction.thrift.CompactorService; import org.apache.accumulo.core.fate.zookeeper.ZooReader; import org.apache.accumulo.core.fate.zookeeper.ZooSession; import org.apache.accumulo.core.lock.ServiceLock; @@ -41,7 +40,14 @@ import org.apache.accumulo.core.rpc.ThriftUtil; import org.apache.accumulo.core.rpc.clients.ThriftClientTypes; import org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction; +import org.apache.accumulo.core.tabletserver.thrift.ActiveCompactionList; import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob; +import org.apache.accumulo.core.tasks.TaskMessage; +import org.apache.accumulo.core.tasks.TaskMessageType; +import org.apache.accumulo.core.tasks.compaction.ActiveCompactionTasks; +import org.apache.accumulo.core.tasks.compaction.CompactionTask; +import org.apache.accumulo.core.tasks.thrift.Task; +import org.apache.accumulo.core.tasks.thrift.TaskRunner; import org.apache.accumulo.core.trace.TraceUtil; import org.apache.accumulo.core.util.threads.ThreadPools; import org.apache.thrift.TException; @@ -96,15 +102,15 @@ public static String getHostPortString(HostAndPort address) { /** * - * @return Optional HostAndPort of Coordinator node if found + * @return Optional HostAndPort of TaskManager node if found */ - public static Optional findCompactionCoordinator(ClientContext context) { + public static Optional findTaskManager(ClientContext context) { final String lockPath = context.getZooKeeperRoot() + Constants.ZMANAGER_LOCK; try { var zk = ZooSession.getAnonymousSession(context.getZooKeepers(), context.getZooKeepersSessionTimeOut()); return ServiceLock.getLockData(zk, ServiceLock.path(lockPath)) - .map(sld -> sld.getAddress(ThriftService.COORDINATOR)); + .map(sld -> sld.getAddress(ThriftService.TASK_MANAGER)); } catch (KeeperException | InterruptedException e) { throw new IllegalStateException(e); } @@ -156,10 +162,14 @@ public static Map> getCompactorAddrs(ClientContext cont */ public static List getActiveCompaction(HostAndPort compactor, ClientContext context) throws ThriftSecurityException { - CompactorService.Client client = null; + TaskRunner.Client client = null; try { - client = ThriftUtil.getClient(ThriftClientTypes.COMPACTOR, compactor, context); - return client.getActiveCompactions(TraceUtil.traceInfo(), context.rpcCreds()); + client = ThriftUtil.getClient(ThriftClientTypes.TASK_RUNNER, compactor, context); + Task task = client.getActiveCompactions(TraceUtil.traceInfo(), context.rpcCreds()); + final ActiveCompactionTasks list = + TaskMessage.fromThiftTask(task, TaskMessageType.COMPACTION_TASK_LIST); + final ActiveCompactionList acl = list.getActiveCompactions(); + return acl.getCompactions(); } catch (ThriftSecurityException e) { throw e; } catch (TException e) { @@ -180,11 +190,13 @@ public static List getActiveCompaction(HostAndPort compactor, public static TExternalCompactionJob getRunningCompaction(HostAndPort compactorAddr, ClientContext context) { - CompactorService.Client client = null; + TaskRunner.Client client = null; try { - client = ThriftUtil.getClient(ThriftClientTypes.COMPACTOR, compactorAddr, context); - TExternalCompactionJob job = - client.getRunningCompaction(TraceUtil.traceInfo(), context.rpcCreds()); + client = ThriftUtil.getClient(ThriftClientTypes.TASK_RUNNER, compactorAddr, context); + Task task = client.getRunningTask(TraceUtil.traceInfo(), context.rpcCreds()); + final CompactionTask compactionTask = + TaskMessage.fromThiftTask(task, TaskMessageType.COMPACTION_TASK); + TExternalCompactionJob job = compactionTask.getCompactionJob(); if (job.getExternalCompactionId() != null) { LOG.debug("Compactor {} is running {}", compactorAddr, job.getExternalCompactionId()); return job; @@ -199,10 +211,10 @@ public static TExternalCompactionJob getRunningCompaction(HostAndPort compactorA private static ExternalCompactionId getRunningCompactionId(HostAndPort compactorAddr, ClientContext context) { - CompactorService.Client client = null; + TaskRunner.Client client = null; try { - client = ThriftUtil.getClient(ThriftClientTypes.COMPACTOR, compactorAddr, context); - String secid = client.getRunningCompactionId(TraceUtil.traceInfo(), context.rpcCreds()); + client = ThriftUtil.getClient(ThriftClientTypes.TASK_RUNNER, compactorAddr, context); + String secid = client.getRunningTaskId(TraceUtil.traceInfo(), context.rpcCreds()); if (!secid.isEmpty()) { return ExternalCompactionId.of(secid); } @@ -216,8 +228,8 @@ private static ExternalCompactionId getRunningCompactionId(HostAndPort compactor /** * This method returns information from the Compactor about the job that is currently running. The - * RunningCompactions are not fully populated. This method is used from the CompactionCoordinator - * on a restart to re-populate the set of running compactions on the compactors. + * RunningCompactions are not fully populated. This method is used from the TaskManager on a + * restart to re-populate the set of running compactions on the compactors. * * @param context server context * @return map of compactor and external compaction jobs @@ -301,10 +313,10 @@ public static int countCompactors(String groupName, ClientContext context) { public static void cancelCompaction(ClientContext context, HostAndPort compactorAddr, String ecid) { - CompactorService.Client client = null; + TaskRunner.Client client = null; try { - client = ThriftUtil.getClient(ThriftClientTypes.COMPACTOR, compactorAddr, context); - client.cancel(TraceUtil.traceInfo(), context.rpcCreds(), ecid); + client = ThriftUtil.getClient(ThriftClientTypes.TASK_RUNNER, compactorAddr, context); + client.cancelTask(TraceUtil.traceInfo(), context.rpcCreds(), ecid); } catch (TException e) { LOG.debug("Failed to cancel compactor {} for {}", compactorAddr, ecid, e); } finally { diff --git a/core/src/main/java/org/apache/accumulo/core/util/json/GsonIgnore.java b/core/src/main/java/org/apache/accumulo/core/util/json/GsonIgnore.java new file mode 100644 index 00000000000..144fcb3c422 --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/core/util/json/GsonIgnore.java @@ -0,0 +1,32 @@ +/* + * 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.util.json; + +import java.lang.annotation.Documented; +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +@Documented +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.FIELD, ElementType.TYPE}) +public @interface GsonIgnore { + +} diff --git a/core/src/main/java/org/apache/accumulo/core/util/json/GsonIgnoreExclusionStrategy.java b/core/src/main/java/org/apache/accumulo/core/util/json/GsonIgnoreExclusionStrategy.java new file mode 100644 index 00000000000..d2b4490c91e --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/core/util/json/GsonIgnoreExclusionStrategy.java @@ -0,0 +1,35 @@ +/* + * 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.util.json; + +import com.google.gson.ExclusionStrategy; +import com.google.gson.FieldAttributes; + +public class GsonIgnoreExclusionStrategy implements ExclusionStrategy { + + @Override + public boolean shouldSkipClass(Class clazz) { + return clazz.getAnnotation(GsonIgnore.class) != null; + } + + @Override + public boolean shouldSkipField(FieldAttributes f) { + return f.getAnnotation(GsonIgnore.class) != null; + } +} diff --git a/core/src/main/scripts/generate-thrift.sh b/core/src/main/scripts/generate-thrift.sh index 0ad5911a33f..fc7816a3928 100755 --- a/core/src/main/scripts/generate-thrift.sh +++ b/core/src/main/scripts/generate-thrift.sh @@ -32,7 +32,7 @@ [[ -z $REQUIRED_THRIFT_VERSION ]] && REQUIRED_THRIFT_VERSION='0.17.0' [[ -z $INCLUDED_MODULES ]] && INCLUDED_MODULES=() [[ -z $BASE_OUTPUT_PACKAGE ]] && BASE_OUTPUT_PACKAGE='org.apache.accumulo.core' -[[ -z $PACKAGES_TO_GENERATE ]] && PACKAGES_TO_GENERATE=(gc manager tabletserver securityImpl clientImpl dataImpl compaction tabletingest tablet tabletscan) +[[ -z $PACKAGES_TO_GENERATE ]] && PACKAGES_TO_GENERATE=(gc manager tabletserver securityImpl clientImpl dataImpl compaction tabletingest tablet tabletscan tasks) [[ -z $BUILD_DIR ]] && BUILD_DIR='target' [[ -z $LANGUAGES_TO_GENERATE ]] && LANGUAGES_TO_GENERATE=(java) [[ -z $FINAL_DIR ]] && FINAL_DIR='src/main' diff --git a/core/src/main/spotbugs/exclude-filter.xml b/core/src/main/spotbugs/exclude-filter.xml index ded70a98dc4..3bf4463988c 100644 --- a/core/src/main/spotbugs/exclude-filter.xml +++ b/core/src/main/spotbugs/exclude-filter.xml @@ -37,6 +37,7 @@ + diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/ActiveCompactionList.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/ActiveCompactionList.java new file mode 100644 index 00000000000..43bd57a1eee --- /dev/null +++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/ActiveCompactionList.java @@ -0,0 +1,461 @@ +/* + * 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.tabletserver.thrift; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) +public class ActiveCompactionList 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("ActiveCompactionList"); + + private static final org.apache.thrift.protocol.TField COMPACTIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("compactions", org.apache.thrift.protocol.TType.LIST, (short)1); + + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new ActiveCompactionListStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new ActiveCompactionListTupleSchemeFactory(); + + public @org.apache.thrift.annotation.Nullable java.util.List compactions; // 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 { + COMPACTIONS((short)1, "compactions"); + + 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: // COMPACTIONS + return COMPACTIONS; + 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.COMPACTIONS, new org.apache.thrift.meta_data.FieldMetaData("compactions", 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, ActiveCompaction.class)))); + metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ActiveCompactionList.class, metaDataMap); + } + + public ActiveCompactionList() { + } + + public ActiveCompactionList( + java.util.List compactions) + { + this(); + this.compactions = compactions; + } + + /** + * Performs a deep copy on other. + */ + public ActiveCompactionList(ActiveCompactionList other) { + if (other.isSetCompactions()) { + java.util.List __this__compactions = new java.util.ArrayList(other.compactions.size()); + for (ActiveCompaction other_element : other.compactions) { + __this__compactions.add(new ActiveCompaction(other_element)); + } + this.compactions = __this__compactions; + } + } + + @Override + public ActiveCompactionList deepCopy() { + return new ActiveCompactionList(this); + } + + @Override + public void clear() { + this.compactions = null; + } + + public int getCompactionsSize() { + return (this.compactions == null) ? 0 : this.compactions.size(); + } + + @org.apache.thrift.annotation.Nullable + public java.util.Iterator getCompactionsIterator() { + return (this.compactions == null) ? null : this.compactions.iterator(); + } + + public void addToCompactions(ActiveCompaction elem) { + if (this.compactions == null) { + this.compactions = new java.util.ArrayList(); + } + this.compactions.add(elem); + } + + @org.apache.thrift.annotation.Nullable + public java.util.List getCompactions() { + return this.compactions; + } + + public ActiveCompactionList setCompactions(@org.apache.thrift.annotation.Nullable java.util.List compactions) { + this.compactions = compactions; + return this; + } + + public void unsetCompactions() { + this.compactions = null; + } + + /** Returns true if field compactions is set (has been assigned a value) and false otherwise */ + public boolean isSetCompactions() { + return this.compactions != null; + } + + public void setCompactionsIsSet(boolean value) { + if (!value) { + this.compactions = null; + } + } + + @Override + public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) { + switch (field) { + case COMPACTIONS: + if (value == null) { + unsetCompactions(); + } else { + setCompactions((java.util.List)value); + } + break; + + } + } + + @org.apache.thrift.annotation.Nullable + @Override + public java.lang.Object getFieldValue(_Fields field) { + switch (field) { + case COMPACTIONS: + return getCompactions(); + + } + 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 COMPACTIONS: + return isSetCompactions(); + } + throw new java.lang.IllegalStateException(); + } + + @Override + public boolean equals(java.lang.Object that) { + if (that instanceof ActiveCompactionList) + return this.equals((ActiveCompactionList)that); + return false; + } + + public boolean equals(ActiveCompactionList that) { + if (that == null) + return false; + if (this == that) + return true; + + boolean this_present_compactions = true && this.isSetCompactions(); + boolean that_present_compactions = true && that.isSetCompactions(); + if (this_present_compactions || that_present_compactions) { + if (!(this_present_compactions && that_present_compactions)) + return false; + if (!this.compactions.equals(that.compactions)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + int hashCode = 1; + + hashCode = hashCode * 8191 + ((isSetCompactions()) ? 131071 : 524287); + if (isSetCompactions()) + hashCode = hashCode * 8191 + compactions.hashCode(); + + return hashCode; + } + + @Override + public int compareTo(ActiveCompactionList other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = java.lang.Boolean.compare(isSetCompactions(), other.isSetCompactions()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetCompactions()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.compactions, other.compactions); + 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("ActiveCompactionList("); + boolean first = true; + + sb.append("compactions:"); + if (this.compactions == null) { + sb.append("null"); + } else { + sb.append(this.compactions); + } + 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 ActiveCompactionListStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + @Override + public ActiveCompactionListStandardScheme getScheme() { + return new ActiveCompactionListStandardScheme(); + } + } + + private static class ActiveCompactionListStandardScheme extends org.apache.thrift.scheme.StandardScheme { + + @Override + public void read(org.apache.thrift.protocol.TProtocol iprot, ActiveCompactionList 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: // COMPACTIONS + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list36 = iprot.readListBegin(); + struct.compactions = new java.util.ArrayList(_list36.size); + @org.apache.thrift.annotation.Nullable ActiveCompaction _elem37; + for (int _i38 = 0; _i38 < _list36.size; ++_i38) + { + _elem37 = new ActiveCompaction(); + _elem37.read(iprot); + struct.compactions.add(_elem37); + } + iprot.readListEnd(); + } + struct.setCompactionsIsSet(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, ActiveCompactionList struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.compactions != null) { + oprot.writeFieldBegin(COMPACTIONS_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.compactions.size())); + for (ActiveCompaction _iter39 : struct.compactions) + { + _iter39.write(oprot); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class ActiveCompactionListTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + @Override + public ActiveCompactionListTupleScheme getScheme() { + return new ActiveCompactionListTupleScheme(); + } + } + + private static class ActiveCompactionListTupleScheme extends org.apache.thrift.scheme.TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, ActiveCompactionList 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.isSetCompactions()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetCompactions()) { + { + oprot.writeI32(struct.compactions.size()); + for (ActiveCompaction _iter40 : struct.compactions) + { + _iter40.write(oprot); + } + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, ActiveCompactionList struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; + java.util.BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + { + org.apache.thrift.protocol.TList _list41 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.compactions = new java.util.ArrayList(_list41.size); + @org.apache.thrift.annotation.Nullable ActiveCompaction _elem42; + for (int _i43 = 0; _i43 < _list41.size; ++_i43) + { + _elem42 = new ActiveCompaction(); + _elem42.read(iprot); + struct.compactions.add(_elem42); + } + } + struct.setCompactionsIsSet(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/tabletserver/thrift/IteratorConfig.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/IteratorConfig.java index d03a8d285e4..9cb3635d09a 100644 --- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/IteratorConfig.java +++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/IteratorConfig.java @@ -354,14 +354,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, IteratorConfig stru case 1: // ITERATORS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list46 = iprot.readListBegin(); - struct.iterators = new java.util.ArrayList(_list46.size); - @org.apache.thrift.annotation.Nullable TIteratorSetting _elem47; - for (int _i48 = 0; _i48 < _list46.size; ++_i48) + org.apache.thrift.protocol.TList _list54 = iprot.readListBegin(); + struct.iterators = new java.util.ArrayList(_list54.size); + @org.apache.thrift.annotation.Nullable TIteratorSetting _elem55; + for (int _i56 = 0; _i56 < _list54.size; ++_i56) { - _elem47 = new TIteratorSetting(); - _elem47.read(iprot); - struct.iterators.add(_elem47); + _elem55 = new TIteratorSetting(); + _elem55.read(iprot); + struct.iterators.add(_elem55); } iprot.readListEnd(); } @@ -390,9 +390,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, IteratorConfig str oprot.writeFieldBegin(ITERATORS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.iterators.size())); - for (TIteratorSetting _iter49 : struct.iterators) + for (TIteratorSetting _iter57 : struct.iterators) { - _iter49.write(oprot); + _iter57.write(oprot); } oprot.writeListEnd(); } @@ -424,9 +424,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, IteratorConfig stru if (struct.isSetIterators()) { { oprot.writeI32(struct.iterators.size()); - for (TIteratorSetting _iter50 : struct.iterators) + for (TIteratorSetting _iter58 : struct.iterators) { - _iter50.write(oprot); + _iter58.write(oprot); } } } @@ -438,14 +438,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, IteratorConfig struc java.util.BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list51 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.iterators = new java.util.ArrayList(_list51.size); - @org.apache.thrift.annotation.Nullable TIteratorSetting _elem52; - for (int _i53 = 0; _i53 < _list51.size; ++_i53) + org.apache.thrift.protocol.TList _list59 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.iterators = new java.util.ArrayList(_list59.size); + @org.apache.thrift.annotation.Nullable TIteratorSetting _elem60; + for (int _i61 = 0; _i61 < _list59.size; ++_i61) { - _elem52 = new TIteratorSetting(); - _elem52.read(iprot); - struct.iterators.add(_elem52); + _elem60 = new TIteratorSetting(); + _elem60.read(iprot); + struct.iterators.add(_elem60); } } struct.setIteratorsIsSet(true); diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TExternalCompactionJob.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TExternalCompactionJob.java index 6a0523c51b2..14c574f5617 100644 --- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TExternalCompactionJob.java +++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TExternalCompactionJob.java @@ -1037,14 +1037,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, TExternalCompaction case 3: // FILES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list54 = iprot.readListBegin(); - struct.files = new java.util.ArrayList(_list54.size); - @org.apache.thrift.annotation.Nullable InputFile _elem55; - for (int _i56 = 0; _i56 < _list54.size; ++_i56) + org.apache.thrift.protocol.TList _list62 = iprot.readListBegin(); + struct.files = new java.util.ArrayList(_list62.size); + @org.apache.thrift.annotation.Nullable InputFile _elem63; + for (int _i64 = 0; _i64 < _list62.size; ++_i64) { - _elem55 = new InputFile(); - _elem55.read(iprot); - struct.files.add(_elem55); + _elem63 = new InputFile(); + _elem63.read(iprot); + struct.files.add(_elem63); } iprot.readListEnd(); } @@ -1097,15 +1097,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, TExternalCompaction case 9: // OVERRIDES if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map57 = iprot.readMapBegin(); - struct.overrides = new java.util.HashMap(2*_map57.size); - @org.apache.thrift.annotation.Nullable java.lang.String _key58; - @org.apache.thrift.annotation.Nullable java.lang.String _val59; - for (int _i60 = 0; _i60 < _map57.size; ++_i60) + org.apache.thrift.protocol.TMap _map65 = iprot.readMapBegin(); + struct.overrides = new java.util.HashMap(2*_map65.size); + @org.apache.thrift.annotation.Nullable java.lang.String _key66; + @org.apache.thrift.annotation.Nullable java.lang.String _val67; + for (int _i68 = 0; _i68 < _map65.size; ++_i68) { - _key58 = iprot.readString(); - _val59 = iprot.readString(); - struct.overrides.put(_key58, _val59); + _key66 = iprot.readString(); + _val67 = iprot.readString(); + struct.overrides.put(_key66, _val67); } iprot.readMapEnd(); } @@ -1144,9 +1144,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, TExternalCompactio oprot.writeFieldBegin(FILES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.files.size())); - for (InputFile _iter61 : struct.files) + for (InputFile _iter69 : struct.files) { - _iter61.write(oprot); + _iter69.write(oprot); } oprot.writeListEnd(); } @@ -1177,10 +1177,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, TExternalCompactio oprot.writeFieldBegin(OVERRIDES_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.overrides.size())); - for (java.util.Map.Entry _iter62 : struct.overrides.entrySet()) + for (java.util.Map.Entry _iter70 : struct.overrides.entrySet()) { - oprot.writeString(_iter62.getKey()); - oprot.writeString(_iter62.getValue()); + oprot.writeString(_iter70.getKey()); + oprot.writeString(_iter70.getValue()); } oprot.writeMapEnd(); } @@ -1242,9 +1242,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, TExternalCompaction if (struct.isSetFiles()) { { oprot.writeI32(struct.files.size()); - for (InputFile _iter63 : struct.files) + for (InputFile _iter71 : struct.files) { - _iter63.write(oprot); + _iter71.write(oprot); } } } @@ -1266,10 +1266,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, TExternalCompaction if (struct.isSetOverrides()) { { oprot.writeI32(struct.overrides.size()); - for (java.util.Map.Entry _iter64 : struct.overrides.entrySet()) + for (java.util.Map.Entry _iter72 : struct.overrides.entrySet()) { - oprot.writeString(_iter64.getKey()); - oprot.writeString(_iter64.getValue()); + oprot.writeString(_iter72.getKey()); + oprot.writeString(_iter72.getValue()); } } } @@ -1290,14 +1290,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, TExternalCompactionJ } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list65 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.files = new java.util.ArrayList(_list65.size); - @org.apache.thrift.annotation.Nullable InputFile _elem66; - for (int _i67 = 0; _i67 < _list65.size; ++_i67) + org.apache.thrift.protocol.TList _list73 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.files = new java.util.ArrayList(_list73.size); + @org.apache.thrift.annotation.Nullable InputFile _elem74; + for (int _i75 = 0; _i75 < _list73.size; ++_i75) { - _elem66 = new InputFile(); - _elem66.read(iprot); - struct.files.add(_elem66); + _elem74 = new InputFile(); + _elem74.read(iprot); + struct.files.add(_elem74); } } struct.setFilesIsSet(true); @@ -1325,15 +1325,15 @@ public void read(org.apache.thrift.protocol.TProtocol prot, TExternalCompactionJ } if (incoming.get(8)) { { - org.apache.thrift.protocol.TMap _map68 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); - struct.overrides = new java.util.HashMap(2*_map68.size); - @org.apache.thrift.annotation.Nullable java.lang.String _key69; - @org.apache.thrift.annotation.Nullable java.lang.String _val70; - for (int _i71 = 0; _i71 < _map68.size; ++_i71) + org.apache.thrift.protocol.TMap _map76 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); + struct.overrides = new java.util.HashMap(2*_map76.size); + @org.apache.thrift.annotation.Nullable java.lang.String _key77; + @org.apache.thrift.annotation.Nullable java.lang.String _val78; + for (int _i79 = 0; _i79 < _map76.size; ++_i79) { - _key69 = iprot.readString(); - _val70 = iprot.readString(); - struct.overrides.put(_key69, _val70); + _key77 = iprot.readString(); + _val78 = iprot.readString(); + struct.overrides.put(_key77, _val78); } } struct.setOverridesIsSet(true); diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TIteratorSetting.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TIteratorSetting.java index cbf08413a56..bcfd60deb46 100644 --- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TIteratorSetting.java +++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TIteratorSetting.java @@ -614,15 +614,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, TIteratorSetting st case 4: // PROPERTIES if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map36 = iprot.readMapBegin(); - struct.properties = new java.util.HashMap(2*_map36.size); - @org.apache.thrift.annotation.Nullable java.lang.String _key37; - @org.apache.thrift.annotation.Nullable java.lang.String _val38; - for (int _i39 = 0; _i39 < _map36.size; ++_i39) + org.apache.thrift.protocol.TMap _map44 = iprot.readMapBegin(); + struct.properties = new java.util.HashMap(2*_map44.size); + @org.apache.thrift.annotation.Nullable java.lang.String _key45; + @org.apache.thrift.annotation.Nullable java.lang.String _val46; + for (int _i47 = 0; _i47 < _map44.size; ++_i47) { - _key37 = iprot.readString(); - _val38 = iprot.readString(); - struct.properties.put(_key37, _val38); + _key45 = iprot.readString(); + _val46 = iprot.readString(); + struct.properties.put(_key45, _val46); } iprot.readMapEnd(); } @@ -664,10 +664,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, TIteratorSetting s oprot.writeFieldBegin(PROPERTIES_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.properties.size())); - for (java.util.Map.Entry _iter40 : struct.properties.entrySet()) + for (java.util.Map.Entry _iter48 : struct.properties.entrySet()) { - oprot.writeString(_iter40.getKey()); - oprot.writeString(_iter40.getValue()); + oprot.writeString(_iter48.getKey()); + oprot.writeString(_iter48.getValue()); } oprot.writeMapEnd(); } @@ -717,10 +717,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, TIteratorSetting st if (struct.isSetProperties()) { { oprot.writeI32(struct.properties.size()); - for (java.util.Map.Entry _iter41 : struct.properties.entrySet()) + for (java.util.Map.Entry _iter49 : struct.properties.entrySet()) { - oprot.writeString(_iter41.getKey()); - oprot.writeString(_iter41.getValue()); + oprot.writeString(_iter49.getKey()); + oprot.writeString(_iter49.getValue()); } } } @@ -744,15 +744,15 @@ public void read(org.apache.thrift.protocol.TProtocol prot, TIteratorSetting str } if (incoming.get(3)) { { - org.apache.thrift.protocol.TMap _map42 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); - struct.properties = new java.util.HashMap(2*_map42.size); - @org.apache.thrift.annotation.Nullable java.lang.String _key43; - @org.apache.thrift.annotation.Nullable java.lang.String _val44; - for (int _i45 = 0; _i45 < _map42.size; ++_i45) + org.apache.thrift.protocol.TMap _map50 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); + struct.properties = new java.util.HashMap(2*_map50.size); + @org.apache.thrift.annotation.Nullable java.lang.String _key51; + @org.apache.thrift.annotation.Nullable java.lang.String _val52; + for (int _i53 = 0; _i53 < _map50.size; ++_i53) { - _key43 = iprot.readString(); - _val44 = iprot.readString(); - struct.properties.put(_key43, _val44); + _key51 = iprot.readString(); + _val52 = iprot.readString(); + struct.properties.put(_key51, _val52); } } struct.setPropertiesIsSet(true); diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TabletServerClientService.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TabletServerClientService.java index b9cb50df149..ad2a09b6c06 100644 --- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TabletServerClientService.java +++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TabletServerClientService.java @@ -5213,14 +5213,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getTabletStats_resu case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list72 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list72.size); - @org.apache.thrift.annotation.Nullable TabletStats _elem73; - for (int _i74 = 0; _i74 < _list72.size; ++_i74) + org.apache.thrift.protocol.TList _list80 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list80.size); + @org.apache.thrift.annotation.Nullable TabletStats _elem81; + for (int _i82 = 0; _i82 < _list80.size; ++_i82) { - _elem73 = new TabletStats(); - _elem73.read(iprot); - struct.success.add(_elem73); + _elem81 = new TabletStats(); + _elem81.read(iprot); + struct.success.add(_elem81); } iprot.readListEnd(); } @@ -5258,9 +5258,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, getTabletStats_res oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (TabletStats _iter75 : struct.success) + for (TabletStats _iter83 : struct.success) { - _iter75.write(oprot); + _iter83.write(oprot); } oprot.writeListEnd(); } @@ -5300,9 +5300,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, getTabletStats_resu if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (TabletStats _iter76 : struct.success) + for (TabletStats _iter84 : struct.success) { - _iter76.write(oprot); + _iter84.write(oprot); } } } @@ -5317,14 +5317,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, getTabletStats_resul java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list77 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList(_list77.size); - @org.apache.thrift.annotation.Nullable TabletStats _elem78; - for (int _i79 = 0; _i79 < _list77.size; ++_i79) + org.apache.thrift.protocol.TList _list85 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list85.size); + @org.apache.thrift.annotation.Nullable TabletStats _elem86; + for (int _i87 = 0; _i87 < _list85.size; ++_i87) { - _elem78 = new TabletStats(); - _elem78.read(iprot); - struct.success.add(_elem78); + _elem86 = new TabletStats(); + _elem86.read(iprot); + struct.success.add(_elem86); } } struct.setSuccessIsSet(true); @@ -8419,13 +8419,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, removeLogs_args str case 3: // FILENAMES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list80 = iprot.readListBegin(); - struct.filenames = new java.util.ArrayList(_list80.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem81; - for (int _i82 = 0; _i82 < _list80.size; ++_i82) + org.apache.thrift.protocol.TList _list88 = iprot.readListBegin(); + struct.filenames = new java.util.ArrayList(_list88.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem89; + for (int _i90 = 0; _i90 < _list88.size; ++_i90) { - _elem81 = iprot.readString(); - struct.filenames.add(_elem81); + _elem89 = iprot.readString(); + struct.filenames.add(_elem89); } iprot.readListEnd(); } @@ -8464,9 +8464,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, removeLogs_args st oprot.writeFieldBegin(FILENAMES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.filenames.size())); - for (java.lang.String _iter83 : struct.filenames) + for (java.lang.String _iter91 : struct.filenames) { - oprot.writeString(_iter83); + oprot.writeString(_iter91); } oprot.writeListEnd(); } @@ -8510,9 +8510,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, removeLogs_args str if (struct.isSetFilenames()) { { oprot.writeI32(struct.filenames.size()); - for (java.lang.String _iter84 : struct.filenames) + for (java.lang.String _iter92 : struct.filenames) { - oprot.writeString(_iter84); + oprot.writeString(_iter92); } } } @@ -8534,13 +8534,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, removeLogs_args stru } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list85 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.filenames = new java.util.ArrayList(_list85.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem86; - for (int _i87 = 0; _i87 < _list85.size; ++_i87) + org.apache.thrift.protocol.TList _list93 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.filenames = new java.util.ArrayList(_list93.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem94; + for (int _i95 = 0; _i95 < _list93.size; ++_i95) { - _elem86 = iprot.readString(); - struct.filenames.add(_elem86); + _elem94 = iprot.readString(); + struct.filenames.add(_elem94); } } struct.setFilenamesIsSet(true); @@ -9373,13 +9373,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getActiveLogs_resul 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(); } @@ -9408,9 +9408,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, getActiveLogs_resu 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(); } @@ -9442,9 +9442,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, getActiveLogs_resul 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); } } } @@ -9456,13 +9456,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, getActiveLogs_result java.util.BitSet incoming = iprot.readBitSet(1); 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); @@ -12586,26 +12586,26 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, startGetSummariesFr case 4: // FILES if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map96 = iprot.readMapBegin(); - struct.files = new java.util.HashMap>(2*_map96.size); - @org.apache.thrift.annotation.Nullable java.lang.String _key97; - @org.apache.thrift.annotation.Nullable java.util.List _val98; - for (int _i99 = 0; _i99 < _map96.size; ++_i99) + org.apache.thrift.protocol.TMap _map104 = iprot.readMapBegin(); + struct.files = new java.util.HashMap>(2*_map104.size); + @org.apache.thrift.annotation.Nullable java.lang.String _key105; + @org.apache.thrift.annotation.Nullable java.util.List _val106; + for (int _i107 = 0; _i107 < _map104.size; ++_i107) { - _key97 = iprot.readString(); + _key105 = iprot.readString(); { - org.apache.thrift.protocol.TList _list100 = iprot.readListBegin(); - _val98 = new java.util.ArrayList(_list100.size); - @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TRowRange _elem101; - for (int _i102 = 0; _i102 < _list100.size; ++_i102) + org.apache.thrift.protocol.TList _list108 = iprot.readListBegin(); + _val106 = new java.util.ArrayList(_list108.size); + @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TRowRange _elem109; + for (int _i110 = 0; _i110 < _list108.size; ++_i110) { - _elem101 = new org.apache.accumulo.core.dataImpl.thrift.TRowRange(); - _elem101.read(iprot); - _val98.add(_elem101); + _elem109 = new org.apache.accumulo.core.dataImpl.thrift.TRowRange(); + _elem109.read(iprot); + _val106.add(_elem109); } iprot.readListEnd(); } - struct.files.put(_key97, _val98); + struct.files.put(_key105, _val106); } iprot.readMapEnd(); } @@ -12649,14 +12649,14 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, startGetSummariesF oprot.writeFieldBegin(FILES_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST, struct.files.size())); - for (java.util.Map.Entry> _iter103 : struct.files.entrySet()) + for (java.util.Map.Entry> _iter111 : struct.files.entrySet()) { - oprot.writeString(_iter103.getKey()); + oprot.writeString(_iter111.getKey()); { - oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, _iter103.getValue().size())); - for (org.apache.accumulo.core.dataImpl.thrift.TRowRange _iter104 : _iter103.getValue()) + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, _iter111.getValue().size())); + for (org.apache.accumulo.core.dataImpl.thrift.TRowRange _iter112 : _iter111.getValue()) { - _iter104.write(oprot); + _iter112.write(oprot); } oprot.writeListEnd(); } @@ -12709,14 +12709,14 @@ public void write(org.apache.thrift.protocol.TProtocol prot, startGetSummariesFr if (struct.isSetFiles()) { { oprot.writeI32(struct.files.size()); - for (java.util.Map.Entry> _iter105 : struct.files.entrySet()) + for (java.util.Map.Entry> _iter113 : struct.files.entrySet()) { - oprot.writeString(_iter105.getKey()); + oprot.writeString(_iter113.getKey()); { - oprot.writeI32(_iter105.getValue().size()); - for (org.apache.accumulo.core.dataImpl.thrift.TRowRange _iter106 : _iter105.getValue()) + oprot.writeI32(_iter113.getValue().size()); + for (org.apache.accumulo.core.dataImpl.thrift.TRowRange _iter114 : _iter113.getValue()) { - _iter106.write(oprot); + _iter114.write(oprot); } } } @@ -12745,25 +12745,25 @@ public void read(org.apache.thrift.protocol.TProtocol prot, startGetSummariesFro } if (incoming.get(3)) { { - org.apache.thrift.protocol.TMap _map107 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST); - struct.files = new java.util.HashMap>(2*_map107.size); - @org.apache.thrift.annotation.Nullable java.lang.String _key108; - @org.apache.thrift.annotation.Nullable java.util.List _val109; - for (int _i110 = 0; _i110 < _map107.size; ++_i110) + org.apache.thrift.protocol.TMap _map115 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST); + struct.files = new java.util.HashMap>(2*_map115.size); + @org.apache.thrift.annotation.Nullable java.lang.String _key116; + @org.apache.thrift.annotation.Nullable java.util.List _val117; + for (int _i118 = 0; _i118 < _map115.size; ++_i118) { - _key108 = iprot.readString(); + _key116 = iprot.readString(); { - org.apache.thrift.protocol.TList _list111 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - _val109 = new java.util.ArrayList(_list111.size); - @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TRowRange _elem112; - for (int _i113 = 0; _i113 < _list111.size; ++_i113) + org.apache.thrift.protocol.TList _list119 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + _val117 = new java.util.ArrayList(_list119.size); + @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TRowRange _elem120; + for (int _i121 = 0; _i121 < _list119.size; ++_i121) { - _elem112 = new org.apache.accumulo.core.dataImpl.thrift.TRowRange(); - _elem112.read(iprot); - _val109.add(_elem112); + _elem120 = new org.apache.accumulo.core.dataImpl.thrift.TRowRange(); + _elem120.read(iprot); + _val117.add(_elem120); } } - struct.files.put(_key108, _val109); + struct.files.put(_key116, _val117); } } struct.setFilesIsSet(true); @@ -14758,14 +14758,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, refreshTablets_args case 3: // TABLETS_TO_REFRESH if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list114 = iprot.readListBegin(); - struct.tabletsToRefresh = 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) + org.apache.thrift.protocol.TList _list122 = iprot.readListBegin(); + struct.tabletsToRefresh = 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) { - _elem115 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(); - _elem115.read(iprot); - struct.tabletsToRefresh.add(_elem115); + _elem123 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(); + _elem123.read(iprot); + struct.tabletsToRefresh.add(_elem123); } iprot.readListEnd(); } @@ -14804,9 +14804,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, refreshTablets_arg oprot.writeFieldBegin(TABLETS_TO_REFRESH_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.tabletsToRefresh.size())); - for (org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _iter117 : struct.tabletsToRefresh) + for (org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _iter125 : struct.tabletsToRefresh) { - _iter117.write(oprot); + _iter125.write(oprot); } oprot.writeListEnd(); } @@ -14850,9 +14850,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, refreshTablets_args if (struct.isSetTabletsToRefresh()) { { oprot.writeI32(struct.tabletsToRefresh.size()); - for (org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _iter118 : struct.tabletsToRefresh) + for (org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _iter126 : struct.tabletsToRefresh) { - _iter118.write(oprot); + _iter126.write(oprot); } } } @@ -14874,14 +14874,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, refreshTablets_args } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list119 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.tabletsToRefresh = 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) + org.apache.thrift.protocol.TList _list127 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.tabletsToRefresh = 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) { - _elem120 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(); - _elem120.read(iprot); - struct.tabletsToRefresh.add(_elem120); + _elem128 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(); + _elem128.read(iprot); + struct.tabletsToRefresh.add(_elem128); } } struct.setTabletsToRefreshIsSet(true); @@ -15223,14 +15223,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, refreshTablets_resu 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) + org.apache.thrift.protocol.TList _list130 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list130.size); + @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _elem131; + for (int _i132 = 0; _i132 < _list130.size; ++_i132) { - _elem123 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(); - _elem123.read(iprot); - struct.success.add(_elem123); + _elem131 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(); + _elem131.read(iprot); + struct.success.add(_elem131); } iprot.readListEnd(); } @@ -15259,9 +15259,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, refreshTablets_res 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) + for (org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _iter133 : struct.success) { - _iter125.write(oprot); + _iter133.write(oprot); } oprot.writeListEnd(); } @@ -15293,9 +15293,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, refreshTablets_resu if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _iter126 : struct.success) + for (org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _iter134 : struct.success) { - _iter126.write(oprot); + _iter134.write(oprot); } } } @@ -15307,14 +15307,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, refreshTablets_resul java.util.BitSet incoming = iprot.readBitSet(1); 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) + org.apache.thrift.protocol.TList _list135 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list135.size); + @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _elem136; + for (int _i137 = 0; _i137 < _list135.size; ++_i137) { - _elem128 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(); - _elem128.read(iprot); - struct.success.add(_elem128); + _elem136 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(); + _elem136.read(iprot); + struct.success.add(_elem136); } } struct.setSuccessIsSet(true); diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tasks/thrift/Task.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tasks/thrift/Task.java new file mode 100644 index 00000000000..e14234aab76 --- /dev/null +++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tasks/thrift/Task.java @@ -0,0 +1,616 @@ +/* + * 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.tasks.thrift; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) +public class Task 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("Task"); + + private static final org.apache.thrift.protocol.TField TASK_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("taskId", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField MESSAGE_TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("messageType", org.apache.thrift.protocol.TType.STRING, (short)2); + private static final org.apache.thrift.protocol.TField MESSAGE_FIELD_DESC = new org.apache.thrift.protocol.TField("message", org.apache.thrift.protocol.TType.STRING, (short)3); + + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new TaskStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new TaskTupleSchemeFactory(); + + public @org.apache.thrift.annotation.Nullable java.lang.String taskId; // required + public @org.apache.thrift.annotation.Nullable java.lang.String messageType; // required + public @org.apache.thrift.annotation.Nullable java.lang.String message; // 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 { + TASK_ID((short)1, "taskId"), + MESSAGE_TYPE((short)2, "messageType"), + MESSAGE((short)3, "message"); + + 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: // TASK_ID + return TASK_ID; + case 2: // MESSAGE_TYPE + return MESSAGE_TYPE; + case 3: // MESSAGE + return MESSAGE; + 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.TASK_ID, new org.apache.thrift.meta_data.FieldMetaData("taskId", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.MESSAGE_TYPE, new org.apache.thrift.meta_data.FieldMetaData("messageType", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.MESSAGE, new org.apache.thrift.meta_data.FieldMetaData("message", 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(Task.class, metaDataMap); + } + + public Task() { + } + + public Task( + java.lang.String taskId, + java.lang.String messageType, + java.lang.String message) + { + this(); + this.taskId = taskId; + this.messageType = messageType; + this.message = message; + } + + /** + * Performs a deep copy on other. + */ + public Task(Task other) { + if (other.isSetTaskId()) { + this.taskId = other.taskId; + } + if (other.isSetMessageType()) { + this.messageType = other.messageType; + } + if (other.isSetMessage()) { + this.message = other.message; + } + } + + @Override + public Task deepCopy() { + return new Task(this); + } + + @Override + public void clear() { + this.taskId = null; + this.messageType = null; + this.message = null; + } + + @org.apache.thrift.annotation.Nullable + public java.lang.String getTaskId() { + return this.taskId; + } + + public Task setTaskId(@org.apache.thrift.annotation.Nullable java.lang.String taskId) { + this.taskId = taskId; + return this; + } + + public void unsetTaskId() { + this.taskId = null; + } + + /** Returns true if field taskId is set (has been assigned a value) and false otherwise */ + public boolean isSetTaskId() { + return this.taskId != null; + } + + public void setTaskIdIsSet(boolean value) { + if (!value) { + this.taskId = null; + } + } + + @org.apache.thrift.annotation.Nullable + public java.lang.String getMessageType() { + return this.messageType; + } + + public Task setMessageType(@org.apache.thrift.annotation.Nullable java.lang.String messageType) { + this.messageType = messageType; + return this; + } + + public void unsetMessageType() { + this.messageType = null; + } + + /** Returns true if field messageType is set (has been assigned a value) and false otherwise */ + public boolean isSetMessageType() { + return this.messageType != null; + } + + public void setMessageTypeIsSet(boolean value) { + if (!value) { + this.messageType = null; + } + } + + @org.apache.thrift.annotation.Nullable + public java.lang.String getMessage() { + return this.message; + } + + public Task setMessage(@org.apache.thrift.annotation.Nullable java.lang.String message) { + this.message = message; + return this; + } + + public void unsetMessage() { + this.message = null; + } + + /** Returns true if field message is set (has been assigned a value) and false otherwise */ + public boolean isSetMessage() { + return this.message != null; + } + + public void setMessageIsSet(boolean value) { + if (!value) { + this.message = null; + } + } + + @Override + public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) { + switch (field) { + case TASK_ID: + if (value == null) { + unsetTaskId(); + } else { + setTaskId((java.lang.String)value); + } + break; + + case MESSAGE_TYPE: + if (value == null) { + unsetMessageType(); + } else { + setMessageType((java.lang.String)value); + } + break; + + case MESSAGE: + if (value == null) { + unsetMessage(); + } else { + setMessage((java.lang.String)value); + } + break; + + } + } + + @org.apache.thrift.annotation.Nullable + @Override + public java.lang.Object getFieldValue(_Fields field) { + switch (field) { + case TASK_ID: + return getTaskId(); + + case MESSAGE_TYPE: + return getMessageType(); + + case MESSAGE: + return getMessage(); + + } + 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 TASK_ID: + return isSetTaskId(); + case MESSAGE_TYPE: + return isSetMessageType(); + case MESSAGE: + return isSetMessage(); + } + throw new java.lang.IllegalStateException(); + } + + @Override + public boolean equals(java.lang.Object that) { + if (that instanceof Task) + return this.equals((Task)that); + return false; + } + + public boolean equals(Task that) { + if (that == null) + return false; + if (this == that) + return true; + + boolean this_present_taskId = true && this.isSetTaskId(); + boolean that_present_taskId = true && that.isSetTaskId(); + if (this_present_taskId || that_present_taskId) { + if (!(this_present_taskId && that_present_taskId)) + return false; + if (!this.taskId.equals(that.taskId)) + return false; + } + + boolean this_present_messageType = true && this.isSetMessageType(); + boolean that_present_messageType = true && that.isSetMessageType(); + if (this_present_messageType || that_present_messageType) { + if (!(this_present_messageType && that_present_messageType)) + return false; + if (!this.messageType.equals(that.messageType)) + return false; + } + + boolean this_present_message = true && this.isSetMessage(); + boolean that_present_message = true && that.isSetMessage(); + if (this_present_message || that_present_message) { + if (!(this_present_message && that_present_message)) + return false; + if (!this.message.equals(that.message)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + int hashCode = 1; + + hashCode = hashCode * 8191 + ((isSetTaskId()) ? 131071 : 524287); + if (isSetTaskId()) + hashCode = hashCode * 8191 + taskId.hashCode(); + + hashCode = hashCode * 8191 + ((isSetMessageType()) ? 131071 : 524287); + if (isSetMessageType()) + hashCode = hashCode * 8191 + messageType.hashCode(); + + hashCode = hashCode * 8191 + ((isSetMessage()) ? 131071 : 524287); + if (isSetMessage()) + hashCode = hashCode * 8191 + message.hashCode(); + + return hashCode; + } + + @Override + public int compareTo(Task other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = java.lang.Boolean.compare(isSetTaskId(), other.isSetTaskId()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTaskId()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.taskId, other.taskId); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = java.lang.Boolean.compare(isSetMessageType(), other.isSetMessageType()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetMessageType()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.messageType, other.messageType); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = java.lang.Boolean.compare(isSetMessage(), other.isSetMessage()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetMessage()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.message, other.message); + 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("Task("); + boolean first = true; + + sb.append("taskId:"); + if (this.taskId == null) { + sb.append("null"); + } else { + sb.append(this.taskId); + } + first = false; + if (!first) sb.append(", "); + sb.append("messageType:"); + if (this.messageType == null) { + sb.append("null"); + } else { + sb.append(this.messageType); + } + first = false; + if (!first) sb.append(", "); + sb.append("message:"); + if (this.message == null) { + sb.append("null"); + } else { + sb.append(this.message); + } + 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 TaskStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + @Override + public TaskStandardScheme getScheme() { + return new TaskStandardScheme(); + } + } + + private static class TaskStandardScheme extends org.apache.thrift.scheme.StandardScheme { + + @Override + public void read(org.apache.thrift.protocol.TProtocol iprot, Task 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: // TASK_ID + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.taskId = iprot.readString(); + struct.setTaskIdIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // MESSAGE_TYPE + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.messageType = iprot.readString(); + struct.setMessageTypeIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // MESSAGE + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.message = iprot.readString(); + struct.setMessageIsSet(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, Task struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.taskId != null) { + oprot.writeFieldBegin(TASK_ID_FIELD_DESC); + oprot.writeString(struct.taskId); + oprot.writeFieldEnd(); + } + if (struct.messageType != null) { + oprot.writeFieldBegin(MESSAGE_TYPE_FIELD_DESC); + oprot.writeString(struct.messageType); + oprot.writeFieldEnd(); + } + if (struct.message != null) { + oprot.writeFieldBegin(MESSAGE_FIELD_DESC); + oprot.writeString(struct.message); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TaskTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + @Override + public TaskTupleScheme getScheme() { + return new TaskTupleScheme(); + } + } + + private static class TaskTupleScheme extends org.apache.thrift.scheme.TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, Task 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.isSetTaskId()) { + optionals.set(0); + } + if (struct.isSetMessageType()) { + optionals.set(1); + } + if (struct.isSetMessage()) { + optionals.set(2); + } + oprot.writeBitSet(optionals, 3); + if (struct.isSetTaskId()) { + oprot.writeString(struct.taskId); + } + if (struct.isSetMessageType()) { + oprot.writeString(struct.messageType); + } + if (struct.isSetMessage()) { + oprot.writeString(struct.message); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, Task 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.taskId = iprot.readString(); + struct.setTaskIdIsSet(true); + } + if (incoming.get(1)) { + struct.messageType = iprot.readString(); + struct.setMessageTypeIsSet(true); + } + if (incoming.get(2)) { + struct.message = iprot.readString(); + struct.setMessageIsSet(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/compaction/thrift/CompactionCoordinatorService.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tasks/thrift/TaskManager.java similarity index 64% rename from core/src/main/thrift-gen-java/org/apache/accumulo/core/compaction/thrift/CompactionCoordinatorService.java rename to core/src/main/thrift-gen-java/org/apache/accumulo/core/tasks/thrift/TaskManager.java index c7da84ef017..11875ff9043 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/tasks/thrift/TaskManager.java @@ -22,44 +22,44 @@ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING * @generated */ -package org.apache.accumulo.core.compaction.thrift; +package org.apache.accumulo.core.tasks.thrift; @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) -public class CompactionCoordinatorService { +public class TaskManager { public interface Iface { - public void compactionCompleted(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats) throws org.apache.thrift.TException; + public Task getTask(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TaskRunnerInfo taskRunner, java.lang.String taskID) throws org.apache.thrift.TException; - public org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob getCompactionJob(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String groupName, java.lang.String compactor, java.lang.String externalCompactionId) throws org.apache.thrift.TException; + public void taskStatus(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long timestamp, Task taskUpdateObject) throws org.apache.thrift.TException; - public void updateCompactionStatus(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, TCompactionStatusUpdate status, long timestamp) throws org.apache.thrift.TException; + public void taskCompleted(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, Task task) throws org.apache.thrift.TException; - public void compactionFailed(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) throws org.apache.thrift.TException; + public void taskFailed(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, Task task) throws org.apache.thrift.TException; - public TExternalCompactionList getRunningCompactions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException; + public void cancelTask(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String taskID) throws org.apache.thrift.TException; - public TExternalCompactionList getCompletedCompactions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException; + public Task getRunningTasks(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException; - 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.thrift.TException; + public Task getCompletedTasks(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException; } public interface AsyncIface { - public void compactionCompleted(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void getTask(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TaskRunnerInfo taskRunner, java.lang.String taskID, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void getCompactionJob(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String groupName, java.lang.String compactor, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void taskStatus(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long timestamp, Task taskUpdateObject, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void updateCompactionStatus(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, TCompactionStatusUpdate status, long timestamp, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void taskCompleted(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, Task task, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void compactionFailed(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void taskFailed(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, Task task, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void getRunningCompactions(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 cancelTask(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String taskID, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void getCompletedCompactions(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 getRunningTasks(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 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 getCompletedTasks(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; } @@ -86,178 +86,165 @@ public Client(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.prot } @Override - public void compactionCompleted(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats) throws org.apache.thrift.TException + public Task getTask(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TaskRunnerInfo taskRunner, java.lang.String taskID) throws org.apache.thrift.TException { - send_compactionCompleted(tinfo, credentials, externalCompactionId, extent, stats); - recv_compactionCompleted(); + send_getTask(tinfo, credentials, taskRunner, taskID); + return recv_getTask(); } - public void send_compactionCompleted(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats) throws org.apache.thrift.TException + public void send_getTask(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TaskRunnerInfo taskRunner, java.lang.String taskID) throws org.apache.thrift.TException { - compactionCompleted_args args = new compactionCompleted_args(); + getTask_args args = new getTask_args(); args.setTinfo(tinfo); args.setCredentials(credentials); - args.setExternalCompactionId(externalCompactionId); - args.setExtent(extent); - args.setStats(stats); - sendBase("compactionCompleted", args); + args.setTaskRunner(taskRunner); + args.setTaskID(taskID); + sendBase("getTask", args); } - public void recv_compactionCompleted() throws org.apache.thrift.TException + public Task recv_getTask() throws org.apache.thrift.TException { - compactionCompleted_result result = new compactionCompleted_result(); - receiveBase(result, "compactionCompleted"); - return; + getTask_result result = new getTask_result(); + receiveBase(result, "getTask"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getTask failed: unknown result"); } @Override - public org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob getCompactionJob(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String groupName, java.lang.String compactor, java.lang.String externalCompactionId) throws org.apache.thrift.TException + public void taskStatus(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long timestamp, Task taskUpdateObject) throws org.apache.thrift.TException { - send_getCompactionJob(tinfo, credentials, groupName, compactor, externalCompactionId); - return recv_getCompactionJob(); + send_taskStatus(tinfo, credentials, timestamp, taskUpdateObject); } - public void send_getCompactionJob(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String groupName, java.lang.String compactor, java.lang.String externalCompactionId) throws org.apache.thrift.TException + public void send_taskStatus(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long timestamp, Task taskUpdateObject) throws org.apache.thrift.TException { - getCompactionJob_args args = new getCompactionJob_args(); + taskStatus_args args = new taskStatus_args(); args.setTinfo(tinfo); args.setCredentials(credentials); - args.setGroupName(groupName); - args.setCompactor(compactor); - args.setExternalCompactionId(externalCompactionId); - sendBase("getCompactionJob", args); - } - - public org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob recv_getCompactionJob() throws org.apache.thrift.TException - { - getCompactionJob_result result = new getCompactionJob_result(); - receiveBase(result, "getCompactionJob"); - if (result.isSetSuccess()) { - return result.success; - } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getCompactionJob failed: unknown result"); + args.setTimestamp(timestamp); + args.setTaskUpdateObject(taskUpdateObject); + sendBaseOneway("taskStatus", args); } @Override - public void updateCompactionStatus(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, TCompactionStatusUpdate status, long timestamp) throws org.apache.thrift.TException + public void taskCompleted(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, Task task) throws org.apache.thrift.TException { - send_updateCompactionStatus(tinfo, credentials, externalCompactionId, status, timestamp); - recv_updateCompactionStatus(); + send_taskCompleted(tinfo, credentials, task); + recv_taskCompleted(); } - public void send_updateCompactionStatus(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, TCompactionStatusUpdate status, long timestamp) throws org.apache.thrift.TException + public void send_taskCompleted(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, Task task) throws org.apache.thrift.TException { - updateCompactionStatus_args args = new updateCompactionStatus_args(); + taskCompleted_args args = new taskCompleted_args(); args.setTinfo(tinfo); args.setCredentials(credentials); - args.setExternalCompactionId(externalCompactionId); - args.setStatus(status); - args.setTimestamp(timestamp); - sendBase("updateCompactionStatus", args); + args.setTask(task); + sendBase("taskCompleted", args); } - public void recv_updateCompactionStatus() throws org.apache.thrift.TException + public void recv_taskCompleted() throws org.apache.thrift.TException { - updateCompactionStatus_result result = new updateCompactionStatus_result(); - receiveBase(result, "updateCompactionStatus"); + taskCompleted_result result = new taskCompleted_result(); + receiveBase(result, "taskCompleted"); return; } @Override - public void compactionFailed(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) throws org.apache.thrift.TException + public void taskFailed(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, Task task) throws org.apache.thrift.TException { - send_compactionFailed(tinfo, credentials, externalCompactionId, extent); - recv_compactionFailed(); + send_taskFailed(tinfo, credentials, task); + recv_taskFailed(); } - public void send_compactionFailed(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) throws org.apache.thrift.TException + public void send_taskFailed(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, Task task) throws org.apache.thrift.TException { - compactionFailed_args args = new compactionFailed_args(); + taskFailed_args args = new taskFailed_args(); args.setTinfo(tinfo); args.setCredentials(credentials); - args.setExternalCompactionId(externalCompactionId); - args.setExtent(extent); - sendBase("compactionFailed", args); + args.setTask(task); + sendBase("taskFailed", args); } - public void recv_compactionFailed() throws org.apache.thrift.TException + public void recv_taskFailed() throws org.apache.thrift.TException { - compactionFailed_result result = new compactionFailed_result(); - receiveBase(result, "compactionFailed"); + taskFailed_result result = new taskFailed_result(); + receiveBase(result, "taskFailed"); return; } @Override - public TExternalCompactionList getRunningCompactions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException + public void cancelTask(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String taskID) throws org.apache.thrift.TException { - send_getRunningCompactions(tinfo, credentials); - return recv_getRunningCompactions(); + send_cancelTask(tinfo, credentials, taskID); + recv_cancelTask(); } - public void send_getRunningCompactions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException + public void send_cancelTask(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String taskID) throws org.apache.thrift.TException { - getRunningCompactions_args args = new getRunningCompactions_args(); + cancelTask_args args = new cancelTask_args(); args.setTinfo(tinfo); args.setCredentials(credentials); - sendBase("getRunningCompactions", args); + args.setTaskID(taskID); + sendBase("cancelTask", args); } - public TExternalCompactionList recv_getRunningCompactions() throws org.apache.thrift.TException + public void recv_cancelTask() throws org.apache.thrift.TException { - getRunningCompactions_result result = new getRunningCompactions_result(); - receiveBase(result, "getRunningCompactions"); - if (result.isSetSuccess()) { - return result.success; - } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getRunningCompactions failed: unknown result"); + cancelTask_result result = new cancelTask_result(); + receiveBase(result, "cancelTask"); + return; } @Override - public TExternalCompactionList getCompletedCompactions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException + public Task getRunningTasks(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException { - send_getCompletedCompactions(tinfo, credentials); - return recv_getCompletedCompactions(); + send_getRunningTasks(tinfo, credentials); + return recv_getRunningTasks(); } - public void send_getCompletedCompactions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException + public void send_getRunningTasks(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException { - getCompletedCompactions_args args = new getCompletedCompactions_args(); + getRunningTasks_args args = new getRunningTasks_args(); args.setTinfo(tinfo); args.setCredentials(credentials); - sendBase("getCompletedCompactions", args); + sendBase("getRunningTasks", args); } - public TExternalCompactionList recv_getCompletedCompactions() throws org.apache.thrift.TException + public Task recv_getRunningTasks() throws org.apache.thrift.TException { - getCompletedCompactions_result result = new getCompletedCompactions_result(); - receiveBase(result, "getCompletedCompactions"); + getRunningTasks_result result = new getRunningTasks_result(); + receiveBase(result, "getRunningTasks"); if (result.isSetSuccess()) { return result.success; } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getCompletedCompactions failed: unknown result"); + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getRunningTasks failed: unknown result"); } @Override - 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.thrift.TException + public Task getCompletedTasks(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException { - send_cancel(tinfo, credentials, externalCompactionId); - recv_cancel(); + send_getCompletedTasks(tinfo, credentials); + return recv_getCompletedTasks(); } - public void send_cancel(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 void send_getCompletedTasks(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException { - cancel_args args = new cancel_args(); + getCompletedTasks_args args = new getCompletedTasks_args(); args.setTinfo(tinfo); args.setCredentials(credentials); - args.setExternalCompactionId(externalCompactionId); - sendBase("cancel", args); + sendBase("getCompletedTasks", args); } - public void recv_cancel() throws org.apache.thrift.TException + public Task recv_getCompletedTasks() throws org.apache.thrift.TException { - cancel_result result = new cancel_result(); - receiveBase(result, "cancel"); - return; + getCompletedTasks_result result = new getCompletedTasks_result(); + receiveBase(result, "getCompletedTasks"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getCompletedTasks failed: unknown result"); } } @@ -280,132 +267,119 @@ public AsyncClient(org.apache.thrift.protocol.TProtocolFactory protocolFactory, } @Override - public void compactionCompleted(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void getTask(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TaskRunnerInfo taskRunner, java.lang.String taskID, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); - compactionCompleted_call method_call = new compactionCompleted_call(tinfo, credentials, externalCompactionId, extent, stats, resultHandler, this, ___protocolFactory, ___transport); + getTask_call method_call = new getTask_call(tinfo, credentials, taskRunner, taskID, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; ___manager.call(method_call); } - public static class compactionCompleted_call extends org.apache.thrift.async.TAsyncMethodCall { + public static class getTask_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; - private org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent; - private org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats; - public compactionCompleted_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats, 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 { + private TaskRunnerInfo taskRunner; + private java.lang.String taskID; + public getTask_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TaskRunnerInfo taskRunner, java.lang.String taskID, 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.externalCompactionId = externalCompactionId; - this.extent = extent; - this.stats = stats; + this.taskRunner = taskRunner; + this.taskID = taskID; } @Override public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("compactionCompleted", org.apache.thrift.protocol.TMessageType.CALL, 0)); - compactionCompleted_args args = new compactionCompleted_args(); + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getTask", org.apache.thrift.protocol.TMessageType.CALL, 0)); + getTask_args args = new getTask_args(); args.setTinfo(tinfo); args.setCredentials(credentials); - args.setExternalCompactionId(externalCompactionId); - args.setExtent(extent); - args.setStats(stats); + args.setTaskRunner(taskRunner); + args.setTaskID(taskID); args.write(prot); prot.writeMessageEnd(); } @Override - public Void getResult() throws org.apache.thrift.TException { + public Task 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_compactionCompleted(); - return null; + return (new Client(prot)).recv_getTask(); } } @Override - public void getCompactionJob(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String groupName, java.lang.String compactor, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void taskStatus(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long timestamp, Task taskUpdateObject, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); - getCompactionJob_call method_call = new getCompactionJob_call(tinfo, credentials, groupName, compactor, externalCompactionId, resultHandler, this, ___protocolFactory, ___transport); + taskStatus_call method_call = new taskStatus_call(tinfo, credentials, timestamp, taskUpdateObject, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; ___manager.call(method_call); } - public static class getCompactionJob_call extends org.apache.thrift.async.TAsyncMethodCall { + public static class taskStatus_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 groupName; - private java.lang.String compactor; - private java.lang.String externalCompactionId; - public getCompactionJob_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String groupName, java.lang.String compactor, 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, false); + private long timestamp; + private Task taskUpdateObject; + public taskStatus_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long timestamp, Task taskUpdateObject, 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.groupName = groupName; - this.compactor = compactor; - this.externalCompactionId = externalCompactionId; + this.timestamp = timestamp; + this.taskUpdateObject = taskUpdateObject; } @Override public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getCompactionJob", org.apache.thrift.protocol.TMessageType.CALL, 0)); - getCompactionJob_args args = new getCompactionJob_args(); + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("taskStatus", org.apache.thrift.protocol.TMessageType.ONEWAY, 0)); + taskStatus_args args = new taskStatus_args(); args.setTinfo(tinfo); args.setCredentials(credentials); - args.setGroupName(groupName); - args.setCompactor(compactor); - args.setExternalCompactionId(externalCompactionId); + args.setTimestamp(timestamp); + args.setTaskUpdateObject(taskUpdateObject); args.write(prot); prot.writeMessageEnd(); } @Override - public org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob getResult() throws org.apache.thrift.TException { + 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 (new Client(prot)).recv_getCompactionJob(); + return null; } } @Override - public void updateCompactionStatus(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, TCompactionStatusUpdate status, long timestamp, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void taskCompleted(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, Task task, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); - updateCompactionStatus_call method_call = new updateCompactionStatus_call(tinfo, credentials, externalCompactionId, status, timestamp, resultHandler, this, ___protocolFactory, ___transport); + taskCompleted_call method_call = new taskCompleted_call(tinfo, credentials, task, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; ___manager.call(method_call); } - public static class updateCompactionStatus_call extends org.apache.thrift.async.TAsyncMethodCall { + public static class taskCompleted_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; - private TCompactionStatusUpdate status; - private long timestamp; - public updateCompactionStatus_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, TCompactionStatusUpdate status, long timestamp, 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 { + private Task task; + public taskCompleted_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, Task task, 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.externalCompactionId = externalCompactionId; - this.status = status; - this.timestamp = timestamp; + this.task = task; } @Override public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("updateCompactionStatus", org.apache.thrift.protocol.TMessageType.CALL, 0)); - updateCompactionStatus_args args = new updateCompactionStatus_args(); + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("taskCompleted", org.apache.thrift.protocol.TMessageType.CALL, 0)); + taskCompleted_args args = new taskCompleted_args(); args.setTinfo(tinfo); args.setCredentials(credentials); - args.setExternalCompactionId(externalCompactionId); - args.setStatus(status); - args.setTimestamp(timestamp); + args.setTask(task); args.write(prot); prot.writeMessageEnd(); } @@ -417,40 +391,37 @@ public Void getResult() throws org.apache.thrift.TException { } 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_updateCompactionStatus(); + (new Client(prot)).recv_taskCompleted(); return null; } } @Override - public void compactionFailed(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void taskFailed(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, Task task, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); - compactionFailed_call method_call = new compactionFailed_call(tinfo, credentials, externalCompactionId, extent, resultHandler, this, ___protocolFactory, ___transport); + taskFailed_call method_call = new taskFailed_call(tinfo, credentials, task, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; ___manager.call(method_call); } - public static class compactionFailed_call extends org.apache.thrift.async.TAsyncMethodCall { + public static class taskFailed_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; - private org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent; - public compactionFailed_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, 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 { + private Task task; + public taskFailed_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, Task task, 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.externalCompactionId = externalCompactionId; - this.extent = extent; + this.task = task; } @Override public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("compactionFailed", org.apache.thrift.protocol.TMessageType.CALL, 0)); - compactionFailed_args args = new compactionFailed_args(); + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("taskFailed", org.apache.thrift.protocol.TMessageType.CALL, 0)); + taskFailed_args args = new taskFailed_args(); args.setTinfo(tinfo); args.setCredentials(credentials); - args.setExternalCompactionId(externalCompactionId); - args.setExtent(extent); + args.setTask(task); args.write(prot); prot.writeMessageEnd(); } @@ -462,61 +433,65 @@ public Void getResult() throws org.apache.thrift.TException { } 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_compactionFailed(); + (new Client(prot)).recv_taskFailed(); return null; } } @Override - public void getRunningCompactions(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 cancelTask(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String taskID, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); - getRunningCompactions_call method_call = new getRunningCompactions_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport); + cancelTask_call method_call = new cancelTask_call(tinfo, credentials, taskID, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; ___manager.call(method_call); } - public static class getRunningCompactions_call extends org.apache.thrift.async.TAsyncMethodCall { + public static class cancelTask_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 getRunningCompactions_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 { + private java.lang.String taskID; + public cancelTask_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String taskID, 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.taskID = taskID; } @Override public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getRunningCompactions", org.apache.thrift.protocol.TMessageType.CALL, 0)); - getRunningCompactions_args args = new getRunningCompactions_args(); + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("cancelTask", org.apache.thrift.protocol.TMessageType.CALL, 0)); + cancelTask_args args = new cancelTask_args(); args.setTinfo(tinfo); args.setCredentials(credentials); + args.setTaskID(taskID); args.write(prot); prot.writeMessageEnd(); } @Override - public TExternalCompactionList getResult() throws org.apache.thrift.TException { + 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 (new Client(prot)).recv_getRunningCompactions(); + (new Client(prot)).recv_cancelTask(); + return null; } } @Override - public void getCompletedCompactions(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 getRunningTasks(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(); - getCompletedCompactions_call method_call = new getCompletedCompactions_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport); + getRunningTasks_call method_call = new getRunningTasks_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; ___manager.call(method_call); } - public static class getCompletedCompactions_call extends org.apache.thrift.async.TAsyncMethodCall { + public static class getRunningTasks_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 getCompletedCompactions_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 { + public getRunningTasks_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; @@ -524,8 +499,8 @@ public getCompletedCompactions_call(org.apache.accumulo.core.clientImpl.thrift.T @Override public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getCompletedCompactions", org.apache.thrift.protocol.TMessageType.CALL, 0)); - getCompletedCompactions_args args = new getCompletedCompactions_args(); + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getRunningTasks", org.apache.thrift.protocol.TMessageType.CALL, 0)); + getRunningTasks_args args = new getRunningTasks_args(); args.setTinfo(tinfo); args.setCredentials(credentials); args.write(prot); @@ -533,55 +508,51 @@ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apa } @Override - public TExternalCompactionList getResult() throws org.apache.thrift.TException { + public Task 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 (new Client(prot)).recv_getCompletedCompactions(); + return (new Client(prot)).recv_getRunningTasks(); } } @Override - 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 getCompletedTasks(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(); - cancel_call method_call = new cancel_call(tinfo, credentials, externalCompactionId, resultHandler, this, ___protocolFactory, ___transport); + getCompletedTasks_call method_call = new getCompletedTasks_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; ___manager.call(method_call); } - public static class cancel_call extends org.apache.thrift.async.TAsyncMethodCall { + public static class getCompletedTasks_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 cancel_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 { + public getCompletedTasks_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; - 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("cancel", org.apache.thrift.protocol.TMessageType.CALL, 0)); - cancel_args args = new cancel_args(); + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getCompletedTasks", org.apache.thrift.protocol.TMessageType.CALL, 0)); + getCompletedTasks_args args = new getCompletedTasks_args(); args.setTinfo(tinfo); args.setCredentials(credentials); - args.setExternalCompactionId(externalCompactionId); args.write(prot); prot.writeMessageEnd(); } @Override - public Void getResult() throws org.apache.thrift.TException { + public Task 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_cancel(); - return null; + return (new Client(prot)).recv_getCompletedTasks(); } } @@ -598,24 +569,24 @@ protected Processor(I iface, java.util.Map java.util.Map> getProcessMap(java.util.Map> processMap) { - processMap.put("compactionCompleted", new compactionCompleted()); - processMap.put("getCompactionJob", new getCompactionJob()); - processMap.put("updateCompactionStatus", new updateCompactionStatus()); - processMap.put("compactionFailed", new compactionFailed()); - processMap.put("getRunningCompactions", new getRunningCompactions()); - processMap.put("getCompletedCompactions", new getCompletedCompactions()); - processMap.put("cancel", new cancel()); + processMap.put("getTask", new getTask()); + processMap.put("taskStatus", new taskStatus()); + processMap.put("taskCompleted", new taskCompleted()); + processMap.put("taskFailed", new taskFailed()); + processMap.put("cancelTask", new cancelTask()); + processMap.put("getRunningTasks", new getRunningTasks()); + processMap.put("getCompletedTasks", new getCompletedTasks()); return processMap; } - public static class compactionCompleted extends org.apache.thrift.ProcessFunction { - public compactionCompleted() { - super("compactionCompleted"); + public static class getTask extends org.apache.thrift.ProcessFunction { + public getTask() { + super("getTask"); } @Override - public compactionCompleted_args getEmptyArgsInstance() { - return new compactionCompleted_args(); + public getTask_args getEmptyArgsInstance() { + return new getTask_args(); } @Override @@ -629,26 +600,26 @@ protected boolean rethrowUnhandledExceptions() { } @Override - public compactionCompleted_result getResult(I iface, compactionCompleted_args args) throws org.apache.thrift.TException { - compactionCompleted_result result = new compactionCompleted_result(); - iface.compactionCompleted(args.tinfo, args.credentials, args.externalCompactionId, args.extent, args.stats); + public getTask_result getResult(I iface, getTask_args args) throws org.apache.thrift.TException { + getTask_result result = new getTask_result(); + result.success = iface.getTask(args.tinfo, args.credentials, args.taskRunner, args.taskID); return result; } } - public static class getCompactionJob extends org.apache.thrift.ProcessFunction { - public getCompactionJob() { - super("getCompactionJob"); + public static class taskStatus extends org.apache.thrift.ProcessFunction { + public taskStatus() { + super("taskStatus"); } @Override - public getCompactionJob_args getEmptyArgsInstance() { - return new getCompactionJob_args(); + public taskStatus_args getEmptyArgsInstance() { + return new taskStatus_args(); } @Override protected boolean isOneway() { - return false; + return true; } @Override @@ -657,21 +628,20 @@ protected boolean rethrowUnhandledExceptions() { } @Override - public getCompactionJob_result getResult(I iface, getCompactionJob_args args) throws org.apache.thrift.TException { - getCompactionJob_result result = new getCompactionJob_result(); - result.success = iface.getCompactionJob(args.tinfo, args.credentials, args.groupName, args.compactor, args.externalCompactionId); - return result; + public org.apache.thrift.TBase getResult(I iface, taskStatus_args args) throws org.apache.thrift.TException { + iface.taskStatus(args.tinfo, args.credentials, args.timestamp, args.taskUpdateObject); + return null; } } - public static class updateCompactionStatus extends org.apache.thrift.ProcessFunction { - public updateCompactionStatus() { - super("updateCompactionStatus"); + public static class taskCompleted extends org.apache.thrift.ProcessFunction { + public taskCompleted() { + super("taskCompleted"); } @Override - public updateCompactionStatus_args getEmptyArgsInstance() { - return new updateCompactionStatus_args(); + public taskCompleted_args getEmptyArgsInstance() { + return new taskCompleted_args(); } @Override @@ -685,21 +655,21 @@ protected boolean rethrowUnhandledExceptions() { } @Override - public updateCompactionStatus_result getResult(I iface, updateCompactionStatus_args args) throws org.apache.thrift.TException { - updateCompactionStatus_result result = new updateCompactionStatus_result(); - iface.updateCompactionStatus(args.tinfo, args.credentials, args.externalCompactionId, args.status, args.timestamp); + public taskCompleted_result getResult(I iface, taskCompleted_args args) throws org.apache.thrift.TException { + taskCompleted_result result = new taskCompleted_result(); + iface.taskCompleted(args.tinfo, args.credentials, args.task); return result; } } - public static class compactionFailed extends org.apache.thrift.ProcessFunction { - public compactionFailed() { - super("compactionFailed"); + public static class taskFailed extends org.apache.thrift.ProcessFunction { + public taskFailed() { + super("taskFailed"); } @Override - public compactionFailed_args getEmptyArgsInstance() { - return new compactionFailed_args(); + public taskFailed_args getEmptyArgsInstance() { + return new taskFailed_args(); } @Override @@ -713,21 +683,21 @@ protected boolean rethrowUnhandledExceptions() { } @Override - public compactionFailed_result getResult(I iface, compactionFailed_args args) throws org.apache.thrift.TException { - compactionFailed_result result = new compactionFailed_result(); - iface.compactionFailed(args.tinfo, args.credentials, args.externalCompactionId, args.extent); + public taskFailed_result getResult(I iface, taskFailed_args args) throws org.apache.thrift.TException { + taskFailed_result result = new taskFailed_result(); + iface.taskFailed(args.tinfo, args.credentials, args.task); return result; } } - public static class getRunningCompactions extends org.apache.thrift.ProcessFunction { - public getRunningCompactions() { - super("getRunningCompactions"); + public static class cancelTask extends org.apache.thrift.ProcessFunction { + public cancelTask() { + super("cancelTask"); } @Override - public getRunningCompactions_args getEmptyArgsInstance() { - return new getRunningCompactions_args(); + public cancelTask_args getEmptyArgsInstance() { + return new cancelTask_args(); } @Override @@ -741,21 +711,21 @@ protected boolean rethrowUnhandledExceptions() { } @Override - public getRunningCompactions_result getResult(I iface, getRunningCompactions_args args) throws org.apache.thrift.TException { - getRunningCompactions_result result = new getRunningCompactions_result(); - result.success = iface.getRunningCompactions(args.tinfo, args.credentials); + public cancelTask_result getResult(I iface, cancelTask_args args) throws org.apache.thrift.TException { + cancelTask_result result = new cancelTask_result(); + iface.cancelTask(args.tinfo, args.credentials, args.taskID); return result; } } - public static class getCompletedCompactions extends org.apache.thrift.ProcessFunction { - public getCompletedCompactions() { - super("getCompletedCompactions"); + public static class getRunningTasks extends org.apache.thrift.ProcessFunction { + public getRunningTasks() { + super("getRunningTasks"); } @Override - public getCompletedCompactions_args getEmptyArgsInstance() { - return new getCompletedCompactions_args(); + public getRunningTasks_args getEmptyArgsInstance() { + return new getRunningTasks_args(); } @Override @@ -769,21 +739,21 @@ protected boolean rethrowUnhandledExceptions() { } @Override - public getCompletedCompactions_result getResult(I iface, getCompletedCompactions_args args) throws org.apache.thrift.TException { - getCompletedCompactions_result result = new getCompletedCompactions_result(); - result.success = iface.getCompletedCompactions(args.tinfo, args.credentials); + public getRunningTasks_result getResult(I iface, getRunningTasks_args args) throws org.apache.thrift.TException { + getRunningTasks_result result = new getRunningTasks_result(); + result.success = iface.getRunningTasks(args.tinfo, args.credentials); return result; } } - public static class cancel extends org.apache.thrift.ProcessFunction { - public cancel() { - super("cancel"); + public static class getCompletedTasks extends org.apache.thrift.ProcessFunction { + public getCompletedTasks() { + super("getCompletedTasks"); } @Override - public cancel_args getEmptyArgsInstance() { - return new cancel_args(); + public getCompletedTasks_args getEmptyArgsInstance() { + return new getCompletedTasks_args(); } @Override @@ -797,9 +767,9 @@ protected boolean rethrowUnhandledExceptions() { } @Override - public cancel_result getResult(I iface, cancel_args args) throws org.apache.thrift.TException { - cancel_result result = new cancel_result(); - iface.cancel(args.tinfo, args.credentials, args.externalCompactionId); + public getCompletedTasks_result getResult(I iface, getCompletedTasks_args args) throws org.apache.thrift.TException { + getCompletedTasks_result result = new getCompletedTasks_result(); + result.success = iface.getCompletedTasks(args.tinfo, args.credentials); return result; } } @@ -817,33 +787,34 @@ protected AsyncProcessor(I iface, java.util.Map java.util.Map> getProcessMap(java.util.Map> processMap) { - processMap.put("compactionCompleted", new compactionCompleted()); - processMap.put("getCompactionJob", new getCompactionJob()); - processMap.put("updateCompactionStatus", new updateCompactionStatus()); - processMap.put("compactionFailed", new compactionFailed()); - processMap.put("getRunningCompactions", new getRunningCompactions()); - processMap.put("getCompletedCompactions", new getCompletedCompactions()); - processMap.put("cancel", new cancel()); + processMap.put("getTask", new getTask()); + processMap.put("taskStatus", new taskStatus()); + processMap.put("taskCompleted", new taskCompleted()); + processMap.put("taskFailed", new taskFailed()); + processMap.put("cancelTask", new cancelTask()); + processMap.put("getRunningTasks", new getRunningTasks()); + processMap.put("getCompletedTasks", new getCompletedTasks()); return processMap; } - public static class compactionCompleted extends org.apache.thrift.AsyncProcessFunction { - public compactionCompleted() { - super("compactionCompleted"); + public static class getTask extends org.apache.thrift.AsyncProcessFunction { + public getTask() { + super("getTask"); } @Override - public compactionCompleted_args getEmptyArgsInstance() { - return new compactionCompleted_args(); + public getTask_args getEmptyArgsInstance() { + return new getTask_args(); } @Override - public org.apache.thrift.async.AsyncMethodCallback getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) { + 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() { + return new org.apache.thrift.async.AsyncMethodCallback() { @Override - public void onComplete(Void o) { - compactionCompleted_result result = new compactionCompleted_result(); + public void onComplete(Task o) { + getTask_result result = new getTask_result(); + result.success = o; try { fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); } catch (org.apache.thrift.transport.TTransportException e) { @@ -858,7 +829,7 @@ public void onComplete(Void o) { public void onError(java.lang.Exception e) { byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; org.apache.thrift.TSerializable msg; - compactionCompleted_result result = new compactionCompleted_result(); + getTask_result result = new getTask_result(); if (e instanceof org.apache.thrift.transport.TTransportException) { _LOGGER.error("TTransportException inside handler", e); fb.close(); @@ -888,62 +859,35 @@ protected boolean isOneway() { } @Override - public void start(I iface, compactionCompleted_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - iface.compactionCompleted(args.tinfo, args.credentials, args.externalCompactionId, args.extent, args.stats,resultHandler); + public void start(I iface, getTask_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + iface.getTask(args.tinfo, args.credentials, args.taskRunner, args.taskID,resultHandler); } } - public static class getCompactionJob extends org.apache.thrift.AsyncProcessFunction { - public getCompactionJob() { - super("getCompactionJob"); + public static class taskStatus extends org.apache.thrift.AsyncProcessFunction { + public taskStatus() { + super("taskStatus"); } @Override - public getCompactionJob_args getEmptyArgsInstance() { - return new getCompactionJob_args(); + public taskStatus_args getEmptyArgsInstance() { + return new taskStatus_args(); } @Override - public org.apache.thrift.async.AsyncMethodCallback getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) { + 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() { + return new org.apache.thrift.async.AsyncMethodCallback() { @Override - public void onComplete(org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob o) { - getCompactionJob_result result = new getCompactionJob_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); - } + public void onComplete(Void o) { } @Override public void onError(java.lang.Exception e) { - byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; - org.apache.thrift.TSerializable msg; - getCompactionJob_result result = new getCompactionJob_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(); + _LOGGER.error("Exception inside oneway handler", e); } } }; @@ -951,23 +895,23 @@ public void onError(java.lang.Exception e) { @Override protected boolean isOneway() { - return false; + return true; } @Override - public void start(I iface, getCompactionJob_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - iface.getCompactionJob(args.tinfo, args.credentials, args.groupName, args.compactor, args.externalCompactionId,resultHandler); + public void start(I iface, taskStatus_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + iface.taskStatus(args.tinfo, args.credentials, args.timestamp, args.taskUpdateObject,resultHandler); } } - public static class updateCompactionStatus extends org.apache.thrift.AsyncProcessFunction { - public updateCompactionStatus() { - super("updateCompactionStatus"); + public static class taskCompleted extends org.apache.thrift.AsyncProcessFunction { + public taskCompleted() { + super("taskCompleted"); } @Override - public updateCompactionStatus_args getEmptyArgsInstance() { - return new updateCompactionStatus_args(); + public taskCompleted_args getEmptyArgsInstance() { + return new taskCompleted_args(); } @Override @@ -976,7 +920,7 @@ public org.apache.thrift.async.AsyncMethodCallback getResultHandler(final return new org.apache.thrift.async.AsyncMethodCallback() { @Override public void onComplete(Void o) { - updateCompactionStatus_result result = new updateCompactionStatus_result(); + taskCompleted_result result = new taskCompleted_result(); try { fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); } catch (org.apache.thrift.transport.TTransportException e) { @@ -991,7 +935,7 @@ public void onComplete(Void o) { public void onError(java.lang.Exception e) { byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; org.apache.thrift.TSerializable msg; - updateCompactionStatus_result result = new updateCompactionStatus_result(); + taskCompleted_result result = new taskCompleted_result(); if (e instanceof org.apache.thrift.transport.TTransportException) { _LOGGER.error("TTransportException inside handler", e); fb.close(); @@ -1021,19 +965,19 @@ protected boolean isOneway() { } @Override - public void start(I iface, updateCompactionStatus_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - iface.updateCompactionStatus(args.tinfo, args.credentials, args.externalCompactionId, args.status, args.timestamp,resultHandler); + public void start(I iface, taskCompleted_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + iface.taskCompleted(args.tinfo, args.credentials, args.task,resultHandler); } } - public static class compactionFailed extends org.apache.thrift.AsyncProcessFunction { - public compactionFailed() { - super("compactionFailed"); + public static class taskFailed extends org.apache.thrift.AsyncProcessFunction { + public taskFailed() { + super("taskFailed"); } @Override - public compactionFailed_args getEmptyArgsInstance() { - return new compactionFailed_args(); + public taskFailed_args getEmptyArgsInstance() { + return new taskFailed_args(); } @Override @@ -1042,7 +986,7 @@ public org.apache.thrift.async.AsyncMethodCallback getResultHandler(final return new org.apache.thrift.async.AsyncMethodCallback() { @Override public void onComplete(Void o) { - compactionFailed_result result = new compactionFailed_result(); + taskFailed_result result = new taskFailed_result(); try { fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); } catch (org.apache.thrift.transport.TTransportException e) { @@ -1057,7 +1001,7 @@ public void onComplete(Void o) { public void onError(java.lang.Exception e) { byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; org.apache.thrift.TSerializable msg; - compactionFailed_result result = new compactionFailed_result(); + taskFailed_result result = new taskFailed_result(); if (e instanceof org.apache.thrift.transport.TTransportException) { _LOGGER.error("TTransportException inside handler", e); fb.close(); @@ -1087,29 +1031,28 @@ protected boolean isOneway() { } @Override - public void start(I iface, compactionFailed_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - iface.compactionFailed(args.tinfo, args.credentials, args.externalCompactionId, args.extent,resultHandler); + public void start(I iface, taskFailed_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + iface.taskFailed(args.tinfo, args.credentials, args.task,resultHandler); } } - public static class getRunningCompactions extends org.apache.thrift.AsyncProcessFunction { - public getRunningCompactions() { - super("getRunningCompactions"); + public static class cancelTask extends org.apache.thrift.AsyncProcessFunction { + public cancelTask() { + super("cancelTask"); } @Override - public getRunningCompactions_args getEmptyArgsInstance() { - return new getRunningCompactions_args(); + public cancelTask_args getEmptyArgsInstance() { + return new cancelTask_args(); } @Override - public org.apache.thrift.async.AsyncMethodCallback getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) { + 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() { + return new org.apache.thrift.async.AsyncMethodCallback() { @Override - public void onComplete(TExternalCompactionList o) { - getRunningCompactions_result result = new getRunningCompactions_result(); - result.success = o; + public void onComplete(Void o) { + cancelTask_result result = new cancelTask_result(); try { fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); } catch (org.apache.thrift.transport.TTransportException e) { @@ -1124,7 +1067,7 @@ public void onComplete(TExternalCompactionList o) { public void onError(java.lang.Exception e) { byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; org.apache.thrift.TSerializable msg; - getRunningCompactions_result result = new getRunningCompactions_result(); + cancelTask_result result = new cancelTask_result(); if (e instanceof org.apache.thrift.transport.TTransportException) { _LOGGER.error("TTransportException inside handler", e); fb.close(); @@ -1154,28 +1097,28 @@ protected boolean isOneway() { } @Override - public void start(I iface, getRunningCompactions_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - iface.getRunningCompactions(args.tinfo, args.credentials,resultHandler); + public void start(I iface, cancelTask_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + iface.cancelTask(args.tinfo, args.credentials, args.taskID,resultHandler); } } - public static class getCompletedCompactions extends org.apache.thrift.AsyncProcessFunction { - public getCompletedCompactions() { - super("getCompletedCompactions"); + public static class getRunningTasks extends org.apache.thrift.AsyncProcessFunction { + public getRunningTasks() { + super("getRunningTasks"); } @Override - public getCompletedCompactions_args getEmptyArgsInstance() { - return new getCompletedCompactions_args(); + public getRunningTasks_args getEmptyArgsInstance() { + return new getRunningTasks_args(); } @Override - public org.apache.thrift.async.AsyncMethodCallback getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) { + 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() { + return new org.apache.thrift.async.AsyncMethodCallback() { @Override - public void onComplete(TExternalCompactionList o) { - getCompletedCompactions_result result = new getCompletedCompactions_result(); + public void onComplete(Task o) { + getRunningTasks_result result = new getRunningTasks_result(); result.success = o; try { fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); @@ -1191,7 +1134,7 @@ public void onComplete(TExternalCompactionList o) { public void onError(java.lang.Exception e) { byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; org.apache.thrift.TSerializable msg; - getCompletedCompactions_result result = new getCompletedCompactions_result(); + getRunningTasks_result result = new getRunningTasks_result(); if (e instanceof org.apache.thrift.transport.TTransportException) { _LOGGER.error("TTransportException inside handler", e); fb.close(); @@ -1221,28 +1164,29 @@ protected boolean isOneway() { } @Override - public void start(I iface, getCompletedCompactions_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - iface.getCompletedCompactions(args.tinfo, args.credentials,resultHandler); + public void start(I iface, getRunningTasks_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + iface.getRunningTasks(args.tinfo, args.credentials,resultHandler); } } - public static class cancel extends org.apache.thrift.AsyncProcessFunction { - public cancel() { - super("cancel"); + public static class getCompletedTasks extends org.apache.thrift.AsyncProcessFunction { + public getCompletedTasks() { + super("getCompletedTasks"); } @Override - public cancel_args getEmptyArgsInstance() { - return new cancel_args(); + public getCompletedTasks_args getEmptyArgsInstance() { + return new getCompletedTasks_args(); } @Override - public org.apache.thrift.async.AsyncMethodCallback getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) { + 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() { + return new org.apache.thrift.async.AsyncMethodCallback() { @Override - public void onComplete(Void o) { - cancel_result result = new cancel_result(); + public void onComplete(Task o) { + getCompletedTasks_result result = new getCompletedTasks_result(); + result.success = o; try { fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); } catch (org.apache.thrift.transport.TTransportException e) { @@ -1257,7 +1201,7 @@ public void onComplete(Void o) { public void onError(java.lang.Exception e) { byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; org.apache.thrift.TSerializable msg; - cancel_result result = new cancel_result(); + getCompletedTasks_result result = new getCompletedTasks_result(); if (e instanceof org.apache.thrift.transport.TTransportException) { _LOGGER.error("TTransportException inside handler", e); fb.close(); @@ -1287,39 +1231,36 @@ protected boolean isOneway() { } @Override - public void start(I iface, cancel_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - iface.cancel(args.tinfo, args.credentials, args.externalCompactionId,resultHandler); + public void start(I iface, getCompletedTasks_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + iface.getCompletedTasks(args.tinfo, args.credentials,resultHandler); } } } @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) - public static class compactionCompleted_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("compactionCompleted_args"); + public static class getTask_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("getTask_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.protocol.TField EXTENT_FIELD_DESC = new org.apache.thrift.protocol.TField("extent", org.apache.thrift.protocol.TType.STRUCT, (short)4); - private static final org.apache.thrift.protocol.TField STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("stats", org.apache.thrift.protocol.TType.STRUCT, (short)5); + private static final org.apache.thrift.protocol.TField TASK_RUNNER_FIELD_DESC = new org.apache.thrift.protocol.TField("taskRunner", org.apache.thrift.protocol.TType.STRUCT, (short)3); + private static final org.apache.thrift.protocol.TField TASK_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("taskID", org.apache.thrift.protocol.TType.STRING, (short)4); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new compactionCompleted_argsStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new compactionCompleted_argsTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getTask_argsStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getTask_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 - public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent; // required - public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats; // required + public @org.apache.thrift.annotation.Nullable TaskRunnerInfo taskRunner; // required + public @org.apache.thrift.annotation.Nullable java.lang.String taskID; // 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"), - EXTENT((short)4, "extent"), - STATS((short)5, "stats"); + TASK_RUNNER((short)3, "taskRunner"), + TASK_ID((short)4, "taskID"); private static final java.util.Map byName = new java.util.HashMap(); @@ -1339,12 +1280,10 @@ public static _Fields findByThriftId(int fieldId) { return TINFO; case 2: // CREDENTIALS return CREDENTIALS; - case 3: // EXTERNAL_COMPACTION_ID - return EXTERNAL_COMPACTION_ID; - case 4: // EXTENT - return EXTENT; - case 5: // STATS - return STATS; + case 3: // TASK_RUNNER + return TASK_RUNNER; + case 4: // TASK_ID + return TASK_ID; default: return null; } @@ -1395,67 +1334,59 @@ 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.EXTERNAL_COMPACTION_ID, new org.apache.thrift.meta_data.FieldMetaData("externalCompactionId", org.apache.thrift.TFieldRequirementType.DEFAULT, + tmpMap.put(_Fields.TASK_RUNNER, new org.apache.thrift.meta_data.FieldMetaData("taskRunner", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TaskRunnerInfo.class))); + tmpMap.put(_Fields.TASK_ID, new org.apache.thrift.meta_data.FieldMetaData("taskID", 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))); - tmpMap.put(_Fields.STATS, new org.apache.thrift.meta_data.FieldMetaData("stats", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.tabletserver.thrift.TCompactionStats.class))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(compactionCompleted_args.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getTask_args.class, metaDataMap); } - public compactionCompleted_args() { + public getTask_args() { } - public compactionCompleted_args( + public getTask_args( org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, - java.lang.String externalCompactionId, - org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, - org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats) + TaskRunnerInfo taskRunner, + java.lang.String taskID) { this(); this.tinfo = tinfo; this.credentials = credentials; - this.externalCompactionId = externalCompactionId; - this.extent = extent; - this.stats = stats; + this.taskRunner = taskRunner; + this.taskID = taskID; } /** * Performs a deep copy on other. */ - public compactionCompleted_args(compactionCompleted_args other) { + public getTask_args(getTask_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; + if (other.isSetTaskRunner()) { + this.taskRunner = new TaskRunnerInfo(other.taskRunner); } - if (other.isSetExtent()) { - this.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(other.extent); - } - if (other.isSetStats()) { - this.stats = new org.apache.accumulo.core.tabletserver.thrift.TCompactionStats(other.stats); + if (other.isSetTaskID()) { + this.taskID = other.taskID; } } @Override - public compactionCompleted_args deepCopy() { - return new compactionCompleted_args(this); + public getTask_args deepCopy() { + return new getTask_args(this); } @Override public void clear() { this.tinfo = null; this.credentials = null; - this.externalCompactionId = null; - this.extent = null; - this.stats = null; + this.taskRunner = null; + this.taskID = null; } @org.apache.thrift.annotation.Nullable @@ -1463,7 +1394,7 @@ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() { return this.tinfo; } - public compactionCompleted_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) { + public getTask_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) { this.tinfo = tinfo; return this; } @@ -1488,7 +1419,7 @@ public org.apache.accumulo.core.securityImpl.thrift.TCredentials getCredentials( return this.credentials; } - public compactionCompleted_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) { + public getTask_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) { this.credentials = credentials; return this; } @@ -1509,77 +1440,52 @@ public void setCredentialsIsSet(boolean value) { } @org.apache.thrift.annotation.Nullable - public java.lang.String getExternalCompactionId() { - return this.externalCompactionId; - } - - public compactionCompleted_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; - } - } - - @org.apache.thrift.annotation.Nullable - public org.apache.accumulo.core.dataImpl.thrift.TKeyExtent getExtent() { - return this.extent; + public TaskRunnerInfo getTaskRunner() { + return this.taskRunner; } - public compactionCompleted_args setExtent(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) { - this.extent = extent; + public getTask_args setTaskRunner(@org.apache.thrift.annotation.Nullable TaskRunnerInfo taskRunner) { + this.taskRunner = taskRunner; return this; } - public void unsetExtent() { - this.extent = null; + public void unsetTaskRunner() { + this.taskRunner = null; } - /** Returns true if field extent is set (has been assigned a value) and false otherwise */ - public boolean isSetExtent() { - return this.extent != null; + /** Returns true if field taskRunner is set (has been assigned a value) and false otherwise */ + public boolean isSetTaskRunner() { + return this.taskRunner != null; } - public void setExtentIsSet(boolean value) { + public void setTaskRunnerIsSet(boolean value) { if (!value) { - this.extent = null; + this.taskRunner = null; } } @org.apache.thrift.annotation.Nullable - public org.apache.accumulo.core.tabletserver.thrift.TCompactionStats getStats() { - return this.stats; + public java.lang.String getTaskID() { + return this.taskID; } - public compactionCompleted_args setStats(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats) { - this.stats = stats; + public getTask_args setTaskID(@org.apache.thrift.annotation.Nullable java.lang.String taskID) { + this.taskID = taskID; return this; } - public void unsetStats() { - this.stats = null; + public void unsetTaskID() { + this.taskID = null; } - /** Returns true if field stats is set (has been assigned a value) and false otherwise */ - public boolean isSetStats() { - return this.stats != null; + /** Returns true if field taskID is set (has been assigned a value) and false otherwise */ + public boolean isSetTaskID() { + return this.taskID != null; } - public void setStatsIsSet(boolean value) { + public void setTaskIDIsSet(boolean value) { if (!value) { - this.stats = null; + this.taskID = null; } } @@ -1602,27 +1508,19 @@ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable } break; - case EXTERNAL_COMPACTION_ID: - if (value == null) { - unsetExternalCompactionId(); - } else { - setExternalCompactionId((java.lang.String)value); - } - break; - - case EXTENT: + case TASK_RUNNER: if (value == null) { - unsetExtent(); + unsetTaskRunner(); } else { - setExtent((org.apache.accumulo.core.dataImpl.thrift.TKeyExtent)value); + setTaskRunner((TaskRunnerInfo)value); } break; - case STATS: + case TASK_ID: if (value == null) { - unsetStats(); + unsetTaskID(); } else { - setStats((org.apache.accumulo.core.tabletserver.thrift.TCompactionStats)value); + setTaskID((java.lang.String)value); } break; @@ -1639,14 +1537,11 @@ public java.lang.Object getFieldValue(_Fields field) { case CREDENTIALS: return getCredentials(); - case EXTERNAL_COMPACTION_ID: - return getExternalCompactionId(); + case TASK_RUNNER: + return getTaskRunner(); - case EXTENT: - return getExtent(); - - case STATS: - return getStats(); + case TASK_ID: + return getTaskID(); } throw new java.lang.IllegalStateException(); @@ -1664,24 +1559,22 @@ public boolean isSet(_Fields field) { return isSetTinfo(); case CREDENTIALS: return isSetCredentials(); - case EXTERNAL_COMPACTION_ID: - return isSetExternalCompactionId(); - case EXTENT: - return isSetExtent(); - case STATS: - return isSetStats(); + case TASK_RUNNER: + return isSetTaskRunner(); + case TASK_ID: + return isSetTaskID(); } throw new java.lang.IllegalStateException(); } @Override public boolean equals(java.lang.Object that) { - if (that instanceof compactionCompleted_args) - return this.equals((compactionCompleted_args)that); + if (that instanceof getTask_args) + return this.equals((getTask_args)that); return false; } - public boolean equals(compactionCompleted_args that) { + public boolean equals(getTask_args that) { if (that == null) return false; if (this == that) @@ -1705,30 +1598,21 @@ public boolean equals(compactionCompleted_args that) { 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; - } - - 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)) + boolean this_present_taskRunner = true && this.isSetTaskRunner(); + boolean that_present_taskRunner = true && that.isSetTaskRunner(); + if (this_present_taskRunner || that_present_taskRunner) { + if (!(this_present_taskRunner && that_present_taskRunner)) return false; - if (!this.extent.equals(that.extent)) + if (!this.taskRunner.equals(that.taskRunner)) return false; } - boolean this_present_stats = true && this.isSetStats(); - boolean that_present_stats = true && that.isSetStats(); - if (this_present_stats || that_present_stats) { - if (!(this_present_stats && that_present_stats)) + boolean this_present_taskID = true && this.isSetTaskID(); + boolean that_present_taskID = true && that.isSetTaskID(); + if (this_present_taskID || that_present_taskID) { + if (!(this_present_taskID && that_present_taskID)) return false; - if (!this.stats.equals(that.stats)) + if (!this.taskID.equals(that.taskID)) return false; } @@ -1747,23 +1631,19 @@ public int hashCode() { if (isSetCredentials()) hashCode = hashCode * 8191 + credentials.hashCode(); - hashCode = hashCode * 8191 + ((isSetExternalCompactionId()) ? 131071 : 524287); - if (isSetExternalCompactionId()) - hashCode = hashCode * 8191 + externalCompactionId.hashCode(); - - hashCode = hashCode * 8191 + ((isSetExtent()) ? 131071 : 524287); - if (isSetExtent()) - hashCode = hashCode * 8191 + extent.hashCode(); + hashCode = hashCode * 8191 + ((isSetTaskRunner()) ? 131071 : 524287); + if (isSetTaskRunner()) + hashCode = hashCode * 8191 + taskRunner.hashCode(); - hashCode = hashCode * 8191 + ((isSetStats()) ? 131071 : 524287); - if (isSetStats()) - hashCode = hashCode * 8191 + stats.hashCode(); + hashCode = hashCode * 8191 + ((isSetTaskID()) ? 131071 : 524287); + if (isSetTaskID()) + hashCode = hashCode * 8191 + taskID.hashCode(); return hashCode; } @Override - public int compareTo(compactionCompleted_args other) { + public int compareTo(getTask_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -1790,32 +1670,22 @@ public int compareTo(compactionCompleted_args other) { 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; - } - } - lastComparison = java.lang.Boolean.compare(isSetExtent(), other.isSetExtent()); + lastComparison = java.lang.Boolean.compare(isSetTaskRunner(), other.isSetTaskRunner()); if (lastComparison != 0) { return lastComparison; } - if (isSetExtent()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.extent, other.extent); + if (isSetTaskRunner()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.taskRunner, other.taskRunner); if (lastComparison != 0) { return lastComparison; } } - lastComparison = java.lang.Boolean.compare(isSetStats(), other.isSetStats()); + lastComparison = java.lang.Boolean.compare(isSetTaskID(), other.isSetTaskID()); if (lastComparison != 0) { return lastComparison; } - if (isSetStats()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.stats, other.stats); + if (isSetTaskID()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.taskID, other.taskID); if (lastComparison != 0) { return lastComparison; } @@ -1841,7 +1711,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("compactionCompleted_args("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("getTask_args("); boolean first = true; sb.append("tinfo:"); @@ -1860,27 +1730,19 @@ public java.lang.String toString() { } first = false; if (!first) sb.append(", "); - sb.append("externalCompactionId:"); - if (this.externalCompactionId == null) { - sb.append("null"); - } else { - sb.append(this.externalCompactionId); - } - first = false; - if (!first) sb.append(", "); - sb.append("extent:"); - if (this.extent == null) { + sb.append("taskRunner:"); + if (this.taskRunner == null) { sb.append("null"); } else { - sb.append(this.extent); + sb.append(this.taskRunner); } first = false; if (!first) sb.append(", "); - sb.append("stats:"); - if (this.stats == null) { + sb.append("taskID:"); + if (this.taskID == null) { sb.append("null"); } else { - sb.append(this.stats); + sb.append(this.taskID); } first = false; sb.append(")"); @@ -1896,11 +1758,8 @@ public void validate() throws org.apache.thrift.TException { if (credentials != null) { credentials.validate(); } - if (extent != null) { - extent.validate(); - } - if (stats != null) { - stats.validate(); + if (taskRunner != null) { + taskRunner.validate(); } } @@ -1920,17 +1779,17 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class compactionCompleted_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class getTask_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public compactionCompleted_argsStandardScheme getScheme() { - return new compactionCompleted_argsStandardScheme(); + public getTask_argsStandardScheme getScheme() { + return new getTask_argsStandardScheme(); } } - private static class compactionCompleted_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class getTask_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { @Override - public void read(org.apache.thrift.protocol.TProtocol iprot, compactionCompleted_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, getTask_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -1958,28 +1817,19 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, compactionCompleted 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; - case 4: // EXTENT + case 3: // TASK_RUNNER 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); + struct.taskRunner = new TaskRunnerInfo(); + struct.taskRunner.read(iprot); + struct.setTaskRunnerIsSet(true); } else { org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; - case 5: // STATS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.stats = new org.apache.accumulo.core.tabletserver.thrift.TCompactionStats(); - struct.stats.read(iprot); - struct.setStatsIsSet(true); + case 4: // TASK_ID + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.taskID = iprot.readString(); + struct.setTaskIDIsSet(true); } else { org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -1996,7 +1846,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, compactionCompleted } @Override - public void write(org.apache.thrift.protocol.TProtocol oprot, compactionCompleted_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, getTask_args struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -2010,19 +1860,14 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, compactionComplete struct.credentials.write(oprot); oprot.writeFieldEnd(); } - if (struct.externalCompactionId != null) { - oprot.writeFieldBegin(EXTERNAL_COMPACTION_ID_FIELD_DESC); - oprot.writeString(struct.externalCompactionId); + if (struct.taskRunner != null) { + oprot.writeFieldBegin(TASK_RUNNER_FIELD_DESC); + struct.taskRunner.write(oprot); oprot.writeFieldEnd(); } - if (struct.extent != null) { - oprot.writeFieldBegin(EXTENT_FIELD_DESC); - struct.extent.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.stats != null) { - oprot.writeFieldBegin(STATS_FIELD_DESC); - struct.stats.write(oprot); + if (struct.taskID != null) { + oprot.writeFieldBegin(TASK_ID_FIELD_DESC); + oprot.writeString(struct.taskID); oprot.writeFieldEnd(); } oprot.writeFieldStop(); @@ -2031,17 +1876,17 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, compactionComplete } - private static class compactionCompleted_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class getTask_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public compactionCompleted_argsTupleScheme getScheme() { - return new compactionCompleted_argsTupleScheme(); + public getTask_argsTupleScheme getScheme() { + return new getTask_argsTupleScheme(); } } - private static class compactionCompleted_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class getTask_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, compactionCompleted_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, getTask_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()) { @@ -2050,37 +1895,31 @@ public void write(org.apache.thrift.protocol.TProtocol prot, compactionCompleted if (struct.isSetCredentials()) { optionals.set(1); } - if (struct.isSetExternalCompactionId()) { + if (struct.isSetTaskRunner()) { optionals.set(2); } - if (struct.isSetExtent()) { + if (struct.isSetTaskID()) { optionals.set(3); } - if (struct.isSetStats()) { - optionals.set(4); - } - oprot.writeBitSet(optionals, 5); + oprot.writeBitSet(optionals, 4); if (struct.isSetTinfo()) { struct.tinfo.write(oprot); } if (struct.isSetCredentials()) { struct.credentials.write(oprot); } - if (struct.isSetExternalCompactionId()) { - oprot.writeString(struct.externalCompactionId); + if (struct.isSetTaskRunner()) { + struct.taskRunner.write(oprot); } - if (struct.isSetExtent()) { - struct.extent.write(oprot); - } - if (struct.isSetStats()) { - struct.stats.write(oprot); + if (struct.isSetTaskID()) { + oprot.writeString(struct.taskID); } } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, compactionCompleted_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, getTask_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(5); + 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); @@ -2092,18 +1931,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, compactionCompleted_ struct.setCredentialsIsSet(true); } if (incoming.get(2)) { - struct.externalCompactionId = iprot.readString(); - struct.setExternalCompactionIdIsSet(true); + struct.taskRunner = new TaskRunnerInfo(); + struct.taskRunner.read(iprot); + struct.setTaskRunnerIsSet(true); } if (incoming.get(3)) { - struct.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(); - struct.extent.read(iprot); - struct.setExtentIsSet(true); - } - if (incoming.get(4)) { - struct.stats = new org.apache.accumulo.core.tabletserver.thrift.TCompactionStats(); - struct.stats.read(iprot); - struct.setStatsIsSet(true); + struct.taskID = iprot.readString(); + struct.setTaskIDIsSet(true); } } } @@ -2114,17 +1948,19 @@ private static S scheme(org.apache. } @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) - public static class compactionCompleted_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("compactionCompleted_result"); + public static class getTask_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("getTask_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.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new compactionCompleted_resultStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new compactionCompleted_resultTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getTask_resultStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getTask_resultTupleSchemeFactory(); + public @org.apache.thrift.annotation.Nullable Task success; // 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"); private static final java.util.Map byName = new java.util.HashMap(); @@ -2140,6 +1976,8 @@ 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; default: return null; } @@ -2181,34 +2019,82 @@ 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, Task.class))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(compactionCompleted_result.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getTask_result.class, metaDataMap); + } + + public getTask_result() { } - public compactionCompleted_result() { + public getTask_result( + Task success) + { + this(); + this.success = success; } /** * Performs a deep copy on other. */ - public compactionCompleted_result(compactionCompleted_result other) { + public getTask_result(getTask_result other) { + if (other.isSetSuccess()) { + this.success = new Task(other.success); + } } @Override - public compactionCompleted_result deepCopy() { - return new compactionCompleted_result(this); + public getTask_result deepCopy() { + return new getTask_result(this); } @Override public void clear() { + this.success = null; + } + + @org.apache.thrift.annotation.Nullable + public Task getSuccess() { + return this.success; + } + + public getTask_result setSuccess(@org.apache.thrift.annotation.Nullable Task 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; + } } @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((Task)value); + } + break; + } } @@ -2216,6 +2102,9 @@ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable @Override public java.lang.Object getFieldValue(_Fields field) { switch (field) { + case SUCCESS: + return getSuccess(); + } throw new java.lang.IllegalStateException(); } @@ -2228,23 +2117,34 @@ public boolean isSet(_Fields field) { } switch (field) { + case SUCCESS: + return isSetSuccess(); } throw new java.lang.IllegalStateException(); } @Override public boolean equals(java.lang.Object that) { - if (that instanceof compactionCompleted_result) - return this.equals((compactionCompleted_result)that); + if (that instanceof getTask_result) + return this.equals((getTask_result)that); return false; } - public boolean equals(compactionCompleted_result that) { + public boolean equals(getTask_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; + } + return true; } @@ -2252,17 +2152,31 @@ public boolean equals(compactionCompleted_result that) { public int hashCode() { int hashCode = 1; + hashCode = hashCode * 8191 + ((isSetSuccess()) ? 131071 : 524287); + if (isSetSuccess()) + hashCode = hashCode * 8191 + success.hashCode(); + return hashCode; } @Override - public int compareTo(compactionCompleted_result other) { + public int compareTo(getTask_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; + } + } return 0; } @@ -2283,9 +2197,16 @@ 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("compactionCompleted_result("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("getTask_result("); boolean first = true; + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; sb.append(")"); return sb.toString(); } @@ -2293,6 +2214,9 @@ public java.lang.String 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 { @@ -2311,17 +2235,17 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class compactionCompleted_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class getTask_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public compactionCompleted_resultStandardScheme getScheme() { - return new compactionCompleted_resultStandardScheme(); + public getTask_resultStandardScheme getScheme() { + return new getTask_resultStandardScheme(); } } - private static class compactionCompleted_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class getTask_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { @Override - public void read(org.apache.thrift.protocol.TProtocol iprot, compactionCompleted_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, getTask_result struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -2331,6 +2255,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, compactionCompleted break; } switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new Task(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -2343,33 +2276,52 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, compactionCompleted } @Override - public void write(org.apache.thrift.protocol.TProtocol oprot, compactionCompleted_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, getTask_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(); + } oprot.writeFieldStop(); oprot.writeStructEnd(); } } - private static class compactionCompleted_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class getTask_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public compactionCompleted_resultTupleScheme getScheme() { - return new compactionCompleted_resultTupleScheme(); + public getTask_resultTupleScheme getScheme() { + return new getTask_resultTupleScheme(); } } - private static class compactionCompleted_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class getTask_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, compactionCompleted_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, getTask_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); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, compactionCompleted_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, getTask_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(1); + if (incoming.get(0)) { + struct.success = new Task(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } } } @@ -2379,31 +2331,28 @@ private static S scheme(org.apache. } @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) - public static class getCompactionJob_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("getCompactionJob_args"); + public static class taskStatus_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("taskStatus_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 GROUP_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("groupName", org.apache.thrift.protocol.TType.STRING, (short)3); - private static final org.apache.thrift.protocol.TField COMPACTOR_FIELD_DESC = new org.apache.thrift.protocol.TField("compactor", org.apache.thrift.protocol.TType.STRING, (short)4); - 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)5); + private static final org.apache.thrift.protocol.TField TIMESTAMP_FIELD_DESC = new org.apache.thrift.protocol.TField("timestamp", org.apache.thrift.protocol.TType.I64, (short)3); + private static final org.apache.thrift.protocol.TField TASK_UPDATE_OBJECT_FIELD_DESC = new org.apache.thrift.protocol.TField("taskUpdateObject", org.apache.thrift.protocol.TType.STRUCT, (short)4); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getCompactionJob_argsStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getCompactionJob_argsTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new taskStatus_argsStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new taskStatus_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 groupName; // required - public @org.apache.thrift.annotation.Nullable java.lang.String compactor; // required - public @org.apache.thrift.annotation.Nullable java.lang.String externalCompactionId; // required + public long timestamp; // required + public @org.apache.thrift.annotation.Nullable Task taskUpdateObject; // 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"), - GROUP_NAME((short)3, "groupName"), - COMPACTOR((short)4, "compactor"), - EXTERNAL_COMPACTION_ID((short)5, "externalCompactionId"); + TIMESTAMP((short)3, "timestamp"), + TASK_UPDATE_OBJECT((short)4, "taskUpdateObject"); private static final java.util.Map byName = new java.util.HashMap(); @@ -2423,12 +2372,10 @@ public static _Fields findByThriftId(int fieldId) { return TINFO; case 2: // CREDENTIALS return CREDENTIALS; - case 3: // GROUP_NAME - return GROUP_NAME; - case 4: // COMPACTOR - return COMPACTOR; - case 5: // EXTERNAL_COMPACTION_ID - return EXTERNAL_COMPACTION_ID; + case 3: // TIMESTAMP + return TIMESTAMP; + case 4: // TASK_UPDATE_OBJECT + return TASK_UPDATE_OBJECT; default: return null; } @@ -2472,6 +2419,8 @@ public java.lang.String getFieldName() { } // isset id assignments + private static final int __TIMESTAMP_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); @@ -2479,67 +2428,60 @@ 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.GROUP_NAME, new org.apache.thrift.meta_data.FieldMetaData("groupName", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); - tmpMap.put(_Fields.COMPACTOR, new org.apache.thrift.meta_data.FieldMetaData("compactor", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); - 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))); + tmpMap.put(_Fields.TIMESTAMP, new org.apache.thrift.meta_data.FieldMetaData("timestamp", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + tmpMap.put(_Fields.TASK_UPDATE_OBJECT, new org.apache.thrift.meta_data.FieldMetaData("taskUpdateObject", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, Task.class))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getCompactionJob_args.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(taskStatus_args.class, metaDataMap); } - public getCompactionJob_args() { + public taskStatus_args() { } - public getCompactionJob_args( + public taskStatus_args( org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, - java.lang.String groupName, - java.lang.String compactor, - java.lang.String externalCompactionId) + long timestamp, + Task taskUpdateObject) { this(); this.tinfo = tinfo; this.credentials = credentials; - this.groupName = groupName; - this.compactor = compactor; - this.externalCompactionId = externalCompactionId; + this.timestamp = timestamp; + setTimestampIsSet(true); + this.taskUpdateObject = taskUpdateObject; } /** * Performs a deep copy on other. */ - public getCompactionJob_args(getCompactionJob_args other) { + public taskStatus_args(taskStatus_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); } - if (other.isSetGroupName()) { - this.groupName = other.groupName; - } - if (other.isSetCompactor()) { - this.compactor = other.compactor; - } - if (other.isSetExternalCompactionId()) { - this.externalCompactionId = other.externalCompactionId; + this.timestamp = other.timestamp; + if (other.isSetTaskUpdateObject()) { + this.taskUpdateObject = new Task(other.taskUpdateObject); } } @Override - public getCompactionJob_args deepCopy() { - return new getCompactionJob_args(this); + public taskStatus_args deepCopy() { + return new taskStatus_args(this); } @Override public void clear() { this.tinfo = null; this.credentials = null; - this.groupName = null; - this.compactor = null; - this.externalCompactionId = null; + setTimestampIsSet(false); + this.timestamp = 0; + this.taskUpdateObject = null; } @org.apache.thrift.annotation.Nullable @@ -2547,7 +2489,7 @@ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() { return this.tinfo; } - public getCompactionJob_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) { + public taskStatus_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) { this.tinfo = tinfo; return this; } @@ -2572,7 +2514,7 @@ public org.apache.accumulo.core.securityImpl.thrift.TCredentials getCredentials( return this.credentials; } - public getCompactionJob_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) { + public taskStatus_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) { this.credentials = credentials; return this; } @@ -2592,78 +2534,51 @@ public void setCredentialsIsSet(boolean value) { } } - @org.apache.thrift.annotation.Nullable - public java.lang.String getGroupName() { - return this.groupName; - } - - public getCompactionJob_args setGroupName(@org.apache.thrift.annotation.Nullable java.lang.String groupName) { - this.groupName = groupName; - return this; - } - - public void unsetGroupName() { - this.groupName = null; - } - - /** Returns true if field groupName is set (has been assigned a value) and false otherwise */ - public boolean isSetGroupName() { - return this.groupName != null; - } - - public void setGroupNameIsSet(boolean value) { - if (!value) { - this.groupName = null; - } - } - - @org.apache.thrift.annotation.Nullable - public java.lang.String getCompactor() { - return this.compactor; + public long getTimestamp() { + return this.timestamp; } - public getCompactionJob_args setCompactor(@org.apache.thrift.annotation.Nullable java.lang.String compactor) { - this.compactor = compactor; + public taskStatus_args setTimestamp(long timestamp) { + this.timestamp = timestamp; + setTimestampIsSet(true); return this; } - public void unsetCompactor() { - this.compactor = null; + public void unsetTimestamp() { + __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __TIMESTAMP_ISSET_ID); } - /** Returns true if field compactor is set (has been assigned a value) and false otherwise */ - public boolean isSetCompactor() { - return this.compactor != null; + /** Returns true if field timestamp is set (has been assigned a value) and false otherwise */ + public boolean isSetTimestamp() { + return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __TIMESTAMP_ISSET_ID); } - public void setCompactorIsSet(boolean value) { - if (!value) { - this.compactor = null; - } + public void setTimestampIsSet(boolean value) { + __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __TIMESTAMP_ISSET_ID, value); } @org.apache.thrift.annotation.Nullable - public java.lang.String getExternalCompactionId() { - return this.externalCompactionId; + public Task getTaskUpdateObject() { + return this.taskUpdateObject; } - public getCompactionJob_args setExternalCompactionId(@org.apache.thrift.annotation.Nullable java.lang.String externalCompactionId) { - this.externalCompactionId = externalCompactionId; + public taskStatus_args setTaskUpdateObject(@org.apache.thrift.annotation.Nullable Task taskUpdateObject) { + this.taskUpdateObject = taskUpdateObject; return this; } - public void unsetExternalCompactionId() { - this.externalCompactionId = null; + public void unsetTaskUpdateObject() { + this.taskUpdateObject = null; } - /** Returns true if field externalCompactionId is set (has been assigned a value) and false otherwise */ - public boolean isSetExternalCompactionId() { - return this.externalCompactionId != null; + /** Returns true if field taskUpdateObject is set (has been assigned a value) and false otherwise */ + public boolean isSetTaskUpdateObject() { + return this.taskUpdateObject != null; } - public void setExternalCompactionIdIsSet(boolean value) { + public void setTaskUpdateObjectIsSet(boolean value) { if (!value) { - this.externalCompactionId = null; + this.taskUpdateObject = null; } } @@ -2686,27 +2601,19 @@ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable } break; - case GROUP_NAME: - if (value == null) { - unsetGroupName(); - } else { - setGroupName((java.lang.String)value); - } - break; - - case COMPACTOR: + case TIMESTAMP: if (value == null) { - unsetCompactor(); + unsetTimestamp(); } else { - setCompactor((java.lang.String)value); + setTimestamp((java.lang.Long)value); } break; - case EXTERNAL_COMPACTION_ID: + case TASK_UPDATE_OBJECT: if (value == null) { - unsetExternalCompactionId(); + unsetTaskUpdateObject(); } else { - setExternalCompactionId((java.lang.String)value); + setTaskUpdateObject((Task)value); } break; @@ -2723,14 +2630,11 @@ public java.lang.Object getFieldValue(_Fields field) { case CREDENTIALS: return getCredentials(); - case GROUP_NAME: - return getGroupName(); - - case COMPACTOR: - return getCompactor(); + case TIMESTAMP: + return getTimestamp(); - case EXTERNAL_COMPACTION_ID: - return getExternalCompactionId(); + case TASK_UPDATE_OBJECT: + return getTaskUpdateObject(); } throw new java.lang.IllegalStateException(); @@ -2748,24 +2652,22 @@ public boolean isSet(_Fields field) { return isSetTinfo(); case CREDENTIALS: return isSetCredentials(); - case GROUP_NAME: - return isSetGroupName(); - case COMPACTOR: - return isSetCompactor(); - case EXTERNAL_COMPACTION_ID: - return isSetExternalCompactionId(); + case TIMESTAMP: + return isSetTimestamp(); + case TASK_UPDATE_OBJECT: + return isSetTaskUpdateObject(); } throw new java.lang.IllegalStateException(); } @Override public boolean equals(java.lang.Object that) { - if (that instanceof getCompactionJob_args) - return this.equals((getCompactionJob_args)that); + if (that instanceof taskStatus_args) + return this.equals((taskStatus_args)that); return false; } - public boolean equals(getCompactionJob_args that) { + public boolean equals(taskStatus_args that) { if (that == null) return false; if (this == that) @@ -2789,30 +2691,21 @@ public boolean equals(getCompactionJob_args that) { return false; } - boolean this_present_groupName = true && this.isSetGroupName(); - boolean that_present_groupName = true && that.isSetGroupName(); - if (this_present_groupName || that_present_groupName) { - if (!(this_present_groupName && that_present_groupName)) - return false; - if (!this.groupName.equals(that.groupName)) - return false; - } - - boolean this_present_compactor = true && this.isSetCompactor(); - boolean that_present_compactor = true && that.isSetCompactor(); - if (this_present_compactor || that_present_compactor) { - if (!(this_present_compactor && that_present_compactor)) + boolean this_present_timestamp = true; + boolean that_present_timestamp = true; + if (this_present_timestamp || that_present_timestamp) { + if (!(this_present_timestamp && that_present_timestamp)) return false; - if (!this.compactor.equals(that.compactor)) + if (this.timestamp != that.timestamp) 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)) + boolean this_present_taskUpdateObject = true && this.isSetTaskUpdateObject(); + boolean that_present_taskUpdateObject = true && that.isSetTaskUpdateObject(); + if (this_present_taskUpdateObject || that_present_taskUpdateObject) { + if (!(this_present_taskUpdateObject && that_present_taskUpdateObject)) return false; - if (!this.externalCompactionId.equals(that.externalCompactionId)) + if (!this.taskUpdateObject.equals(that.taskUpdateObject)) return false; } @@ -2831,23 +2724,17 @@ public int hashCode() { if (isSetCredentials()) hashCode = hashCode * 8191 + credentials.hashCode(); - hashCode = hashCode * 8191 + ((isSetGroupName()) ? 131071 : 524287); - if (isSetGroupName()) - hashCode = hashCode * 8191 + groupName.hashCode(); - - hashCode = hashCode * 8191 + ((isSetCompactor()) ? 131071 : 524287); - if (isSetCompactor()) - hashCode = hashCode * 8191 + compactor.hashCode(); + hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(timestamp); - hashCode = hashCode * 8191 + ((isSetExternalCompactionId()) ? 131071 : 524287); - if (isSetExternalCompactionId()) - hashCode = hashCode * 8191 + externalCompactionId.hashCode(); + hashCode = hashCode * 8191 + ((isSetTaskUpdateObject()) ? 131071 : 524287); + if (isSetTaskUpdateObject()) + hashCode = hashCode * 8191 + taskUpdateObject.hashCode(); return hashCode; } @Override - public int compareTo(getCompactionJob_args other) { + public int compareTo(taskStatus_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -2874,32 +2761,22 @@ public int compareTo(getCompactionJob_args other) { return lastComparison; } } - lastComparison = java.lang.Boolean.compare(isSetGroupName(), other.isSetGroupName()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetGroupName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.groupName, other.groupName); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = java.lang.Boolean.compare(isSetCompactor(), other.isSetCompactor()); + lastComparison = java.lang.Boolean.compare(isSetTimestamp(), other.isSetTimestamp()); if (lastComparison != 0) { return lastComparison; } - if (isSetCompactor()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.compactor, other.compactor); + if (isSetTimestamp()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.timestamp, other.timestamp); if (lastComparison != 0) { return lastComparison; } } - lastComparison = java.lang.Boolean.compare(isSetExternalCompactionId(), other.isSetExternalCompactionId()); + lastComparison = java.lang.Boolean.compare(isSetTaskUpdateObject(), other.isSetTaskUpdateObject()); if (lastComparison != 0) { return lastComparison; } - if (isSetExternalCompactionId()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.externalCompactionId, other.externalCompactionId); + if (isSetTaskUpdateObject()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.taskUpdateObject, other.taskUpdateObject); if (lastComparison != 0) { return lastComparison; } @@ -2925,7 +2802,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("getCompactionJob_args("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("taskStatus_args("); boolean first = true; sb.append("tinfo:"); @@ -2944,27 +2821,15 @@ public java.lang.String toString() { } first = false; if (!first) sb.append(", "); - sb.append("groupName:"); - if (this.groupName == null) { - sb.append("null"); - } else { - sb.append(this.groupName); - } - first = false; - if (!first) sb.append(", "); - sb.append("compactor:"); - if (this.compactor == null) { - sb.append("null"); - } else { - sb.append(this.compactor); - } + sb.append("timestamp:"); + sb.append(this.timestamp); first = false; if (!first) sb.append(", "); - sb.append("externalCompactionId:"); - if (this.externalCompactionId == null) { + sb.append("taskUpdateObject:"); + if (this.taskUpdateObject == null) { sb.append("null"); } else { - sb.append(this.externalCompactionId); + sb.append(this.taskUpdateObject); } first = false; sb.append(")"); @@ -2980,6 +2845,9 @@ public void validate() throws org.apache.thrift.TException { if (credentials != null) { credentials.validate(); } + if (taskUpdateObject != null) { + taskUpdateObject.validate(); + } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { @@ -2992,23 +2860,25 @@ 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 getCompactionJob_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class taskStatus_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public getCompactionJob_argsStandardScheme getScheme() { - return new getCompactionJob_argsStandardScheme(); + public taskStatus_argsStandardScheme getScheme() { + return new taskStatus_argsStandardScheme(); } } - private static class getCompactionJob_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class taskStatus_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { @Override - public void read(org.apache.thrift.protocol.TProtocol iprot, getCompactionJob_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, taskStatus_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -3036,26 +2906,19 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getCompactionJob_ar org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; - case 3: // GROUP_NAME - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.groupName = iprot.readString(); - struct.setGroupNameIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 4: // COMPACTOR - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.compactor = iprot.readString(); - struct.setCompactorIsSet(true); + case 3: // TIMESTAMP + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.timestamp = iprot.readI64(); + struct.setTimestampIsSet(true); } else { org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; - case 5: // EXTERNAL_COMPACTION_ID - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.externalCompactionId = iprot.readString(); - struct.setExternalCompactionIdIsSet(true); + case 4: // TASK_UPDATE_OBJECT + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.taskUpdateObject = new Task(); + struct.taskUpdateObject.read(iprot); + struct.setTaskUpdateObjectIsSet(true); } else { org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -3072,7 +2935,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getCompactionJob_ar } @Override - public void write(org.apache.thrift.protocol.TProtocol oprot, getCompactionJob_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, taskStatus_args struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -3086,19 +2949,12 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, getCompactionJob_a struct.credentials.write(oprot); oprot.writeFieldEnd(); } - if (struct.groupName != null) { - oprot.writeFieldBegin(GROUP_NAME_FIELD_DESC); - oprot.writeString(struct.groupName); - oprot.writeFieldEnd(); - } - if (struct.compactor != null) { - oprot.writeFieldBegin(COMPACTOR_FIELD_DESC); - oprot.writeString(struct.compactor); - oprot.writeFieldEnd(); - } - if (struct.externalCompactionId != null) { - oprot.writeFieldBegin(EXTERNAL_COMPACTION_ID_FIELD_DESC); - oprot.writeString(struct.externalCompactionId); + oprot.writeFieldBegin(TIMESTAMP_FIELD_DESC); + oprot.writeI64(struct.timestamp); + oprot.writeFieldEnd(); + if (struct.taskUpdateObject != null) { + oprot.writeFieldBegin(TASK_UPDATE_OBJECT_FIELD_DESC); + struct.taskUpdateObject.write(oprot); oprot.writeFieldEnd(); } oprot.writeFieldStop(); @@ -3107,17 +2963,17 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, getCompactionJob_a } - private static class getCompactionJob_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class taskStatus_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public getCompactionJob_argsTupleScheme getScheme() { - return new getCompactionJob_argsTupleScheme(); + public taskStatus_argsTupleScheme getScheme() { + return new taskStatus_argsTupleScheme(); } } - private static class getCompactionJob_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class taskStatus_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, getCompactionJob_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, taskStatus_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()) { @@ -3126,37 +2982,31 @@ public void write(org.apache.thrift.protocol.TProtocol prot, getCompactionJob_ar if (struct.isSetCredentials()) { optionals.set(1); } - if (struct.isSetGroupName()) { + if (struct.isSetTimestamp()) { optionals.set(2); } - if (struct.isSetCompactor()) { + if (struct.isSetTaskUpdateObject()) { optionals.set(3); } - if (struct.isSetExternalCompactionId()) { - optionals.set(4); - } - oprot.writeBitSet(optionals, 5); + oprot.writeBitSet(optionals, 4); if (struct.isSetTinfo()) { struct.tinfo.write(oprot); } if (struct.isSetCredentials()) { struct.credentials.write(oprot); } - if (struct.isSetGroupName()) { - oprot.writeString(struct.groupName); - } - if (struct.isSetCompactor()) { - oprot.writeString(struct.compactor); + if (struct.isSetTimestamp()) { + oprot.writeI64(struct.timestamp); } - if (struct.isSetExternalCompactionId()) { - oprot.writeString(struct.externalCompactionId); + if (struct.isSetTaskUpdateObject()) { + struct.taskUpdateObject.write(oprot); } } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, getCompactionJob_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, taskStatus_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(5); + 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); @@ -3168,399 +3018,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, getCompactionJob_arg struct.setCredentialsIsSet(true); } if (incoming.get(2)) { - struct.groupName = iprot.readString(); - struct.setGroupNameIsSet(true); + struct.timestamp = iprot.readI64(); + struct.setTimestampIsSet(true); } if (incoming.get(3)) { - struct.compactor = iprot.readString(); - struct.setCompactorIsSet(true); - } - if (incoming.get(4)) { - 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(); - } - } - - @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) - public static class getCompactionJob_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("getCompactionJob_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.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getCompactionJob_resultStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getCompactionJob_resultTupleSchemeFactory(); - - public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob success; // 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"); - - 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; - 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, org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob.class))); - metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getCompactionJob_result.class, metaDataMap); - } - - public getCompactionJob_result() { - } - - public getCompactionJob_result( - org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public getCompactionJob_result(getCompactionJob_result other) { - if (other.isSetSuccess()) { - this.success = new org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob(other.success); - } - } - - @Override - public getCompactionJob_result deepCopy() { - return new getCompactionJob_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - @org.apache.thrift.annotation.Nullable - public org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob getSuccess() { - return this.success; - } - - public getCompactionJob_result setSuccess(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob 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; - } - } - - @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((org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob)value); - } - break; - - } - } - - @org.apache.thrift.annotation.Nullable - @Override - public java.lang.Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return getSuccess(); - - } - 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(); - } - throw new java.lang.IllegalStateException(); - } - - @Override - public boolean equals(java.lang.Object that) { - if (that instanceof getCompactionJob_result) - return this.equals((getCompactionJob_result)that); - return false; - } - - public boolean equals(getCompactionJob_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; - } - - return true; - } - - @Override - public int hashCode() { - int hashCode = 1; - - hashCode = hashCode * 8191 + ((isSetSuccess()) ? 131071 : 524287); - if (isSetSuccess()) - hashCode = hashCode * 8191 + success.hashCode(); - - return hashCode; - } - - @Override - public int compareTo(getCompactionJob_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; - } - } - 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("getCompactionJob_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - 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 getCompactionJob_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - @Override - public getCompactionJob_resultStandardScheme getScheme() { - return new getCompactionJob_resultStandardScheme(); - } - } - - private static class getCompactionJob_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { - - @Override - public void read(org.apache.thrift.protocol.TProtocol iprot, getCompactionJob_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 org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob(); - struct.success.read(iprot); - struct.setSuccessIsSet(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, getCompactionJob_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(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class getCompactionJob_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - @Override - public getCompactionJob_resultTupleScheme getScheme() { - return new getCompactionJob_resultTupleScheme(); - } - } - - private static class getCompactionJob_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, getCompactionJob_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); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetSuccess()) { - struct.success.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, getCompactionJob_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(1); - if (incoming.get(0)) { - struct.success = new org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); + struct.taskUpdateObject = new Task(); + struct.taskUpdateObject.read(iprot); + struct.setTaskUpdateObjectIsSet(true); } } } @@ -3571,31 +3035,25 @@ private static S scheme(org.apache. } @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) - public static class updateCompactionStatus_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("updateCompactionStatus_args"); + public static class taskCompleted_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("taskCompleted_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.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)4); - private static final org.apache.thrift.protocol.TField TIMESTAMP_FIELD_DESC = new org.apache.thrift.protocol.TField("timestamp", org.apache.thrift.protocol.TType.I64, (short)5); + private static final org.apache.thrift.protocol.TField TASK_FIELD_DESC = new org.apache.thrift.protocol.TField("task", org.apache.thrift.protocol.TType.STRUCT, (short)3); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new updateCompactionStatus_argsStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new updateCompactionStatus_argsTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new taskCompleted_argsStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new taskCompleted_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 - public @org.apache.thrift.annotation.Nullable TCompactionStatusUpdate status; // required - public long timestamp; // required + public @org.apache.thrift.annotation.Nullable Task task; // 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"), - STATUS((short)4, "status"), - TIMESTAMP((short)5, "timestamp"); + TASK((short)3, "task"); private static final java.util.Map byName = new java.util.HashMap(); @@ -3615,12 +3073,8 @@ public static _Fields findByThriftId(int fieldId) { return TINFO; case 2: // CREDENTIALS return CREDENTIALS; - case 3: // EXTERNAL_COMPACTION_ID - return EXTERNAL_COMPACTION_ID; - case 4: // STATUS - return STATUS; - case 5: // TIMESTAMP - return TIMESTAMP; + case 3: // TASK + return TASK; default: return null; } @@ -3664,8 +3118,6 @@ public java.lang.String getFieldName() { } // isset id assignments - private static final int __TIMESTAMP_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); @@ -3673,68 +3125,51 @@ 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.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))); - tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TCompactionStatusUpdate.class))); - tmpMap.put(_Fields.TIMESTAMP, new org.apache.thrift.meta_data.FieldMetaData("timestamp", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + tmpMap.put(_Fields.TASK, new org.apache.thrift.meta_data.FieldMetaData("task", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, Task.class))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(updateCompactionStatus_args.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(taskCompleted_args.class, metaDataMap); } - public updateCompactionStatus_args() { + public taskCompleted_args() { } - public updateCompactionStatus_args( + public taskCompleted_args( org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, - java.lang.String externalCompactionId, - TCompactionStatusUpdate status, - long timestamp) + Task task) { this(); this.tinfo = tinfo; this.credentials = credentials; - this.externalCompactionId = externalCompactionId; - this.status = status; - this.timestamp = timestamp; - setTimestampIsSet(true); + this.task = task; } /** * Performs a deep copy on other. */ - public updateCompactionStatus_args(updateCompactionStatus_args other) { - __isset_bitfield = other.__isset_bitfield; + public taskCompleted_args(taskCompleted_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; + if (other.isSetTask()) { + this.task = new Task(other.task); } - if (other.isSetStatus()) { - this.status = new TCompactionStatusUpdate(other.status); - } - this.timestamp = other.timestamp; } @Override - public updateCompactionStatus_args deepCopy() { - return new updateCompactionStatus_args(this); + public taskCompleted_args deepCopy() { + return new taskCompleted_args(this); } @Override public void clear() { this.tinfo = null; this.credentials = null; - this.externalCompactionId = null; - this.status = null; - setTimestampIsSet(false); - this.timestamp = 0; + this.task = null; } @org.apache.thrift.annotation.Nullable @@ -3742,7 +3177,7 @@ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() { return this.tinfo; } - public updateCompactionStatus_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) { + public taskCompleted_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) { this.tinfo = tinfo; return this; } @@ -3767,7 +3202,7 @@ public org.apache.accumulo.core.securityImpl.thrift.TCredentials getCredentials( return this.credentials; } - public updateCompactionStatus_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) { + public taskCompleted_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) { this.credentials = credentials; return this; } @@ -3788,76 +3223,28 @@ public void setCredentialsIsSet(boolean value) { } @org.apache.thrift.annotation.Nullable - public java.lang.String getExternalCompactionId() { - return this.externalCompactionId; - } - - public updateCompactionStatus_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; - } - } - - @org.apache.thrift.annotation.Nullable - public TCompactionStatusUpdate getStatus() { - return this.status; - } - - public updateCompactionStatus_args setStatus(@org.apache.thrift.annotation.Nullable TCompactionStatusUpdate status) { - this.status = status; - return this; + public Task getTask() { + return this.task; } - - public void unsetStatus() { - this.status = null; - } - - /** Returns true if field status is set (has been assigned a value) and false otherwise */ - public boolean isSetStatus() { - return this.status != null; - } - - public void setStatusIsSet(boolean value) { - if (!value) { - this.status = null; - } - } - - public long getTimestamp() { - return this.timestamp; - } - - public updateCompactionStatus_args setTimestamp(long timestamp) { - this.timestamp = timestamp; - setTimestampIsSet(true); + + public taskCompleted_args setTask(@org.apache.thrift.annotation.Nullable Task task) { + this.task = task; return this; } - public void unsetTimestamp() { - __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __TIMESTAMP_ISSET_ID); + public void unsetTask() { + this.task = null; } - /** Returns true if field timestamp is set (has been assigned a value) and false otherwise */ - public boolean isSetTimestamp() { - return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __TIMESTAMP_ISSET_ID); + /** Returns true if field task is set (has been assigned a value) and false otherwise */ + public boolean isSetTask() { + return this.task != null; } - public void setTimestampIsSet(boolean value) { - __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __TIMESTAMP_ISSET_ID, value); + public void setTaskIsSet(boolean value) { + if (!value) { + this.task = null; + } } @Override @@ -3879,27 +3266,11 @@ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable } break; - case EXTERNAL_COMPACTION_ID: - if (value == null) { - unsetExternalCompactionId(); - } else { - setExternalCompactionId((java.lang.String)value); - } - break; - - case STATUS: - if (value == null) { - unsetStatus(); - } else { - setStatus((TCompactionStatusUpdate)value); - } - break; - - case TIMESTAMP: + case TASK: if (value == null) { - unsetTimestamp(); + unsetTask(); } else { - setTimestamp((java.lang.Long)value); + setTask((Task)value); } break; @@ -3916,14 +3287,8 @@ public java.lang.Object getFieldValue(_Fields field) { case CREDENTIALS: return getCredentials(); - case EXTERNAL_COMPACTION_ID: - return getExternalCompactionId(); - - case STATUS: - return getStatus(); - - case TIMESTAMP: - return getTimestamp(); + case TASK: + return getTask(); } throw new java.lang.IllegalStateException(); @@ -3941,24 +3306,20 @@ public boolean isSet(_Fields field) { return isSetTinfo(); case CREDENTIALS: return isSetCredentials(); - case EXTERNAL_COMPACTION_ID: - return isSetExternalCompactionId(); - case STATUS: - return isSetStatus(); - case TIMESTAMP: - return isSetTimestamp(); + case TASK: + return isSetTask(); } throw new java.lang.IllegalStateException(); } @Override public boolean equals(java.lang.Object that) { - if (that instanceof updateCompactionStatus_args) - return this.equals((updateCompactionStatus_args)that); + if (that instanceof taskCompleted_args) + return this.equals((taskCompleted_args)that); return false; } - public boolean equals(updateCompactionStatus_args that) { + public boolean equals(taskCompleted_args that) { if (that == null) return false; if (this == that) @@ -3982,30 +3343,12 @@ public boolean equals(updateCompactionStatus_args that) { 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; - } - - boolean this_present_status = true && this.isSetStatus(); - boolean that_present_status = true && that.isSetStatus(); - if (this_present_status || that_present_status) { - if (!(this_present_status && that_present_status)) - return false; - if (!this.status.equals(that.status)) - return false; - } - - boolean this_present_timestamp = true; - boolean that_present_timestamp = true; - if (this_present_timestamp || that_present_timestamp) { - if (!(this_present_timestamp && that_present_timestamp)) + boolean this_present_task = true && this.isSetTask(); + boolean that_present_task = true && that.isSetTask(); + if (this_present_task || that_present_task) { + if (!(this_present_task && that_present_task)) return false; - if (this.timestamp != that.timestamp) + if (!this.task.equals(that.task)) return false; } @@ -4024,21 +3367,15 @@ public int hashCode() { if (isSetCredentials()) hashCode = hashCode * 8191 + credentials.hashCode(); - hashCode = hashCode * 8191 + ((isSetExternalCompactionId()) ? 131071 : 524287); - if (isSetExternalCompactionId()) - hashCode = hashCode * 8191 + externalCompactionId.hashCode(); - - hashCode = hashCode * 8191 + ((isSetStatus()) ? 131071 : 524287); - if (isSetStatus()) - hashCode = hashCode * 8191 + status.hashCode(); - - hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(timestamp); + hashCode = hashCode * 8191 + ((isSetTask()) ? 131071 : 524287); + if (isSetTask()) + hashCode = hashCode * 8191 + task.hashCode(); return hashCode; } @Override - public int compareTo(updateCompactionStatus_args other) { + public int compareTo(taskCompleted_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -4065,32 +3402,12 @@ public int compareTo(updateCompactionStatus_args other) { 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; - } - } - lastComparison = java.lang.Boolean.compare(isSetStatus(), other.isSetStatus()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStatus()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, other.status); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = java.lang.Boolean.compare(isSetTimestamp(), other.isSetTimestamp()); + lastComparison = java.lang.Boolean.compare(isSetTask(), other.isSetTask()); if (lastComparison != 0) { return lastComparison; } - if (isSetTimestamp()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.timestamp, other.timestamp); + if (isSetTask()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.task, other.task); if (lastComparison != 0) { return lastComparison; } @@ -4116,7 +3433,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("updateCompactionStatus_args("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("taskCompleted_args("); boolean first = true; sb.append("tinfo:"); @@ -4135,25 +3452,13 @@ public java.lang.String toString() { } first = false; if (!first) sb.append(", "); - sb.append("externalCompactionId:"); - if (this.externalCompactionId == null) { - sb.append("null"); - } else { - sb.append(this.externalCompactionId); - } - first = false; - if (!first) sb.append(", "); - sb.append("status:"); - if (this.status == null) { + sb.append("task:"); + if (this.task == null) { sb.append("null"); } else { - sb.append(this.status); + sb.append(this.task); } first = false; - if (!first) sb.append(", "); - sb.append("timestamp:"); - sb.append(this.timestamp); - first = false; sb.append(")"); return sb.toString(); } @@ -4167,8 +3472,8 @@ public void validate() throws org.apache.thrift.TException { if (credentials != null) { credentials.validate(); } - if (status != null) { - status.validate(); + if (task != null) { + task.validate(); } } @@ -4182,25 +3487,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 updateCompactionStatus_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class taskCompleted_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public updateCompactionStatus_argsStandardScheme getScheme() { - return new updateCompactionStatus_argsStandardScheme(); + public taskCompleted_argsStandardScheme getScheme() { + return new taskCompleted_argsStandardScheme(); } } - private static class updateCompactionStatus_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class taskCompleted_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { @Override - public void read(org.apache.thrift.protocol.TProtocol iprot, updateCompactionStatus_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, taskCompleted_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -4228,27 +3531,11 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, updateCompactionSta 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; - case 4: // STATUS + case 3: // TASK if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.status = new TCompactionStatusUpdate(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 5: // TIMESTAMP - if (schemeField.type == org.apache.thrift.protocol.TType.I64) { - struct.timestamp = iprot.readI64(); - struct.setTimestampIsSet(true); + struct.task = new Task(); + struct.task.read(iprot); + struct.setTaskIsSet(true); } else { org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -4265,7 +3552,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, updateCompactionSta } @Override - public void write(org.apache.thrift.protocol.TProtocol oprot, updateCompactionStatus_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, taskCompleted_args struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -4279,36 +3566,28 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, updateCompactionSt struct.credentials.write(oprot); oprot.writeFieldEnd(); } - if (struct.externalCompactionId != null) { - oprot.writeFieldBegin(EXTERNAL_COMPACTION_ID_FIELD_DESC); - oprot.writeString(struct.externalCompactionId); - oprot.writeFieldEnd(); - } - if (struct.status != null) { - oprot.writeFieldBegin(STATUS_FIELD_DESC); - struct.status.write(oprot); + if (struct.task != null) { + oprot.writeFieldBegin(TASK_FIELD_DESC); + struct.task.write(oprot); oprot.writeFieldEnd(); } - oprot.writeFieldBegin(TIMESTAMP_FIELD_DESC); - oprot.writeI64(struct.timestamp); - oprot.writeFieldEnd(); oprot.writeFieldStop(); oprot.writeStructEnd(); } } - private static class updateCompactionStatus_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class taskCompleted_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public updateCompactionStatus_argsTupleScheme getScheme() { - return new updateCompactionStatus_argsTupleScheme(); + public taskCompleted_argsTupleScheme getScheme() { + return new taskCompleted_argsTupleScheme(); } } - private static class updateCompactionStatus_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class taskCompleted_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, updateCompactionStatus_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, taskCompleted_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()) { @@ -4317,37 +3596,25 @@ public void write(org.apache.thrift.protocol.TProtocol prot, updateCompactionSta if (struct.isSetCredentials()) { optionals.set(1); } - if (struct.isSetExternalCompactionId()) { + if (struct.isSetTask()) { optionals.set(2); } - if (struct.isSetStatus()) { - optionals.set(3); - } - if (struct.isSetTimestamp()) { - optionals.set(4); - } - oprot.writeBitSet(optionals, 5); + 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); - } - if (struct.isSetStatus()) { - struct.status.write(oprot); - } - if (struct.isSetTimestamp()) { - oprot.writeI64(struct.timestamp); + if (struct.isSetTask()) { + struct.task.write(oprot); } } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, updateCompactionStatus_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, taskCompleted_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(5); + 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); @@ -4359,17 +3626,9 @@ public void read(org.apache.thrift.protocol.TProtocol prot, updateCompactionStat struct.setCredentialsIsSet(true); } if (incoming.get(2)) { - struct.externalCompactionId = iprot.readString(); - struct.setExternalCompactionIdIsSet(true); - } - if (incoming.get(3)) { - struct.status = new TCompactionStatusUpdate(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } - if (incoming.get(4)) { - struct.timestamp = iprot.readI64(); - struct.setTimestampIsSet(true); + struct.task = new Task(); + struct.task.read(iprot); + struct.setTaskIsSet(true); } } } @@ -4380,12 +3639,12 @@ private static S scheme(org.apache. } @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) - public static class updateCompactionStatus_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("updateCompactionStatus_result"); + public static class taskCompleted_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("taskCompleted_result"); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new updateCompactionStatus_resultStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new updateCompactionStatus_resultTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new taskCompleted_resultStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new taskCompleted_resultTupleSchemeFactory(); /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ @@ -4451,21 +3710,21 @@ public java.lang.String getFieldName() { 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(updateCompactionStatus_result.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(taskCompleted_result.class, metaDataMap); } - public updateCompactionStatus_result() { + public taskCompleted_result() { } /** * Performs a deep copy on other. */ - public updateCompactionStatus_result(updateCompactionStatus_result other) { + public taskCompleted_result(taskCompleted_result other) { } @Override - public updateCompactionStatus_result deepCopy() { - return new updateCompactionStatus_result(this); + public taskCompleted_result deepCopy() { + return new taskCompleted_result(this); } @Override @@ -4500,12 +3759,12 @@ public boolean isSet(_Fields field) { @Override public boolean equals(java.lang.Object that) { - if (that instanceof updateCompactionStatus_result) - return this.equals((updateCompactionStatus_result)that); + if (that instanceof taskCompleted_result) + return this.equals((taskCompleted_result)that); return false; } - public boolean equals(updateCompactionStatus_result that) { + public boolean equals(taskCompleted_result that) { if (that == null) return false; if (this == that) @@ -4522,7 +3781,7 @@ public int hashCode() { } @Override - public int compareTo(updateCompactionStatus_result other) { + public int compareTo(taskCompleted_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -4549,7 +3808,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("updateCompactionStatus_result("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("taskCompleted_result("); boolean first = true; sb.append(")"); @@ -4577,17 +3836,17 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class updateCompactionStatus_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class taskCompleted_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public updateCompactionStatus_resultStandardScheme getScheme() { - return new updateCompactionStatus_resultStandardScheme(); + public taskCompleted_resultStandardScheme getScheme() { + return new taskCompleted_resultStandardScheme(); } } - private static class updateCompactionStatus_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class taskCompleted_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { @Override - public void read(org.apache.thrift.protocol.TProtocol iprot, updateCompactionStatus_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, taskCompleted_result struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -4609,7 +3868,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, updateCompactionSta } @Override - public void write(org.apache.thrift.protocol.TProtocol oprot, updateCompactionStatus_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, taskCompleted_result struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -4619,22 +3878,22 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, updateCompactionSt } - private static class updateCompactionStatus_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class taskCompleted_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public updateCompactionStatus_resultTupleScheme getScheme() { - return new updateCompactionStatus_resultTupleScheme(); + public taskCompleted_resultTupleScheme getScheme() { + return new taskCompleted_resultTupleScheme(); } } - private static class updateCompactionStatus_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class taskCompleted_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, updateCompactionStatus_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, taskCompleted_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, updateCompactionStatus_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, taskCompleted_result struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; } } @@ -4645,28 +3904,25 @@ private static S scheme(org.apache. } @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) - public static class compactionFailed_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("compactionFailed_args"); + public static class taskFailed_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("taskFailed_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.protocol.TField EXTENT_FIELD_DESC = new org.apache.thrift.protocol.TField("extent", org.apache.thrift.protocol.TType.STRUCT, (short)4); + private static final org.apache.thrift.protocol.TField TASK_FIELD_DESC = new org.apache.thrift.protocol.TField("task", org.apache.thrift.protocol.TType.STRUCT, (short)3); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new compactionFailed_argsStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new compactionFailed_argsTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new taskFailed_argsStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new taskFailed_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 - public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent; // required + public @org.apache.thrift.annotation.Nullable Task task; // 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"), - EXTENT((short)4, "extent"); + TASK((short)3, "task"); private static final java.util.Map byName = new java.util.HashMap(); @@ -4686,10 +3942,8 @@ public static _Fields findByThriftId(int fieldId) { return TINFO; case 2: // CREDENTIALS return CREDENTIALS; - case 3: // EXTERNAL_COMPACTION_ID - return EXTERNAL_COMPACTION_ID; - case 4: // EXTENT - return EXTENT; + case 3: // TASK + return TASK; default: return null; } @@ -4740,59 +3994,51 @@ 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.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))); - 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))); + tmpMap.put(_Fields.TASK, new org.apache.thrift.meta_data.FieldMetaData("task", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, Task.class))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(compactionFailed_args.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(taskFailed_args.class, metaDataMap); } - public compactionFailed_args() { + public taskFailed_args() { } - public compactionFailed_args( + public taskFailed_args( org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, - java.lang.String externalCompactionId, - org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) + Task task) { this(); this.tinfo = tinfo; this.credentials = credentials; - this.externalCompactionId = externalCompactionId; - this.extent = extent; + this.task = task; } /** * Performs a deep copy on other. */ - public compactionFailed_args(compactionFailed_args other) { + public taskFailed_args(taskFailed_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; - } - if (other.isSetExtent()) { - this.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(other.extent); + if (other.isSetTask()) { + this.task = new Task(other.task); } } @Override - public compactionFailed_args deepCopy() { - return new compactionFailed_args(this); + public taskFailed_args deepCopy() { + return new taskFailed_args(this); } @Override public void clear() { this.tinfo = null; this.credentials = null; - this.externalCompactionId = null; - this.extent = null; + this.task = null; } @org.apache.thrift.annotation.Nullable @@ -4800,7 +4046,7 @@ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() { return this.tinfo; } - public compactionFailed_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) { + public taskFailed_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) { this.tinfo = tinfo; return this; } @@ -4825,7 +4071,7 @@ public org.apache.accumulo.core.securityImpl.thrift.TCredentials getCredentials( return this.credentials; } - public compactionFailed_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) { + public taskFailed_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) { this.credentials = credentials; return this; } @@ -4846,52 +4092,27 @@ public void setCredentialsIsSet(boolean value) { } @org.apache.thrift.annotation.Nullable - public java.lang.String getExternalCompactionId() { - return this.externalCompactionId; - } - - public compactionFailed_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; - } - } - - @org.apache.thrift.annotation.Nullable - public org.apache.accumulo.core.dataImpl.thrift.TKeyExtent getExtent() { - return this.extent; + public Task getTask() { + return this.task; } - public compactionFailed_args setExtent(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) { - this.extent = extent; + public taskFailed_args setTask(@org.apache.thrift.annotation.Nullable Task task) { + this.task = task; return this; } - public void unsetExtent() { - this.extent = null; + public void unsetTask() { + this.task = null; } - /** Returns true if field extent is set (has been assigned a value) and false otherwise */ - public boolean isSetExtent() { - return this.extent != null; + /** Returns true if field task is set (has been assigned a value) and false otherwise */ + public boolean isSetTask() { + return this.task != null; } - public void setExtentIsSet(boolean value) { + public void setTaskIsSet(boolean value) { if (!value) { - this.extent = null; + this.task = null; } } @@ -4914,19 +4135,11 @@ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable } break; - case EXTERNAL_COMPACTION_ID: - if (value == null) { - unsetExternalCompactionId(); - } else { - setExternalCompactionId((java.lang.String)value); - } - break; - - case EXTENT: + case TASK: if (value == null) { - unsetExtent(); + unsetTask(); } else { - setExtent((org.apache.accumulo.core.dataImpl.thrift.TKeyExtent)value); + setTask((Task)value); } break; @@ -4943,11 +4156,8 @@ public java.lang.Object getFieldValue(_Fields field) { case CREDENTIALS: return getCredentials(); - case EXTERNAL_COMPACTION_ID: - return getExternalCompactionId(); - - case EXTENT: - return getExtent(); + case TASK: + return getTask(); } throw new java.lang.IllegalStateException(); @@ -4965,22 +4175,20 @@ public boolean isSet(_Fields field) { return isSetTinfo(); case CREDENTIALS: return isSetCredentials(); - case EXTERNAL_COMPACTION_ID: - return isSetExternalCompactionId(); - case EXTENT: - return isSetExtent(); + case TASK: + return isSetTask(); } throw new java.lang.IllegalStateException(); } @Override public boolean equals(java.lang.Object that) { - if (that instanceof compactionFailed_args) - return this.equals((compactionFailed_args)that); + if (that instanceof taskFailed_args) + return this.equals((taskFailed_args)that); return false; } - public boolean equals(compactionFailed_args that) { + public boolean equals(taskFailed_args that) { if (that == null) return false; if (this == that) @@ -5004,21 +4212,12 @@ public boolean equals(compactionFailed_args that) { 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; - } - - 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)) + boolean this_present_task = true && this.isSetTask(); + boolean that_present_task = true && that.isSetTask(); + if (this_present_task || that_present_task) { + if (!(this_present_task && that_present_task)) return false; - if (!this.extent.equals(that.extent)) + if (!this.task.equals(that.task)) return false; } @@ -5037,19 +4236,15 @@ public int hashCode() { if (isSetCredentials()) hashCode = hashCode * 8191 + credentials.hashCode(); - hashCode = hashCode * 8191 + ((isSetExternalCompactionId()) ? 131071 : 524287); - if (isSetExternalCompactionId()) - hashCode = hashCode * 8191 + externalCompactionId.hashCode(); - - hashCode = hashCode * 8191 + ((isSetExtent()) ? 131071 : 524287); - if (isSetExtent()) - hashCode = hashCode * 8191 + extent.hashCode(); + hashCode = hashCode * 8191 + ((isSetTask()) ? 131071 : 524287); + if (isSetTask()) + hashCode = hashCode * 8191 + task.hashCode(); return hashCode; } @Override - public int compareTo(compactionFailed_args other) { + public int compareTo(taskFailed_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -5076,22 +4271,12 @@ public int compareTo(compactionFailed_args other) { 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; - } - } - lastComparison = java.lang.Boolean.compare(isSetExtent(), other.isSetExtent()); + lastComparison = java.lang.Boolean.compare(isSetTask(), other.isSetTask()); if (lastComparison != 0) { return lastComparison; } - if (isSetExtent()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.extent, other.extent); + if (isSetTask()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.task, other.task); if (lastComparison != 0) { return lastComparison; } @@ -5117,7 +4302,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("compactionFailed_args("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("taskFailed_args("); boolean first = true; sb.append("tinfo:"); @@ -5136,19 +4321,11 @@ public java.lang.String toString() { } first = false; if (!first) sb.append(", "); - sb.append("externalCompactionId:"); - if (this.externalCompactionId == null) { - sb.append("null"); - } else { - sb.append(this.externalCompactionId); - } - first = false; - if (!first) sb.append(", "); - sb.append("extent:"); - if (this.extent == null) { + sb.append("task:"); + if (this.task == null) { sb.append("null"); } else { - sb.append(this.extent); + sb.append(this.task); } first = false; sb.append(")"); @@ -5164,8 +4341,8 @@ public void validate() throws org.apache.thrift.TException { if (credentials != null) { credentials.validate(); } - if (extent != null) { - extent.validate(); + if (task != null) { + task.validate(); } } @@ -5185,17 +4362,17 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class compactionFailed_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class taskFailed_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public compactionFailed_argsStandardScheme getScheme() { - return new compactionFailed_argsStandardScheme(); + public taskFailed_argsStandardScheme getScheme() { + return new taskFailed_argsStandardScheme(); } } - private static class compactionFailed_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class taskFailed_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { @Override - public void read(org.apache.thrift.protocol.TProtocol iprot, compactionFailed_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, taskFailed_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -5223,19 +4400,11 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, compactionFailed_ar 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; - case 4: // EXTENT + case 3: // TASK 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); + struct.task = new Task(); + struct.task.read(iprot); + struct.setTaskIsSet(true); } else { org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -5252,7 +4421,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, compactionFailed_ar } @Override - public void write(org.apache.thrift.protocol.TProtocol oprot, compactionFailed_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, taskFailed_args struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -5266,14 +4435,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, compactionFailed_a struct.credentials.write(oprot); oprot.writeFieldEnd(); } - if (struct.externalCompactionId != null) { - oprot.writeFieldBegin(EXTERNAL_COMPACTION_ID_FIELD_DESC); - oprot.writeString(struct.externalCompactionId); - oprot.writeFieldEnd(); - } - if (struct.extent != null) { - oprot.writeFieldBegin(EXTENT_FIELD_DESC); - struct.extent.write(oprot); + if (struct.task != null) { + oprot.writeFieldBegin(TASK_FIELD_DESC); + struct.task.write(oprot); oprot.writeFieldEnd(); } oprot.writeFieldStop(); @@ -5282,17 +4446,17 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, compactionFailed_a } - private static class compactionFailed_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class taskFailed_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public compactionFailed_argsTupleScheme getScheme() { - return new compactionFailed_argsTupleScheme(); + public taskFailed_argsTupleScheme getScheme() { + return new taskFailed_argsTupleScheme(); } } - private static class compactionFailed_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class taskFailed_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, compactionFailed_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, taskFailed_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()) { @@ -5301,31 +4465,25 @@ public void write(org.apache.thrift.protocol.TProtocol prot, compactionFailed_ar if (struct.isSetCredentials()) { optionals.set(1); } - if (struct.isSetExternalCompactionId()) { + if (struct.isSetTask()) { optionals.set(2); } - if (struct.isSetExtent()) { - optionals.set(3); - } - oprot.writeBitSet(optionals, 4); + 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); - } - if (struct.isSetExtent()) { - struct.extent.write(oprot); + if (struct.isSetTask()) { + struct.task.write(oprot); } } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, compactionFailed_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, taskFailed_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); + 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); @@ -5337,13 +4495,9 @@ public void read(org.apache.thrift.protocol.TProtocol prot, compactionFailed_arg struct.setCredentialsIsSet(true); } if (incoming.get(2)) { - struct.externalCompactionId = iprot.readString(); - struct.setExternalCompactionIdIsSet(true); - } - if (incoming.get(3)) { - struct.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(); - struct.extent.read(iprot); - struct.setExtentIsSet(true); + struct.task = new Task(); + struct.task.read(iprot); + struct.setTaskIsSet(true); } } } @@ -5354,12 +4508,12 @@ private static S scheme(org.apache. } @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) - public static class compactionFailed_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("compactionFailed_result"); + public static class taskFailed_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("taskFailed_result"); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new compactionFailed_resultStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new compactionFailed_resultTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new taskFailed_resultStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new taskFailed_resultTupleSchemeFactory(); /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ @@ -5425,21 +4579,21 @@ public java.lang.String getFieldName() { 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(compactionFailed_result.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(taskFailed_result.class, metaDataMap); } - public compactionFailed_result() { + public taskFailed_result() { } /** * Performs a deep copy on other. */ - public compactionFailed_result(compactionFailed_result other) { + public taskFailed_result(taskFailed_result other) { } @Override - public compactionFailed_result deepCopy() { - return new compactionFailed_result(this); + public taskFailed_result deepCopy() { + return new taskFailed_result(this); } @Override @@ -5474,12 +4628,12 @@ public boolean isSet(_Fields field) { @Override public boolean equals(java.lang.Object that) { - if (that instanceof compactionFailed_result) - return this.equals((compactionFailed_result)that); + if (that instanceof taskFailed_result) + return this.equals((taskFailed_result)that); return false; } - public boolean equals(compactionFailed_result that) { + public boolean equals(taskFailed_result that) { if (that == null) return false; if (this == that) @@ -5496,7 +4650,7 @@ public int hashCode() { } @Override - public int compareTo(compactionFailed_result other) { + public int compareTo(taskFailed_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -5523,7 +4677,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("compactionFailed_result("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("taskFailed_result("); boolean first = true; sb.append(")"); @@ -5551,17 +4705,17 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class compactionFailed_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class taskFailed_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public compactionFailed_resultStandardScheme getScheme() { - return new compactionFailed_resultStandardScheme(); + public taskFailed_resultStandardScheme getScheme() { + return new taskFailed_resultStandardScheme(); } } - private static class compactionFailed_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class taskFailed_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { @Override - public void read(org.apache.thrift.protocol.TProtocol iprot, compactionFailed_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, taskFailed_result struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -5583,7 +4737,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, compactionFailed_re } @Override - public void write(org.apache.thrift.protocol.TProtocol oprot, compactionFailed_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, taskFailed_result struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -5593,22 +4747,22 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, compactionFailed_r } - private static class compactionFailed_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class taskFailed_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public compactionFailed_resultTupleScheme getScheme() { - return new compactionFailed_resultTupleScheme(); + public taskFailed_resultTupleScheme getScheme() { + return new taskFailed_resultTupleScheme(); } } - private static class compactionFailed_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class taskFailed_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, compactionFailed_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, taskFailed_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, compactionFailed_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, taskFailed_result struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; } } @@ -5619,22 +4773,25 @@ private static S scheme(org.apache. } @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) - public static class getRunningCompactions_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("getRunningCompactions_args"); + public static class cancelTask_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("cancelTask_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 TASK_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("taskID", org.apache.thrift.protocol.TType.STRING, (short)3); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getRunningCompactions_argsStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getRunningCompactions_argsTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new cancelTask_argsStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new cancelTask_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 taskID; // 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"), + TASK_ID((short)3, "taskID"); private static final java.util.Map byName = new java.util.HashMap(); @@ -5654,6 +4811,8 @@ public static _Fields findByThriftId(int fieldId) { return TINFO; case 2: // CREDENTIALS return CREDENTIALS; + case 3: // TASK_ID + return TASK_ID; default: return null; } @@ -5704,43 +4863,51 @@ 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.TASK_ID, new org.apache.thrift.meta_data.FieldMetaData("taskID", 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(getRunningCompactions_args.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(cancelTask_args.class, metaDataMap); } - public getRunningCompactions_args() { + public cancelTask_args() { } - public getRunningCompactions_args( + public cancelTask_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.lang.String taskID) { this(); this.tinfo = tinfo; this.credentials = credentials; + this.taskID = taskID; } /** * Performs a deep copy on other. */ - public getRunningCompactions_args(getRunningCompactions_args other) { + public cancelTask_args(cancelTask_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.isSetTaskID()) { + this.taskID = other.taskID; + } } @Override - public getRunningCompactions_args deepCopy() { - return new getRunningCompactions_args(this); + public cancelTask_args deepCopy() { + return new cancelTask_args(this); } @Override public void clear() { this.tinfo = null; this.credentials = null; + this.taskID = null; } @org.apache.thrift.annotation.Nullable @@ -5748,7 +4915,7 @@ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() { return this.tinfo; } - public getRunningCompactions_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) { + public cancelTask_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) { this.tinfo = tinfo; return this; } @@ -5773,7 +4940,7 @@ public org.apache.accumulo.core.securityImpl.thrift.TCredentials getCredentials( return this.credentials; } - public getRunningCompactions_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) { + public cancelTask_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) { this.credentials = credentials; return this; } @@ -5793,6 +4960,31 @@ public void setCredentialsIsSet(boolean value) { } } + @org.apache.thrift.annotation.Nullable + public java.lang.String getTaskID() { + return this.taskID; + } + + public cancelTask_args setTaskID(@org.apache.thrift.annotation.Nullable java.lang.String taskID) { + this.taskID = taskID; + return this; + } + + public void unsetTaskID() { + this.taskID = null; + } + + /** Returns true if field taskID is set (has been assigned a value) and false otherwise */ + public boolean isSetTaskID() { + return this.taskID != null; + } + + public void setTaskIDIsSet(boolean value) { + if (!value) { + this.taskID = null; + } + } + @Override public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) { switch (field) { @@ -5812,6 +5004,14 @@ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable } break; + case TASK_ID: + if (value == null) { + unsetTaskID(); + } else { + setTaskID((java.lang.String)value); + } + break; + } } @@ -5825,6 +5025,9 @@ public java.lang.Object getFieldValue(_Fields field) { case CREDENTIALS: return getCredentials(); + case TASK_ID: + return getTaskID(); + } throw new java.lang.IllegalStateException(); } @@ -5841,18 +5044,20 @@ public boolean isSet(_Fields field) { return isSetTinfo(); case CREDENTIALS: return isSetCredentials(); + case TASK_ID: + return isSetTaskID(); } throw new java.lang.IllegalStateException(); } @Override public boolean equals(java.lang.Object that) { - if (that instanceof getRunningCompactions_args) - return this.equals((getRunningCompactions_args)that); + if (that instanceof cancelTask_args) + return this.equals((cancelTask_args)that); return false; } - public boolean equals(getRunningCompactions_args that) { + public boolean equals(cancelTask_args that) { if (that == null) return false; if (this == that) @@ -5876,6 +5081,15 @@ public boolean equals(getRunningCompactions_args that) { return false; } + boolean this_present_taskID = true && this.isSetTaskID(); + boolean that_present_taskID = true && that.isSetTaskID(); + if (this_present_taskID || that_present_taskID) { + if (!(this_present_taskID && that_present_taskID)) + return false; + if (!this.taskID.equals(that.taskID)) + return false; + } + return true; } @@ -5891,11 +5105,15 @@ public int hashCode() { if (isSetCredentials()) hashCode = hashCode * 8191 + credentials.hashCode(); + hashCode = hashCode * 8191 + ((isSetTaskID()) ? 131071 : 524287); + if (isSetTaskID()) + hashCode = hashCode * 8191 + taskID.hashCode(); + return hashCode; } @Override - public int compareTo(getRunningCompactions_args other) { + public int compareTo(cancelTask_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -5922,6 +5140,16 @@ public int compareTo(getRunningCompactions_args other) { return lastComparison; } } + lastComparison = java.lang.Boolean.compare(isSetTaskID(), other.isSetTaskID()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTaskID()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.taskID, other.taskID); + if (lastComparison != 0) { + return lastComparison; + } + } return 0; } @@ -5943,7 +5171,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("getRunningCompactions_args("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("cancelTask_args("); boolean first = true; sb.append("tinfo:"); @@ -5961,6 +5189,14 @@ public java.lang.String toString() { sb.append(this.credentials); } first = false; + if (!first) sb.append(", "); + sb.append("taskID:"); + if (this.taskID == null) { + sb.append("null"); + } else { + sb.append(this.taskID); + } + first = false; sb.append(")"); return sb.toString(); } @@ -5992,17 +5228,17 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class getRunningCompactions_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class cancelTask_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public getRunningCompactions_argsStandardScheme getScheme() { - return new getRunningCompactions_argsStandardScheme(); + public cancelTask_argsStandardScheme getScheme() { + return new cancelTask_argsStandardScheme(); } } - private static class getRunningCompactions_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class cancelTask_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { @Override - public void read(org.apache.thrift.protocol.TProtocol iprot, getRunningCompactions_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, cancelTask_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -6030,6 +5266,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getRunningCompactio org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; + case 3: // TASK_ID + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.taskID = iprot.readString(); + struct.setTaskIDIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -6042,7 +5286,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getRunningCompactio } @Override - public void write(org.apache.thrift.protocol.TProtocol oprot, getRunningCompactions_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, cancelTask_args struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -6056,23 +5300,28 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, getRunningCompacti struct.credentials.write(oprot); oprot.writeFieldEnd(); } + if (struct.taskID != null) { + oprot.writeFieldBegin(TASK_ID_FIELD_DESC); + oprot.writeString(struct.taskID); + oprot.writeFieldEnd(); + } oprot.writeFieldStop(); oprot.writeStructEnd(); } } - private static class getRunningCompactions_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class cancelTask_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public getRunningCompactions_argsTupleScheme getScheme() { - return new getRunningCompactions_argsTupleScheme(); + public cancelTask_argsTupleScheme getScheme() { + return new cancelTask_argsTupleScheme(); } } - private static class getRunningCompactions_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class cancelTask_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, getRunningCompactions_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, cancelTask_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()) { @@ -6081,19 +5330,25 @@ public void write(org.apache.thrift.protocol.TProtocol prot, getRunningCompactio if (struct.isSetCredentials()) { optionals.set(1); } - oprot.writeBitSet(optionals, 2); + if (struct.isSetTaskID()) { + optionals.set(2); + } + oprot.writeBitSet(optionals, 3); if (struct.isSetTinfo()) { struct.tinfo.write(oprot); } if (struct.isSetCredentials()) { struct.credentials.write(oprot); } + if (struct.isSetTaskID()) { + oprot.writeString(struct.taskID); + } } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, getRunningCompactions_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, cancelTask_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); @@ -6104,6 +5359,10 @@ public void read(org.apache.thrift.protocol.TProtocol prot, getRunningCompaction struct.credentials.read(iprot); struct.setCredentialsIsSet(true); } + if (incoming.get(2)) { + struct.taskID = iprot.readString(); + struct.setTaskIDIsSet(true); + } } } @@ -6113,19 +5372,17 @@ private static S scheme(org.apache. } @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) - public static class getRunningCompactions_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("getRunningCompactions_result"); + public static class cancelTask_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("cancelTask_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.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getRunningCompactions_resultStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getRunningCompactions_resultTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new cancelTask_resultStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new cancelTask_resultTupleSchemeFactory(); - public @org.apache.thrift.annotation.Nullable TExternalCompactionList success; // 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"); +; private static final java.util.Map byName = new java.util.HashMap(); @@ -6141,8 +5398,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; default: return null; } @@ -6184,82 +5439,34 @@ 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, TExternalCompactionList.class))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getRunningCompactions_result.class, metaDataMap); - } - - public getRunningCompactions_result() { - } - - public getRunningCompactions_result( - TExternalCompactionList success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public getRunningCompactions_result(getRunningCompactions_result other) { - if (other.isSetSuccess()) { - this.success = new TExternalCompactionList(other.success); - } - } - - @Override - public getRunningCompactions_result deepCopy() { - return new getRunningCompactions_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - @org.apache.thrift.annotation.Nullable - public TExternalCompactionList getSuccess() { - return this.success; + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(cancelTask_result.class, metaDataMap); } - - public getRunningCompactions_result setSuccess(@org.apache.thrift.annotation.Nullable TExternalCompactionList success) { - this.success = success; - return this; + + public cancelTask_result() { } - public void unsetSuccess() { - this.success = null; + /** + * Performs a deep copy on other. + */ + public cancelTask_result(cancelTask_result other) { } - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean isSetSuccess() { - return this.success != null; + @Override + public cancelTask_result deepCopy() { + return new cancelTask_result(this); } - public void setSuccessIsSet(boolean value) { - if (!value) { - this.success = null; - } + @Override + public void clear() { } @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((TExternalCompactionList)value); - } - break; - } } @@ -6267,9 +5474,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(); - } throw new java.lang.IllegalStateException(); } @@ -6282,34 +5486,23 @@ public boolean isSet(_Fields field) { } switch (field) { - case SUCCESS: - return isSetSuccess(); } throw new java.lang.IllegalStateException(); } @Override public boolean equals(java.lang.Object that) { - if (that instanceof getRunningCompactions_result) - return this.equals((getRunningCompactions_result)that); + if (that instanceof cancelTask_result) + return this.equals((cancelTask_result)that); return false; } - public boolean equals(getRunningCompactions_result that) { + public boolean equals(cancelTask_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; - } - return true; } @@ -6317,31 +5510,17 @@ public boolean equals(getRunningCompactions_result that) { public int hashCode() { int hashCode = 1; - hashCode = hashCode * 8191 + ((isSetSuccess()) ? 131071 : 524287); - if (isSetSuccess()) - hashCode = hashCode * 8191 + success.hashCode(); - return hashCode; } @Override - public int compareTo(getRunningCompactions_result other) { + public int compareTo(cancelTask_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; - } - } return 0; } @@ -6362,16 +5541,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("getRunningCompactions_result("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("cancelTask_result("); boolean first = true; - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; sb.append(")"); return sb.toString(); } @@ -6379,9 +5551,6 @@ public java.lang.String 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 { @@ -6400,17 +5569,17 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class getRunningCompactions_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class cancelTask_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public getRunningCompactions_resultStandardScheme getScheme() { - return new getRunningCompactions_resultStandardScheme(); + public cancelTask_resultStandardScheme getScheme() { + return new cancelTask_resultStandardScheme(); } } - private static class getRunningCompactions_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class cancelTask_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { @Override - public void read(org.apache.thrift.protocol.TProtocol iprot, getRunningCompactions_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, cancelTask_result struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -6420,15 +5589,6 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getRunningCompactio break; } switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new TExternalCompactionList(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -6441,52 +5601,33 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getRunningCompactio } @Override - public void write(org.apache.thrift.protocol.TProtocol oprot, getRunningCompactions_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, cancelTask_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(); - } oprot.writeFieldStop(); oprot.writeStructEnd(); } } - private static class getRunningCompactions_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class cancelTask_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public getRunningCompactions_resultTupleScheme getScheme() { - return new getRunningCompactions_resultTupleScheme(); + public cancelTask_resultTupleScheme getScheme() { + return new cancelTask_resultTupleScheme(); } } - private static class getRunningCompactions_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class cancelTask_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, getRunningCompactions_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, cancelTask_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); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetSuccess()) { - struct.success.write(oprot); - } } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, getRunningCompactions_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, cancelTask_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(1); - if (incoming.get(0)) { - struct.success = new TExternalCompactionList(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } } } @@ -6496,14 +5637,14 @@ private static S scheme(org.apache. } @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) - public static class getCompletedCompactions_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("getCompletedCompactions_args"); + public static class getRunningTasks_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("getRunningTasks_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 getCompletedCompactions_argsStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getCompletedCompactions_argsTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getRunningTasks_argsStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getRunningTasks_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 @@ -6582,13 +5723,13 @@ public java.lang.String getFieldName() { 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(getCompletedCompactions_args.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getRunningTasks_args.class, metaDataMap); } - public getCompletedCompactions_args() { + public getRunningTasks_args() { } - public getCompletedCompactions_args( + public getRunningTasks_args( org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) { @@ -6600,7 +5741,7 @@ public getCompletedCompactions_args( /** * Performs a deep copy on other. */ - public getCompletedCompactions_args(getCompletedCompactions_args other) { + public getRunningTasks_args(getRunningTasks_args other) { if (other.isSetTinfo()) { this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo); } @@ -6610,8 +5751,8 @@ public getCompletedCompactions_args(getCompletedCompactions_args other) { } @Override - public getCompletedCompactions_args deepCopy() { - return new getCompletedCompactions_args(this); + public getRunningTasks_args deepCopy() { + return new getRunningTasks_args(this); } @Override @@ -6625,7 +5766,7 @@ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() { return this.tinfo; } - public getCompletedCompactions_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) { + public getRunningTasks_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) { this.tinfo = tinfo; return this; } @@ -6650,7 +5791,7 @@ public org.apache.accumulo.core.securityImpl.thrift.TCredentials getCredentials( return this.credentials; } - public getCompletedCompactions_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) { + public getRunningTasks_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) { this.credentials = credentials; return this; } @@ -6724,12 +5865,12 @@ public boolean isSet(_Fields field) { @Override public boolean equals(java.lang.Object that) { - if (that instanceof getCompletedCompactions_args) - return this.equals((getCompletedCompactions_args)that); + if (that instanceof getRunningTasks_args) + return this.equals((getRunningTasks_args)that); return false; } - public boolean equals(getCompletedCompactions_args that) { + public boolean equals(getRunningTasks_args that) { if (that == null) return false; if (this == that) @@ -6772,7 +5913,7 @@ public int hashCode() { } @Override - public int compareTo(getCompletedCompactions_args other) { + public int compareTo(getRunningTasks_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -6820,7 +5961,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("getCompletedCompactions_args("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("getRunningTasks_args("); boolean first = true; sb.append("tinfo:"); @@ -6869,17 +6010,17 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class getCompletedCompactions_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class getRunningTasks_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public getCompletedCompactions_argsStandardScheme getScheme() { - return new getCompletedCompactions_argsStandardScheme(); + public getRunningTasks_argsStandardScheme getScheme() { + return new getRunningTasks_argsStandardScheme(); } } - private static class getCompletedCompactions_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class getRunningTasks_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { @Override - public void read(org.apache.thrift.protocol.TProtocol iprot, getCompletedCompactions_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, getRunningTasks_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -6919,7 +6060,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getCompletedCompact } @Override - public void write(org.apache.thrift.protocol.TProtocol oprot, getCompletedCompactions_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, getRunningTasks_args struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -6939,17 +6080,17 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, getCompletedCompac } - private static class getCompletedCompactions_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class getRunningTasks_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public getCompletedCompactions_argsTupleScheme getScheme() { - return new getCompletedCompactions_argsTupleScheme(); + public getRunningTasks_argsTupleScheme getScheme() { + return new getRunningTasks_argsTupleScheme(); } } - private static class getCompletedCompactions_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class getRunningTasks_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, getCompletedCompactions_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, getRunningTasks_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()) { @@ -6968,7 +6109,7 @@ public void write(org.apache.thrift.protocol.TProtocol prot, getCompletedCompact } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, getCompletedCompactions_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, getRunningTasks_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)) { @@ -6990,15 +6131,15 @@ private static S scheme(org.apache. } @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) - public static class getCompletedCompactions_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("getCompletedCompactions_result"); + public static class getRunningTasks_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("getRunningTasks_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.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getCompletedCompactions_resultStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getCompletedCompactions_resultTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getRunningTasks_resultStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getRunningTasks_resultTupleSchemeFactory(); - public @org.apache.thrift.annotation.Nullable TExternalCompactionList success; // required + public @org.apache.thrift.annotation.Nullable Task success; // 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 { @@ -7067,16 +6208,16 @@ public java.lang.String getFieldName() { 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, TExternalCompactionList.class))); + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, Task.class))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getCompletedCompactions_result.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getRunningTasks_result.class, metaDataMap); } - public getCompletedCompactions_result() { + public getRunningTasks_result() { } - public getCompletedCompactions_result( - TExternalCompactionList success) + public getRunningTasks_result( + Task success) { this(); this.success = success; @@ -7085,15 +6226,15 @@ public getCompletedCompactions_result( /** * Performs a deep copy on other. */ - public getCompletedCompactions_result(getCompletedCompactions_result other) { + public getRunningTasks_result(getRunningTasks_result other) { if (other.isSetSuccess()) { - this.success = new TExternalCompactionList(other.success); + this.success = new Task(other.success); } } @Override - public getCompletedCompactions_result deepCopy() { - return new getCompletedCompactions_result(this); + public getRunningTasks_result deepCopy() { + return new getRunningTasks_result(this); } @Override @@ -7102,11 +6243,11 @@ public void clear() { } @org.apache.thrift.annotation.Nullable - public TExternalCompactionList getSuccess() { + public Task getSuccess() { return this.success; } - public getCompletedCompactions_result setSuccess(@org.apache.thrift.annotation.Nullable TExternalCompactionList success) { + public getRunningTasks_result setSuccess(@org.apache.thrift.annotation.Nullable Task success) { this.success = success; return this; } @@ -7133,7 +6274,7 @@ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable if (value == null) { unsetSuccess(); } else { - setSuccess((TExternalCompactionList)value); + setSuccess((Task)value); } break; @@ -7167,12 +6308,12 @@ public boolean isSet(_Fields field) { @Override public boolean equals(java.lang.Object that) { - if (that instanceof getCompletedCompactions_result) - return this.equals((getCompletedCompactions_result)that); + if (that instanceof getRunningTasks_result) + return this.equals((getRunningTasks_result)that); return false; } - public boolean equals(getCompletedCompactions_result that) { + public boolean equals(getRunningTasks_result that) { if (that == null) return false; if (this == that) @@ -7202,7 +6343,7 @@ public int hashCode() { } @Override - public int compareTo(getCompletedCompactions_result other) { + public int compareTo(getRunningTasks_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -7239,7 +6380,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("getCompletedCompactions_result("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("getRunningTasks_result("); boolean first = true; sb.append("success:"); @@ -7277,17 +6418,17 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class getCompletedCompactions_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class getRunningTasks_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public getCompletedCompactions_resultStandardScheme getScheme() { - return new getCompletedCompactions_resultStandardScheme(); + public getRunningTasks_resultStandardScheme getScheme() { + return new getRunningTasks_resultStandardScheme(); } } - private static class getCompletedCompactions_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class getRunningTasks_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { @Override - public void read(org.apache.thrift.protocol.TProtocol iprot, getCompletedCompactions_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, getRunningTasks_result struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -7299,7 +6440,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getCompletedCompact switch (schemeField.id) { case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new TExternalCompactionList(); + struct.success = new Task(); struct.success.read(iprot); struct.setSuccessIsSet(true); } else { @@ -7318,7 +6459,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getCompletedCompact } @Override - public void write(org.apache.thrift.protocol.TProtocol oprot, getCompletedCompactions_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, getRunningTasks_result struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -7333,17 +6474,17 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, getCompletedCompac } - private static class getCompletedCompactions_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class getRunningTasks_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public getCompletedCompactions_resultTupleScheme getScheme() { - return new getCompletedCompactions_resultTupleScheme(); + public getRunningTasks_resultTupleScheme getScheme() { + return new getRunningTasks_resultTupleScheme(); } } - private static class getCompletedCompactions_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class getRunningTasks_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, getCompletedCompactions_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, getRunningTasks_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()) { @@ -7356,11 +6497,11 @@ public void write(org.apache.thrift.protocol.TProtocol prot, getCompletedCompact } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, getCompletedCompactions_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, getRunningTasks_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(1); if (incoming.get(0)) { - struct.success = new TExternalCompactionList(); + struct.success = new Task(); struct.success.read(iprot); struct.setSuccessIsSet(true); } @@ -7373,25 +6514,22 @@ private static S scheme(org.apache. } @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) - public static class cancel_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("cancel_args"); + public static class getCompletedTasks_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("getCompletedTasks_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 cancel_argsStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new cancel_argsTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getCompletedTasks_argsStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getCompletedTasks_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"); + CREDENTIALS((short)2, "credentials"); private static final java.util.Map byName = new java.util.HashMap(); @@ -7411,8 +6549,6 @@ public static _Fields findByThriftId(int fieldId) { return TINFO; case 2: // CREDENTIALS return CREDENTIALS; - case 3: // EXTERNAL_COMPACTION_ID - return EXTERNAL_COMPACTION_ID; default: return null; } @@ -7463,51 +6599,43 @@ 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.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(cancel_args.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getCompletedTasks_args.class, metaDataMap); } - public cancel_args() { + public getCompletedTasks_args() { } - public cancel_args( + public getCompletedTasks_args( org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, - org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, - java.lang.String externalCompactionId) + org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) { this(); this.tinfo = tinfo; this.credentials = credentials; - this.externalCompactionId = externalCompactionId; } /** * Performs a deep copy on other. */ - public cancel_args(cancel_args other) { + public getCompletedTasks_args(getCompletedTasks_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 cancel_args deepCopy() { - return new cancel_args(this); + public getCompletedTasks_args deepCopy() { + return new getCompletedTasks_args(this); } @Override public void clear() { this.tinfo = null; this.credentials = null; - this.externalCompactionId = null; } @org.apache.thrift.annotation.Nullable @@ -7515,7 +6643,7 @@ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() { return this.tinfo; } - public cancel_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) { + public getCompletedTasks_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) { this.tinfo = tinfo; return this; } @@ -7540,7 +6668,7 @@ public org.apache.accumulo.core.securityImpl.thrift.TCredentials getCredentials( return this.credentials; } - public cancel_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) { + public getCompletedTasks_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) { this.credentials = credentials; return this; } @@ -7560,31 +6688,6 @@ public void setCredentialsIsSet(boolean value) { } } - @org.apache.thrift.annotation.Nullable - public java.lang.String getExternalCompactionId() { - return this.externalCompactionId; - } - - public cancel_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) { @@ -7604,14 +6707,6 @@ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable } break; - case EXTERNAL_COMPACTION_ID: - if (value == null) { - unsetExternalCompactionId(); - } else { - setExternalCompactionId((java.lang.String)value); - } - break; - } } @@ -7625,9 +6720,6 @@ public java.lang.Object getFieldValue(_Fields field) { case CREDENTIALS: return getCredentials(); - case EXTERNAL_COMPACTION_ID: - return getExternalCompactionId(); - } throw new java.lang.IllegalStateException(); } @@ -7644,20 +6736,18 @@ public boolean isSet(_Fields field) { 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 cancel_args) - return this.equals((cancel_args)that); + if (that instanceof getCompletedTasks_args) + return this.equals((getCompletedTasks_args)that); return false; } - public boolean equals(cancel_args that) { + public boolean equals(getCompletedTasks_args that) { if (that == null) return false; if (this == that) @@ -7681,15 +6771,6 @@ public boolean equals(cancel_args that) { 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; } @@ -7705,15 +6786,11 @@ public int hashCode() { 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(cancel_args other) { + public int compareTo(getCompletedTasks_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -7740,16 +6817,6 @@ public int compareTo(cancel_args other) { 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; } @@ -7771,7 +6838,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("cancel_args("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("getCompletedTasks_args("); boolean first = true; sb.append("tinfo:"); @@ -7789,14 +6856,6 @@ public java.lang.String toString() { 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(); } @@ -7828,17 +6887,17 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class cancel_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class getCompletedTasks_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public cancel_argsStandardScheme getScheme() { - return new cancel_argsStandardScheme(); + public getCompletedTasks_argsStandardScheme getScheme() { + return new getCompletedTasks_argsStandardScheme(); } } - private static class cancel_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class getCompletedTasks_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { @Override - public void read(org.apache.thrift.protocol.TProtocol iprot, cancel_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, getCompletedTasks_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -7866,14 +6925,6 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, cancel_args struct) 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); } @@ -7886,7 +6937,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, cancel_args struct) } @Override - public void write(org.apache.thrift.protocol.TProtocol oprot, cancel_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, getCompletedTasks_args struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -7900,28 +6951,23 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, cancel_args struct 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 cancel_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class getCompletedTasks_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public cancel_argsTupleScheme getScheme() { - return new cancel_argsTupleScheme(); + public getCompletedTasks_argsTupleScheme getScheme() { + return new getCompletedTasks_argsTupleScheme(); } } - private static class cancel_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class getCompletedTasks_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, cancel_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, getCompletedTasks_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()) { @@ -7930,25 +6976,19 @@ public void write(org.apache.thrift.protocol.TProtocol prot, cancel_args struct) if (struct.isSetCredentials()) { optionals.set(1); } - if (struct.isSetExternalCompactionId()) { - optionals.set(2); - } - oprot.writeBitSet(optionals, 3); + oprot.writeBitSet(optionals, 2); 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, cancel_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, getCompletedTasks_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); + 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); @@ -7959,10 +6999,6 @@ public void read(org.apache.thrift.protocol.TProtocol prot, cancel_args struct) struct.credentials.read(iprot); struct.setCredentialsIsSet(true); } - if (incoming.get(2)) { - struct.externalCompactionId = iprot.readString(); - struct.setExternalCompactionIdIsSet(true); - } } } @@ -7972,17 +7008,19 @@ private static S scheme(org.apache. } @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) - public static class cancel_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("cancel_result"); + public static class getCompletedTasks_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("getCompletedTasks_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.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new cancel_resultStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new cancel_resultTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getCompletedTasks_resultStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getCompletedTasks_resultTupleSchemeFactory(); + public @org.apache.thrift.annotation.Nullable Task success; // 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"); private static final java.util.Map byName = new java.util.HashMap(); @@ -7998,6 +7036,8 @@ 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; default: return null; } @@ -8039,34 +7079,82 @@ 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, Task.class))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(cancel_result.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getCompletedTasks_result.class, metaDataMap); + } + + public getCompletedTasks_result() { } - public cancel_result() { + public getCompletedTasks_result( + Task success) + { + this(); + this.success = success; } /** * Performs a deep copy on other. */ - public cancel_result(cancel_result other) { + public getCompletedTasks_result(getCompletedTasks_result other) { + if (other.isSetSuccess()) { + this.success = new Task(other.success); + } } @Override - public cancel_result deepCopy() { - return new cancel_result(this); + public getCompletedTasks_result deepCopy() { + return new getCompletedTasks_result(this); } @Override public void clear() { + this.success = null; + } + + @org.apache.thrift.annotation.Nullable + public Task getSuccess() { + return this.success; + } + + public getCompletedTasks_result setSuccess(@org.apache.thrift.annotation.Nullable Task 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; + } } @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((Task)value); + } + break; + } } @@ -8074,6 +7162,9 @@ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable @Override public java.lang.Object getFieldValue(_Fields field) { switch (field) { + case SUCCESS: + return getSuccess(); + } throw new java.lang.IllegalStateException(); } @@ -8086,23 +7177,34 @@ public boolean isSet(_Fields field) { } switch (field) { + case SUCCESS: + return isSetSuccess(); } throw new java.lang.IllegalStateException(); } @Override public boolean equals(java.lang.Object that) { - if (that instanceof cancel_result) - return this.equals((cancel_result)that); + if (that instanceof getCompletedTasks_result) + return this.equals((getCompletedTasks_result)that); return false; } - public boolean equals(cancel_result that) { + public boolean equals(getCompletedTasks_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; + } + return true; } @@ -8110,17 +7212,31 @@ public boolean equals(cancel_result that) { public int hashCode() { int hashCode = 1; + hashCode = hashCode * 8191 + ((isSetSuccess()) ? 131071 : 524287); + if (isSetSuccess()) + hashCode = hashCode * 8191 + success.hashCode(); + return hashCode; } @Override - public int compareTo(cancel_result other) { + public int compareTo(getCompletedTasks_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; + } + } return 0; } @@ -8141,9 +7257,16 @@ 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("cancel_result("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("getCompletedTasks_result("); boolean first = true; + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; sb.append(")"); return sb.toString(); } @@ -8151,6 +7274,9 @@ public java.lang.String 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 { @@ -8169,17 +7295,17 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class cancel_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class getCompletedTasks_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public cancel_resultStandardScheme getScheme() { - return new cancel_resultStandardScheme(); + public getCompletedTasks_resultStandardScheme getScheme() { + return new getCompletedTasks_resultStandardScheme(); } } - private static class cancel_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class getCompletedTasks_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { @Override - public void read(org.apache.thrift.protocol.TProtocol iprot, cancel_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, getCompletedTasks_result struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -8189,6 +7315,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, cancel_result struc break; } switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new Task(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -8201,33 +7336,52 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, cancel_result struc } @Override - public void write(org.apache.thrift.protocol.TProtocol oprot, cancel_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, getCompletedTasks_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(); + } oprot.writeFieldStop(); oprot.writeStructEnd(); } } - private static class cancel_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class getCompletedTasks_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public cancel_resultTupleScheme getScheme() { - return new cancel_resultTupleScheme(); + public getCompletedTasks_resultTupleScheme getScheme() { + return new getCompletedTasks_resultTupleScheme(); } } - private static class cancel_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class getCompletedTasks_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, cancel_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, getCompletedTasks_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); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, cancel_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, getCompletedTasks_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(1); + if (incoming.get(0)) { + struct.success = new Task(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } } } diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/compaction/thrift/CompactorService.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tasks/thrift/TaskRunner.java similarity index 78% rename from core/src/main/thrift-gen-java/org/apache/accumulo/core/compaction/thrift/CompactorService.java rename to core/src/main/thrift-gen-java/org/apache/accumulo/core/tasks/thrift/TaskRunner.java index d7224ae3375..c73afb051f2 100644 --- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/compaction/thrift/CompactorService.java +++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tasks/thrift/TaskRunner.java @@ -22,32 +22,32 @@ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING * @generated */ -package org.apache.accumulo.core.compaction.thrift; +package org.apache.accumulo.core.tasks.thrift; @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) -public class CompactorService { +public class TaskRunner { public interface Iface { - public org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob getRunningCompaction(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 Task getRunningTask(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 java.lang.String getRunningCompactionId(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 java.lang.String getRunningTaskId(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 java.util.List getActiveCompactions(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 void cancelTask(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String taskID) throws org.apache.thrift.TException; - 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.thrift.TException; + public Task getActiveCompactions(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 interface AsyncIface { - public void getRunningCompaction(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 getRunningTask(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 getRunningCompactionId(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 getRunningTaskId(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 getActiveCompactions(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 cancelTask(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String taskID, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - 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 getActiveCompactions(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; } @@ -74,63 +74,86 @@ public Client(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.prot } @Override - public org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob getRunningCompaction(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 Task getRunningTask(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_getRunningCompaction(tinfo, credentials); - return recv_getRunningCompaction(); + send_getRunningTask(tinfo, credentials); + return recv_getRunningTask(); } - public void send_getRunningCompaction(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException + public void send_getRunningTask(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException { - getRunningCompaction_args args = new getRunningCompaction_args(); + getRunningTask_args args = new getRunningTask_args(); args.setTinfo(tinfo); args.setCredentials(credentials); - sendBase("getRunningCompaction", args); + sendBase("getRunningTask", args); } - public org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob recv_getRunningCompaction() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException + public Task recv_getRunningTask() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException { - getRunningCompaction_result result = new getRunningCompaction_result(); - receiveBase(result, "getRunningCompaction"); + getRunningTask_result result = new getRunningTask_result(); + receiveBase(result, "getRunningTask"); 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, "getRunningCompaction failed: unknown result"); + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getRunningTask failed: unknown result"); } @Override - public java.lang.String getRunningCompactionId(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 java.lang.String getRunningTaskId(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_getRunningCompactionId(tinfo, credentials); - return recv_getRunningCompactionId(); + send_getRunningTaskId(tinfo, credentials); + return recv_getRunningTaskId(); } - public void send_getRunningCompactionId(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException + public void send_getRunningTaskId(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException { - getRunningCompactionId_args args = new getRunningCompactionId_args(); + getRunningTaskId_args args = new getRunningTaskId_args(); args.setTinfo(tinfo); args.setCredentials(credentials); - sendBase("getRunningCompactionId", args); + sendBase("getRunningTaskId", args); } - public java.lang.String recv_getRunningCompactionId() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException + public java.lang.String recv_getRunningTaskId() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException { - getRunningCompactionId_result result = new getRunningCompactionId_result(); - receiveBase(result, "getRunningCompactionId"); + getRunningTaskId_result result = new getRunningTaskId_result(); + receiveBase(result, "getRunningTaskId"); 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, "getRunningCompactionId failed: unknown result"); + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getRunningTaskId failed: unknown result"); } @Override - public java.util.List getActiveCompactions(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 void cancelTask(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String taskID) throws org.apache.thrift.TException + { + send_cancelTask(tinfo, credentials, taskID); + recv_cancelTask(); + } + + public void send_cancelTask(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String taskID) throws org.apache.thrift.TException + { + cancelTask_args args = new cancelTask_args(); + args.setTinfo(tinfo); + args.setCredentials(credentials); + args.setTaskID(taskID); + sendBase("cancelTask", args); + } + + public void recv_cancelTask() throws org.apache.thrift.TException + { + cancelTask_result result = new cancelTask_result(); + receiveBase(result, "cancelTask"); + return; + } + + @Override + public Task getActiveCompactions(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_getActiveCompactions(tinfo, credentials); return recv_getActiveCompactions(); @@ -144,7 +167,7 @@ public void send_getActiveCompactions(org.apache.accumulo.core.clientImpl.thrift sendBase("getActiveCompactions", args); } - public java.util.List recv_getActiveCompactions() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException + public Task recv_getActiveCompactions() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException { getActiveCompactions_result result = new getActiveCompactions_result(); receiveBase(result, "getActiveCompactions"); @@ -157,29 +180,6 @@ public java.util.List { @@ -200,17 +200,17 @@ public AsyncClient(org.apache.thrift.protocol.TProtocolFactory protocolFactory, } @Override - public void getRunningCompaction(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 getRunningTask(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(); - getRunningCompaction_call method_call = new getRunningCompaction_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport); + getRunningTask_call method_call = new getRunningTask_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; ___manager.call(method_call); } - public static class getRunningCompaction_call extends org.apache.thrift.async.TAsyncMethodCall { + public static class getRunningTask_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 getRunningCompaction_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 { + public getRunningTask_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; @@ -218,8 +218,8 @@ public getRunningCompaction_call(org.apache.accumulo.core.clientImpl.thrift.TInf @Override public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getRunningCompaction", org.apache.thrift.protocol.TMessageType.CALL, 0)); - getRunningCompaction_args args = new getRunningCompaction_args(); + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getRunningTask", org.apache.thrift.protocol.TMessageType.CALL, 0)); + getRunningTask_args args = new getRunningTask_args(); args.setTinfo(tinfo); args.setCredentials(credentials); args.write(prot); @@ -227,28 +227,28 @@ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apa } @Override - public org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob getResult() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException { + public Task 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_getRunningCompaction(); + return (new Client(prot)).recv_getRunningTask(); } } @Override - public void getRunningCompactionId(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 getRunningTaskId(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(); - getRunningCompactionId_call method_call = new getRunningCompactionId_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport); + getRunningTaskId_call method_call = new getRunningTaskId_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; ___manager.call(method_call); } - public static class getRunningCompactionId_call extends org.apache.thrift.async.TAsyncMethodCall { + public static class getRunningTaskId_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 getRunningCompactionId_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 { + public getRunningTaskId_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; @@ -256,8 +256,8 @@ public getRunningCompactionId_call(org.apache.accumulo.core.clientImpl.thrift.TI @Override public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getRunningCompactionId", org.apache.thrift.protocol.TMessageType.CALL, 0)); - getRunningCompactionId_args args = new getRunningCompactionId_args(); + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getRunningTaskId", org.apache.thrift.protocol.TMessageType.CALL, 0)); + getRunningTaskId_args args = new getRunningTaskId_args(); args.setTinfo(tinfo); args.setCredentials(credentials); args.write(prot); @@ -271,87 +271,87 @@ public java.lang.String getResult() throws org.apache.accumulo.core.clientImpl.t } 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_getRunningCompactionId(); + return (new Client(prot)).recv_getRunningTaskId(); } } @Override - public void getActiveCompactions(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 cancelTask(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String taskID, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); - getActiveCompactions_call method_call = new getActiveCompactions_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport); + cancelTask_call method_call = new cancelTask_call(tinfo, credentials, taskID, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; ___manager.call(method_call); } - public static class getActiveCompactions_call extends org.apache.thrift.async.TAsyncMethodCall> { + public static class cancelTask_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 getActiveCompactions_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 { + private java.lang.String taskID; + public cancelTask_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String taskID, 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.taskID = taskID; } @Override public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getActiveCompactions", org.apache.thrift.protocol.TMessageType.CALL, 0)); - getActiveCompactions_args args = new getActiveCompactions_args(); + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("cancelTask", org.apache.thrift.protocol.TMessageType.CALL, 0)); + cancelTask_args args = new cancelTask_args(); args.setTinfo(tinfo); args.setCredentials(credentials); + args.setTaskID(taskID); args.write(prot); prot.writeMessageEnd(); } @Override - public java.util.List getResult() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException { + 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 (new Client(prot)).recv_getActiveCompactions(); + (new Client(prot)).recv_cancelTask(); + return null; } } @Override - 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 getActiveCompactions(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(); - cancel_call method_call = new cancel_call(tinfo, credentials, externalCompactionId, resultHandler, this, ___protocolFactory, ___transport); + getActiveCompactions_call method_call = new getActiveCompactions_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; ___manager.call(method_call); } - public static class cancel_call extends org.apache.thrift.async.TAsyncMethodCall { + public static class getActiveCompactions_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 cancel_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 { + public getActiveCompactions_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; - 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("cancel", org.apache.thrift.protocol.TMessageType.CALL, 0)); - cancel_args args = new cancel_args(); + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getActiveCompactions", org.apache.thrift.protocol.TMessageType.CALL, 0)); + getActiveCompactions_args args = new getActiveCompactions_args(); args.setTinfo(tinfo); args.setCredentials(credentials); - args.setExternalCompactionId(externalCompactionId); args.write(prot); prot.writeMessageEnd(); } @Override - public Void getResult() throws org.apache.thrift.TException { + public Task 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); - (new Client(prot)).recv_cancel(); - return null; + return (new Client(prot)).recv_getActiveCompactions(); } } @@ -368,21 +368,21 @@ protected Processor(I iface, java.util.Map java.util.Map> getProcessMap(java.util.Map> processMap) { - processMap.put("getRunningCompaction", new getRunningCompaction()); - processMap.put("getRunningCompactionId", new getRunningCompactionId()); + processMap.put("getRunningTask", new getRunningTask()); + processMap.put("getRunningTaskId", new getRunningTaskId()); + processMap.put("cancelTask", new cancelTask()); processMap.put("getActiveCompactions", new getActiveCompactions()); - processMap.put("cancel", new cancel()); return processMap; } - public static class getRunningCompaction extends org.apache.thrift.ProcessFunction { - public getRunningCompaction() { - super("getRunningCompaction"); + public static class getRunningTask extends org.apache.thrift.ProcessFunction { + public getRunningTask() { + super("getRunningTask"); } @Override - public getRunningCompaction_args getEmptyArgsInstance() { - return new getRunningCompaction_args(); + public getRunningTask_args getEmptyArgsInstance() { + return new getRunningTask_args(); } @Override @@ -396,10 +396,10 @@ protected boolean rethrowUnhandledExceptions() { } @Override - public getRunningCompaction_result getResult(I iface, getRunningCompaction_args args) throws org.apache.thrift.TException { - getRunningCompaction_result result = new getRunningCompaction_result(); + public getRunningTask_result getResult(I iface, getRunningTask_args args) throws org.apache.thrift.TException { + getRunningTask_result result = new getRunningTask_result(); try { - result.success = iface.getRunningCompaction(args.tinfo, args.credentials); + result.success = iface.getRunningTask(args.tinfo, args.credentials); } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) { result.sec = sec; } @@ -407,14 +407,14 @@ public getRunningCompaction_result getResult(I iface, getRunningCompaction_args } } - public static class getRunningCompactionId extends org.apache.thrift.ProcessFunction { - public getRunningCompactionId() { - super("getRunningCompactionId"); + public static class getRunningTaskId extends org.apache.thrift.ProcessFunction { + public getRunningTaskId() { + super("getRunningTaskId"); } @Override - public getRunningCompactionId_args getEmptyArgsInstance() { - return new getRunningCompactionId_args(); + public getRunningTaskId_args getEmptyArgsInstance() { + return new getRunningTaskId_args(); } @Override @@ -428,10 +428,10 @@ protected boolean rethrowUnhandledExceptions() { } @Override - public getRunningCompactionId_result getResult(I iface, getRunningCompactionId_args args) throws org.apache.thrift.TException { - getRunningCompactionId_result result = new getRunningCompactionId_result(); + public getRunningTaskId_result getResult(I iface, getRunningTaskId_args args) throws org.apache.thrift.TException { + getRunningTaskId_result result = new getRunningTaskId_result(); try { - result.success = iface.getRunningCompactionId(args.tinfo, args.credentials); + result.success = iface.getRunningTaskId(args.tinfo, args.credentials); } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) { result.sec = sec; } @@ -439,14 +439,14 @@ public getRunningCompactionId_result getResult(I iface, getRunningCompactionId_a } } - public static class getActiveCompactions extends org.apache.thrift.ProcessFunction { - public getActiveCompactions() { - super("getActiveCompactions"); + public static class cancelTask extends org.apache.thrift.ProcessFunction { + public cancelTask() { + super("cancelTask"); } @Override - public getActiveCompactions_args getEmptyArgsInstance() { - return new getActiveCompactions_args(); + public cancelTask_args getEmptyArgsInstance() { + return new cancelTask_args(); } @Override @@ -460,25 +460,21 @@ protected boolean rethrowUnhandledExceptions() { } @Override - public getActiveCompactions_result getResult(I iface, getActiveCompactions_args args) throws org.apache.thrift.TException { - getActiveCompactions_result result = new getActiveCompactions_result(); - try { - result.success = iface.getActiveCompactions(args.tinfo, args.credentials); - } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) { - result.sec = sec; - } + public cancelTask_result getResult(I iface, cancelTask_args args) throws org.apache.thrift.TException { + cancelTask_result result = new cancelTask_result(); + iface.cancelTask(args.tinfo, args.credentials, args.taskID); return result; } } - public static class cancel extends org.apache.thrift.ProcessFunction { - public cancel() { - super("cancel"); + public static class getActiveCompactions extends org.apache.thrift.ProcessFunction { + public getActiveCompactions() { + super("getActiveCompactions"); } @Override - public cancel_args getEmptyArgsInstance() { - return new cancel_args(); + public getActiveCompactions_args getEmptyArgsInstance() { + return new getActiveCompactions_args(); } @Override @@ -492,9 +488,13 @@ protected boolean rethrowUnhandledExceptions() { } @Override - public cancel_result getResult(I iface, cancel_args args) throws org.apache.thrift.TException { - cancel_result result = new cancel_result(); - iface.cancel(args.tinfo, args.credentials, args.externalCompactionId); + public getActiveCompactions_result getResult(I iface, getActiveCompactions_args args) throws org.apache.thrift.TException { + getActiveCompactions_result result = new getActiveCompactions_result(); + try { + result.success = iface.getActiveCompactions(args.tinfo, args.credentials); + } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) { + result.sec = sec; + } return result; } } @@ -512,30 +512,30 @@ protected AsyncProcessor(I iface, java.util.Map java.util.Map> getProcessMap(java.util.Map> processMap) { - processMap.put("getRunningCompaction", new getRunningCompaction()); - processMap.put("getRunningCompactionId", new getRunningCompactionId()); + processMap.put("getRunningTask", new getRunningTask()); + processMap.put("getRunningTaskId", new getRunningTaskId()); + processMap.put("cancelTask", new cancelTask()); processMap.put("getActiveCompactions", new getActiveCompactions()); - processMap.put("cancel", new cancel()); return processMap; } - public static class getRunningCompaction extends org.apache.thrift.AsyncProcessFunction { - public getRunningCompaction() { - super("getRunningCompaction"); + public static class getRunningTask extends org.apache.thrift.AsyncProcessFunction { + public getRunningTask() { + super("getRunningTask"); } @Override - public getRunningCompaction_args getEmptyArgsInstance() { - return new getRunningCompaction_args(); + public getRunningTask_args getEmptyArgsInstance() { + return new getRunningTask_args(); } @Override - public org.apache.thrift.async.AsyncMethodCallback getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) { + 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() { + return new org.apache.thrift.async.AsyncMethodCallback() { @Override - public void onComplete(org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob o) { - getRunningCompaction_result result = new getRunningCompaction_result(); + public void onComplete(Task o) { + getRunningTask_result result = new getRunningTask_result(); result.success = o; try { fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); @@ -551,7 +551,7 @@ public void onComplete(org.apache.accumulo.core.tabletserver.thrift.TExternalCom public void onError(java.lang.Exception e) { byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; org.apache.thrift.TSerializable msg; - getRunningCompaction_result result = new getRunningCompaction_result(); + getRunningTask_result result = new getRunningTask_result(); if (e instanceof org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) { result.sec = (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) e; result.setSecIsSet(true); @@ -585,19 +585,19 @@ protected boolean isOneway() { } @Override - public void start(I iface, getRunningCompaction_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - iface.getRunningCompaction(args.tinfo, args.credentials,resultHandler); + public void start(I iface, getRunningTask_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + iface.getRunningTask(args.tinfo, args.credentials,resultHandler); } } - public static class getRunningCompactionId extends org.apache.thrift.AsyncProcessFunction { - public getRunningCompactionId() { - super("getRunningCompactionId"); + public static class getRunningTaskId extends org.apache.thrift.AsyncProcessFunction { + public getRunningTaskId() { + super("getRunningTaskId"); } @Override - public getRunningCompactionId_args getEmptyArgsInstance() { - return new getRunningCompactionId_args(); + public getRunningTaskId_args getEmptyArgsInstance() { + return new getRunningTaskId_args(); } @Override @@ -606,7 +606,7 @@ public org.apache.thrift.async.AsyncMethodCallback getResultHa return new org.apache.thrift.async.AsyncMethodCallback() { @Override public void onComplete(java.lang.String o) { - getRunningCompactionId_result result = new getRunningCompactionId_result(); + getRunningTaskId_result result = new getRunningTaskId_result(); result.success = o; try { fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); @@ -622,7 +622,7 @@ public void onComplete(java.lang.String o) { public void onError(java.lang.Exception e) { byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; org.apache.thrift.TSerializable msg; - getRunningCompactionId_result result = new getRunningCompactionId_result(); + getRunningTaskId_result result = new getRunningTaskId_result(); if (e instanceof org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) { result.sec = (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) e; result.setSecIsSet(true); @@ -656,29 +656,28 @@ protected boolean isOneway() { } @Override - public void start(I iface, getRunningCompactionId_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - iface.getRunningCompactionId(args.tinfo, args.credentials,resultHandler); + public void start(I iface, getRunningTaskId_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + iface.getRunningTaskId(args.tinfo, args.credentials,resultHandler); } } - public static class getActiveCompactions extends org.apache.thrift.AsyncProcessFunction> { - public getActiveCompactions() { - super("getActiveCompactions"); + public static class cancelTask extends org.apache.thrift.AsyncProcessFunction { + public cancelTask() { + super("cancelTask"); } @Override - public getActiveCompactions_args getEmptyArgsInstance() { - return new getActiveCompactions_args(); + public cancelTask_args getEmptyArgsInstance() { + return new cancelTask_args(); } @Override - public org.apache.thrift.async.AsyncMethodCallback> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) { + 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>() { + return new org.apache.thrift.async.AsyncMethodCallback() { @Override - public void onComplete(java.util.List o) { - getActiveCompactions_result result = new getActiveCompactions_result(); - result.success = o; + public void onComplete(Void o) { + cancelTask_result result = new cancelTask_result(); try { fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); } catch (org.apache.thrift.transport.TTransportException e) { @@ -693,12 +692,8 @@ public void onComplete(java.util.List> resultHandler) throws org.apache.thrift.TException { - iface.getActiveCompactions(args.tinfo, args.credentials,resultHandler); + public void start(I iface, cancelTask_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + iface.cancelTask(args.tinfo, args.credentials, args.taskID,resultHandler); } } - public static class cancel extends org.apache.thrift.AsyncProcessFunction { - public cancel() { - super("cancel"); + public static class getActiveCompactions extends org.apache.thrift.AsyncProcessFunction { + public getActiveCompactions() { + super("getActiveCompactions"); } @Override - public cancel_args getEmptyArgsInstance() { - return new cancel_args(); + public getActiveCompactions_args getEmptyArgsInstance() { + return new getActiveCompactions_args(); } @Override - public org.apache.thrift.async.AsyncMethodCallback getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) { + 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() { + return new org.apache.thrift.async.AsyncMethodCallback() { @Override - public void onComplete(Void o) { - cancel_result result = new cancel_result(); + public void onComplete(Task o) { + getActiveCompactions_result result = new getActiveCompactions_result(); + result.success = o; try { fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); } catch (org.apache.thrift.transport.TTransportException e) { @@ -763,8 +759,12 @@ public void onComplete(Void o) { public void onError(java.lang.Exception e) { byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; org.apache.thrift.TSerializable msg; - cancel_result result = new cancel_result(); - if (e instanceof org.apache.thrift.transport.TTransportException) { + getActiveCompactions_result result = new getActiveCompactions_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; @@ -793,22 +793,22 @@ protected boolean isOneway() { } @Override - public void start(I iface, cancel_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - iface.cancel(args.tinfo, args.credentials, args.externalCompactionId,resultHandler); + public void start(I iface, getActiveCompactions_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + iface.getActiveCompactions(args.tinfo, args.credentials,resultHandler); } } } @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) - public static class getRunningCompaction_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("getRunningCompaction_args"); + public static class getRunningTask_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("getRunningTask_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 getRunningCompaction_argsStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getRunningCompaction_argsTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getRunningTask_argsStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getRunningTask_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 @@ -887,13 +887,13 @@ public java.lang.String getFieldName() { 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(getRunningCompaction_args.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getRunningTask_args.class, metaDataMap); } - public getRunningCompaction_args() { + public getRunningTask_args() { } - public getRunningCompaction_args( + public getRunningTask_args( org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) { @@ -905,7 +905,7 @@ public getRunningCompaction_args( /** * Performs a deep copy on other. */ - public getRunningCompaction_args(getRunningCompaction_args other) { + public getRunningTask_args(getRunningTask_args other) { if (other.isSetTinfo()) { this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo); } @@ -915,8 +915,8 @@ public getRunningCompaction_args(getRunningCompaction_args other) { } @Override - public getRunningCompaction_args deepCopy() { - return new getRunningCompaction_args(this); + public getRunningTask_args deepCopy() { + return new getRunningTask_args(this); } @Override @@ -930,7 +930,7 @@ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() { return this.tinfo; } - public getRunningCompaction_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) { + public getRunningTask_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) { this.tinfo = tinfo; return this; } @@ -955,7 +955,7 @@ public org.apache.accumulo.core.securityImpl.thrift.TCredentials getCredentials( return this.credentials; } - public getRunningCompaction_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) { + public getRunningTask_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) { this.credentials = credentials; return this; } @@ -1029,12 +1029,12 @@ public boolean isSet(_Fields field) { @Override public boolean equals(java.lang.Object that) { - if (that instanceof getRunningCompaction_args) - return this.equals((getRunningCompaction_args)that); + if (that instanceof getRunningTask_args) + return this.equals((getRunningTask_args)that); return false; } - public boolean equals(getRunningCompaction_args that) { + public boolean equals(getRunningTask_args that) { if (that == null) return false; if (this == that) @@ -1077,7 +1077,7 @@ public int hashCode() { } @Override - public int compareTo(getRunningCompaction_args other) { + public int compareTo(getRunningTask_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -1125,7 +1125,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("getRunningCompaction_args("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("getRunningTask_args("); boolean first = true; sb.append("tinfo:"); @@ -1174,17 +1174,17 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class getRunningCompaction_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class getRunningTask_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public getRunningCompaction_argsStandardScheme getScheme() { - return new getRunningCompaction_argsStandardScheme(); + public getRunningTask_argsStandardScheme getScheme() { + return new getRunningTask_argsStandardScheme(); } } - private static class getRunningCompaction_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class getRunningTask_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { @Override - public void read(org.apache.thrift.protocol.TProtocol iprot, getRunningCompaction_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, getRunningTask_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -1224,7 +1224,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getRunningCompactio } @Override - public void write(org.apache.thrift.protocol.TProtocol oprot, getRunningCompaction_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, getRunningTask_args struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -1244,17 +1244,17 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, getRunningCompacti } - private static class getRunningCompaction_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class getRunningTask_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public getRunningCompaction_argsTupleScheme getScheme() { - return new getRunningCompaction_argsTupleScheme(); + public getRunningTask_argsTupleScheme getScheme() { + return new getRunningTask_argsTupleScheme(); } } - private static class getRunningCompaction_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class getRunningTask_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, getRunningCompaction_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, getRunningTask_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()) { @@ -1273,7 +1273,7 @@ public void write(org.apache.thrift.protocol.TProtocol prot, getRunningCompactio } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, getRunningCompaction_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, getRunningTask_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)) { @@ -1295,16 +1295,16 @@ private static S scheme(org.apache. } @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) - public static class getRunningCompaction_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("getRunningCompaction_result"); + public static class getRunningTask_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("getRunningTask_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 getRunningCompaction_resultStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getRunningCompaction_resultTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getRunningTask_resultStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getRunningTask_resultTupleSchemeFactory(); - public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob success; // required + public @org.apache.thrift.annotation.Nullable Task 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. */ @@ -1377,18 +1377,18 @@ public java.lang.String getFieldName() { 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, org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob.class))); + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, Task.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(getRunningCompaction_result.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getRunningTask_result.class, metaDataMap); } - public getRunningCompaction_result() { + public getRunningTask_result() { } - public getRunningCompaction_result( - org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob success, + public getRunningTask_result( + Task success, org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) { this(); @@ -1399,9 +1399,9 @@ public getRunningCompaction_result( /** * Performs a deep copy on other. */ - public getRunningCompaction_result(getRunningCompaction_result other) { + public getRunningTask_result(getRunningTask_result other) { if (other.isSetSuccess()) { - this.success = new org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob(other.success); + this.success = new Task(other.success); } if (other.isSetSec()) { this.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(other.sec); @@ -1409,8 +1409,8 @@ public getRunningCompaction_result(getRunningCompaction_result other) { } @Override - public getRunningCompaction_result deepCopy() { - return new getRunningCompaction_result(this); + public getRunningTask_result deepCopy() { + return new getRunningTask_result(this); } @Override @@ -1420,11 +1420,11 @@ public void clear() { } @org.apache.thrift.annotation.Nullable - public org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob getSuccess() { + public Task getSuccess() { return this.success; } - public getRunningCompaction_result setSuccess(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob success) { + public getRunningTask_result setSuccess(@org.apache.thrift.annotation.Nullable Task success) { this.success = success; return this; } @@ -1449,7 +1449,7 @@ public org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException getSec return this.sec; } - public getRunningCompaction_result setSec(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) { + public getRunningTask_result setSec(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) { this.sec = sec; return this; } @@ -1476,7 +1476,7 @@ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable if (value == null) { unsetSuccess(); } else { - setSuccess((org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob)value); + setSuccess((Task)value); } break; @@ -1523,12 +1523,12 @@ public boolean isSet(_Fields field) { @Override public boolean equals(java.lang.Object that) { - if (that instanceof getRunningCompaction_result) - return this.equals((getRunningCompaction_result)that); + if (that instanceof getRunningTask_result) + return this.equals((getRunningTask_result)that); return false; } - public boolean equals(getRunningCompaction_result that) { + public boolean equals(getRunningTask_result that) { if (that == null) return false; if (this == that) @@ -1571,7 +1571,7 @@ public int hashCode() { } @Override - public int compareTo(getRunningCompaction_result other) { + public int compareTo(getRunningTask_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -1618,7 +1618,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("getRunningCompaction_result("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("getRunningTask_result("); boolean first = true; sb.append("success:"); @@ -1664,17 +1664,17 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class getRunningCompaction_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class getRunningTask_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public getRunningCompaction_resultStandardScheme getScheme() { - return new getRunningCompaction_resultStandardScheme(); + public getRunningTask_resultStandardScheme getScheme() { + return new getRunningTask_resultStandardScheme(); } } - private static class getRunningCompaction_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class getRunningTask_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { @Override - public void read(org.apache.thrift.protocol.TProtocol iprot, getRunningCompaction_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, getRunningTask_result struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -1686,7 +1686,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getRunningCompactio switch (schemeField.id) { case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob(); + struct.success = new Task(); struct.success.read(iprot); struct.setSuccessIsSet(true); } else { @@ -1714,7 +1714,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getRunningCompactio } @Override - public void write(org.apache.thrift.protocol.TProtocol oprot, getRunningCompaction_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, getRunningTask_result struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -1734,17 +1734,17 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, getRunningCompacti } - private static class getRunningCompaction_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class getRunningTask_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public getRunningCompaction_resultTupleScheme getScheme() { - return new getRunningCompaction_resultTupleScheme(); + public getRunningTask_resultTupleScheme getScheme() { + return new getRunningTask_resultTupleScheme(); } } - private static class getRunningCompaction_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class getRunningTask_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, getRunningCompaction_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, getRunningTask_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()) { @@ -1763,11 +1763,11 @@ public void write(org.apache.thrift.protocol.TProtocol prot, getRunningCompactio } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, getRunningCompaction_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, getRunningTask_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 org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob(); + struct.success = new Task(); struct.success.read(iprot); struct.setSuccessIsSet(true); } @@ -1785,14 +1785,14 @@ private static S scheme(org.apache. } @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) - public static class getRunningCompactionId_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("getRunningCompactionId_args"); + public static class getRunningTaskId_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("getRunningTaskId_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 getRunningCompactionId_argsStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getRunningCompactionId_argsTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getRunningTaskId_argsStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getRunningTaskId_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 @@ -1871,13 +1871,13 @@ public java.lang.String getFieldName() { 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(getRunningCompactionId_args.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getRunningTaskId_args.class, metaDataMap); } - public getRunningCompactionId_args() { + public getRunningTaskId_args() { } - public getRunningCompactionId_args( + public getRunningTaskId_args( org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) { @@ -1889,7 +1889,7 @@ public getRunningCompactionId_args( /** * Performs a deep copy on other. */ - public getRunningCompactionId_args(getRunningCompactionId_args other) { + public getRunningTaskId_args(getRunningTaskId_args other) { if (other.isSetTinfo()) { this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo); } @@ -1899,8 +1899,8 @@ public getRunningCompactionId_args(getRunningCompactionId_args other) { } @Override - public getRunningCompactionId_args deepCopy() { - return new getRunningCompactionId_args(this); + public getRunningTaskId_args deepCopy() { + return new getRunningTaskId_args(this); } @Override @@ -1914,7 +1914,7 @@ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() { return this.tinfo; } - public getRunningCompactionId_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) { + public getRunningTaskId_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) { this.tinfo = tinfo; return this; } @@ -1939,7 +1939,7 @@ public org.apache.accumulo.core.securityImpl.thrift.TCredentials getCredentials( return this.credentials; } - public getRunningCompactionId_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) { + public getRunningTaskId_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) { this.credentials = credentials; return this; } @@ -2013,12 +2013,12 @@ public boolean isSet(_Fields field) { @Override public boolean equals(java.lang.Object that) { - if (that instanceof getRunningCompactionId_args) - return this.equals((getRunningCompactionId_args)that); + if (that instanceof getRunningTaskId_args) + return this.equals((getRunningTaskId_args)that); return false; } - public boolean equals(getRunningCompactionId_args that) { + public boolean equals(getRunningTaskId_args that) { if (that == null) return false; if (this == that) @@ -2061,7 +2061,7 @@ public int hashCode() { } @Override - public int compareTo(getRunningCompactionId_args other) { + public int compareTo(getRunningTaskId_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -2109,7 +2109,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("getRunningCompactionId_args("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("getRunningTaskId_args("); boolean first = true; sb.append("tinfo:"); @@ -2158,17 +2158,17 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class getRunningCompactionId_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class getRunningTaskId_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public getRunningCompactionId_argsStandardScheme getScheme() { - return new getRunningCompactionId_argsStandardScheme(); + public getRunningTaskId_argsStandardScheme getScheme() { + return new getRunningTaskId_argsStandardScheme(); } } - private static class getRunningCompactionId_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class getRunningTaskId_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { @Override - public void read(org.apache.thrift.protocol.TProtocol iprot, getRunningCompactionId_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, getRunningTaskId_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -2208,7 +2208,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getRunningCompactio } @Override - public void write(org.apache.thrift.protocol.TProtocol oprot, getRunningCompactionId_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, getRunningTaskId_args struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -2228,17 +2228,17 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, getRunningCompacti } - private static class getRunningCompactionId_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class getRunningTaskId_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public getRunningCompactionId_argsTupleScheme getScheme() { - return new getRunningCompactionId_argsTupleScheme(); + public getRunningTaskId_argsTupleScheme getScheme() { + return new getRunningTaskId_argsTupleScheme(); } } - private static class getRunningCompactionId_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class getRunningTaskId_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, getRunningCompactionId_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, getRunningTaskId_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()) { @@ -2257,7 +2257,7 @@ public void write(org.apache.thrift.protocol.TProtocol prot, getRunningCompactio } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, getRunningCompactionId_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, getRunningTaskId_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)) { @@ -2279,14 +2279,14 @@ private static S scheme(org.apache. } @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) - public static class getRunningCompactionId_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("getRunningCompactionId_result"); + public static class getRunningTaskId_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("getRunningTaskId_result"); private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRING, (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 getRunningCompactionId_resultStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getRunningCompactionId_resultTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getRunningTaskId_resultStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getRunningTaskId_resultTupleSchemeFactory(); public @org.apache.thrift.annotation.Nullable java.lang.String success; // required public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec; // required @@ -2365,13 +2365,13 @@ public java.lang.String getFieldName() { 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(getRunningCompactionId_result.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getRunningTaskId_result.class, metaDataMap); } - public getRunningCompactionId_result() { + public getRunningTaskId_result() { } - public getRunningCompactionId_result( + public getRunningTaskId_result( java.lang.String success, org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) { @@ -2383,7 +2383,7 @@ public getRunningCompactionId_result( /** * Performs a deep copy on other. */ - public getRunningCompactionId_result(getRunningCompactionId_result other) { + public getRunningTaskId_result(getRunningTaskId_result other) { if (other.isSetSuccess()) { this.success = other.success; } @@ -2393,8 +2393,8 @@ public getRunningCompactionId_result(getRunningCompactionId_result other) { } @Override - public getRunningCompactionId_result deepCopy() { - return new getRunningCompactionId_result(this); + public getRunningTaskId_result deepCopy() { + return new getRunningTaskId_result(this); } @Override @@ -2408,7 +2408,7 @@ public java.lang.String getSuccess() { return this.success; } - public getRunningCompactionId_result setSuccess(@org.apache.thrift.annotation.Nullable java.lang.String success) { + public getRunningTaskId_result setSuccess(@org.apache.thrift.annotation.Nullable java.lang.String success) { this.success = success; return this; } @@ -2433,7 +2433,7 @@ public org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException getSec return this.sec; } - public getRunningCompactionId_result setSec(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) { + public getRunningTaskId_result setSec(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) { this.sec = sec; return this; } @@ -2507,12 +2507,12 @@ public boolean isSet(_Fields field) { @Override public boolean equals(java.lang.Object that) { - if (that instanceof getRunningCompactionId_result) - return this.equals((getRunningCompactionId_result)that); + if (that instanceof getRunningTaskId_result) + return this.equals((getRunningTaskId_result)that); return false; } - public boolean equals(getRunningCompactionId_result that) { + public boolean equals(getRunningTaskId_result that) { if (that == null) return false; if (this == that) @@ -2555,7 +2555,7 @@ public int hashCode() { } @Override - public int compareTo(getRunningCompactionId_result other) { + public int compareTo(getRunningTaskId_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -2602,7 +2602,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("getRunningCompactionId_result("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("getRunningTaskId_result("); boolean first = true; sb.append("success:"); @@ -2645,17 +2645,17 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class getRunningCompactionId_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class getRunningTaskId_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public getRunningCompactionId_resultStandardScheme getScheme() { - return new getRunningCompactionId_resultStandardScheme(); + public getRunningTaskId_resultStandardScheme getScheme() { + return new getRunningTaskId_resultStandardScheme(); } } - private static class getRunningCompactionId_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class getRunningTaskId_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { @Override - public void read(org.apache.thrift.protocol.TProtocol iprot, getRunningCompactionId_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, getRunningTaskId_result struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -2694,7 +2694,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getRunningCompactio } @Override - public void write(org.apache.thrift.protocol.TProtocol oprot, getRunningCompactionId_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, getRunningTaskId_result struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -2714,17 +2714,17 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, getRunningCompacti } - private static class getRunningCompactionId_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class getRunningTaskId_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public getRunningCompactionId_resultTupleScheme getScheme() { - return new getRunningCompactionId_resultTupleScheme(); + public getRunningTaskId_resultTupleScheme getScheme() { + return new getRunningTaskId_resultTupleScheme(); } } - private static class getRunningCompactionId_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class getRunningTaskId_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, getRunningCompactionId_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, getRunningTaskId_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()) { @@ -2743,7 +2743,7 @@ public void write(org.apache.thrift.protocol.TProtocol prot, getRunningCompactio } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, getRunningCompactionId_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, getRunningTaskId_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)) { @@ -2764,22 +2764,25 @@ private static S scheme(org.apache. } @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) - public static class getActiveCompactions_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("getActiveCompactions_args"); + public static class cancelTask_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("cancelTask_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)2); - private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.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 TASK_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("taskID", org.apache.thrift.protocol.TType.STRING, (short)3); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getActiveCompactions_argsStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getActiveCompactions_argsTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new cancelTask_argsStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new cancelTask_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 taskID; // 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)2, "tinfo"), - CREDENTIALS((short)1, "credentials"); + TINFO((short)1, "tinfo"), + CREDENTIALS((short)2, "credentials"), + TASK_ID((short)3, "taskID"); private static final java.util.Map byName = new java.util.HashMap(); @@ -2795,10 +2798,12 @@ public enum _Fields implements org.apache.thrift.TFieldIdEnum { @org.apache.thrift.annotation.Nullable public static _Fields findByThriftId(int fieldId) { switch(fieldId) { - case 2: // TINFO + case 1: // TINFO return TINFO; - case 1: // CREDENTIALS + case 2: // CREDENTIALS return CREDENTIALS; + case 3: // TASK_ID + return TASK_ID; default: return null; } @@ -2849,43 +2854,51 @@ 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.TASK_ID, new org.apache.thrift.meta_data.FieldMetaData("taskID", 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(getActiveCompactions_args.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(cancelTask_args.class, metaDataMap); } - public getActiveCompactions_args() { + public cancelTask_args() { } - public getActiveCompactions_args( + public cancelTask_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.lang.String taskID) { this(); this.tinfo = tinfo; this.credentials = credentials; + this.taskID = taskID; } /** * Performs a deep copy on other. */ - public getActiveCompactions_args(getActiveCompactions_args other) { + public cancelTask_args(cancelTask_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.isSetTaskID()) { + this.taskID = other.taskID; + } } @Override - public getActiveCompactions_args deepCopy() { - return new getActiveCompactions_args(this); + public cancelTask_args deepCopy() { + return new cancelTask_args(this); } @Override public void clear() { this.tinfo = null; this.credentials = null; + this.taskID = null; } @org.apache.thrift.annotation.Nullable @@ -2893,7 +2906,7 @@ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() { return this.tinfo; } - public getActiveCompactions_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) { + public cancelTask_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) { this.tinfo = tinfo; return this; } @@ -2918,7 +2931,7 @@ public org.apache.accumulo.core.securityImpl.thrift.TCredentials getCredentials( return this.credentials; } - public getActiveCompactions_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) { + public cancelTask_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) { this.credentials = credentials; return this; } @@ -2938,6 +2951,31 @@ public void setCredentialsIsSet(boolean value) { } } + @org.apache.thrift.annotation.Nullable + public java.lang.String getTaskID() { + return this.taskID; + } + + public cancelTask_args setTaskID(@org.apache.thrift.annotation.Nullable java.lang.String taskID) { + this.taskID = taskID; + return this; + } + + public void unsetTaskID() { + this.taskID = null; + } + + /** Returns true if field taskID is set (has been assigned a value) and false otherwise */ + public boolean isSetTaskID() { + return this.taskID != null; + } + + public void setTaskIDIsSet(boolean value) { + if (!value) { + this.taskID = null; + } + } + @Override public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) { switch (field) { @@ -2957,6 +2995,14 @@ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable } break; + case TASK_ID: + if (value == null) { + unsetTaskID(); + } else { + setTaskID((java.lang.String)value); + } + break; + } } @@ -2970,6 +3016,9 @@ public java.lang.Object getFieldValue(_Fields field) { case CREDENTIALS: return getCredentials(); + case TASK_ID: + return getTaskID(); + } throw new java.lang.IllegalStateException(); } @@ -2986,18 +3035,20 @@ public boolean isSet(_Fields field) { return isSetTinfo(); case CREDENTIALS: return isSetCredentials(); + case TASK_ID: + return isSetTaskID(); } throw new java.lang.IllegalStateException(); } @Override public boolean equals(java.lang.Object that) { - if (that instanceof getActiveCompactions_args) - return this.equals((getActiveCompactions_args)that); + if (that instanceof cancelTask_args) + return this.equals((cancelTask_args)that); return false; } - public boolean equals(getActiveCompactions_args that) { + public boolean equals(cancelTask_args that) { if (that == null) return false; if (this == that) @@ -3021,6 +3072,15 @@ public boolean equals(getActiveCompactions_args that) { return false; } + boolean this_present_taskID = true && this.isSetTaskID(); + boolean that_present_taskID = true && that.isSetTaskID(); + if (this_present_taskID || that_present_taskID) { + if (!(this_present_taskID && that_present_taskID)) + return false; + if (!this.taskID.equals(that.taskID)) + return false; + } + return true; } @@ -3036,11 +3096,15 @@ public int hashCode() { if (isSetCredentials()) hashCode = hashCode * 8191 + credentials.hashCode(); + hashCode = hashCode * 8191 + ((isSetTaskID()) ? 131071 : 524287); + if (isSetTaskID()) + hashCode = hashCode * 8191 + taskID.hashCode(); + return hashCode; } @Override - public int compareTo(getActiveCompactions_args other) { + public int compareTo(cancelTask_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -3067,6 +3131,16 @@ public int compareTo(getActiveCompactions_args other) { return lastComparison; } } + lastComparison = java.lang.Boolean.compare(isSetTaskID(), other.isSetTaskID()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTaskID()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.taskID, other.taskID); + if (lastComparison != 0) { + return lastComparison; + } + } return 0; } @@ -3088,7 +3162,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("getActiveCompactions_args("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("cancelTask_args("); boolean first = true; sb.append("tinfo:"); @@ -3106,6 +3180,14 @@ public java.lang.String toString() { sb.append(this.credentials); } first = false; + if (!first) sb.append(", "); + sb.append("taskID:"); + if (this.taskID == null) { + sb.append("null"); + } else { + sb.append(this.taskID); + } + first = false; sb.append(")"); return sb.toString(); } @@ -3137,17 +3219,17 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class getActiveCompactions_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class cancelTask_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public getActiveCompactions_argsStandardScheme getScheme() { - return new getActiveCompactions_argsStandardScheme(); + public cancelTask_argsStandardScheme getScheme() { + return new cancelTask_argsStandardScheme(); } } - private static class getActiveCompactions_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class cancelTask_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { @Override - public void read(org.apache.thrift.protocol.TProtocol iprot, getActiveCompactions_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, cancelTask_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -3157,7 +3239,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getActiveCompaction break; } switch (schemeField.id) { - case 2: // TINFO + 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); @@ -3166,7 +3248,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getActiveCompaction org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; - case 1: // CREDENTIALS + 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); @@ -3175,6 +3257,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getActiveCompaction org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; + case 3: // TASK_ID + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.taskID = iprot.readString(); + struct.setTaskIDIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -3187,18 +3277,23 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getActiveCompaction } @Override - public void write(org.apache.thrift.protocol.TProtocol oprot, getActiveCompactions_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, cancelTask_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.tinfo != null) { - oprot.writeFieldBegin(TINFO_FIELD_DESC); - struct.tinfo.write(oprot); + if (struct.taskID != null) { + oprot.writeFieldBegin(TASK_ID_FIELD_DESC); + oprot.writeString(struct.taskID); oprot.writeFieldEnd(); } oprot.writeFieldStop(); @@ -3207,17 +3302,17 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, getActiveCompactio } - private static class getActiveCompactions_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class cancelTask_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public getActiveCompactions_argsTupleScheme getScheme() { - return new getActiveCompactions_argsTupleScheme(); + public cancelTask_argsTupleScheme getScheme() { + return new cancelTask_argsTupleScheme(); } } - private static class getActiveCompactions_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class cancelTask_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, getActiveCompactions_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, cancelTask_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()) { @@ -3226,19 +3321,25 @@ public void write(org.apache.thrift.protocol.TProtocol prot, getActiveCompaction if (struct.isSetCredentials()) { optionals.set(1); } - oprot.writeBitSet(optionals, 2); + if (struct.isSetTaskID()) { + optionals.set(2); + } + oprot.writeBitSet(optionals, 3); if (struct.isSetTinfo()) { struct.tinfo.write(oprot); } if (struct.isSetCredentials()) { struct.credentials.write(oprot); } + if (struct.isSetTaskID()) { + oprot.writeString(struct.taskID); + } } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, getActiveCompactions_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, cancelTask_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); @@ -3249,6 +3350,10 @@ public void read(org.apache.thrift.protocol.TProtocol prot, getActiveCompactions struct.credentials.read(iprot); struct.setCredentialsIsSet(true); } + if (incoming.get(2)) { + struct.taskID = iprot.readString(); + struct.setTaskIDIsSet(true); + } } } @@ -3258,22 +3363,17 @@ private static S scheme(org.apache. } @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) - public static class getActiveCompactions_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("getActiveCompactions_result"); + public static class cancelTask_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("cancelTask_result"); - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (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 getActiveCompactions_resultStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getActiveCompactions_resultTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new cancelTask_resultStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new cancelTask_resultTupleSchemeFactory(); - public @org.apache.thrift.annotation.Nullable java.util.List 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(); @@ -3289,10 +3389,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; default: return null; } @@ -3334,208 +3430,70 @@ 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.ListMetaData(org.apache.thrift.protocol.TType.LIST, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction.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(getActiveCompactions_result.class, metaDataMap); - } - - public getActiveCompactions_result() { + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(cancelTask_result.class, metaDataMap); } - public getActiveCompactions_result( - java.util.List success, - org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) - { - this(); - this.success = success; - this.sec = sec; + public cancelTask_result() { } /** * Performs a deep copy on other. */ - public getActiveCompactions_result(getActiveCompactions_result other) { - if (other.isSetSuccess()) { - java.util.List __this__success = new java.util.ArrayList(other.success.size()); - for (org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction other_element : other.success) { - __this__success.add(new org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction(other_element)); - } - this.success = __this__success; - } - if (other.isSetSec()) { - this.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(other.sec); - } + public cancelTask_result(cancelTask_result other) { } @Override - public getActiveCompactions_result deepCopy() { - return new getActiveCompactions_result(this); + public cancelTask_result deepCopy() { + return new cancelTask_result(this); } @Override public void clear() { - this.success = null; - this.sec = null; } - public int getSuccessSize() { - return (this.success == null) ? 0 : this.success.size(); + @Override + public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) { + switch (field) { + } } @org.apache.thrift.annotation.Nullable - public java.util.Iterator getSuccessIterator() { - return (this.success == null) ? null : this.success.iterator(); + @Override + public java.lang.Object getFieldValue(_Fields field) { + switch (field) { + } + throw new java.lang.IllegalStateException(); } - public void addToSuccess(org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction elem) { - if (this.success == null) { - this.success = new java.util.ArrayList(); + /** 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(); } - this.success.add(elem); - } - @org.apache.thrift.annotation.Nullable - public java.util.List getSuccess() { - return this.success; - } - - public getActiveCompactions_result setSuccess(@org.apache.thrift.annotation.Nullable java.util.List 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 getActiveCompactions_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.util.List)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(); + switch (field) { + } + throw new java.lang.IllegalStateException(); } @Override public boolean equals(java.lang.Object that) { - if (that instanceof getActiveCompactions_result) - return this.equals((getActiveCompactions_result)that); + if (that instanceof cancelTask_result) + return this.equals((cancelTask_result)that); return false; } - public boolean equals(getActiveCompactions_result that) { + public boolean equals(cancelTask_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; } @@ -3543,45 +3501,17 @@ public boolean equals(getActiveCompactions_result that) { 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(getActiveCompactions_result other) { + public int compareTo(cancelTask_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; } @@ -3602,24 +3532,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("getActiveCompactions_result("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("cancelTask_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(); } @@ -3645,17 +3560,17 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class getActiveCompactions_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class cancelTask_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public getActiveCompactions_resultStandardScheme getScheme() { - return new getActiveCompactions_resultStandardScheme(); + public cancelTask_resultStandardScheme getScheme() { + return new cancelTask_resultStandardScheme(); } } - private static class getActiveCompactions_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class cancelTask_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { @Override - public void read(org.apache.thrift.protocol.TProtocol iprot, getActiveCompactions_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, cancelTask_result struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -3665,34 +3580,6 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getActiveCompaction break; } switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { - { - org.apache.thrift.protocol.TList _list20 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list20.size); - @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction _elem21; - for (int _i22 = 0; _i22 < _list20.size; ++_i22) - { - _elem21 = new org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction(); - _elem21.read(iprot); - struct.success.add(_elem21); - } - 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; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -3705,90 +3592,33 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getActiveCompaction } @Override - public void write(org.apache.thrift.protocol.TProtocol oprot, getActiveCompactions_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, cancelTask_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.tabletserver.thrift.ActiveCompaction _iter23 : struct.success) - { - _iter23.write(oprot); - } - oprot.writeListEnd(); - } - oprot.writeFieldEnd(); - } - if (struct.sec != null) { - oprot.writeFieldBegin(SEC_FIELD_DESC); - struct.sec.write(oprot); - oprot.writeFieldEnd(); - } oprot.writeFieldStop(); oprot.writeStructEnd(); } } - private static class getActiveCompactions_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class cancelTask_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public getActiveCompactions_resultTupleScheme getScheme() { - return new getActiveCompactions_resultTupleScheme(); + public cancelTask_resultTupleScheme getScheme() { + return new cancelTask_resultTupleScheme(); } } - private static class getActiveCompactions_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class cancelTask_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, getActiveCompactions_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, cancelTask_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.writeI32(struct.success.size()); - for (org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction _iter24 : struct.success) - { - _iter24.write(oprot); - } - } - } - if (struct.isSetSec()) { - struct.sec.write(oprot); - } } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, getActiveCompactions_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, cancelTask_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)) { - { - org.apache.thrift.protocol.TList _list25 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList(_list25.size); - @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction _elem26; - for (int _i27 = 0; _i27 < _list25.size; ++_i27) - { - _elem26 = new org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction(); - _elem26.read(iprot); - struct.success.add(_elem26); - } - } - struct.setSuccessIsSet(true); - } - if (incoming.get(1)) { - struct.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(); - struct.sec.read(iprot); - struct.setSecIsSet(true); - } } } @@ -3798,25 +3628,22 @@ private static S scheme(org.apache. } @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) - public static class cancel_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("cancel_args"); + public static class getActiveCompactions_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("getActiveCompactions_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 cancel_argsStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new cancel_argsTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getActiveCompactions_argsStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getActiveCompactions_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"); + CREDENTIALS((short)2, "credentials"); private static final java.util.Map byName = new java.util.HashMap(); @@ -3836,8 +3663,6 @@ public static _Fields findByThriftId(int fieldId) { return TINFO; case 2: // CREDENTIALS return CREDENTIALS; - case 3: // EXTERNAL_COMPACTION_ID - return EXTERNAL_COMPACTION_ID; default: return null; } @@ -3888,51 +3713,43 @@ 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.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(cancel_args.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getActiveCompactions_args.class, metaDataMap); } - public cancel_args() { + public getActiveCompactions_args() { } - public cancel_args( + public getActiveCompactions_args( org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, - org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, - java.lang.String externalCompactionId) + org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) { this(); this.tinfo = tinfo; this.credentials = credentials; - this.externalCompactionId = externalCompactionId; } /** * Performs a deep copy on other. */ - public cancel_args(cancel_args other) { + public getActiveCompactions_args(getActiveCompactions_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 cancel_args deepCopy() { - return new cancel_args(this); + public getActiveCompactions_args deepCopy() { + return new getActiveCompactions_args(this); } @Override public void clear() { this.tinfo = null; this.credentials = null; - this.externalCompactionId = null; } @org.apache.thrift.annotation.Nullable @@ -3940,7 +3757,7 @@ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() { return this.tinfo; } - public cancel_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) { + public getActiveCompactions_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) { this.tinfo = tinfo; return this; } @@ -3965,7 +3782,7 @@ public org.apache.accumulo.core.securityImpl.thrift.TCredentials getCredentials( return this.credentials; } - public cancel_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) { + public getActiveCompactions_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) { this.credentials = credentials; return this; } @@ -3985,41 +3802,16 @@ public void setCredentialsIsSet(boolean value) { } } - @org.apache.thrift.annotation.Nullable - public java.lang.String getExternalCompactionId() { - return this.externalCompactionId; - } - - public cancel_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; + @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) { @@ -4029,14 +3821,6 @@ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable } break; - case EXTERNAL_COMPACTION_ID: - if (value == null) { - unsetExternalCompactionId(); - } else { - setExternalCompactionId((java.lang.String)value); - } - break; - } } @@ -4050,9 +3834,6 @@ public java.lang.Object getFieldValue(_Fields field) { case CREDENTIALS: return getCredentials(); - case EXTERNAL_COMPACTION_ID: - return getExternalCompactionId(); - } throw new java.lang.IllegalStateException(); } @@ -4069,20 +3850,18 @@ public boolean isSet(_Fields field) { 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 cancel_args) - return this.equals((cancel_args)that); + if (that instanceof getActiveCompactions_args) + return this.equals((getActiveCompactions_args)that); return false; } - public boolean equals(cancel_args that) { + public boolean equals(getActiveCompactions_args that) { if (that == null) return false; if (this == that) @@ -4106,15 +3885,6 @@ public boolean equals(cancel_args that) { 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; } @@ -4130,15 +3900,11 @@ public int hashCode() { 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(cancel_args other) { + public int compareTo(getActiveCompactions_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -4165,16 +3931,6 @@ public int compareTo(cancel_args other) { 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; } @@ -4196,7 +3952,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("cancel_args("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("getActiveCompactions_args("); boolean first = true; sb.append("tinfo:"); @@ -4214,14 +3970,6 @@ public java.lang.String toString() { 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(); } @@ -4253,17 +4001,17 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class cancel_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class getActiveCompactions_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public cancel_argsStandardScheme getScheme() { - return new cancel_argsStandardScheme(); + public getActiveCompactions_argsStandardScheme getScheme() { + return new getActiveCompactions_argsStandardScheme(); } } - private static class cancel_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class getActiveCompactions_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { @Override - public void read(org.apache.thrift.protocol.TProtocol iprot, cancel_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, getActiveCompactions_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -4291,14 +4039,6 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, cancel_args struct) 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); } @@ -4311,7 +4051,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, cancel_args struct) } @Override - public void write(org.apache.thrift.protocol.TProtocol oprot, cancel_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, getActiveCompactions_args struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -4325,28 +4065,23 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, cancel_args struct 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 cancel_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class getActiveCompactions_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public cancel_argsTupleScheme getScheme() { - return new cancel_argsTupleScheme(); + public getActiveCompactions_argsTupleScheme getScheme() { + return new getActiveCompactions_argsTupleScheme(); } } - private static class cancel_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class getActiveCompactions_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, cancel_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, getActiveCompactions_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()) { @@ -4355,25 +4090,19 @@ public void write(org.apache.thrift.protocol.TProtocol prot, cancel_args struct) if (struct.isSetCredentials()) { optionals.set(1); } - if (struct.isSetExternalCompactionId()) { - optionals.set(2); - } - oprot.writeBitSet(optionals, 3); + oprot.writeBitSet(optionals, 2); 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, cancel_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, getActiveCompactions_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); + 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); @@ -4384,10 +4113,6 @@ public void read(org.apache.thrift.protocol.TProtocol prot, cancel_args struct) struct.credentials.read(iprot); struct.setCredentialsIsSet(true); } - if (incoming.get(2)) { - struct.externalCompactionId = iprot.readString(); - struct.setExternalCompactionIdIsSet(true); - } } } @@ -4397,17 +4122,22 @@ private static S scheme(org.apache. } @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) - public static class cancel_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("cancel_result"); + public static class getActiveCompactions_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("getActiveCompactions_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 cancel_resultStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new cancel_resultTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getActiveCompactions_resultStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getActiveCompactions_resultTupleSchemeFactory(); + public @org.apache.thrift.annotation.Nullable Task 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(); @@ -4423,6 +4153,10 @@ 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; default: return null; } @@ -4464,34 +4198,123 @@ 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, Task.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(cancel_result.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getActiveCompactions_result.class, metaDataMap); } - public cancel_result() { + public getActiveCompactions_result() { + } + + public getActiveCompactions_result( + Task success, + org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) + { + this(); + this.success = success; + this.sec = sec; } /** * Performs a deep copy on other. */ - public cancel_result(cancel_result other) { + public getActiveCompactions_result(getActiveCompactions_result other) { + if (other.isSetSuccess()) { + this.success = new Task(other.success); + } + if (other.isSetSec()) { + this.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(other.sec); + } } @Override - public cancel_result deepCopy() { - return new cancel_result(this); + public getActiveCompactions_result deepCopy() { + return new getActiveCompactions_result(this); } @Override public void clear() { + this.success = null; + this.sec = null; + } + + @org.apache.thrift.annotation.Nullable + public Task getSuccess() { + return this.success; + } + + public getActiveCompactions_result setSuccess(@org.apache.thrift.annotation.Nullable Task 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 getActiveCompactions_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((Task)value); + } + break; + + case SEC: + if (value == null) { + unsetSec(); + } else { + setSec((org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException)value); + } + break; + } } @@ -4499,6 +4322,12 @@ 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(); + } throw new java.lang.IllegalStateException(); } @@ -4511,23 +4340,45 @@ public boolean isSet(_Fields field) { } 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 cancel_result) - return this.equals((cancel_result)that); + if (that instanceof getActiveCompactions_result) + return this.equals((getActiveCompactions_result)that); return false; } - public boolean equals(cancel_result that) { + public boolean equals(getActiveCompactions_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; } @@ -4535,17 +4386,45 @@ public boolean equals(cancel_result that) { 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(cancel_result other) { + public int compareTo(getActiveCompactions_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; } @@ -4566,9 +4445,24 @@ 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("cancel_result("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("getActiveCompactions_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(); } @@ -4576,6 +4470,9 @@ public java.lang.String 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 { @@ -4594,17 +4491,17 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class cancel_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class getActiveCompactions_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public cancel_resultStandardScheme getScheme() { - return new cancel_resultStandardScheme(); + public getActiveCompactions_resultStandardScheme getScheme() { + return new getActiveCompactions_resultStandardScheme(); } } - private static class cancel_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class getActiveCompactions_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { @Override - public void read(org.apache.thrift.protocol.TProtocol iprot, cancel_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, getActiveCompactions_result struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -4614,6 +4511,24 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, cancel_result struc break; } switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new Task(); + 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); } @@ -4626,33 +4541,68 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, cancel_result struc } @Override - public void write(org.apache.thrift.protocol.TProtocol oprot, cancel_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, getActiveCompactions_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 cancel_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + private static class getActiveCompactions_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { @Override - public cancel_resultTupleScheme getScheme() { - return new cancel_resultTupleScheme(); + public getActiveCompactions_resultTupleScheme getScheme() { + return new getActiveCompactions_resultTupleScheme(); } } - private static class cancel_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class getActiveCompactions_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, cancel_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, getActiveCompactions_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, cancel_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, getActiveCompactions_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 Task(); + 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); + } } } diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tasks/thrift/TaskRunnerInfo.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tasks/thrift/TaskRunnerInfo.java new file mode 100644 index 00000000000..5ab18d184f6 --- /dev/null +++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tasks/thrift/TaskRunnerInfo.java @@ -0,0 +1,732 @@ +/* + * 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.tasks.thrift; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) +public class TaskRunnerInfo 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("TaskRunnerInfo"); + + private static final org.apache.thrift.protocol.TField HOSTNAME_FIELD_DESC = new org.apache.thrift.protocol.TField("hostname", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField PORT_FIELD_DESC = new org.apache.thrift.protocol.TField("port", org.apache.thrift.protocol.TType.I32, (short)2); + private static final org.apache.thrift.protocol.TField WORKER_TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("workerType", org.apache.thrift.protocol.TType.I32, (short)3); + private static final org.apache.thrift.protocol.TField RESOURCE_GROUP_FIELD_DESC = new org.apache.thrift.protocol.TField("resourceGroup", org.apache.thrift.protocol.TType.STRING, (short)4); + + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new TaskRunnerInfoStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new TaskRunnerInfoTupleSchemeFactory(); + + public @org.apache.thrift.annotation.Nullable java.lang.String hostname; // required + public int port; // required + /** + * + * @see WorkerType + */ + public @org.apache.thrift.annotation.Nullable WorkerType workerType; // required + public @org.apache.thrift.annotation.Nullable java.lang.String resourceGroup; // 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 { + HOSTNAME((short)1, "hostname"), + PORT((short)2, "port"), + /** + * + * @see WorkerType + */ + WORKER_TYPE((short)3, "workerType"), + RESOURCE_GROUP((short)4, "resourceGroup"); + + 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: // HOSTNAME + return HOSTNAME; + case 2: // PORT + return PORT; + case 3: // WORKER_TYPE + return WORKER_TYPE; + case 4: // RESOURCE_GROUP + return RESOURCE_GROUP; + 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 __PORT_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.HOSTNAME, new org.apache.thrift.meta_data.FieldMetaData("hostname", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.PORT, new org.apache.thrift.meta_data.FieldMetaData("port", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.WORKER_TYPE, new org.apache.thrift.meta_data.FieldMetaData("workerType", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, WorkerType.class))); + tmpMap.put(_Fields.RESOURCE_GROUP, new org.apache.thrift.meta_data.FieldMetaData("resourceGroup", 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(TaskRunnerInfo.class, metaDataMap); + } + + public TaskRunnerInfo() { + } + + public TaskRunnerInfo( + java.lang.String hostname, + int port, + WorkerType workerType, + java.lang.String resourceGroup) + { + this(); + this.hostname = hostname; + this.port = port; + setPortIsSet(true); + this.workerType = workerType; + this.resourceGroup = resourceGroup; + } + + /** + * Performs a deep copy on other. + */ + public TaskRunnerInfo(TaskRunnerInfo other) { + __isset_bitfield = other.__isset_bitfield; + if (other.isSetHostname()) { + this.hostname = other.hostname; + } + this.port = other.port; + if (other.isSetWorkerType()) { + this.workerType = other.workerType; + } + if (other.isSetResourceGroup()) { + this.resourceGroup = other.resourceGroup; + } + } + + @Override + public TaskRunnerInfo deepCopy() { + return new TaskRunnerInfo(this); + } + + @Override + public void clear() { + this.hostname = null; + setPortIsSet(false); + this.port = 0; + this.workerType = null; + this.resourceGroup = null; + } + + @org.apache.thrift.annotation.Nullable + public java.lang.String getHostname() { + return this.hostname; + } + + public TaskRunnerInfo setHostname(@org.apache.thrift.annotation.Nullable java.lang.String hostname) { + this.hostname = hostname; + return this; + } + + public void unsetHostname() { + this.hostname = null; + } + + /** Returns true if field hostname is set (has been assigned a value) and false otherwise */ + public boolean isSetHostname() { + return this.hostname != null; + } + + public void setHostnameIsSet(boolean value) { + if (!value) { + this.hostname = null; + } + } + + public int getPort() { + return this.port; + } + + public TaskRunnerInfo setPort(int port) { + this.port = port; + setPortIsSet(true); + return this; + } + + public void unsetPort() { + __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __PORT_ISSET_ID); + } + + /** Returns true if field port is set (has been assigned a value) and false otherwise */ + public boolean isSetPort() { + return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __PORT_ISSET_ID); + } + + public void setPortIsSet(boolean value) { + __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __PORT_ISSET_ID, value); + } + + /** + * + * @see WorkerType + */ + @org.apache.thrift.annotation.Nullable + public WorkerType getWorkerType() { + return this.workerType; + } + + /** + * + * @see WorkerType + */ + public TaskRunnerInfo setWorkerType(@org.apache.thrift.annotation.Nullable WorkerType workerType) { + this.workerType = workerType; + return this; + } + + public void unsetWorkerType() { + this.workerType = null; + } + + /** Returns true if field workerType is set (has been assigned a value) and false otherwise */ + public boolean isSetWorkerType() { + return this.workerType != null; + } + + public void setWorkerTypeIsSet(boolean value) { + if (!value) { + this.workerType = null; + } + } + + @org.apache.thrift.annotation.Nullable + public java.lang.String getResourceGroup() { + return this.resourceGroup; + } + + public TaskRunnerInfo setResourceGroup(@org.apache.thrift.annotation.Nullable java.lang.String resourceGroup) { + this.resourceGroup = resourceGroup; + return this; + } + + public void unsetResourceGroup() { + this.resourceGroup = null; + } + + /** Returns true if field resourceGroup is set (has been assigned a value) and false otherwise */ + public boolean isSetResourceGroup() { + return this.resourceGroup != null; + } + + public void setResourceGroupIsSet(boolean value) { + if (!value) { + this.resourceGroup = null; + } + } + + @Override + public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) { + switch (field) { + case HOSTNAME: + if (value == null) { + unsetHostname(); + } else { + setHostname((java.lang.String)value); + } + break; + + case PORT: + if (value == null) { + unsetPort(); + } else { + setPort((java.lang.Integer)value); + } + break; + + case WORKER_TYPE: + if (value == null) { + unsetWorkerType(); + } else { + setWorkerType((WorkerType)value); + } + break; + + case RESOURCE_GROUP: + if (value == null) { + unsetResourceGroup(); + } else { + setResourceGroup((java.lang.String)value); + } + break; + + } + } + + @org.apache.thrift.annotation.Nullable + @Override + public java.lang.Object getFieldValue(_Fields field) { + switch (field) { + case HOSTNAME: + return getHostname(); + + case PORT: + return getPort(); + + case WORKER_TYPE: + return getWorkerType(); + + case RESOURCE_GROUP: + return getResourceGroup(); + + } + 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 HOSTNAME: + return isSetHostname(); + case PORT: + return isSetPort(); + case WORKER_TYPE: + return isSetWorkerType(); + case RESOURCE_GROUP: + return isSetResourceGroup(); + } + throw new java.lang.IllegalStateException(); + } + + @Override + public boolean equals(java.lang.Object that) { + if (that instanceof TaskRunnerInfo) + return this.equals((TaskRunnerInfo)that); + return false; + } + + public boolean equals(TaskRunnerInfo that) { + if (that == null) + return false; + if (this == that) + return true; + + boolean this_present_hostname = true && this.isSetHostname(); + boolean that_present_hostname = true && that.isSetHostname(); + if (this_present_hostname || that_present_hostname) { + if (!(this_present_hostname && that_present_hostname)) + return false; + if (!this.hostname.equals(that.hostname)) + return false; + } + + boolean this_present_port = true; + boolean that_present_port = true; + if (this_present_port || that_present_port) { + if (!(this_present_port && that_present_port)) + return false; + if (this.port != that.port) + return false; + } + + boolean this_present_workerType = true && this.isSetWorkerType(); + boolean that_present_workerType = true && that.isSetWorkerType(); + if (this_present_workerType || that_present_workerType) { + if (!(this_present_workerType && that_present_workerType)) + return false; + if (!this.workerType.equals(that.workerType)) + return false; + } + + boolean this_present_resourceGroup = true && this.isSetResourceGroup(); + boolean that_present_resourceGroup = true && that.isSetResourceGroup(); + if (this_present_resourceGroup || that_present_resourceGroup) { + if (!(this_present_resourceGroup && that_present_resourceGroup)) + return false; + if (!this.resourceGroup.equals(that.resourceGroup)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + int hashCode = 1; + + hashCode = hashCode * 8191 + ((isSetHostname()) ? 131071 : 524287); + if (isSetHostname()) + hashCode = hashCode * 8191 + hostname.hashCode(); + + hashCode = hashCode * 8191 + port; + + hashCode = hashCode * 8191 + ((isSetWorkerType()) ? 131071 : 524287); + if (isSetWorkerType()) + hashCode = hashCode * 8191 + workerType.getValue(); + + hashCode = hashCode * 8191 + ((isSetResourceGroup()) ? 131071 : 524287); + if (isSetResourceGroup()) + hashCode = hashCode * 8191 + resourceGroup.hashCode(); + + return hashCode; + } + + @Override + public int compareTo(TaskRunnerInfo other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = java.lang.Boolean.compare(isSetHostname(), other.isSetHostname()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetHostname()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.hostname, other.hostname); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = java.lang.Boolean.compare(isSetPort(), other.isSetPort()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetPort()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.port, other.port); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = java.lang.Boolean.compare(isSetWorkerType(), other.isSetWorkerType()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetWorkerType()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.workerType, other.workerType); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = java.lang.Boolean.compare(isSetResourceGroup(), other.isSetResourceGroup()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetResourceGroup()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.resourceGroup, other.resourceGroup); + 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("TaskRunnerInfo("); + boolean first = true; + + sb.append("hostname:"); + if (this.hostname == null) { + sb.append("null"); + } else { + sb.append(this.hostname); + } + first = false; + if (!first) sb.append(", "); + sb.append("port:"); + sb.append(this.port); + first = false; + if (!first) sb.append(", "); + sb.append("workerType:"); + if (this.workerType == null) { + sb.append("null"); + } else { + sb.append(this.workerType); + } + first = false; + if (!first) sb.append(", "); + sb.append("resourceGroup:"); + if (this.resourceGroup == null) { + sb.append("null"); + } else { + sb.append(this.resourceGroup); + } + 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 TaskRunnerInfoStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + @Override + public TaskRunnerInfoStandardScheme getScheme() { + return new TaskRunnerInfoStandardScheme(); + } + } + + private static class TaskRunnerInfoStandardScheme extends org.apache.thrift.scheme.StandardScheme { + + @Override + public void read(org.apache.thrift.protocol.TProtocol iprot, TaskRunnerInfo 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: // HOSTNAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.hostname = iprot.readString(); + struct.setHostnameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // PORT + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.port = iprot.readI32(); + struct.setPortIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // WORKER_TYPE + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.workerType = org.apache.accumulo.core.tasks.thrift.WorkerType.findByValue(iprot.readI32()); + struct.setWorkerTypeIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // RESOURCE_GROUP + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.resourceGroup = iprot.readString(); + struct.setResourceGroupIsSet(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, TaskRunnerInfo struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.hostname != null) { + oprot.writeFieldBegin(HOSTNAME_FIELD_DESC); + oprot.writeString(struct.hostname); + oprot.writeFieldEnd(); + } + oprot.writeFieldBegin(PORT_FIELD_DESC); + oprot.writeI32(struct.port); + oprot.writeFieldEnd(); + if (struct.workerType != null) { + oprot.writeFieldBegin(WORKER_TYPE_FIELD_DESC); + oprot.writeI32(struct.workerType.getValue()); + oprot.writeFieldEnd(); + } + if (struct.resourceGroup != null) { + oprot.writeFieldBegin(RESOURCE_GROUP_FIELD_DESC); + oprot.writeString(struct.resourceGroup); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TaskRunnerInfoTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + @Override + public TaskRunnerInfoTupleScheme getScheme() { + return new TaskRunnerInfoTupleScheme(); + } + } + + private static class TaskRunnerInfoTupleScheme extends org.apache.thrift.scheme.TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TaskRunnerInfo 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.isSetHostname()) { + optionals.set(0); + } + if (struct.isSetPort()) { + optionals.set(1); + } + if (struct.isSetWorkerType()) { + optionals.set(2); + } + if (struct.isSetResourceGroup()) { + optionals.set(3); + } + oprot.writeBitSet(optionals, 4); + if (struct.isSetHostname()) { + oprot.writeString(struct.hostname); + } + if (struct.isSetPort()) { + oprot.writeI32(struct.port); + } + if (struct.isSetWorkerType()) { + oprot.writeI32(struct.workerType.getValue()); + } + if (struct.isSetResourceGroup()) { + oprot.writeString(struct.resourceGroup); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TaskRunnerInfo 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.hostname = iprot.readString(); + struct.setHostnameIsSet(true); + } + if (incoming.get(1)) { + struct.port = iprot.readI32(); + struct.setPortIsSet(true); + } + if (incoming.get(2)) { + struct.workerType = org.apache.accumulo.core.tasks.thrift.WorkerType.findByValue(iprot.readI32()); + struct.setWorkerTypeIsSet(true); + } + if (incoming.get(3)) { + struct.resourceGroup = iprot.readString(); + struct.setResourceGroupIsSet(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/tasks/thrift/WorkerType.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tasks/thrift/WorkerType.java new file mode 100644 index 00000000000..8b605cd8659 --- /dev/null +++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tasks/thrift/WorkerType.java @@ -0,0 +1,64 @@ +/* + * 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.tasks.thrift; + + +public enum WorkerType implements org.apache.thrift.TEnum { + COMPACTION(0), + LOG_SORTING(1), + SPLIT_POINT_CALCULATION(2); + + private final int value; + + private WorkerType(int value) { + this.value = value; + } + + /** + * Get the integer value of this enum value, as defined in the Thrift IDL. + */ + @Override + public int getValue() { + return value; + } + + /** + * Find a the enum type by its integer value, as defined in the Thrift IDL. + * @return null if the value is not found. + */ + @org.apache.thrift.annotation.Nullable + public static WorkerType findByValue(int value) { + switch (value) { + case 0: + return COMPACTION; + case 1: + return LOG_SORTING; + case 2: + return SPLIT_POINT_CALCULATION; + default: + return null; + } + } +} diff --git a/core/src/main/thrift/compaction-coordinator.thrift b/core/src/main/thrift/compaction-coordinator.thrift index 47bf06af15e..7f50fcb56b4 100644 --- a/core/src/main/thrift/compaction-coordinator.thrift +++ b/core/src/main/thrift/compaction-coordinator.thrift @@ -60,101 +60,3 @@ struct TExternalCompactionList { exception UnknownCompactionIdException {} -service CompactionCoordinatorService { - - /* - * Called by Compactor on successful completion of compaction job - */ - void compactionCompleted( - 1:client.TInfo tinfo - 2:security.TCredentials credentials - 3:string externalCompactionId - 4:data.TKeyExtent extent - 5:tabletserver.TCompactionStats stats - ) - - /* - * Called by Compactor to get the next compaction job - */ - tabletserver.TExternalCompactionJob getCompactionJob( - 1:client.TInfo tinfo - 2:security.TCredentials credentials - 3:string groupName - 4:string compactor - 5:string externalCompactionId - ) - - /* - * Called by Compactor to update the Coordinator with the state of the compaction - */ - void updateCompactionStatus( - 1:client.TInfo tinfo - 2:security.TCredentials credentials - 3:string externalCompactionId - 4:TCompactionStatusUpdate status - 5:i64 timestamp - ) - - /* - * Called by Compactor on unsuccessful completion of compaction job - */ - void compactionFailed( - 1:client.TInfo tinfo - 2:security.TCredentials credentials - 3:string externalCompactionId - 4:data.TKeyExtent extent - ) - - /* - * Called by the Monitor to get progress information - */ - TExternalCompactionList getRunningCompactions( - 1:client.TInfo tinfo - 2:security.TCredentials credentials - ) - - /* - * Called by the Monitor to get progress information - */ - TExternalCompactionList getCompletedCompactions( - 1:client.TInfo tinfo - 2:security.TCredentials credentials - ) - - void cancel( - 1:client.TInfo tinfo - 2:security.TCredentials credentials - 3:string externalCompactionId - ) - -} - -service CompactorService { - - tabletserver.TExternalCompactionJob getRunningCompaction( - 1:client.TInfo tinfo - 2:security.TCredentials credentials - ) throws ( - 1:client.ThriftSecurityException sec - ) - - string getRunningCompactionId( - 1:client.TInfo tinfo - 2:security.TCredentials credentials - ) throws ( - 1:client.ThriftSecurityException sec - ) - - list getActiveCompactions( - 2:client.TInfo tinfo - 1:security.TCredentials credentials - ) throws ( - 1:client.ThriftSecurityException sec - ) - - void cancel( - 1:client.TInfo tinfo - 2:security.TCredentials credentials - 3:string externalCompactionId - ) -} diff --git a/core/src/main/thrift/tabletserver.thrift b/core/src/main/thrift/tabletserver.thrift index accb311c8b3..80d5bf632f7 100644 --- a/core/src/main/thrift/tabletserver.thrift +++ b/core/src/main/thrift/tabletserver.thrift @@ -85,6 +85,10 @@ struct ActiveCompaction { 12:i64 timesPaused } +struct ActiveCompactionList { + 1:list compactions +} + struct TIteratorSetting { 1:i32 priority 2:string name diff --git a/core/src/main/thrift/tasks.thrift b/core/src/main/thrift/tasks.thrift new file mode 100644 index 00000000000..3a3cef6a54d --- /dev/null +++ b/core/src/main/thrift/tasks.thrift @@ -0,0 +1,127 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +namespace java org.apache.accumulo.core.tasks.thrift +namespace cpp org.apache.accumulo.core.tasks.thrift + +include "security.thrift" +include "client.thrift" +include "tabletserver.thrift" + +enum WorkerType { + COMPACTION + LOG_SORTING + SPLIT_POINT_CALCULATION +} + +struct TaskRunnerInfo { + 1:string hostname + 2:i32 port + 3:WorkerType workerType + 4:string resourceGroup +} + +struct Task { + 1:string taskId + 2:string messageType + 3:string message +} + +service TaskManager { + + Task getTask( + 1:client.TInfo tinfo + 2:security.TCredentials credentials + 3:TaskRunnerInfo taskRunner + 4:string taskID + ) + + oneway void taskStatus( + 1:client.TInfo tinfo + 2:security.TCredentials credentials + 3:i64 timestamp + 4:Task taskUpdateObject + ) + + void taskCompleted( + 1:client.TInfo tinfo + 2:security.TCredentials credentials + 3:Task task + ) + + void taskFailed( + 1:client.TInfo tinfo + 2:security.TCredentials credentials + 3:Task task + ) + + void cancelTask( + 1:client.TInfo tinfo + 2:security.TCredentials credentials + 3:string taskID + ) + + /* + * Called by the Monitor to get progress information + */ + Task getRunningTasks( + 1:client.TInfo tinfo + 2:security.TCredentials credentials + ) + + /* + * Called by the Monitor to get progress information + */ + Task getCompletedTasks( + 1:client.TInfo tinfo + 2:security.TCredentials credentials + ) + +} + +service TaskRunner { + + Task getRunningTask( + 1:client.TInfo tinfo + 2:security.TCredentials credentials + ) throws ( + 1:client.ThriftSecurityException sec + ) + + string getRunningTaskId( + 1:client.TInfo tinfo + 2:security.TCredentials credentials + ) throws ( + 1:client.ThriftSecurityException sec + ) + + void cancelTask( + 1:client.TInfo tinfo + 2:security.TCredentials credentials + 3:string taskID + ) + + Task getActiveCompactions( + 1:client.TInfo tinfo + 2:security.TCredentials credentials + ) throws ( + 1:client.ThriftSecurityException sec + ) + + +} diff --git a/core/src/test/java/org/apache/accumulo/core/lock/ServiceLockDataTest.java b/core/src/test/java/org/apache/accumulo/core/lock/ServiceLockDataTest.java index a13f1ca4443..5b3f85e46bc 100644 --- a/core/src/test/java/org/apache/accumulo/core/lock/ServiceLockDataTest.java +++ b/core/src/test/java/org/apache/accumulo/core/lock/ServiceLockDataTest.java @@ -115,9 +115,9 @@ public void testMultipleServiceConstructorWithGroup() throws Exception { assertEquals(HostAndPort.fromString("127.0.0.1:9998"), ss.getAddress(ThriftService.TABLET_SCAN)); assertEquals("ns1", ss.getGroup(ThriftService.TABLET_SCAN)); - assertNull(ss.getAddressString(ThriftService.COMPACTOR)); - assertNull(ss.getAddress(ThriftService.COMPACTOR)); - assertNull(ss.getGroup(ThriftService.COMPACTOR)); + assertNull(ss.getAddressString(ThriftService.TASK_RUNNER)); + assertNull(ss.getAddress(ThriftService.TASK_RUNNER)); + assertNull(ss.getGroup(ThriftService.TASK_RUNNER)); } @Test diff --git a/core/src/test/java/org/apache/accumulo/core/tasks/CompactionTasksSerializationTest.java b/core/src/test/java/org/apache/accumulo/core/tasks/CompactionTasksSerializationTest.java new file mode 100644 index 00000000000..a7033f0caf3 --- /dev/null +++ b/core/src/test/java/org/apache/accumulo/core/tasks/CompactionTasksSerializationTest.java @@ -0,0 +1,82 @@ +/* + * 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.tasks; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; + +import org.apache.accumulo.core.data.TableId; +import org.apache.accumulo.core.dataImpl.KeyExtent; +import org.apache.accumulo.core.iteratorsImpl.system.SystemIteratorUtil; +import org.apache.accumulo.core.spi.compaction.CompactionKind; +import org.apache.accumulo.core.tabletserver.thrift.InputFile; +import org.apache.accumulo.core.tabletserver.thrift.IteratorConfig; +import org.apache.accumulo.core.tabletserver.thrift.TCompactionKind; +import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob; +import org.apache.accumulo.core.tasks.compaction.CompactionTask; +import org.apache.accumulo.core.tasks.thrift.Task; +import org.apache.hadoop.io.Text; +import org.junit.jupiter.api.Test; + +public class CompactionTasksSerializationTest { + + @Test + public void testCompactionTask() throws Exception { + + TExternalCompactionJob job = new TExternalCompactionJob(); + + job.setFateTxId(123456789L); + job.setExtent(new KeyExtent(TableId.of("2B"), new Text("3"), new Text("2")).toThrift()); + job.setKind(TCompactionKind.valueOf(CompactionKind.SYSTEM.name())); + List files = new ArrayList<>(); + files.add(new InputFile("hdfs://fake/path/to/output/fileA.rf", 32768, 1000, + System.currentTimeMillis())); + files.add(new InputFile("hdfs://fake/path/to/output/fileB.rf", 32768, 1000, + System.currentTimeMillis())); + job.setFiles(files); + IteratorConfig iteratorSettings = SystemIteratorUtil.toIteratorConfig(List.of()); + job.setIteratorSettings(iteratorSettings); + job.setOutputFile("hdfs://fake/path/to/output/fileC.rf"); + Map overrides = new HashMap<>(); + overrides.put("override1", "value1"); + overrides.put("overrides2", "value2"); + job.setOverrides(overrides); + job.setPropagateDeletes(true); + + CompactionTask task = TaskMessageType.COMPACTION_TASK.getTaskMessage(); + task.setTaskId(UUID.randomUUID().toString()); + task.setCompactionJob(job); + + Task to = task.toThriftTask(); + assertEquals(TaskMessageType.COMPACTION_TASK.name(), to.getMessageType()); + System.out.println(to.getMessage()); + + CompactionTask task2 = TaskMessage.fromThiftTask(to, TaskMessageType.COMPACTION_TASK); + + assertEquals(task.getTaskId(), task2.getTaskId()); + assertEquals(task.getMessageType(), task2.getMessageType()); + assertEquals(task.getCompactionJob(), task2.getCompactionJob()); + } + +} diff --git a/minicluster/pom.xml b/minicluster/pom.xml index 8ce479da1ff..2f6c3ce6ad0 100644 --- a/minicluster/pom.xml +++ b/minicluster/pom.xml @@ -47,10 +47,6 @@ commons-io commons-io - - org.apache.accumulo - accumulo-compactor - org.apache.accumulo accumulo-core @@ -75,6 +71,10 @@ org.apache.accumulo accumulo-start + + org.apache.accumulo + accumulo-task-runner + org.apache.accumulo accumulo-tserver diff --git a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterControl.java b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterControl.java index de16f12e541..d679248dc74 100644 --- a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterControl.java +++ b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterControl.java @@ -33,14 +33,15 @@ import java.util.concurrent.TimeoutException; import org.apache.accumulo.cluster.ClusterControl; -import org.apache.accumulo.compactor.Compactor; import org.apache.accumulo.core.conf.Property; +import org.apache.accumulo.core.tasks.thrift.WorkerType; import org.apache.accumulo.gc.SimpleGarbageCollector; import org.apache.accumulo.manager.Manager; import org.apache.accumulo.minicluster.ServerType; import org.apache.accumulo.miniclusterImpl.MiniAccumuloClusterImpl.ProcessInfo; import org.apache.accumulo.monitor.Monitor; import org.apache.accumulo.server.util.Admin; +import org.apache.accumulo.tasks.TaskRunner; import org.apache.accumulo.tserver.ScanServer; import org.apache.accumulo.tserver.TabletServer; import org.apache.zookeeper.server.ZooKeeperServerMain; @@ -145,7 +146,7 @@ public synchronized void start(ServerType server, Map configOverr } else { switch (server) { case COMPACTOR: - classToUse = Compactor.class; + classToUse = TaskRunner.class; break; case SCAN_SERVER: classToUse = ScanServer.class; @@ -233,7 +234,10 @@ public synchronized void start(ServerType server, Map configOverr int count = 0; for (int i = processes.size(); count < limit && i < e.getValue(); i++, ++count) { processes.add(cluster._exec(classToUse, server, configOverrides, "-o", - Property.COMPACTOR_GROUP_NAME.getKey() + "=" + e.getKey()).getProcess()); + Property.TASK_RUNNER_GROUP_NAME.getKey() + "=" + e.getKey(), "-o", + Property.TASK_RUNNER_WORKER_TYPE.getKey() + "=" + + WorkerType.COMPACTION.toString()) + .getProcess()); } } } diff --git a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloConfigImpl.java b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloConfigImpl.java index 895b1b6f4a4..adf2984c63d 100644 --- a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloConfigImpl.java +++ b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloConfigImpl.java @@ -164,7 +164,7 @@ MiniAccumuloConfigImpl initialize() { mergePropWithRandomPort(Property.MONITOR_PORT.getKey()); mergePropWithRandomPort(Property.GC_PORT.getKey()); - mergeProp(Property.COMPACTOR_PORTSEARCH.getKey(), "true"); + mergeProp(Property.TASK_RUNNER_PORTSEARCH.getKey(), "true"); mergeProp(Property.MANAGER_COMPACTION_SERVICE_PRIORITY_QUEUE_SIZE.getKey(), Property.MANAGER_COMPACTION_SERVICE_PRIORITY_QUEUE_SIZE.getDefaultValue()); diff --git a/pom.xml b/pom.xml index 108fc5788e0..b8bfe887756 100644 --- a/pom.xml +++ b/pom.xml @@ -85,11 +85,11 @@ iterator-test-harness minicluster server/base - server/compactor server/gc server/manager server/monitor server/native + server/task-runner server/tserver shell start @@ -328,11 +328,6 @@ junit 4.13.2 - - org.apache.accumulo - accumulo-compactor - ${project.version} - org.apache.accumulo accumulo-core @@ -389,6 +384,11 @@ accumulo-start ${project.version} + + org.apache.accumulo + accumulo-task-runner + ${project.version} + org.apache.accumulo accumulo-test 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 d52b47d3949..20cafb99d09 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 @@ -19,8 +19,6 @@ package org.apache.accumulo.server.rpc; import org.apache.accumulo.core.clientImpl.thrift.ClientService; -import org.apache.accumulo.core.compaction.thrift.CompactionCoordinatorService; -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.ManagerClientService; @@ -29,6 +27,8 @@ import org.apache.accumulo.core.tabletingest.thrift.TabletIngestClientService; import org.apache.accumulo.core.tabletscan.thrift.TabletScanClientService; import org.apache.accumulo.core.tabletserver.thrift.TabletServerClientService; +import org.apache.accumulo.core.tasks.thrift.TaskManager; +import org.apache.accumulo.core.tasks.thrift.TaskRunner; import org.apache.accumulo.core.trace.TraceUtil; import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.server.client.ClientServiceHandler; @@ -66,12 +66,6 @@ public > TProcessor getTProcessor( public static final ThriftProcessorTypes CLIENT = new ThriftProcessorTypes<>(ThriftClientTypes.CLIENT); - private static final ThriftProcessorTypes COMPACTOR = - new ThriftProcessorTypes<>(ThriftClientTypes.COMPACTOR); - - private static final ThriftProcessorTypes COORDINATOR = - new ThriftProcessorTypes<>(ThriftClientTypes.COORDINATOR); - private static final ThriftProcessorTypes FATE = new ThriftProcessorTypes<>(ThriftClientTypes.FATE); @@ -95,11 +89,17 @@ public > TProcessor getTProcessor( public static final ThriftProcessorTypes TABLET_MGMT = new ThriftProcessorTypes<>(ThriftClientTypes.TABLET_MGMT); - public static TMultiplexedProcessor getCompactorTProcessor(CompactorService.Iface serviceHandler, + public static final ThriftProcessorTypes TASK_RUNNER = + new ThriftProcessorTypes<>(ThriftClientTypes.TASK_RUNNER); + + public static final ThriftProcessorTypes TASK_MANAGER = + new ThriftProcessorTypes<>(ThriftClientTypes.TASK_MANAGER); + + public static TMultiplexedProcessor getCompactorTProcessor(TaskRunner.Iface serviceHandler, ServerContext context) { TMultiplexedProcessor muxProcessor = new TMultiplexedProcessor(); - muxProcessor.registerProcessor(COMPACTOR.getServiceName(), COMPACTOR.getTProcessor( - CompactorService.Processor.class, CompactorService.Iface.class, serviceHandler, context)); + muxProcessor.registerProcessor(TASK_RUNNER.getServiceName(), TASK_RUNNER.getTProcessor( + TaskRunner.Processor.class, TaskRunner.Iface.class, serviceHandler, context)); return muxProcessor; } @@ -112,14 +112,13 @@ public static TMultiplexedProcessor getGcTProcessor(GCMonitorService.Iface servi } public static TMultiplexedProcessor getManagerTProcessor(FateService.Iface fateServiceHandler, - CompactionCoordinatorService.Iface coordinatorServiceHandler, - ManagerClientService.Iface managerServiceHandler, ServerContext context) { + TaskManager.Iface taskMgrServiceHandler, ManagerClientService.Iface managerServiceHandler, + ServerContext context) { TMultiplexedProcessor muxProcessor = new TMultiplexedProcessor(); muxProcessor.registerProcessor(FATE.getServiceName(), FATE.getTProcessor( FateService.Processor.class, FateService.Iface.class, fateServiceHandler, context)); - muxProcessor.registerProcessor(COORDINATOR.getServiceName(), - COORDINATOR.getTProcessor(CompactionCoordinatorService.Processor.class, - CompactionCoordinatorService.Iface.class, coordinatorServiceHandler, context)); + muxProcessor.registerProcessor(TASK_MANAGER.getServiceName(), TASK_MANAGER.getTProcessor( + TaskManager.Processor.class, TaskManager.Iface.class, taskMgrServiceHandler, context)); muxProcessor.registerProcessor(MANAGER.getServiceName(), MANAGER.getTProcessor(ManagerClientService.Processor.class, ManagerClientService.Iface.class, managerServiceHandler, context)); diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ECAdmin.java b/server/base/src/main/java/org/apache/accumulo/server/util/ECAdmin.java index 416bbf39a2f..b37a7b7bf79 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/ECAdmin.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/ECAdmin.java @@ -18,7 +18,6 @@ */ package org.apache.accumulo.server.util; -import org.apache.accumulo.core.compaction.thrift.CompactionCoordinatorService; import org.apache.accumulo.core.compaction.thrift.TExternalCompactionList; import org.apache.accumulo.core.dataImpl.KeyExtent; import org.apache.accumulo.core.metadata.schema.ExternalCompactionId; @@ -26,6 +25,11 @@ import org.apache.accumulo.core.rpc.clients.ThriftClientTypes; import org.apache.accumulo.core.singletons.SingletonManager; import org.apache.accumulo.core.singletons.SingletonManager.Mode; +import org.apache.accumulo.core.tasks.TaskMessage; +import org.apache.accumulo.core.tasks.TaskMessageType; +import org.apache.accumulo.core.tasks.compaction.CompactionTasksRunning; +import org.apache.accumulo.core.tasks.thrift.Task; +import org.apache.accumulo.core.tasks.thrift.TaskManager; import org.apache.accumulo.core.trace.TraceUtil; import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil; import org.apache.accumulo.core.util.compaction.RunningCompaction; @@ -131,16 +135,16 @@ public void execute(final String[] args) { } private void cancelCompaction(ServerContext context, String ecid) { - CompactionCoordinatorService.Client coordinatorClient = null; + TaskManager.Client taskManagerClient = null; ecid = ExternalCompactionId.from(ecid).canonical(); try { - coordinatorClient = getCoordinatorClient(context); - coordinatorClient.cancel(TraceUtil.traceInfo(), context.rpcCreds(), ecid); - System.out.println("Cancel sent to coordinator for " + ecid); + taskManagerClient = getTaskManagerClient(context); + taskManagerClient.cancelTask(TraceUtil.traceInfo(), context.rpcCreds(), ecid); + System.out.println("Cancel sent to TaskManager for " + ecid); } catch (Exception e) { throw new IllegalStateException("Exception calling cancel compaction for " + ecid, e); } finally { - ThriftUtil.returnClient(coordinatorClient, context); + ThriftUtil.returnClient(taskManagerClient, context); } } @@ -154,11 +158,14 @@ private void listCompactorsByQueue(ServerContext context) { } private void runningCompactions(ServerContext context, boolean details) { - CompactionCoordinatorService.Client coordinatorClient = null; + TaskManager.Client taskManagerClient = null; TExternalCompactionList running; try { - coordinatorClient = getCoordinatorClient(context); - running = coordinatorClient.getRunningCompactions(TraceUtil.traceInfo(), context.rpcCreds()); + taskManagerClient = getTaskManagerClient(context); + Task task = taskManagerClient.getRunningTasks(TraceUtil.traceInfo(), context.rpcCreds()); + final CompactionTasksRunning list = + TaskMessage.fromThiftTask(task, TaskMessageType.COMPACTION_TASKS_RUNNING); + running = list.getRunning(); if (running == null) { System.out.println("No running compactions found."); return; @@ -191,23 +198,23 @@ private void runningCompactions(ServerContext context, boolean details) { } catch (Exception e) { throw new IllegalStateException("Unable to get running compactions.", e); } finally { - ThriftUtil.returnClient(coordinatorClient, context); + ThriftUtil.returnClient(taskManagerClient, context); } } - private CompactionCoordinatorService.Client getCoordinatorClient(ServerContext context) { - var coordinatorHost = ExternalCompactionUtil.findCompactionCoordinator(context); - if (coordinatorHost.isEmpty()) { - throw new IllegalStateException("Unable to find coordinator. Check that it is running."); + private TaskManager.Client getTaskManagerClient(ServerContext context) { + var taskManagerHost = ExternalCompactionUtil.findTaskManager(context); + if (taskManagerHost.isEmpty()) { + throw new IllegalStateException("Unable to find TaskManager. Check that it is running."); } - HostAndPort address = coordinatorHost.orElseThrow(); - CompactionCoordinatorService.Client coordinatorClient; + HostAndPort address = taskManagerHost.orElseThrow(); + TaskManager.Client taskManagerClient; try { - coordinatorClient = ThriftUtil.getClient(ThriftClientTypes.COORDINATOR, address, context); + taskManagerClient = ThriftUtil.getClient(ThriftClientTypes.TASK_MANAGER, address, context); } catch (Exception e) { - throw new IllegalStateException("Unable to get Compaction coordinator at " + address, e); + throw new IllegalStateException("Unable to get TaskManager at " + address, e); } - System.out.println("Connected to coordinator at " + address); - return coordinatorClient; + System.out.println("Connected to TaskManager at " + address); + return taskManagerClient; } } diff --git a/server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java b/server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java deleted file mode 100644 index e5d2ccbdfc3..00000000000 --- a/server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java +++ /dev/null @@ -1,878 +0,0 @@ -/* - * 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.compactor; - -import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly; -import static java.util.concurrent.TimeUnit.MINUTES; -import static org.apache.accumulo.core.util.LazySingletons.RANDOM; - -import java.io.IOException; -import java.lang.reflect.InvocationTargetException; -import java.net.UnknownHostException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.TreeMap; -import java.util.UUID; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ScheduledThreadPoolExecutor; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicReference; -import java.util.concurrent.atomic.LongAdder; -import java.util.function.Supplier; - -import org.apache.accumulo.core.Constants; -import org.apache.accumulo.core.cli.ConfigOpts; -import org.apache.accumulo.core.client.AccumuloSecurityException; -import org.apache.accumulo.core.client.IteratorSetting; -import org.apache.accumulo.core.client.TableNotFoundException; -import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode; -import org.apache.accumulo.core.clientImpl.thrift.TInfo; -import org.apache.accumulo.core.clientImpl.thrift.TableOperation; -import org.apache.accumulo.core.clientImpl.thrift.TableOperationExceptionType; -import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException; -import org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException; -import org.apache.accumulo.core.compaction.thrift.CompactionCoordinatorService; -import org.apache.accumulo.core.compaction.thrift.CompactionCoordinatorService.Client; -import org.apache.accumulo.core.compaction.thrift.CompactorService; -import org.apache.accumulo.core.compaction.thrift.TCompactionState; -import org.apache.accumulo.core.compaction.thrift.TCompactionStatusUpdate; -import org.apache.accumulo.core.compaction.thrift.UnknownCompactionIdException; -import org.apache.accumulo.core.conf.AccumuloConfiguration; -import org.apache.accumulo.core.conf.ConfigurationCopy; -import org.apache.accumulo.core.conf.Property; -import org.apache.accumulo.core.conf.SiteConfiguration; -import org.apache.accumulo.core.data.NamespaceId; -import org.apache.accumulo.core.data.TableId; -import org.apache.accumulo.core.dataImpl.KeyExtent; -import org.apache.accumulo.core.fate.FateTxId; -import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; -import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeExistsPolicy; -import org.apache.accumulo.core.iteratorsImpl.system.SystemIteratorUtil; -import org.apache.accumulo.core.lock.ServiceLock; -import org.apache.accumulo.core.lock.ServiceLock.LockLossReason; -import org.apache.accumulo.core.lock.ServiceLock.LockWatcher; -import org.apache.accumulo.core.lock.ServiceLockData; -import org.apache.accumulo.core.lock.ServiceLockData.ThriftService; -import org.apache.accumulo.core.metadata.ReferencedTabletFile; -import org.apache.accumulo.core.metadata.StoredTabletFile; -import org.apache.accumulo.core.metadata.schema.DataFileValue; -import org.apache.accumulo.core.metadata.schema.ExternalCompactionId; -import org.apache.accumulo.core.metadata.schema.TabletMetadata; -import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType; -import org.apache.accumulo.core.metrics.MetricsProducer; -import org.apache.accumulo.core.metrics.MetricsUtil; -import org.apache.accumulo.core.rpc.ThriftUtil; -import org.apache.accumulo.core.rpc.clients.ThriftClientTypes; -import org.apache.accumulo.core.securityImpl.thrift.TCredentials; -import org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction; -import org.apache.accumulo.core.tabletserver.thrift.TCompactionKind; -import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats; -import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob; -import org.apache.accumulo.core.trace.TraceUtil; -import org.apache.accumulo.core.util.Halt; -import org.apache.accumulo.core.util.UtilWaitThread; -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.server.AbstractServer; -import org.apache.accumulo.server.compaction.CompactionConfigStorage; -import org.apache.accumulo.server.compaction.CompactionInfo; -import org.apache.accumulo.server.compaction.CompactionWatcher; -import org.apache.accumulo.server.compaction.FileCompactor; -import org.apache.accumulo.server.compaction.PausedCompactionMetrics; -import org.apache.accumulo.server.compaction.RetryableThriftCall; -import org.apache.accumulo.server.compaction.RetryableThriftCall.RetriesExceededException; -import org.apache.accumulo.server.conf.TableConfiguration; -import org.apache.accumulo.server.fs.VolumeManager; -import org.apache.accumulo.server.rpc.ServerAddress; -import org.apache.accumulo.server.rpc.TServerUtils; -import org.apache.accumulo.server.rpc.ThriftProcessorTypes; -import org.apache.accumulo.server.security.SecurityOperation; -import org.apache.hadoop.fs.Path; -import org.apache.thrift.TException; -import org.apache.thrift.transport.TTransportException; -import org.apache.zookeeper.KeeperException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.google.common.base.Preconditions; -import com.google.common.net.HostAndPort; - -import io.micrometer.core.instrument.LongTaskTimer; -import io.micrometer.core.instrument.MeterRegistry; - -public class Compactor extends AbstractServer implements MetricsProducer, CompactorService.Iface { - - private static final Logger LOG = LoggerFactory.getLogger(Compactor.class); - private static final long TIME_BETWEEN_CANCEL_CHECKS = MINUTES.toMillis(5); - - private static final long TEN_MEGABYTES = 10485760; - - protected static final CompactionJobHolder JOB_HOLDER = new CompactionJobHolder(); - - private final UUID compactorId = UUID.randomUUID(); - private final AccumuloConfiguration aconf; - protected final AtomicReference currentCompactionId = - new AtomicReference<>(); - private final CompactionWatcher watcher; - - private SecurityOperation security; - private ServiceLock compactorLock; - private ServerAddress compactorAddress = null; - private PausedCompactionMetrics pausedMetrics; - - // Exposed for tests - protected volatile boolean shutdown = false; - - private final AtomicBoolean compactionRunning = new AtomicBoolean(false); - - protected Compactor(ConfigOpts opts, String[] args) { - this(opts, args, null); - } - - protected Compactor(ConfigOpts opts, String[] args, AccumuloConfiguration conf) { - super("compactor", opts, args); - aconf = conf == null ? super.getConfiguration() : conf; - setupSecurity(); - watcher = new CompactionWatcher(aconf); - var schedExecutor = - ThreadPools.getServerThreadPools().createGeneralScheduledExecutorService(aconf); - startCancelChecker(schedExecutor, TIME_BETWEEN_CANCEL_CHECKS); - printStartupMsg(); - } - - @Override - protected String getResourceGroupPropertyValue(SiteConfiguration conf) { - return conf.get(Property.COMPACTOR_GROUP_NAME); - } - - @Override - public AccumuloConfiguration getConfiguration() { - return aconf; - } - - @Override - public void registerMetrics(MeterRegistry registry) { - super.registerMetrics(registry); - LongTaskTimer timer = LongTaskTimer.builder(METRICS_COMPACTOR_MAJC_STUCK) - .description("Number and duration of stuck major compactions").register(registry); - CompactionWatcher.setTimer(timer); - } - - protected void setupSecurity() { - security = getContext().getSecurityOperation(); - } - - protected void startCancelChecker(ScheduledThreadPoolExecutor schedExecutor, - long timeBetweenChecks) { - ThreadPools.watchCriticalScheduledTask(schedExecutor.scheduleWithFixedDelay( - this::checkIfCanceled, 0, timeBetweenChecks, TimeUnit.MILLISECONDS)); - } - - protected void checkIfCanceled() { - TExternalCompactionJob job = JOB_HOLDER.getJob(); - if (job != null) { - try { - var extent = KeyExtent.fromThrift(job.getExtent()); - var ecid = ExternalCompactionId.of(job.getExternalCompactionId()); - - TabletMetadata tabletMeta = - getContext().getAmple().readTablet(extent, ColumnType.ECOMP, ColumnType.PREV_ROW); - if (tabletMeta == null || !tabletMeta.getExternalCompactions().containsKey(ecid)) { - // table was deleted OR tablet was split or merged OR tablet no longer thinks compaction - // is running for some reason - LOG.info("Cancelling compaction {} that no longer has a metadata entry at {}", ecid, - extent); - JOB_HOLDER.cancel(job.getExternalCompactionId()); - return; - } - - if (job.getKind() == TCompactionKind.USER) { - - var cconf = CompactionConfigStorage.getConfig(getContext(), job.getFateTxId()); - - if (cconf == null) { - LOG.info("Cancelling compaction {} for user compaction that no longer exists {} {}", - ecid, FateTxId.formatTid(job.getFateTxId()), extent); - JOB_HOLDER.cancel(job.getExternalCompactionId()); - } - } - } catch (RuntimeException | KeeperException e) { - LOG.warn("Failed to check if compaction {} for {} was canceled.", - job.getExternalCompactionId(), KeyExtent.fromThrift(job.getExtent()), e); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException(e); - } - } - } - - protected void printStartupMsg() { - LOG.info("Version " + Constants.VERSION); - LOG.info("Instance " + getContext().getInstanceID()); - } - - /** - * Set up nodes and locks in ZooKeeper for this Compactor - * - * @param clientAddress address of this Compactor - * @throws KeeperException zookeeper error - * @throws InterruptedException thread interrupted - */ - protected void announceExistence(HostAndPort clientAddress) - throws KeeperException, InterruptedException { - - String hostPort = ExternalCompactionUtil.getHostPortString(clientAddress); - - ZooReaderWriter zoo = getContext().getZooReaderWriter(); - String compactorQueuePath = - getContext().getZooKeeperRoot() + Constants.ZCOMPACTORS + "/" + this.getResourceGroup(); - String zPath = compactorQueuePath + "/" + hostPort; - - try { - zoo.mkdirs(compactorQueuePath); - zoo.putPersistentData(zPath, new byte[] {}, NodeExistsPolicy.SKIP); - } catch (KeeperException e) { - if (e.code() == KeeperException.Code.NOAUTH) { - LOG.error("Failed to write to ZooKeeper. Ensure that" - + " accumulo.properties, specifically instance.secret, is consistent."); - } - throw e; - } - - compactorLock = new ServiceLock(getContext().getZooReaderWriter().getZooKeeper(), - ServiceLock.path(zPath), compactorId); - LockWatcher lw = new LockWatcher() { - @Override - public void lostLock(final LockLossReason reason) { - Halt.halt(1, () -> { - LOG.error("Compactor lost lock (reason = {}), exiting.", reason); - getContext().getLowMemoryDetector().logGCInfo(getConfiguration()); - }); - } - - @Override - public void unableToMonitorLockNode(final Exception e) { - Halt.halt(1, () -> LOG.error("Lost ability to monitor Compactor lock, exiting.", e)); - } - }; - - try { - for (int i = 0; i < 25; i++) { - zoo.putPersistentData(zPath, new byte[0], NodeExistsPolicy.SKIP); - - if (compactorLock.tryLock(lw, new ServiceLockData(compactorId, hostPort, - ThriftService.COMPACTOR, this.getResourceGroup()))) { - LOG.debug("Obtained Compactor lock {}", compactorLock.getLockPath()); - return; - } - LOG.info("Waiting for Compactor 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 tablet server lock, exiting.", e); - throw new RuntimeException(e); - } - } - - /** - * Start this Compactors thrift service to handle incoming client requests - * - * @return address of this compactor client service - * @throws UnknownHostException host unknown - */ - protected ServerAddress startCompactorClientService() throws UnknownHostException { - var processor = ThriftProcessorTypes.getCompactorTProcessor(this, getContext()); - Property maxMessageSizeProperty = - (getConfiguration().get(Property.COMPACTOR_MAX_MESSAGE_SIZE) != null - ? Property.COMPACTOR_MAX_MESSAGE_SIZE : Property.GENERAL_MAX_MESSAGE_SIZE); - ServerAddress sp = TServerUtils.startServer(getContext(), getHostname(), - Property.COMPACTOR_CLIENTPORT, processor, this.getClass().getSimpleName(), - "Thrift Client Server", Property.COMPACTOR_PORTSEARCH, Property.COMPACTOR_MINTHREADS, - Property.COMPACTOR_MINTHREADS_TIMEOUT, Property.COMPACTOR_THREADCHECK, - maxMessageSizeProperty); - LOG.info("address = {}", sp.address); - return sp; - } - - /** - * Cancel the compaction with this id. - * - * @param externalCompactionId compaction id - * @throws UnknownCompactionIdException if the externalCompactionId does not match the currently - * executing compaction - * @throws TException thrift error - */ - private void cancel(String externalCompactionId) throws TException { - if (JOB_HOLDER.cancel(externalCompactionId)) { - LOG.info("Cancel requested for compaction job {}", externalCompactionId); - } else { - throw new UnknownCompactionIdException(); - } - } - - @Override - public void cancel(TInfo tinfo, TCredentials credentials, String externalCompactionId) - throws TException { - TableId tableId = JOB_HOLDER.getTableId(); - try { - NamespaceId nsId = getContext().getNamespaceId(tableId); - if (!security.canCompact(credentials, tableId, nsId)) { - throw new AccumuloSecurityException(credentials.getPrincipal(), - SecurityErrorCode.PERMISSION_DENIED).asThriftException(); - } - } catch (TableNotFoundException e) { - throw new ThriftTableOperationException(tableId.canonical(), null, - TableOperation.COMPACT_CANCEL, TableOperationExceptionType.NOTFOUND, e.getMessage()); - } - - cancel(externalCompactionId); - } - - /** - * Send an update to the CompactionCoordinator for this job - * - * @param job compactionJob - * @param update status update - * @throws RetriesExceededException thrown when retries have been exceeded - */ - protected void updateCompactionState(TExternalCompactionJob job, TCompactionStatusUpdate update) - throws RetriesExceededException { - RetryableThriftCall thriftCall = - new RetryableThriftCall<>(1000, RetryableThriftCall.MAX_WAIT_TIME, 25, () -> { - Client coordinatorClient = getCoordinatorClient(); - try { - coordinatorClient.updateCompactionStatus(TraceUtil.traceInfo(), getContext().rpcCreds(), - job.getExternalCompactionId(), update, System.currentTimeMillis()); - return ""; - } finally { - ThriftUtil.returnClient(coordinatorClient, getContext()); - } - }); - thriftCall.run(); - } - - /** - * Notify the CompactionCoordinator the job failed - * - * @param job current compaction job - * @throws RetriesExceededException thrown when retries have been exceeded - */ - protected void updateCompactionFailed(TExternalCompactionJob job) - throws RetriesExceededException { - RetryableThriftCall thriftCall = - new RetryableThriftCall<>(1000, RetryableThriftCall.MAX_WAIT_TIME, 25, () -> { - Client coordinatorClient = getCoordinatorClient(); - try { - coordinatorClient.compactionFailed(TraceUtil.traceInfo(), getContext().rpcCreds(), - job.getExternalCompactionId(), job.extent); - return ""; - } finally { - ThriftUtil.returnClient(coordinatorClient, getContext()); - } - }); - thriftCall.run(); - } - - /** - * Update the CompactionCoordinator with the stats from the completed job - * - * @param job current compaction job - * @param stats compaction stats - * @throws RetriesExceededException thrown when retries have been exceeded - */ - protected void updateCompactionCompleted(TExternalCompactionJob job, TCompactionStats stats) - throws RetriesExceededException { - RetryableThriftCall thriftCall = - new RetryableThriftCall<>(1000, RetryableThriftCall.MAX_WAIT_TIME, 25, () -> { - Client coordinatorClient = getCoordinatorClient(); - try { - coordinatorClient.compactionCompleted(TraceUtil.traceInfo(), getContext().rpcCreds(), - job.getExternalCompactionId(), job.extent, stats); - return ""; - } finally { - ThriftUtil.returnClient(coordinatorClient, getContext()); - } - }); - thriftCall.run(); - } - - /** - * Get the next job to run - * - * @param uuid uuid supplier - * @return CompactionJob - * @throws RetriesExceededException thrown when retries have been exceeded - */ - protected TExternalCompactionJob getNextJob(Supplier uuid) throws RetriesExceededException { - final long startingWaitTime = - getConfiguration().getTimeInMillis(Property.COMPACTOR_MIN_JOB_WAIT_TIME); - final long maxWaitTime = - getConfiguration().getTimeInMillis(Property.COMPACTOR_MAX_JOB_WAIT_TIME); - - RetryableThriftCall nextJobThriftCall = - new RetryableThriftCall<>(startingWaitTime, maxWaitTime, 0, () -> { - Client coordinatorClient = getCoordinatorClient(); - try { - ExternalCompactionId eci = ExternalCompactionId.generate(uuid.get()); - LOG.trace("Attempting to get next job, eci = {}", eci); - currentCompactionId.set(eci); - return coordinatorClient.getCompactionJob(TraceUtil.traceInfo(), - getContext().rpcCreds(), this.getResourceGroup(), - ExternalCompactionUtil.getHostPortString(compactorAddress.getAddress()), - eci.toString()); - } catch (Exception e) { - currentCompactionId.set(null); - throw e; - } finally { - ThriftUtil.returnClient(coordinatorClient, getContext()); - } - }); - return nextJobThriftCall.run(); - } - - /** - * Get the client to the CompactionCoordinator - * - * @return compaction coordinator client - * @throws TTransportException when unable to get client - */ - protected CompactionCoordinatorService.Client getCoordinatorClient() throws TTransportException { - var coordinatorHost = ExternalCompactionUtil.findCompactionCoordinator(getContext()); - if (coordinatorHost.isEmpty()) { - throw new TTransportException("Unable to get CompactionCoordinator address from ZooKeeper"); - } - LOG.trace("CompactionCoordinator address is: {}", coordinatorHost.orElseThrow()); - return ThriftUtil.getClient(ThriftClientTypes.COORDINATOR, coordinatorHost.orElseThrow(), - getContext()); - } - - /** - * Create compaction runnable - * - * @param job compaction job - * @param totalInputEntries object to capture total entries - * @param totalInputBytes object to capture input file size - * @param started started latch - * @param stopped stopped latch - * @param err reference to error - * @return Runnable compaction job - */ - protected Runnable createCompactionJob(final TExternalCompactionJob job, - final LongAdder totalInputEntries, final LongAdder totalInputBytes, - final CountDownLatch started, final CountDownLatch stopped, - final AtomicReference err) { - - return () -> { - // Its only expected that a single compaction runs at a time. Multiple compactions running - // at a time could cause odd behavior like out of order and unexpected thrift calls to the - // coordinator. This is a sanity check to ensure the expectation is met. Should this check - // ever fail, it means there is a bug elsewhere. - Preconditions.checkState(compactionRunning.compareAndSet(false, true)); - try { - LOG.info("Starting up compaction runnable for job: {}", job); - TCompactionStatusUpdate update = - new TCompactionStatusUpdate(TCompactionState.STARTED, "Compaction started", -1, -1, -1); - updateCompactionState(job, update); - var extent = KeyExtent.fromThrift(job.getExtent()); - final AccumuloConfiguration aConfig; - final TableConfiguration tConfig = getContext().getTableConfiguration(extent.tableId()); - - if (!job.getOverrides().isEmpty()) { - aConfig = new ConfigurationCopy(tConfig); - job.getOverrides().forEach(((ConfigurationCopy) aConfig)::set); - LOG.debug("Overriding table properties with {}", job.getOverrides()); - } else { - aConfig = tConfig; - } - - final ReferencedTabletFile outputFile = - new ReferencedTabletFile(new Path(job.getOutputFile())); - - final Map files = new TreeMap<>(); - job.getFiles().forEach(f -> { - files.put(new StoredTabletFile(f.getMetadataFileEntry()), - new DataFileValue(f.getSize(), f.getEntries(), f.getTimestamp())); - totalInputEntries.add(f.getEntries()); - totalInputBytes.add(f.getSize()); - }); - - final List iters = new ArrayList<>(); - job.getIteratorSettings().getIterators() - .forEach(tis -> iters.add(SystemIteratorUtil.toIteratorSetting(tis))); - - ExtCEnv cenv = new ExtCEnv(JOB_HOLDER, this.getResourceGroup()); - FileCompactor compactor = - new FileCompactor(getContext(), extent, files, outputFile, job.isPropagateDeletes(), - cenv, iters, aConfig, tConfig.getCryptoService(), pausedMetrics); - - LOG.trace("Starting compactor"); - started.countDown(); - - org.apache.accumulo.server.compaction.CompactionStats stat = compactor.call(); - TCompactionStats cs = new TCompactionStats(); - cs.setEntriesRead(stat.getEntriesRead()); - cs.setEntriesWritten(stat.getEntriesWritten()); - cs.setFileSize(stat.getFileSize()); - JOB_HOLDER.setStats(cs); - - LOG.info("Compaction completed successfully {} ", job.getExternalCompactionId()); - // Update state when completed - TCompactionStatusUpdate update2 = new TCompactionStatusUpdate(TCompactionState.SUCCEEDED, - "Compaction completed successfully", -1, -1, -1); - updateCompactionState(job, update2); - } catch (FileCompactor.CompactionCanceledException cce) { - LOG.debug("Compaction canceled {}", job.getExternalCompactionId()); - } catch (Exception e) { - LOG.error("Compaction failed", e); - err.set(e); - } finally { - stopped.countDown(); - Preconditions.checkState(compactionRunning.compareAndSet(true, false)); - } - }; - } - - /** - * Returns the number of seconds to wait in between progress checks based on input file sizes - * - * @param numBytes number of bytes in input file - * @return number of seconds to wait between progress checks - */ - static long calculateProgressCheckTime(long numBytes) { - return Math.max(1, (numBytes / TEN_MEGABYTES)); - } - - protected Supplier getNextId() { - return UUID::randomUUID; - } - - protected long getWaitTimeBetweenCompactionChecks() { - // get the total number of compactors assigned to this group - int numCompactors = - ExternalCompactionUtil.countCompactors(this.getResourceGroup(), getContext()); - // Aim for around 3 compactors checking in every second - long sleepTime = numCompactors * 1000L / 3; - // Ensure a compactor sleeps at least around a second - sleepTime = Math.max(1000, sleepTime); - // Ensure a compactor sleep not too much more than 5 mins - sleepTime = Math.min(300_000L, sleepTime); - // Add some random jitter to the sleep time, that averages out to sleep time. This will spread - // compactors out evenly over time. - sleepTime = (long) (.9 * sleepTime + sleepTime * .2 * RANDOM.get().nextDouble()); - LOG.trace("Sleeping {}ms based on {} compactors", sleepTime, numCompactors); - return sleepTime; - } - - @Override - public void run() { - - try { - compactorAddress = startCompactorClientService(); - } catch (UnknownHostException e1) { - throw new RuntimeException("Failed to start the compactor client service", e1); - } - final HostAndPort clientAddress = compactorAddress.getAddress(); - - try { - announceExistence(clientAddress); - } catch (KeeperException | InterruptedException e) { - throw new RuntimeException("Error registering compactor in ZooKeeper", e); - } - - try { - MetricsUtil.initializeMetrics(getContext().getConfiguration(), this.applicationName, - clientAddress); - pausedMetrics = new PausedCompactionMetrics(); - MetricsUtil.initializeProducers(this, pausedMetrics); - } catch (ClassNotFoundException | InstantiationException | IllegalAccessException - | IllegalArgumentException | InvocationTargetException | NoSuchMethodException - | SecurityException e1) { - LOG.error("Error initializing metrics, metrics will not be emitted.", e1); - } - - LOG.info("Compactor started, waiting for work"); - try { - - final AtomicReference err = new AtomicReference<>(); - - while (!shutdown) { - currentCompactionId.set(null); - err.set(null); - JOB_HOLDER.reset(); - - TExternalCompactionJob job; - try { - job = getNextJob(getNextId()); - if (!job.isSetExternalCompactionId()) { - LOG.trace("No external compactions in group {}", this.getResourceGroup()); - UtilWaitThread.sleep(getWaitTimeBetweenCompactionChecks()); - continue; - } - if (!job.getExternalCompactionId().equals(currentCompactionId.get().toString())) { - throw new IllegalStateException("Returned eci " + job.getExternalCompactionId() - + " does not match supplied eci " + currentCompactionId.get()); - } - } catch (RetriesExceededException e2) { - LOG.warn("Retries exceeded getting next job. Retrying..."); - continue; - } - LOG.debug("Received next compaction job: {}", job); - - final LongAdder totalInputEntries = new LongAdder(); - final LongAdder totalInputBytes = new LongAdder(); - final CountDownLatch started = new CountDownLatch(1); - final CountDownLatch stopped = new CountDownLatch(1); - - final Thread compactionThread = Threads.createThread( - "Compaction job for tablet " + job.getExtent().toString(), - createCompactionJob(job, totalInputEntries, totalInputBytes, started, stopped, err)); - - JOB_HOLDER.set(job, compactionThread); - - try { - compactionThread.start(); // start the compactionThread - started.await(); // wait until the compactor is started - final long inputEntries = totalInputEntries.sum(); - final long waitTime = calculateProgressCheckTime(totalInputBytes.sum()); - LOG.debug("Progress checks will occur every {} seconds", waitTime); - String percentComplete = "unknown"; - - while (!stopped.await(waitTime, TimeUnit.SECONDS)) { - List running = - org.apache.accumulo.server.compaction.FileCompactor.getRunningCompactions(); - if (!running.isEmpty()) { - // Compaction has started. There should only be one in the list - CompactionInfo info = running.get(0); - if (info != null) { - if (inputEntries > 0) { - percentComplete = - Float.toString((info.getEntriesRead() / (float) inputEntries) * 100); - } - String message = String.format( - "Compaction in progress, read %d of %d input entries ( %s %s ), written %d entries, paused %d times", - info.getEntriesRead(), inputEntries, percentComplete, "%", - info.getEntriesWritten(), info.getTimesPaused()); - watcher.run(); - try { - LOG.debug("Updating coordinator with compaction progress: {}.", message); - TCompactionStatusUpdate update = - new TCompactionStatusUpdate(TCompactionState.IN_PROGRESS, message, - inputEntries, info.getEntriesRead(), info.getEntriesWritten()); - updateCompactionState(job, update); - } catch (RetriesExceededException e) { - LOG.warn("Error updating coordinator with compaction progress, error: {}", - e.getMessage()); - } - } - } else { - LOG.error("Waiting on compaction thread to finish, but no RUNNING compaction"); - } - } - compactionThread.join(); - LOG.trace("Compaction thread finished."); - // Run the watcher again to clear out the finished compaction and set the - // stuck count to zero. - watcher.run(); - - if (err.get() != null) { - // maybe the error occured because the table was deleted or something like that, so - // force a cancel check to possibly reduce noise in the logs - checkIfCanceled(); - } - - if (compactionThread.isInterrupted() || JOB_HOLDER.isCancelled() - || (err.get() != null && err.get().getClass().equals(InterruptedException.class))) { - LOG.warn("Compaction thread was interrupted, sending CANCELLED state"); - try { - TCompactionStatusUpdate update = new TCompactionStatusUpdate( - TCompactionState.CANCELLED, "Compaction cancelled", -1, -1, -1); - updateCompactionState(job, update); - updateCompactionFailed(job); - } catch (RetriesExceededException e) { - LOG.error("Error updating coordinator with compaction cancellation.", e); - } finally { - currentCompactionId.set(null); - } - } else if (err.get() != null) { - try { - LOG.info("Updating coordinator with compaction failure."); - TCompactionStatusUpdate update = new TCompactionStatusUpdate(TCompactionState.FAILED, - "Compaction failed due to: " + err.get().getMessage(), -1, -1, -1); - updateCompactionState(job, update); - updateCompactionFailed(job); - } catch (RetriesExceededException e) { - LOG.error("Error updating coordinator with compaction failure.", e); - } finally { - currentCompactionId.set(null); - } - } else { - try { - LOG.trace("Updating coordinator with compaction completion."); - updateCompactionCompleted(job, JOB_HOLDER.getStats()); - } catch (RetriesExceededException e) { - LOG.error( - "Error updating coordinator with compaction completion, cancelling compaction.", - e); - try { - cancel(job.getExternalCompactionId()); - } catch (TException e1) { - LOG.error("Error cancelling compaction.", e1); - } - } finally { - currentCompactionId.set(null); - } - } - } catch (RuntimeException e1) { - LOG.error( - "Compactor thread was interrupted waiting for compaction to start, cancelling job", - e1); - try { - cancel(job.getExternalCompactionId()); - } catch (TException e2) { - LOG.error("Error cancelling compaction.", e2); - } - } finally { - currentCompactionId.set(null); - // In the case where there is an error in the foreground code the background compaction - // may still be running. Must cancel it before starting another iteration of the loop to - // avoid multiple threads updating shared state. - while (compactionThread.isAlive()) { - compactionThread.interrupt(); - compactionThread.join(1000); - } - } - - } - - } catch (Exception e) { - LOG.error("Unhandled error occurred in Compactor", e); - } finally { - // Shutdown local thrift server - LOG.info("Stopping Thrift Servers"); - if (compactorAddress.server != null) { - compactorAddress.server.stop(); - } - - try { - LOG.debug("Closing filesystems"); - VolumeManager mgr = getContext().getVolumeManager(); - if (null != mgr) { - mgr.close(); - } - } catch (IOException e) { - LOG.warn("Failed to close filesystem : {}", e.getMessage(), e); - } - - getContext().getLowMemoryDetector().logGCInfo(getConfiguration()); - LOG.info("stop requested. exiting ... "); - try { - if (null != compactorLock) { - compactorLock.unlock(); - } - } catch (Exception e) { - LOG.warn("Failed to release compactor lock", e); - } - } - - } - - public static void main(String[] args) throws Exception { - try (Compactor compactor = new Compactor(new ConfigOpts(), args)) { - compactor.runServer(); - } - } - - @Override - public List getActiveCompactions(TInfo tinfo, TCredentials credentials) - throws ThriftSecurityException, TException { - if (!security.canPerformSystemActions(credentials)) { - throw new AccumuloSecurityException(credentials.getPrincipal(), - SecurityErrorCode.PERMISSION_DENIED).asThriftException(); - } - - List compactions = - org.apache.accumulo.server.compaction.FileCompactor.getRunningCompactions(); - List ret = new ArrayList<>(compactions.size()); - - for (CompactionInfo compactionInfo : compactions) { - ret.add(compactionInfo.toThrift()); - } - - return ret; - } - - /** - * Called by a CompactionCoordinator to get the running compaction - * - * @param tinfo trace info - * @param credentials caller credentials - * @return current compaction job or empty compaction job is none running - */ - @Override - public TExternalCompactionJob getRunningCompaction(TInfo tinfo, TCredentials credentials) - throws ThriftSecurityException, TException { - // do not expect users to call this directly, expect other tservers to call this method - if (!security.canPerformSystemActions(credentials)) { - throw new AccumuloSecurityException(credentials.getPrincipal(), - SecurityErrorCode.PERMISSION_DENIED).asThriftException(); - } - - // Return what is currently running, does not wait for jobs in the process of reserving. This - // method is called by a coordinator starting up to determine what is currently running on all - // compactors. - - TExternalCompactionJob job = null; - synchronized (JOB_HOLDER) { - job = JOB_HOLDER.getJob(); - } - - if (null == job) { - return new TExternalCompactionJob(); - } else { - return job; - } - } - - @Override - public String getRunningCompactionId(TInfo tinfo, TCredentials credentials) - throws ThriftSecurityException, TException { - // do not expect users to call this directly, expect other tservers to call this method - if (!security.canPerformSystemActions(credentials)) { - throw new AccumuloSecurityException(credentials.getPrincipal(), - SecurityErrorCode.PERMISSION_DENIED).asThriftException(); - } - - // Any returned id must cover the time period from before a job is reserved until after it - // commits. This method is called to detect dead compactions and depends on this behavior. - // For the purpose of detecting dead compactions its ok if ids are returned that never end up - // being related to a running compaction. - ExternalCompactionId eci = currentCompactionId.get(); - if (null == eci) { - return ""; - } else { - return eci.canonical(); - } - } -} 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 0f539e199f5..31fe3f17a58 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 @@ -115,13 +115,13 @@ import org.apache.accumulo.core.util.Retry; import org.apache.accumulo.core.util.threads.ThreadPools; import org.apache.accumulo.core.util.threads.Threads; -import org.apache.accumulo.manager.compaction.coordinator.CompactionCoordinator; import org.apache.accumulo.manager.compaction.queue.CompactionJobQueues; import org.apache.accumulo.manager.metrics.ManagerMetrics; import org.apache.accumulo.manager.recovery.RecoveryManager; import org.apache.accumulo.manager.split.Splitter; import org.apache.accumulo.manager.state.TableCounts; import org.apache.accumulo.manager.tableOps.TraceRepo; +import org.apache.accumulo.manager.tasks.TaskManager; import org.apache.accumulo.manager.upgrade.PreUpgradeValidation; import org.apache.accumulo.manager.upgrade.UpgradeCoordinator; import org.apache.accumulo.server.AbstractServer; @@ -350,7 +350,7 @@ synchronized void setManagerState(ManagerState newState) { private FateServiceHandler fateServiceHandler; private ManagerClientServiceHandler managerClientHandler; - private CompactionCoordinator compactionCoordinator; + private TaskManager taskManager; private int assignedOrHosted(TableId tableId) { int result = 0; @@ -1171,16 +1171,15 @@ public void run() { // Start the Manager's Fate Service fateServiceHandler = new FateServiceHandler(this); managerClientHandler = new ManagerClientServiceHandler(this); - compactionCoordinator = - new CompactionCoordinator(context, tserverSet, security, compactionJobQueues); + taskManager = new TaskManager(context, tserverSet, security, compactionJobQueues); // Start the Manager's Client service // Ensure that calls before the manager gets the lock fail ManagerClientService.Iface haProxy = HighlyAvailableServiceWrapper.service(managerClientHandler, this); ServerAddress sa; - var processor = ThriftProcessorTypes.getManagerTProcessor(fateServiceHandler, - compactionCoordinator, haProxy, getContext()); + var processor = ThriftProcessorTypes.getManagerTProcessor(fateServiceHandler, taskManager, + haProxy, getContext()); try { sa = TServerUtils.startServer(context, getHostname(), Property.MANAGER_CLIENTPORT, processor, @@ -1238,10 +1237,9 @@ public void run() { Thread.currentThread().interrupt(); } - // Don't call run on the CompactionCoordinator until we have tservers. - Thread compactionCoordinatorThread = - Threads.createThread("CompactionCoordinator Thread", compactionCoordinator); - compactionCoordinatorThread.start(); + // Don't call run on the TaskManager until we have tservers. + Thread taskManagerThread = Threads.createThread("TaskManager Thread", taskManager); + taskManagerThread.start(); ZooReaderWriter zReaderWriter = context.getZooReaderWriter(); @@ -1355,8 +1353,8 @@ boolean canSuspendTablets() { String address = sa.address.toString(); UUID uuid = sld.getServerUUID(ThriftService.MANAGER); ServiceDescriptors descriptors = new ServiceDescriptors(); - for (ThriftService svc : new ThriftService[] {ThriftService.MANAGER, ThriftService.COORDINATOR, - ThriftService.FATE}) { + for (ThriftService svc : new ThriftService[] {ThriftService.MANAGER, ThriftService.FATE, + ThriftService.TASK_MANAGER}) { descriptors.addService(new ServiceDescriptor(uuid, svc, address, this.getResourceGroup())); } @@ -1392,11 +1390,11 @@ boolean canSuspendTablets() { tableInformationStatusPool.shutdownNow(); - compactionCoordinator.shutdown(); + taskManager.shutdown(); try { - compactionCoordinatorThread.join(); + taskManagerThread.join(); } catch (InterruptedException e) { - log.error("Exception compaction coordinator thread", e); + log.error("Exception TaskManager thread", e); } // Signal that we want it to stop, and wait for it to do so. @@ -1619,7 +1617,7 @@ private ServiceLockData getManagerLock(final ServiceLockPath zManagerLoc) public void update(LiveTServerSet current, Set deleted, Set added) { - compactionCoordinator.updateTServerSet(current, deleted, added); + taskManager.updateTServerSet(current, deleted, added); // if we have deleted or added tservers, then adjust our dead server list if (!deleted.isEmpty() || !added.isEmpty()) { diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/DeadCompactionDetector.java b/server/manager/src/main/java/org/apache/accumulo/manager/tasks/DeadCompactionDetector.java similarity index 93% rename from server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/DeadCompactionDetector.java rename to server/manager/src/main/java/org/apache/accumulo/manager/tasks/DeadCompactionDetector.java index 98540e1b93f..7cf15c1ae78 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/DeadCompactionDetector.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/tasks/DeadCompactionDetector.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.accumulo.manager.compaction.coordinator; +package org.apache.accumulo.manager.tasks; import java.util.Collection; import java.util.HashMap; @@ -44,14 +44,14 @@ public class DeadCompactionDetector { private static final Logger log = LoggerFactory.getLogger(DeadCompactionDetector.class); private final ServerContext context; - private final CompactionCoordinator coordinator; + private final TaskManager taskManager; private final ScheduledThreadPoolExecutor schedExecutor; private final ConcurrentHashMap deadCompactions; - public DeadCompactionDetector(ServerContext context, CompactionCoordinator coordinator, + public DeadCompactionDetector(ServerContext context, TaskManager taskManager, ScheduledThreadPoolExecutor stpe) { this.context = context; - this.coordinator = coordinator; + this.taskManager = taskManager; this.schedExecutor = stpe; this.deadCompactions = new ConcurrentHashMap<>(); } @@ -120,13 +120,13 @@ private void detectDeadCompactions() { tabletCompactions.forEach((eci, v) -> { log.warn("Compaction {} believed to be dead, failing it.", eci); }); - coordinator.compactionFailed(tabletCompactions); + taskManager.compactionFailed(tabletCompactions); this.deadCompactions.keySet().removeAll(toFail); } public void start() { long interval = this.context.getConfiguration() - .getTimeInMillis(Property.COMPACTION_COORDINATOR_DEAD_COMPACTOR_CHECK_INTERVAL); + .getTimeInMillis(Property.TASK_MANAGER_DEAD_COMPACTOR_CHECK_INTERVAL); ThreadPools.watchCriticalScheduledTask(schedExecutor.scheduleWithFixedDelay(() -> { try { 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/tasks/TaskManager.java similarity index 81% rename from server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/CompactionCoordinator.java rename to server/manager/src/main/java/org/apache/accumulo/manager/tasks/TaskManager.java index 7f0be24740f..16405dbbb41 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/tasks/TaskManager.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.accumulo.manager.compaction.coordinator; +package org.apache.accumulo.manager.tasks; import static java.util.concurrent.TimeUnit.MILLISECONDS; import static java.util.concurrent.TimeUnit.MINUTES; @@ -65,7 +65,6 @@ import org.apache.accumulo.core.clientImpl.thrift.TableOperationExceptionType; import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException; import org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException; -import org.apache.accumulo.core.compaction.thrift.CompactionCoordinatorService; import org.apache.accumulo.core.compaction.thrift.TCompactionState; import org.apache.accumulo.core.compaction.thrift.TCompactionStatusUpdate; import org.apache.accumulo.core.compaction.thrift.TExternalCompaction; @@ -99,6 +98,17 @@ import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats; import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob; import org.apache.accumulo.core.tabletserver.thrift.TabletServerClientService; +import org.apache.accumulo.core.tasks.TaskMessage; +import org.apache.accumulo.core.tasks.TaskMessageType; +import org.apache.accumulo.core.tasks.compaction.CompactionTask; +import org.apache.accumulo.core.tasks.compaction.CompactionTaskCompleted; +import org.apache.accumulo.core.tasks.compaction.CompactionTaskFailed; +import org.apache.accumulo.core.tasks.compaction.CompactionTaskStatus; +import org.apache.accumulo.core.tasks.compaction.CompactionTasksCompleted; +import org.apache.accumulo.core.tasks.compaction.CompactionTasksRunning; +import org.apache.accumulo.core.tasks.thrift.Task; +import org.apache.accumulo.core.tasks.thrift.TaskRunnerInfo; +import org.apache.accumulo.core.tasks.thrift.WorkerType; import org.apache.accumulo.core.util.Retry; import org.apache.accumulo.core.util.UtilWaitThread; import org.apache.accumulo.core.util.cache.Caches.CacheName; @@ -116,6 +126,7 @@ import org.apache.accumulo.server.tablets.TabletNameGenerator; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +import org.apache.thrift.TApplicationException; import org.apache.thrift.TException; import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; @@ -131,9 +142,10 @@ import com.google.common.net.HostAndPort; import com.google.common.util.concurrent.MoreExecutors; -public class CompactionCoordinator implements CompactionCoordinatorService.Iface, Runnable { +public class TaskManager + implements org.apache.accumulo.core.tasks.thrift.TaskManager.Iface, Runnable { - private static final Logger LOG = LoggerFactory.getLogger(CompactionCoordinator.class); + private static final Logger LOG = LoggerFactory.getLogger(TaskManager.class); private static final long FIFTEEN_MINUTES = TimeUnit.MINUTES.toMillis(15); /* @@ -170,8 +182,8 @@ public class CompactionCoordinator implements CompactionCoordinatorService.Iface private LoadingCache compactionConfigCache; private final Cache checked_tablet_dir_cache; - public CompactionCoordinator(ServerContext ctx, LiveTServerSet tservers, - SecurityOperation security, CompactionJobQueues jobQueues) { + public TaskManager(ServerContext ctx, LiveTServerSet tservers, SecurityOperation security, + CompactionJobQueues jobQueues) { this.ctx = ctx; this.tserverSet = tservers; this.schedExecutor = this.ctx.getScheduledExecutor(); @@ -253,7 +265,7 @@ private void processRefreshes(Ample.DataLevel dataLevel) { } }); - String logId = "Coordinator:" + dataLevel; + String logId = "TaskManager:" + dataLevel; ThreadPoolExecutor threadPool = ctx.threadPools().createFixedThreadPool(10, "Tablet refresh " + logId, false); try { @@ -280,7 +292,7 @@ public void run() { startCompactionCleaner(schedExecutor); startRunningCleaner(schedExecutor); - // On a re-start of the coordinator it's possible that external compactions are in-progress. + // On a re-start of the TaskManager it's possible that external compactions are in-progress. // Attempt to get the running compactions on the compactors and then resolve which tserver // the external compaction came from to re-populate the RUNNING collection. LOG.info("Checking for running external compactions"); @@ -293,7 +305,7 @@ public void run() { running.forEach(rc -> { TCompactionStatusUpdate update = new TCompactionStatusUpdate(); update.setState(TCompactionState.IN_PROGRESS); - update.setMessage("Coordinator restarted, compaction found in progress"); + update.setMessage("TaskManager restarted, compaction found in progress"); rc.addUpdate(System.currentTimeMillis(), update); RUNNING_CACHE.put(ExternalCompactionId.of(rc.getJob().getExternalCompactionId()), rc); }); @@ -314,7 +326,7 @@ public void run() { if ((now - v) > getMissingCompactorWarningTime()) { // ELASTICITY_TODO may want to consider of the group has any jobs queued OR if the group // still exist in configuration - LOG.warn("No compactors have checked in with coordinator for group {} in {}ms", k, + LOG.warn("No compactors have checked in with TaskManager for group {} in {}ms", k, getMissingCompactorWarningTime()); } }); @@ -340,7 +352,7 @@ protected long getMissingCompactorWarningTime() { protected long getTServerCheckInterval() { return this.ctx.getConfiguration() - .getTimeInMillis(Property.COMPACTION_COORDINATOR_TSERVER_COMPACTION_CHECK_INTERVAL); + .getTimeInMillis(Property.TASK_MANAGER_TSERVER_COMPACTION_CHECK_INTERVAL); } /** @@ -356,77 +368,91 @@ public void updateTServerSet(LiveTServerSet current, Set delete } - /** - * Return the next compaction job from the queue to a Compactor - * - * @param groupName group - * @param compactorAddress compactor address - * @throws ThriftSecurityException when permission error - * @return compaction job - */ @Override - public TExternalCompactionJob getCompactionJob(TInfo tinfo, TCredentials credentials, - String groupName, String compactorAddress, String externalCompactionId) - throws ThriftSecurityException { + public Task getTask(TInfo tinfo, TCredentials credentials, TaskRunnerInfo taskRunner, + String taskID) throws TException { // do not expect users to call this directly, expect compactors to call this method if (!security.canPerformSystemActions(credentials)) { throw new AccumuloSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED).asThriftException(); } - final String group = groupName.intern(); - LOG.trace("getCompactionJob called for group {} by compactor {}", group, compactorAddress); - TIME_COMPACTOR_LAST_CHECKED.put(group, System.currentTimeMillis()); - TExternalCompactionJob result = null; + final String taskRunnerAddress = taskRunner.getHostname() + ":" + taskRunner.getPort(); + final WorkerType workerType = taskRunner.getWorkerType(); + final String group = taskRunner.getResourceGroup().intern(); - CompactionJobQueues.MetaJob metaJob = - jobQueues.poll(CompactionExecutorIdImpl.externalId(groupName)); + switch (workerType) { + case COMPACTION: + LOG.trace("getTask called for group {} by compactor {}", group, taskRunnerAddress); + TIME_COMPACTOR_LAST_CHECKED.put(group, System.currentTimeMillis()); - while (metaJob != null) { + TExternalCompactionJob result = null; - Optional compactionConfig = getCompactionConfig(metaJob); + CompactionJobQueues.MetaJob metaJob = + jobQueues.poll(CompactionExecutorIdImpl.externalId(group)); - // this method may reread the metadata, do not use the metadata in metaJob for anything after - // this method - ExternalCompactionMetadata ecm = null; + while (metaJob != null) { - var kind = metaJob.getJob().getKind(); + Optional compactionConfig = getCompactionConfig(metaJob); - // Only reserve user compactions when the config is present. When compactions are canceled the - // config is deleted. - if (kind == CompactionKind.SYSTEM - || (kind == CompactionKind.USER && compactionConfig.isPresent())) { - ecm = reserveCompaction(metaJob, compactorAddress, externalCompactionId); - } + // this method may reread the metadata, do not use the metadata in metaJob for anything + // after this method + ExternalCompactionMetadata ecm = null; - if (ecm != null) { - result = createThriftJob(externalCompactionId, ecm, metaJob, compactionConfig); - // It is possible that by the time this added that the the compactor that made this request - // is dead. In this cases the compaction is not actually running. - RUNNING_CACHE.put(ExternalCompactionId.of(result.getExternalCompactionId()), - new RunningCompaction(result, compactorAddress, group)); - LOG.debug("Returning external job {} to {} with {} files", result.externalCompactionId, - compactorAddress, ecm.getJobFiles().size()); - break; - } else { - LOG.debug("Unable to reserve compaction job for {}, pulling another off the queue ", - metaJob.getTabletMetadata().getExtent()); - metaJob = jobQueues.poll(CompactionExecutorIdImpl.externalId(groupName)); - } - } + var kind = metaJob.getJob().getKind(); - if (metaJob == null) { - LOG.debug("No jobs found in group {} ", group); - } + // Only reserve user compactions when the config is present. When compactions are canceled + // the config is deleted. + if (kind == CompactionKind.SYSTEM + || (kind == CompactionKind.USER && compactionConfig.isPresent())) { + ecm = reserveCompaction(metaJob, taskRunnerAddress, taskID); + } - if (result == null) { - LOG.trace("No jobs found for group {}, returning empty job to compactor {}", group, - compactorAddress); - result = new TExternalCompactionJob(); - } + if (ecm != null) { + result = createThriftJob(taskID, ecm, metaJob, compactionConfig); + // It is possible that by the time this added that the the compactor that made this + // request is dead. In this cases the compaction is not actually running. + RUNNING_CACHE.put(ExternalCompactionId.of(result.getExternalCompactionId()), + new RunningCompaction(result, taskRunnerAddress, group)); + LOG.debug("Returning external job {} to {} with {} files", result.externalCompactionId, + taskRunnerAddress, ecm.getJobFiles().size()); + break; + } else { + LOG.debug("Unable to reserve compaction job for {}, pulling another off the queue ", + metaJob.getTabletMetadata().getExtent()); + metaJob = jobQueues.poll(CompactionExecutorIdImpl.externalId(group)); + } + } - return result; + if (metaJob == null) { + LOG.debug("No jobs found in group {} ", group); + } + + CompactionTask task = TaskMessageType.COMPACTION_TASK.getTaskMessage(); + task.setTaskId(taskID); + + if (result == null) { + LOG.trace("No compaction jobs found for group {}, returning empty job to compactor {}", + group, taskRunnerAddress); + task.setCompactionJob(new TExternalCompactionJob()); + } else { + task.setCompactionJob(result); + } + + return task.toThriftTask(); + case LOG_SORTING: + // ELASTICITY_TODO + // ResourceGroup is ignored for log sorting. Find the next log sorting task and return it + return null; + case SPLIT_POINT_CALCULATION: + // ELASTICITY_TODO + // ResourceGroup could be ignored for split point calculation. Find the next split point + // calculation task and return it + return null; + default: + return null; + } } @@ -724,70 +750,84 @@ private Optional getCompactionConfig(CompactionJobQueues.MetaJ * * @param tinfo trace info * @param credentials tcredentials object - * @param externalCompactionId compaction id - * @param textent tablet extent - * @param stats compaction stats + * @param task object * @throws ThriftSecurityException when permission error */ @Override - public void compactionCompleted(TInfo tinfo, TCredentials credentials, - String externalCompactionId, TKeyExtent textent, TCompactionStats stats) - throws ThriftSecurityException { + public void taskCompleted(TInfo tinfo, TCredentials credentials, Task task) throws TException { // do not expect users to call this directly, expect other tservers to call this method if (!security.canPerformSystemActions(credentials)) { throw new AccumuloSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED).asThriftException(); } + TaskMessageType messageType = TaskMessageType.valueOf(task.getMessageType()); + switch (messageType) { + case COMPACTION_TASK_COMPLETED: + CompactionTaskCompleted compactionTask = TaskMessage.fromThiftTask(task, messageType); + final TExternalCompactionJob job = compactionTask.getCompactionJob(); + final String externalCompactionId = job.getExternalCompactionId(); + final TCompactionStats stats = compactionTask.getCompactionStats(); + + var extent = KeyExtent.fromThrift(job.getExtent()); + LOG.info("Compaction completed, id: {}, stats: {}, extent: {}", externalCompactionId, stats, + extent); + final var ecid = ExternalCompactionId.of(externalCompactionId); + + var tabletMeta = + ctx.getAmple().readTablet(extent, ECOMP, SELECTED, LOCATION, FILES, COMPACTED, OPID); + + if (!canCommitCompaction(ecid, tabletMeta)) { + return; + } - var extent = KeyExtent.fromThrift(textent); - LOG.info("Compaction completed, id: {}, stats: {}, extent: {}", externalCompactionId, stats, - extent); - final var ecid = ExternalCompactionId.of(externalCompactionId); - - var tabletMeta = - ctx.getAmple().readTablet(extent, ECOMP, SELECTED, LOCATION, FILES, COMPACTED, OPID); + ExternalCompactionMetadata ecm = tabletMeta.getExternalCompactions().get(ecid); - if (!canCommitCompaction(ecid, tabletMeta)) { - return; - } + // ELASTICITY_TODO this code does not handle race conditions or faults. Need to ensure + // refresh + // happens in the case of manager process death between commit and refresh. + ReferencedTabletFile newDatafile = + TabletNameGenerator.computeCompactionFileDest(ecm.getCompactTmpName()); - ExternalCompactionMetadata ecm = tabletMeta.getExternalCompactions().get(ecid); + Optional optionalNewFile; + try { + optionalNewFile = renameOrDeleteFile(stats, ecm, newDatafile); + } catch (IOException e) { + LOG.warn("Can not commit complete compaction {} because unable to delete or rename {} ", + ecid, ecm.getCompactTmpName(), e); + compactionFailed(Map.of(ecid, extent)); + return; + } - // ELASTICITY_TODO this code does not handle race conditions or faults. Need to ensure refresh - // happens in the case of manager process death between commit and refresh. - ReferencedTabletFile newDatafile = - TabletNameGenerator.computeCompactionFileDest(ecm.getCompactTmpName()); + RefreshWriter refreshWriter = new RefreshWriter(ecid, extent); - Optional optionalNewFile; - try { - optionalNewFile = renameOrDeleteFile(stats, ecm, newDatafile); - } catch (IOException e) { - LOG.warn("Can not commit complete compaction {} because unable to delete or rename {} ", ecid, - ecm.getCompactTmpName(), e); - compactionFailed(Map.of(ecid, extent)); - return; - } + try { + tabletMeta = commitCompaction(stats, ecid, tabletMeta, optionalNewFile, refreshWriter); + } catch (RuntimeException e) { + LOG.warn("Failed to commit complete compaction {} {}", ecid, extent, e); + compactionFailed(Map.of(ecid, extent)); + } - RefreshWriter refreshWriter = new RefreshWriter(ecid, extent); + if (ecm.getKind() != CompactionKind.USER) { + refreshTablet(tabletMeta); + } - try { - tabletMeta = commitCompaction(stats, ecid, tabletMeta, optionalNewFile, refreshWriter); - } catch (RuntimeException e) { - LOG.warn("Failed to commit complete compaction {} {}", ecid, extent, e); - compactionFailed(Map.of(ecid, extent)); - } + // if a refresh entry was written, it can be removed after the tablet was refreshed + refreshWriter.deleteRefresh(); - if (ecm.getKind() != CompactionKind.USER) { - refreshTablet(tabletMeta); + // It's possible that RUNNING might not have an entry for this ecid in the case + // of a TaskManager restart when the TaskManager can't find the TServer for the + // corresponding external compaction. + recordCompletion(ecid); + break; + case COMPACTION_TASKS_COMPLETED: + case COMPACTION_TASK: + case COMPACTION_TASKS_RUNNING: + case COMPACTION_TASK_FAILED: + case COMPACTION_TASK_LIST: + case COMPACTION_TASK_STATUS: + default: + throw new TApplicationException(TApplicationException.INVALID_MESSAGE_TYPE); } - - // if a refresh entry was written, it can be removed after the tablet was refreshed - refreshWriter.deleteRefresh(); - - // It's possible that RUNNING might not have an entry for this ecid in the case - // of a coordinator restart when the Coordinator can't find the TServer for the - // corresponding external compaction. - recordCompletion(ecid); } private Optional renameOrDeleteFile(TCompactionStats stats, @@ -1020,20 +1060,37 @@ private void updateTabletForCompaction(TCompactionStats stats, ExternalCompactio } @Override - public void compactionFailed(TInfo tinfo, TCredentials credentials, String externalCompactionId, - TKeyExtent extent) throws ThriftSecurityException { + public void taskFailed(TInfo tinfo, TCredentials credentials, Task task) throws TException { // do not expect users to call this directly, expect other tservers to call this method if (!security.canPerformSystemActions(credentials)) { throw new AccumuloSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED).asThriftException(); } - LOG.info("Compaction failed, id: {}", externalCompactionId); - final var ecid = ExternalCompactionId.of(externalCompactionId); - compactionFailed(Map.of(ecid, KeyExtent.fromThrift(extent))); - // ELASTICITIY_TODO need to open an issue about making the GC clean up tmp files. The tablet - // currently cleans up tmp files on tablet load. With tablets never loading possibly but still - // compacting dying compactors may still leave tmp files behind. + TaskMessageType messageType = TaskMessageType.valueOf(task.getMessageType()); + switch (messageType) { + case COMPACTION_TASK_FAILED: + final CompactionTaskFailed compactionTask = TaskMessage.fromThiftTask(task, messageType); + final TExternalCompactionJob job = compactionTask.getCompactionJob(); + + LOG.info("Compaction failed, id: {}", job.getExternalCompactionId()); + final var ecid = ExternalCompactionId.of(job.getExternalCompactionId()); + compactionFailed(Map.of(ecid, KeyExtent.fromThrift(job.getExtent()))); + + // ELASTICITIY_TODO need to open an issue about making the GC clean up tmp files. The tablet + // currently cleans up tmp files on tablet load. With tablets never loading possibly but + // still + // compacting dying compactors may still leave tmp files behind. + break; + case COMPACTION_TASK: + case COMPACTION_TASKS_COMPLETED: + case COMPACTION_TASKS_RUNNING: + case COMPACTION_TASK_COMPLETED: + case COMPACTION_TASK_LIST: + case COMPACTION_TASK_STATUS: + default: + throw new TApplicationException(TApplicationException.INVALID_MESSAGE_TYPE); + } } void compactionFailed(Map compactions) { @@ -1068,30 +1125,38 @@ void compactionFailed(Map compactions) { compactions.forEach((k, v) -> recordCompletion(k)); } - /** - * Compactor calls to update the status of the assigned compaction - * - * @param tinfo trace info - * @param credentials tcredentials object - * @param externalCompactionId compaction id - * @param update compaction status update - * @param timestamp timestamp of the message - * @throws ThriftSecurityException when permission error - */ @Override - public void updateCompactionStatus(TInfo tinfo, TCredentials credentials, - String externalCompactionId, TCompactionStatusUpdate update, long timestamp) - throws ThriftSecurityException { + public void taskStatus(TInfo tinfo, TCredentials credentials, long timestamp, + Task taskUpdateObject) throws TException { // do not expect users to call this directly, expect other tservers to call this method if (!security.canPerformSystemActions(credentials)) { throw new AccumuloSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED).asThriftException(); } - LOG.debug("Compaction status update, id: {}, timestamp: {}, update: {}", externalCompactionId, - timestamp, update); - final RunningCompaction rc = RUNNING_CACHE.get(ExternalCompactionId.of(externalCompactionId)); - if (null != rc) { - rc.addUpdate(timestamp, update); + TaskMessageType messageType = TaskMessageType.valueOf(taskUpdateObject.getMessageType()); + switch (messageType) { + case COMPACTION_TASK_STATUS: + final CompactionTaskStatus statusMsg = + TaskMessage.fromThiftTask(taskUpdateObject, messageType); + final TCompactionStatusUpdate update = statusMsg.getCompactionStatus(); + final String externalCompactionId = statusMsg.getTaskId(); + + LOG.debug("Compaction status update, id: {}, timestamp: {}, update: {}", + externalCompactionId, timestamp, update); + final RunningCompaction rc = + RUNNING_CACHE.get(ExternalCompactionId.of(externalCompactionId)); + if (null != rc) { + rc.addUpdate(timestamp, update); + } + break; + case COMPACTION_TASK: + case COMPACTION_TASKS_COMPLETED: + case COMPACTION_TASKS_RUNNING: + case COMPACTION_TASK_COMPLETED: + case COMPACTION_TASK_FAILED: + case COMPACTION_TASK_LIST: + default: + throw new TApplicationException(TApplicationException.INVALID_MESSAGE_TYPE); } } @@ -1141,8 +1206,8 @@ protected void cleanUpRunning() { * @throws ThriftSecurityException permission error */ @Override - public TExternalCompactionList getRunningCompactions(TInfo tinfo, TCredentials credentials) - throws ThriftSecurityException { + public Task getRunningTasks(TInfo tinfo, TCredentials credentials) + throws ThriftSecurityException, TException { // do not expect users to call this directly, expect other tservers to call this method if (!security.canPerformSystemActions(credentials)) { throw new AccumuloSecurityException(credentials.getPrincipal(), @@ -1158,7 +1223,11 @@ public TExternalCompactionList getRunningCompactions(TInfo tinfo, TCredentials c trc.setJob(rc.getJob()); result.putToCompactions(ecid.canonical(), trc); }); - return result; + + CompactionTasksRunning running = TaskMessageType.COMPACTION_TASKS_RUNNING.getTaskMessage(); + running.setRunning(result); + return running.toThriftTask(); + } /** @@ -1170,8 +1239,8 @@ public TExternalCompactionList getRunningCompactions(TInfo tinfo, TCredentials c * @throws ThriftSecurityException permission error */ @Override - public TExternalCompactionList getCompletedCompactions(TInfo tinfo, TCredentials credentials) - throws ThriftSecurityException { + public Task getCompletedTasks(TInfo tinfo, TCredentials credentials) + throws ThriftSecurityException, TException { // do not expect users to call this directly, expect other tservers to call this method if (!security.canPerformSystemActions(credentials)) { throw new AccumuloSecurityException(credentials.getPrincipal(), @@ -1186,11 +1255,15 @@ public TExternalCompactionList getCompletedCompactions(TInfo tinfo, TCredentials trc.setUpdates(rc.getUpdates()); result.putToCompactions(ecid.canonical(), trc); }); - return result; + + CompactionTasksCompleted completed = + TaskMessageType.COMPACTION_TASKS_COMPLETED.getTaskMessage(); + completed.setCompleted(result); + return completed.toThriftTask(); } @Override - public void cancel(TInfo tinfo, TCredentials credentials, String externalCompactionId) + public void cancelTask(TInfo tinfo, TCredentials credentials, String externalCompactionId) throws TException { var runningCompaction = RUNNING_CACHE.get(ExternalCompactionId.of(externalCompactionId)); var extent = KeyExtent.fromThrift(runningCompaction.getJob().getExtent()); diff --git a/server/manager/src/test/java/org/apache/accumulo/manager/compaction/CompactionCoordinatorTest.java b/server/manager/src/test/java/org/apache/accumulo/manager/compaction/CompactionCoordinatorTest.java index 6ef94028862..56767c05467 100644 --- a/server/manager/src/test/java/org/apache/accumulo/manager/compaction/CompactionCoordinatorTest.java +++ b/server/manager/src/test/java/org/apache/accumulo/manager/compaction/CompactionCoordinatorTest.java @@ -18,6 +18,8 @@ */ package org.apache.accumulo.manager.compaction; +//ELASTICITY_TODO: Rename this to TaskManagerTest, put in +// org.apache.accumulo.manager.tasks package public class CompactionCoordinatorTest { // ELASTICITY_TODO this test was no longer compiling with all the changes to // CompactionCoordinator. Its contents were deleted to get things compiling, however need to go diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java index cd99b55d86f..b9802629d1d 100644 --- a/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java @@ -47,7 +47,6 @@ import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.cli.ConfigOpts; -import org.apache.accumulo.core.compaction.thrift.CompactionCoordinatorService; import org.apache.accumulo.core.compaction.thrift.TExternalCompaction; import org.apache.accumulo.core.compaction.thrift.TExternalCompactionList; import org.apache.accumulo.core.conf.Property; @@ -71,6 +70,11 @@ import org.apache.accumulo.core.tabletscan.thrift.TabletScanClientService; import org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction; import org.apache.accumulo.core.tabletserver.thrift.TabletServerClientService.Client; +import org.apache.accumulo.core.tasks.TaskMessage; +import org.apache.accumulo.core.tasks.TaskMessageType; +import org.apache.accumulo.core.tasks.compaction.CompactionTasksRunning; +import org.apache.accumulo.core.tasks.thrift.Task; +import org.apache.accumulo.core.tasks.thrift.TaskManager; import org.apache.accumulo.core.trace.TraceUtil; import org.apache.accumulo.core.util.Halt; import org.apache.accumulo.core.util.Pair; @@ -175,7 +179,7 @@ public boolean add(Pair obj) { private GCStatus gcStatus; private Optional coordinatorHost = Optional.empty(); private long coordinatorCheckNanos = 0L; - private CompactionCoordinatorService.Client coordinatorClient; + private TaskManager.Client coordinatorClient; private final String coordinatorMissingMsg = "Error getting the compaction coordinator. Check that it is running. It is not " + "started automatically with other cluster processes so must be started by running " @@ -387,7 +391,7 @@ public void fetchData() { Optional previousHost; if (System.nanoTime() - coordinatorCheckNanos > fetchTimeNanos) { previousHost = coordinatorHost; - coordinatorHost = ExternalCompactionUtil.findCompactionCoordinator(context); + coordinatorHost = ExternalCompactionUtil.findTaskManager(context); coordinatorCheckNanos = System.nanoTime(); if (previousHost.isEmpty() && coordinatorHost.isPresent()) { log.info("External Compaction Coordinator found at {}", coordinatorHost.orElseThrow()); @@ -675,7 +679,10 @@ public synchronized Map fetchRunningInfo() { var client = getCoordinator(ccHost); TExternalCompactionList running; try { - running = client.getRunningCompactions(TraceUtil.traceInfo(), getContext().rpcCreds()); + Task task = client.getRunningTasks(TraceUtil.traceInfo(), getContext().rpcCreds()); + final CompactionTasksRunning list = + TaskMessage.fromThiftTask(task, TaskMessageType.COMPACTION_TASKS_RUNNING); + running = list.getRunning(); } catch (Exception e) { throw new IllegalStateException("Unable to get running compactions from " + ccHost, e); } @@ -692,11 +699,11 @@ public Map getEcRunningMap() { return ecRunningMap; } - private CompactionCoordinatorService.Client getCoordinator(HostAndPort address) { + private TaskManager.Client getCoordinator(HostAndPort address) { if (coordinatorClient == null) { try { coordinatorClient = - ThriftUtil.getClient(ThriftClientTypes.COORDINATOR, address, getContext()); + ThriftUtil.getClient(ThriftClientTypes.TASK_MANAGER, address, getContext()); } catch (Exception e) { log.error("Unable to get Compaction coordinator at {}", address); throw new IllegalStateException(coordinatorMissingMsg, e); diff --git a/server/compactor/.gitignore b/server/task-runner/.gitignore similarity index 100% rename from server/compactor/.gitignore rename to server/task-runner/.gitignore diff --git a/server/compactor/pom.xml b/server/task-runner/pom.xml similarity index 97% rename from server/compactor/pom.xml rename to server/task-runner/pom.xml index e1c61106ab4..e3597c030d9 100644 --- a/server/compactor/pom.xml +++ b/server/task-runner/pom.xml @@ -27,8 +27,8 @@ 4.0.0-SNAPSHOT ../../pom.xml - accumulo-compactor - Apache Accumulo Compactor + accumulo-task-runner + Apache Accumulo Task Runner com.google.auto.service diff --git a/server/task-runner/src/main/java/org/apache/accumulo/tasks/TaskRunner.java b/server/task-runner/src/main/java/org/apache/accumulo/tasks/TaskRunner.java new file mode 100644 index 00000000000..b5e9fc203d5 --- /dev/null +++ b/server/task-runner/src/main/java/org/apache/accumulo/tasks/TaskRunner.java @@ -0,0 +1,646 @@ +/* + * 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.tasks; + +import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly; +import static java.util.concurrent.TimeUnit.MINUTES; +import static org.apache.accumulo.core.util.LazySingletons.RANDOM; + +import java.io.IOException; +import java.lang.reflect.InvocationTargetException; +import java.net.UnknownHostException; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Supplier; + +import org.apache.accumulo.core.Constants; +import org.apache.accumulo.core.cli.ConfigOpts; +import org.apache.accumulo.core.client.AccumuloSecurityException; +import org.apache.accumulo.core.client.TableNotFoundException; +import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode; +import org.apache.accumulo.core.clientImpl.thrift.TInfo; +import org.apache.accumulo.core.clientImpl.thrift.TableOperation; +import org.apache.accumulo.core.clientImpl.thrift.TableOperationExceptionType; +import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException; +import org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException; +import org.apache.accumulo.core.conf.AccumuloConfiguration; +import org.apache.accumulo.core.conf.Property; +import org.apache.accumulo.core.conf.SiteConfiguration; +import org.apache.accumulo.core.data.NamespaceId; +import org.apache.accumulo.core.data.TableId; +import org.apache.accumulo.core.dataImpl.KeyExtent; +import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; +import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeExistsPolicy; +import org.apache.accumulo.core.lock.ServiceLock; +import org.apache.accumulo.core.lock.ServiceLock.LockLossReason; +import org.apache.accumulo.core.lock.ServiceLock.LockWatcher; +import org.apache.accumulo.core.lock.ServiceLockData; +import org.apache.accumulo.core.lock.ServiceLockData.ThriftService; +import org.apache.accumulo.core.metadata.schema.ExternalCompactionId; +import org.apache.accumulo.core.metrics.MetricsProducer; +import org.apache.accumulo.core.metrics.MetricsUtil; +import org.apache.accumulo.core.rpc.ThriftUtil; +import org.apache.accumulo.core.rpc.clients.ThriftClientTypes; +import org.apache.accumulo.core.securityImpl.thrift.TCredentials; +import org.apache.accumulo.core.tabletserver.thrift.ActiveCompactionList; +import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob; +import org.apache.accumulo.core.tasks.TaskMessage; +import org.apache.accumulo.core.tasks.TaskMessageType; +import org.apache.accumulo.core.tasks.compaction.ActiveCompactionTasks; +import org.apache.accumulo.core.tasks.compaction.CompactionTask; +import org.apache.accumulo.core.tasks.thrift.Task; +import org.apache.accumulo.core.tasks.thrift.TaskManager; +import org.apache.accumulo.core.tasks.thrift.TaskManager.Client; +import org.apache.accumulo.core.tasks.thrift.TaskRunnerInfo; +import org.apache.accumulo.core.tasks.thrift.WorkerType; +import org.apache.accumulo.core.trace.TraceUtil; +import org.apache.accumulo.core.util.Halt; +import org.apache.accumulo.core.util.UtilWaitThread; +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.server.AbstractServer; +import org.apache.accumulo.server.compaction.CompactionInfo; +import org.apache.accumulo.server.compaction.CompactionWatcher; +import org.apache.accumulo.server.compaction.PausedCompactionMetrics; +import org.apache.accumulo.server.compaction.RetryableThriftCall; +import org.apache.accumulo.server.compaction.RetryableThriftCall.RetriesExceededException; +import org.apache.accumulo.server.fs.VolumeManager; +import org.apache.accumulo.server.rpc.ServerAddress; +import org.apache.accumulo.server.rpc.TServerUtils; +import org.apache.accumulo.server.rpc.ThriftProcessorTypes; +import org.apache.accumulo.server.security.SecurityOperation; +import org.apache.accumulo.tasks.jobs.CompactionJob; +import org.apache.accumulo.tasks.jobs.Job; +import org.apache.thrift.TException; +import org.apache.thrift.transport.TTransportException; +import org.apache.zookeeper.KeeperException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.net.HostAndPort; + +import io.micrometer.core.instrument.LongTaskTimer; +import io.micrometer.core.instrument.MeterRegistry; + +public class TaskRunner extends AbstractServer implements MetricsProducer, + org.apache.accumulo.core.tasks.thrift.TaskRunner.Iface, TaskRunnerProcess { + + private static final Logger LOG = LoggerFactory.getLogger(TaskRunner.class); + private static final AtomicReference> CURRENTLY_EXECUTING_TASK = new AtomicReference<>(); + + private final UUID taskRunnerId = UUID.randomUUID(); + private final AccumuloConfiguration aconf; + protected final AtomicReference currentTaskId = new AtomicReference<>(); + private final WorkerType workerType; + + private SecurityOperation security; + private ServiceLock taskRunnerLock; + private ServerAddress taskRunnerAddress = null; + private PausedCompactionMetrics pausedMetrics; + private CompactionWatcher watcher; + + // Exposed for tests + protected volatile boolean shutdown = false; + + protected TaskRunner(ConfigOpts opts, String[] args) { + this(opts, args, null); + } + + protected TaskRunner(ConfigOpts opts, String[] args, AccumuloConfiguration conf) { + super("TaskRunner", opts, args); + String runnerType = getTaskWorkerTypePropertyValue(); + workerType = WorkerType.valueOf(runnerType); + LOG.debug("Starting TaskRunner of type: {}", workerType); + aconf = conf == null ? super.getConfiguration() : conf; + setupSecurity(); + switch (workerType) { + case COMPACTION: + watcher = new CompactionWatcher(aconf); + var schedExecutor = + ThreadPools.getServerThreadPools().createGeneralScheduledExecutorService(aconf); + startCancelChecker(schedExecutor, getTimeBetweenCancelChecks()); + break; + case LOG_SORTING: + break; + case SPLIT_POINT_CALCULATION: + break; + default: + break; + + } + printStartupMsg(); + } + + protected long getTimeBetweenCancelChecks() { + return MINUTES.toMillis(5); + } + + @Override + protected String getResourceGroupPropertyValue(SiteConfiguration conf) { + return conf.get(Property.TASK_RUNNER_GROUP_NAME); + } + + protected String getTaskWorkerTypePropertyValue() { + return getContext().getSiteConfiguration().get(Property.TASK_RUNNER_WORKER_TYPE); + } + + @Override + public AccumuloConfiguration getConfiguration() { + return aconf; + } + + @Override + public void registerMetrics(MeterRegistry registry) { + super.registerMetrics(registry); + switch (workerType) { + case COMPACTION: + LongTaskTimer timer = LongTaskTimer.builder(METRICS_COMPACTOR_MAJC_STUCK) + .description("Number and duration of stuck major compactions").register(registry); + CompactionWatcher.setTimer(timer); + break; + case LOG_SORTING: + break; + case SPLIT_POINT_CALCULATION: + break; + default: + break; + + } + } + + protected void setupSecurity() { + security = getContext().getSecurityOperation(); + } + + protected void startCancelChecker(ScheduledThreadPoolExecutor schedExecutor, + long timeBetweenChecks) { + LOG.debug("Starting compaction job cancel checker at {}ms intervals", timeBetweenChecks); + ThreadPools.watchCriticalScheduledTask( + schedExecutor.scheduleWithFixedDelay(() -> CompactionJob.checkIfCanceled(getContext()), 0, + timeBetweenChecks, TimeUnit.MILLISECONDS)); + } + + @Override + public CompactionWatcher getCompactionWatcher() { + return this.watcher; + } + + @Override + public PausedCompactionMetrics getPausedCompactionMetrics() { + return pausedMetrics; + } + + protected void printStartupMsg() { + LOG.info("Version " + Constants.VERSION); + LOG.info("Instance " + getContext().getInstanceID()); + } + + /** + * Set up nodes and locks in ZooKeeper for this TaskRunner + * + * @param clientAddress address of this TaskRunner + * @throws KeeperException zookeeper error + * @throws InterruptedException thread interrupted + */ + protected void announceExistence(HostAndPort clientAddress) + throws KeeperException, InterruptedException { + + String hostPort = ExternalCompactionUtil.getHostPortString(clientAddress); + + ZooReaderWriter zoo = getContext().getZooReaderWriter(); + String zPath = null; + switch (workerType) { + case COMPACTION: + String compactorQueuePath = + getContext().getZooKeeperRoot() + Constants.ZCOMPACTORS + "/" + this.getResourceGroup(); + zPath = compactorQueuePath + "/" + hostPort; + + try { + zoo.mkdirs(compactorQueuePath); + zoo.putPersistentData(zPath, new byte[] {}, NodeExistsPolicy.SKIP); + } catch (KeeperException e) { + if (e.code() == KeeperException.Code.NOAUTH) { + LOG.error("Failed to write to ZooKeeper. Ensure that" + + " accumulo.properties, specifically instance.secret, is consistent."); + } + throw e; + } + break; + case LOG_SORTING: + // ELASTICITY_TODO + break; + case SPLIT_POINT_CALCULATION: + // ELASTICITY_TODO + break; + default: + throw new RuntimeException("Unknown worker type: " + workerType); + + } + + if (zPath == null) { + throw new RuntimeException("TaskRunner node not created in ZooKeeper"); + } + + taskRunnerLock = new ServiceLock(getContext().getZooReaderWriter().getZooKeeper(), + ServiceLock.path(zPath), taskRunnerId); + LockWatcher lw = new LockWatcher() { + @Override + public void lostLock(final LockLossReason reason) { + Halt.halt(1, () -> { + LOG.error("TaskRunner lost lock (reason = {}), exiting.", reason); + getContext().getLowMemoryDetector().logGCInfo(getConfiguration()); + }); + } + + @Override + public void unableToMonitorLockNode(final Exception e) { + Halt.halt(1, () -> LOG.error("Lost ability to monitor TaskRunner lock, exiting.", e)); + } + }; + + try { + for (int i = 0; i < 25; i++) { + zoo.putPersistentData(zPath, new byte[0], NodeExistsPolicy.SKIP); + + if (taskRunnerLock.tryLock(lw, new ServiceLockData(taskRunnerId, hostPort, + ThriftService.TASK_RUNNER, this.getResourceGroup()))) { + LOG.debug("Obtained TaskRunner lock {}", taskRunnerLock.getLockPath()); + return; + } + LOG.info("Waiting for TaskRunner 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 TaskRunner server lock, exiting.", e); + throw new RuntimeException(e); + } + } + + /** + * Start this servers thrift service to handle incoming client requests + * + * @return address of this TaskRunner client service + * @throws UnknownHostException host unknown + */ + protected ServerAddress startThriftClientService() throws UnknownHostException { + var processor = ThriftProcessorTypes.getCompactorTProcessor(this, getContext()); + Property maxMessageSizeProperty = + (getConfiguration().get(Property.TASK_RUNNER_MAX_MESSAGE_SIZE) != null + ? Property.TASK_RUNNER_MAX_MESSAGE_SIZE : Property.GENERAL_MAX_MESSAGE_SIZE); + ServerAddress sp = TServerUtils.startServer(getContext(), getHostname(), + Property.TASK_RUNNER_CLIENTPORT, processor, this.getClass().getSimpleName(), + "Thrift Client Server", Property.TASK_RUNNER_PORTSEARCH, Property.TASK_RUNNER_MINTHREADS, + Property.TASK_RUNNER_MINTHREADS_TIMEOUT, Property.TASK_RUNNER_THREADCHECK, + maxMessageSizeProperty); + LOG.info("address = {}", sp.address); + return sp; + } + + @Override + public void cancelTask(TInfo tinfo, TCredentials credentials, String externalCompactionId) + throws TException { + Job job = CURRENTLY_EXECUTING_TASK.get(); + switch (workerType) { + case COMPACTION: + TableId tableId = + KeyExtent.fromThrift(((CompactionJob) job).getJobDetails().getExtent()).tableId(); + try { + NamespaceId nsId = getContext().getNamespaceId(tableId); + if (!security.canCompact(credentials, tableId, nsId)) { + throw new AccumuloSecurityException(credentials.getPrincipal(), + SecurityErrorCode.PERMISSION_DENIED).asThriftException(); + } + } catch (TableNotFoundException e) { + throw new ThriftTableOperationException(tableId.canonical(), null, + TableOperation.COMPACT_CANCEL, TableOperationExceptionType.NOTFOUND, e.getMessage()); + } + + job.cancel(externalCompactionId); + break; + case LOG_SORTING: + // ELASTICITY_TODO + break; + case SPLIT_POINT_CALCULATION: + // ELASTICITY_TODO + break; + default: + break; + + } + } + + /** + * Get the next job to run + * + * @param uuid uuid supplier + * @return CompactionJob + * @throws RetriesExceededException thrown when retries have been exceeded + */ + protected Job getNextJob(Supplier uuid) throws RetriesExceededException { + final long startingWaitTime = + getConfiguration().getTimeInMillis(Property.TASK_RUNNER_MIN_JOB_WAIT_TIME); + final long maxWaitTime = + getConfiguration().getTimeInMillis(Property.TASK_RUNNER_MAX_JOB_WAIT_TIME); + + RetryableThriftCall> nextJobThriftCall = + new RetryableThriftCall<>(startingWaitTime, maxWaitTime, 0, () -> { + Client taskManagerClient = getTaskManagerClient(); + try { + // ELASTICITY_TODO: Change ExternalCompactionId to a more generic task id + ExternalCompactionId eci = ExternalCompactionId.generate(uuid.get()); + LOG.trace("Attempting to get next job, eci = {}", eci); + TaskRunnerInfo runner = new TaskRunnerInfo(taskRunnerAddress.getAddress().getHost(), + taskRunnerAddress.getAddress().getPort(), this.workerType, this.getResourceGroup()); + Task task = taskManagerClient.getTask(TraceUtil.traceInfo(), getContext().rpcCreds(), + runner, eci.toString()); + + switch (this.workerType) { + case COMPACTION: + final CompactionTask compactionTask = + TaskMessage.fromThiftTask(task, TaskMessageType.COMPACTION_TASK); + currentTaskId.set(eci); + LOG.debug("Received task for eci: {}, job:{}", eci.toString(), + compactionTask.getCompactionJob()); + return new CompactionJob(this, compactionTask, currentTaskId); + case LOG_SORTING: + // ELASTICITY_TODO + return null; + case SPLIT_POINT_CALCULATION: + // ELASTICITY_TODO + return null; + default: + return null; + } + + } catch (Exception e) { + currentTaskId.set(null); + throw e; + } finally { + ThriftUtil.returnClient(taskManagerClient, getContext()); + } + }); + return nextJobThriftCall.run(); + } + + /** + * Get the client to the TaskManager + * + * @return TaskManager client + * @throws TTransportException when unable to get client + */ + @Override + public TaskManager.Client getTaskManagerClient() throws TTransportException { + var taskManagerHost = ExternalCompactionUtil.findTaskManager(getContext()); + if (taskManagerHost.isEmpty()) { + throw new TTransportException("Unable to get TaskManager address from ZooKeeper"); + } + LOG.trace("TaskManager address is: {}", taskManagerHost.orElseThrow()); + return ThriftUtil.getClient(ThriftClientTypes.TASK_MANAGER, taskManagerHost.orElseThrow(), + getContext()); + } + + protected Supplier getNextId() { + return UUID::randomUUID; + } + + protected long getWaitTimeBetweenCompactionChecks() { + // get the total number of compactors assigned to this group + int numCompactors = + ExternalCompactionUtil.countCompactors(this.getResourceGroup(), getContext()); + // Aim for around 3 compactors checking in every second + long sleepTime = numCompactors * 1000L / 3; + // Ensure a compactor sleeps at least around a second + sleepTime = Math.max(1000, sleepTime); + // Ensure a compactor sleep not too much more than 5 mins + sleepTime = Math.min(300_000L, sleepTime); + // Add some random jitter to the sleep time, that averages out to sleep time. This will spread + // compactors out evenly over time. + sleepTime = (long) (.9 * sleepTime + sleepTime * .2 * RANDOM.get().nextDouble()); + LOG.trace("Sleeping {}ms based on {} compactors", sleepTime, numCompactors); + return sleepTime; + } + + @Override + public void run() { + + try { + taskRunnerAddress = startThriftClientService(); + } catch (UnknownHostException e1) { + throw new RuntimeException("Failed to start the TaskRunner client service", e1); + } + final HostAndPort clientAddress = taskRunnerAddress.getAddress(); + + try { + announceExistence(clientAddress); + } catch (KeeperException | InterruptedException e) { + throw new RuntimeException("Error registering TaskRunner in ZooKeeper", e); + } + + if (this.workerType == WorkerType.COMPACTION) { + // ELASTICITY_TODO: There are no metrics for the other types + try { + MetricsUtil.initializeMetrics(getContext().getConfiguration(), this.applicationName, + clientAddress); + pausedMetrics = new PausedCompactionMetrics(); + MetricsUtil.initializeProducers(this, pausedMetrics); + } catch (ClassNotFoundException | InstantiationException | IllegalAccessException + | IllegalArgumentException | InvocationTargetException | NoSuchMethodException + | SecurityException e1) { + LOG.error("Error initializing metrics, metrics will not be emitted.", e1); + } + } + + LOG.info("TaskRunner started, waiting for {} work, resource group: {}", workerType, + this.getResourceGroup()); + try { + + while (!shutdown) { + + switch (this.workerType) { + case COMPACTION: { + currentTaskId.set(null); + CURRENTLY_EXECUTING_TASK.set(null); + CompactionJob job; + try { + job = (CompactionJob) getNextJob(getNextId()); + if (!job.getJobDetails().isSetExternalCompactionId()) { + LOG.trace("No external compactions in group {}", this.getResourceGroup()); + UtilWaitThread.sleep(getWaitTimeBetweenCompactionChecks()); + continue; + } + if (!job.getJobDetails().getExternalCompactionId() + .equals(currentTaskId.get().toString())) { + throw new IllegalStateException( + "Returned eci " + job.getJobDetails().getExternalCompactionId() + + " does not match supplied eci " + currentTaskId.get()); + } + } catch (RetriesExceededException e2) { + LOG.warn("Retries exceeded getting next job. Retrying..."); + continue; + } + LOG.debug("Received next compaction job: {}", job); + + final Thread compactionThread = Threads.createThread( + "Compaction job for tablet " + job.getJobDetails().getExtent().toString(), + job.createJob()); + + CURRENTLY_EXECUTING_TASK.set(job); + + job.executeJob(compactionThread); + break; + } + case LOG_SORTING: + break; + case SPLIT_POINT_CALCULATION: + break; + default: + break; + } + } + + } catch (Exception e) { + LOG.error("Unhandled error occurred in TaskRunner", e); + } finally { + // Shutdown local thrift server + LOG.info("Stopping Thrift Servers"); + if (taskRunnerAddress.server != null) { + taskRunnerAddress.server.stop(); + } + + try { + LOG.debug("Closing filesystems"); + VolumeManager mgr = getContext().getVolumeManager(); + if (null != mgr) { + mgr.close(); + } + } catch (IOException e) { + LOG.warn("Failed to close filesystem : {}", e.getMessage(), e); + } + + getContext().getLowMemoryDetector().logGCInfo(getConfiguration()); + LOG.info("stop requested. exiting ... "); + try { + if (null != taskRunnerLock) { + taskRunnerLock.unlock(); + } + } catch (Exception e) { + LOG.warn("Failed to release TaskRunner lock", e); + } + } + + } + + public static void main(String[] args) throws Exception { + try (TaskRunner taskRunner = new TaskRunner(new ConfigOpts(), args)) { + taskRunner.runServer(); + } + } + + @Override + public Task getActiveCompactions(TInfo tinfo, TCredentials credentials) + throws ThriftSecurityException, TException { + if (!security.canPerformSystemActions(credentials)) { + throw new AccumuloSecurityException(credentials.getPrincipal(), + SecurityErrorCode.PERMISSION_DENIED).asThriftException(); + } + + List compactions = + org.apache.accumulo.server.compaction.FileCompactor.getRunningCompactions(); + + ActiveCompactionList list = new ActiveCompactionList(); + compactions.forEach(c -> list.addToCompactions(c.toThrift())); + + ActiveCompactionTasks tasks = TaskMessageType.COMPACTION_TASK_LIST.getTaskMessage(); + tasks.setActiveCompactions(list); + + return tasks.toThriftTask(); + } + + @Override + public Task getRunningTask(TInfo tinfo, TCredentials credentials) + throws ThriftSecurityException, TException { + // do not expect users to call this directly, expect other tservers to call this method + if (!security.canPerformSystemActions(credentials)) { + throw new AccumuloSecurityException(credentials.getPrincipal(), + SecurityErrorCode.PERMISSION_DENIED).asThriftException(); + } + + // Return what is currently running, does not wait for jobs in the process of reserving. This + // method is called by a TaskManager starting up to determine what is currently running on all + // TaskRunners. + + Job job = CURRENTLY_EXECUTING_TASK.get(); + + switch (workerType) { + case COMPACTION: + CompactionTask task = TaskMessageType.COMPACTION_TASK.getTaskMessage(); + if (null == job) { + task.setCompactionJob(new TExternalCompactionJob()); + } else { + TExternalCompactionJob tjob = ((CompactionJob) job).getJobDetails(); + task.setTaskId(tjob.getExternalCompactionId()); + task.setCompactionJob(tjob); + } + return task.toThriftTask(); + case LOG_SORTING: + // ELASTICITY_TODO + return null; + case SPLIT_POINT_CALCULATION: + // ELASTICITY_TODO + return null; + default: + return null; + + } + } + + @Override + public String getRunningTaskId(TInfo tinfo, TCredentials credentials) + throws ThriftSecurityException, TException { + // do not expect users to call this directly, expect other tservers to call this method + if (!security.canPerformSystemActions(credentials)) { + throw new AccumuloSecurityException(credentials.getPrincipal(), + SecurityErrorCode.PERMISSION_DENIED).asThriftException(); + } + + // Any returned id must cover the time period from before a job is reserved until after it + // commits. This method is called to detect dead compactions and depends on this behavior. + // For the purpose of detecting dead compactions its ok if ids are returned that never end up + // being related to a running compaction. + ExternalCompactionId eci = currentTaskId.get(); + if (null == eci) { + return ""; + } else { + return eci.canonical(); + } + } + + @Override + public void shutdown() { + this.shutdown = true; + } + +} diff --git a/server/compactor/src/main/java/org/apache/accumulo/compactor/CompactorExecutable.java b/server/task-runner/src/main/java/org/apache/accumulo/tasks/TaskRunnerExecutable.java similarity index 81% rename from server/compactor/src/main/java/org/apache/accumulo/compactor/CompactorExecutable.java rename to server/task-runner/src/main/java/org/apache/accumulo/tasks/TaskRunnerExecutable.java index d7c6b400fad..0a55d9e114c 100644 --- a/server/compactor/src/main/java/org/apache/accumulo/compactor/CompactorExecutable.java +++ b/server/task-runner/src/main/java/org/apache/accumulo/tasks/TaskRunnerExecutable.java @@ -16,18 +16,18 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.accumulo.compactor; +package org.apache.accumulo.tasks; import org.apache.accumulo.start.spi.KeywordExecutable; import com.google.auto.service.AutoService; @AutoService(KeywordExecutable.class) -public class CompactorExecutable implements KeywordExecutable { +public class TaskRunnerExecutable implements KeywordExecutable { @Override public String keyword() { - return "compactor"; + return "task_runner"; } @Override @@ -37,13 +37,12 @@ public UsageGroup usageGroup() { @Override public String description() { - return "Starts Accumulo Compactor"; + return "Starts Accumulo TaskRunner"; } @Override public void execute(final String[] args) throws Exception { - System.err.println("WARNING: External compaction processes are experimental"); - Compactor.main(args); + TaskRunner.main(args); } } diff --git a/server/task-runner/src/main/java/org/apache/accumulo/tasks/TaskRunnerProcess.java b/server/task-runner/src/main/java/org/apache/accumulo/tasks/TaskRunnerProcess.java new file mode 100644 index 00000000000..ae6f2084a0a --- /dev/null +++ b/server/task-runner/src/main/java/org/apache/accumulo/tasks/TaskRunnerProcess.java @@ -0,0 +1,45 @@ +/* + * 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.tasks; + +import org.apache.accumulo.core.conf.AccumuloConfiguration; +import org.apache.accumulo.core.tasks.thrift.TaskManager; +import org.apache.accumulo.server.ServerContext; +import org.apache.accumulo.server.compaction.CompactionWatcher; +import org.apache.accumulo.server.compaction.PausedCompactionMetrics; +import org.apache.thrift.transport.TTransportException; + +// This is the view of the TaskWorker that the Job's get +public interface TaskRunnerProcess { + + ServerContext getContext(); + + AccumuloConfiguration getConfiguration(); + + String getResourceGroup(); + + TaskManager.Client getTaskManagerClient() throws TTransportException; + + CompactionWatcher getCompactionWatcher(); + + PausedCompactionMetrics getPausedCompactionMetrics(); + + // Exposed for tests + void shutdown(); +} diff --git a/server/task-runner/src/main/java/org/apache/accumulo/tasks/jobs/CompactionJob.java b/server/task-runner/src/main/java/org/apache/accumulo/tasks/jobs/CompactionJob.java new file mode 100644 index 00000000000..450ff9baf09 --- /dev/null +++ b/server/task-runner/src/main/java/org/apache/accumulo/tasks/jobs/CompactionJob.java @@ -0,0 +1,457 @@ +/* + * 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.tasks.jobs; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.atomic.LongAdder; + +import org.apache.accumulo.core.client.IteratorSetting; +import org.apache.accumulo.core.compaction.thrift.TCompactionState; +import org.apache.accumulo.core.compaction.thrift.TCompactionStatusUpdate; +import org.apache.accumulo.core.compaction.thrift.UnknownCompactionIdException; +import org.apache.accumulo.core.conf.AccumuloConfiguration; +import org.apache.accumulo.core.conf.ConfigurationCopy; +import org.apache.accumulo.core.dataImpl.KeyExtent; +import org.apache.accumulo.core.fate.FateTxId; +import org.apache.accumulo.core.iteratorsImpl.system.SystemIteratorUtil; +import org.apache.accumulo.core.metadata.ReferencedTabletFile; +import org.apache.accumulo.core.metadata.StoredTabletFile; +import org.apache.accumulo.core.metadata.schema.DataFileValue; +import org.apache.accumulo.core.metadata.schema.ExternalCompactionId; +import org.apache.accumulo.core.metadata.schema.TabletMetadata; +import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType; +import org.apache.accumulo.core.rpc.ThriftUtil; +import org.apache.accumulo.core.tabletserver.thrift.TCompactionKind; +import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats; +import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob; +import org.apache.accumulo.core.tasks.TaskMessageType; +import org.apache.accumulo.core.tasks.compaction.CompactionTask; +import org.apache.accumulo.core.tasks.compaction.CompactionTaskCompleted; +import org.apache.accumulo.core.tasks.compaction.CompactionTaskFailed; +import org.apache.accumulo.core.tasks.compaction.CompactionTaskStatus; +import org.apache.accumulo.core.tasks.thrift.TaskManager.Client; +import org.apache.accumulo.core.trace.TraceUtil; +import org.apache.accumulo.server.ServerContext; +import org.apache.accumulo.server.compaction.CompactionConfigStorage; +import org.apache.accumulo.server.compaction.CompactionInfo; +import org.apache.accumulo.server.compaction.FileCompactor; +import org.apache.accumulo.server.compaction.RetryableThriftCall; +import org.apache.accumulo.server.compaction.RetryableThriftCall.RetriesExceededException; +import org.apache.accumulo.server.conf.TableConfiguration; +import org.apache.accumulo.tasks.TaskRunnerProcess; +import org.apache.hadoop.fs.Path; +import org.apache.thrift.TException; +import org.apache.zookeeper.KeeperException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.base.Preconditions; + +public class CompactionJob extends Job { + + private static final Logger LOG = LoggerFactory.getLogger(CompactionJob.class); + private static final long TEN_MEGABYTES = 10485760; + + protected static final CompactionJobHolder JOB_HOLDER = new CompactionJobHolder(); + + public static void checkIfCanceled(ServerContext ctx) { + TExternalCompactionJob job = JOB_HOLDER.getJob(); + if (job != null) { + try { + var extent = KeyExtent.fromThrift(job.getExtent()); + var ecid = ExternalCompactionId.of(job.getExternalCompactionId()); + + TabletMetadata tabletMeta = + ctx.getAmple().readTablet(extent, ColumnType.ECOMP, ColumnType.PREV_ROW); + if (tabletMeta == null || !tabletMeta.getExternalCompactions().containsKey(ecid)) { + // table was deleted OR tablet was split or merged OR tablet no longer thinks compaction + // is running for some reason + LOG.info("Cancelling compaction {} that no longer has a metadata entry at {}", ecid, + extent); + JOB_HOLDER.cancel(job.getExternalCompactionId()); + return; + } + + if (job.getKind() == TCompactionKind.USER) { + + var cconf = CompactionConfigStorage.getConfig(ctx, job.getFateTxId()); + + if (cconf == null) { + LOG.info("Cancelling compaction {} for user compaction that no longer exists {} {}", + ecid, FateTxId.formatTid(job.getFateTxId()), extent); + JOB_HOLDER.cancel(job.getExternalCompactionId()); + } + } + } catch (RuntimeException | KeeperException e) { + LOG.warn("Failed to check if compaction {} for {} was canceled.", + job.getExternalCompactionId(), KeyExtent.fromThrift(job.getExtent()), e); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + } + } + + /** + * Returns the number of seconds to wait in between progress checks based on input file sizes + * + * @param numBytes number of bytes in input file + * @return number of seconds to wait between progress checks + */ + public static long calculateProgressCheckTime(long numBytes) { + return Math.max(1, (numBytes / TEN_MEGABYTES)); + } + + protected final LongAdder totalInputEntries = new LongAdder(); + protected final LongAdder totalInputBytes = new LongAdder(); + protected final CountDownLatch started = new CountDownLatch(1); + protected final CountDownLatch stopped = new CountDownLatch(1); + protected final AtomicReference errorRef = new AtomicReference<>(); + protected final AtomicBoolean compactionRunning = new AtomicBoolean(false); + + private final TExternalCompactionJob details; + private final AtomicReference currentCompactionId; + + public CompactionJob(TaskRunnerProcess worker, CompactionTask msg, + AtomicReference currentCompactionId) throws TException { + super(worker, msg); + this.details = msg.getCompactionJob(); + this.currentCompactionId = currentCompactionId; + } + + public TExternalCompactionJob getJobDetails() { + return this.details; + } + + @Override + public Runnable createJob() throws TException { + errorRef.set(null); + return createCompactionJob(msg.getCompactionJob(), totalInputEntries, totalInputBytes, started, + stopped, errorRef); + } + + @Override + public void executeJob(Thread executionThread) throws InterruptedException { + try { + JOB_HOLDER.set(this.details, executionThread); + executionThread.start(); // start the compactionThread + started.await(); // wait until the compactor is started + final long inputEntries = totalInputEntries.sum(); + final long waitTime = calculateProgressCheckTime(totalInputBytes.sum()); + LOG.debug("Progress checks will occur every {} seconds", waitTime); + String percentComplete = "unknown"; + + while (!stopped.await(waitTime, TimeUnit.SECONDS)) { + List running = + org.apache.accumulo.server.compaction.FileCompactor.getRunningCompactions(); + if (!running.isEmpty()) { + // Compaction has started. There should only be one in the list + CompactionInfo info = running.get(0); + if (info != null) { + if (inputEntries > 0) { + percentComplete = + Float.toString((info.getEntriesRead() / (float) inputEntries) * 100); + } + String message = String.format( + "Compaction in progress, read %d of %d input entries ( %s %s ), written %d entries, paused %d times", + info.getEntriesRead(), inputEntries, percentComplete, "%", info.getEntriesWritten(), + info.getTimesPaused()); + getTaskWorker().getCompactionWatcher().run(); + try { + LOG.debug("Updating TaskManager with compaction progress: {}.", message); + TCompactionStatusUpdate update = + new TCompactionStatusUpdate(TCompactionState.IN_PROGRESS, message, inputEntries, + info.getEntriesRead(), info.getEntriesWritten()); + updateCompactionState(details, update); + } catch (RetriesExceededException e) { + LOG.warn("Error updating TaskManager with compaction progress, error: {}", + e.getMessage()); + } + } + } else { + LOG.error("Waiting on compaction thread to finish, but no RUNNING compaction"); + } + } + executionThread.join(); + LOG.trace("Compaction thread finished."); + // Run the watcher again to clear out the finished compaction and set the + // stuck count to zero. + getTaskWorker().getCompactionWatcher().run(); + + if (errorRef.get() != null) { + // maybe the error occured because the table was deleted or something like that, so + // force a cancel check to possibly reduce noise in the logs + checkIfCanceled(getTaskWorker().getContext()); + } + + if (executionThread.isInterrupted() || JOB_HOLDER.isCancelled() || (errorRef.get() != null + && errorRef.get().getClass().equals(InterruptedException.class))) { + LOG.warn("Compaction thread was interrupted, sending CANCELLED state"); + try { + TCompactionStatusUpdate update = new TCompactionStatusUpdate(TCompactionState.CANCELLED, + "Compaction cancelled", -1, -1, -1); + updateCompactionState(details, update); + updateCompactionFailed(details); + } catch (RetriesExceededException e) { + LOG.error("Error updating TaskManager with compaction cancellation.", e); + } finally { + currentCompactionId.set(null); + } + } else if (errorRef.get() != null) { + try { + LOG.info("Updating TaskManager with compaction failure."); + TCompactionStatusUpdate update = new TCompactionStatusUpdate(TCompactionState.FAILED, + "Compaction failed due to: " + errorRef.get().getMessage(), -1, -1, -1); + updateCompactionState(details, update); + updateCompactionFailed(details); + } catch (RetriesExceededException e) { + LOG.error("Error updating TaskManager with compaction failure.", e); + } finally { + currentCompactionId.set(null); + } + } else { + try { + LOG.trace("Updating TaskManager with compaction completion."); + updateCompactionCompleted(details, JOB_HOLDER.getStats()); + } catch (RetriesExceededException e) { + LOG.error("Error updating TaskManager with compaction completion, cancelling compaction.", + e); + try { + cancel(details.getExternalCompactionId()); + } catch (TException e1) { + LOG.error("Error cancelling compaction.", e1); + } + } finally { + currentCompactionId.set(null); + } + } + } catch (RuntimeException e1) { + LOG.error("Compactor thread was interrupted waiting for compaction to start, cancelling job", + e1); + try { + cancel(details.getExternalCompactionId()); + } catch (TException e2) { + LOG.error("Error cancelling compaction.", e2); + } + } finally { + currentCompactionId.set(null); + // In the case where there is an error in the foreground code the background compaction + // may still be running. Must cancel it before starting another iteration of the loop to + // avoid multiple threads updating shared state. + while (executionThread.isAlive()) { + executionThread.interrupt(); + executionThread.join(1000); + } + } + + } + + /** + * Cancel the compaction with this id. + * + * @param externalCompactionId compaction id + * @throws UnknownCompactionIdException if the externalCompactionId does not match the currently + * executing compaction + * @throws TException thrift error + */ + @Override + public void cancel(String externalCompactionId) throws TException { + if (JOB_HOLDER.cancel(externalCompactionId)) { + LOG.info("Cancel requested for compaction job {}", externalCompactionId); + } else { + throw new UnknownCompactionIdException(); + } + } + + /** + * Create compaction runnable + * + * @param job compaction job + * @param totalInputEntries object to capture total entries + * @param totalInputBytes object to capture input file size + * @param started started latch + * @param stopped stopped latch + * @param err reference to error + * @return Runnable compaction job + */ + private Runnable createCompactionJob(final TExternalCompactionJob job, + final LongAdder totalInputEntries, final LongAdder totalInputBytes, + final CountDownLatch started, final CountDownLatch stopped, + final AtomicReference err) { + + return () -> { + // Its only expected that a single compaction runs at a time. Multiple compactions running + // at a time could cause odd behavior like out of order and unexpected thrift calls to the + // TaskManager. This is a sanity check to ensure the expectation is met. Should this check + // ever fail, it means there is a bug elsewhere. + Preconditions.checkState(compactionRunning.compareAndSet(false, true)); + try { + LOG.info("Starting up compaction runnable for job: {}", job); + TCompactionStatusUpdate update = + new TCompactionStatusUpdate(TCompactionState.STARTED, "Compaction started", -1, -1, -1); + updateCompactionState(job, update); + var extent = KeyExtent.fromThrift(job.getExtent()); + final AccumuloConfiguration aConfig; + final TableConfiguration tConfig = + getTaskWorker().getContext().getTableConfiguration(extent.tableId()); + + if (!job.getOverrides().isEmpty()) { + aConfig = new ConfigurationCopy(tConfig); + job.getOverrides().forEach(((ConfigurationCopy) aConfig)::set); + LOG.debug("Overriding table properties with {}", job.getOverrides()); + } else { + aConfig = tConfig; + } + + final ReferencedTabletFile outputFile = + new ReferencedTabletFile(new Path(job.getOutputFile())); + + final Map files = new TreeMap<>(); + job.getFiles().forEach(f -> { + files.put(new StoredTabletFile(f.getMetadataFileEntry()), + new DataFileValue(f.getSize(), f.getEntries(), f.getTimestamp())); + totalInputEntries.add(f.getEntries()); + totalInputBytes.add(f.getSize()); + }); + + final List iters = new ArrayList<>(); + job.getIteratorSettings().getIterators() + .forEach(tis -> iters.add(SystemIteratorUtil.toIteratorSetting(tis))); + + ExtCEnv cenv = new ExtCEnv(JOB_HOLDER, getTaskWorker().getResourceGroup()); + FileCompactor compactor = new FileCompactor(getTaskWorker().getContext(), extent, files, + outputFile, job.isPropagateDeletes(), cenv, iters, aConfig, tConfig.getCryptoService(), + getTaskWorker().getPausedCompactionMetrics()); + + LOG.trace("Starting compactor"); + started.countDown(); + + org.apache.accumulo.server.compaction.CompactionStats stat = compactor.call(); + TCompactionStats cs = new TCompactionStats(); + cs.setEntriesRead(stat.getEntriesRead()); + cs.setEntriesWritten(stat.getEntriesWritten()); + cs.setFileSize(stat.getFileSize()); + JOB_HOLDER.setStats(cs); + + LOG.info("Compaction completed successfully {} ", job.getExternalCompactionId()); + // Update state when completed + TCompactionStatusUpdate update2 = new TCompactionStatusUpdate(TCompactionState.SUCCEEDED, + "Compaction completed successfully", -1, -1, -1); + updateCompactionState(job, update2); + } catch (FileCompactor.CompactionCanceledException cce) { + LOG.debug("Compaction canceled {}", job.getExternalCompactionId()); + } catch (Exception e) { + LOG.error("Compaction failed", e); + err.set(e); + } finally { + stopped.countDown(); + Preconditions.checkState(compactionRunning.compareAndSet(true, false)); + } + }; + } + + /** + * Send an update to the TaskManager for this job + * + * @param job compactionJob + * @param update status update + * @throws RetriesExceededException thrown when retries have been exceeded + */ + protected void updateCompactionState(TExternalCompactionJob job, TCompactionStatusUpdate update) + throws RetriesExceededException { + RetryableThriftCall thriftCall = + new RetryableThriftCall<>(1000, RetryableThriftCall.MAX_WAIT_TIME, 25, () -> { + Client taskManagerClient = getTaskWorker().getTaskManagerClient(); + try { + CompactionTaskStatus status = TaskMessageType.COMPACTION_TASK_STATUS.getTaskMessage(); + status.setTaskId(job.getExternalCompactionId()); + status.setCompactionStatus(update); + taskManagerClient.taskStatus(TraceUtil.traceInfo(), + getTaskWorker().getContext().rpcCreds(), System.currentTimeMillis(), + status.toThriftTask()); + return ""; + } finally { + ThriftUtil.returnClient(taskManagerClient, getTaskWorker().getContext()); + } + }); + thriftCall.run(); + } + + /** + * Notify the TaskManager the job failed + * + * @param job current compaction job + * @throws RetriesExceededException thrown when retries have been exceeded + */ + protected void updateCompactionFailed(TExternalCompactionJob job) + throws RetriesExceededException { + RetryableThriftCall thriftCall = + new RetryableThriftCall<>(1000, RetryableThriftCall.MAX_WAIT_TIME, 25, () -> { + Client taskManagerClient = getTaskWorker().getTaskManagerClient(); + try { + CompactionTaskFailed failedMsg = + TaskMessageType.COMPACTION_TASK_FAILED.getTaskMessage(); + failedMsg.setTaskId(job.getExternalCompactionId()); + failedMsg.setCompactionJob(job); + taskManagerClient.taskFailed(TraceUtil.traceInfo(), + getTaskWorker().getContext().rpcCreds(), failedMsg.toThriftTask()); + return ""; + } finally { + ThriftUtil.returnClient(taskManagerClient, getTaskWorker().getContext()); + } + }); + thriftCall.run(); + } + + /** + * Update the TaskManager with the stats from the completed job + * + * @param job current compaction job + * @param stats compaction stats + * @throws RetriesExceededException thrown when retries have been exceeded + */ + protected void updateCompactionCompleted(TExternalCompactionJob job, TCompactionStats stats) + throws RetriesExceededException { + RetryableThriftCall thriftCall = + new RetryableThriftCall<>(1000, RetryableThriftCall.MAX_WAIT_TIME, 25, () -> { + Client taskManagerClient = getTaskWorker().getTaskManagerClient(); + try { + CompactionTaskCompleted completedMsg = + TaskMessageType.COMPACTION_TASK_COMPLETED.getTaskMessage(); + completedMsg.setTaskId(job.getExternalCompactionId()); + completedMsg.setCompactionJob(job); + completedMsg.setCompactionStats(stats); + taskManagerClient.taskCompleted(TraceUtil.traceInfo(), + getTaskWorker().getContext().rpcCreds(), completedMsg.toThriftTask()); + return ""; + } finally { + ThriftUtil.returnClient(taskManagerClient, getTaskWorker().getContext()); + } + }); + thriftCall.run(); + } + +} diff --git a/server/compactor/src/main/java/org/apache/accumulo/compactor/CompactionJobHolder.java b/server/task-runner/src/main/java/org/apache/accumulo/tasks/jobs/CompactionJobHolder.java similarity index 98% rename from server/compactor/src/main/java/org/apache/accumulo/compactor/CompactionJobHolder.java rename to server/task-runner/src/main/java/org/apache/accumulo/tasks/jobs/CompactionJobHolder.java index d609424c4bc..b8808430f68 100644 --- a/server/compactor/src/main/java/org/apache/accumulo/compactor/CompactionJobHolder.java +++ b/server/task-runner/src/main/java/org/apache/accumulo/tasks/jobs/CompactionJobHolder.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.accumulo.compactor; +package org.apache.accumulo.tasks.jobs; import java.util.Objects; diff --git a/server/compactor/src/main/java/org/apache/accumulo/compactor/ExtCEnv.java b/server/task-runner/src/main/java/org/apache/accumulo/tasks/jobs/ExtCEnv.java similarity index 98% rename from server/compactor/src/main/java/org/apache/accumulo/compactor/ExtCEnv.java rename to server/task-runner/src/main/java/org/apache/accumulo/tasks/jobs/ExtCEnv.java index 59235b6cecc..ba73f3e230d 100644 --- a/server/compactor/src/main/java/org/apache/accumulo/compactor/ExtCEnv.java +++ b/server/task-runner/src/main/java/org/apache/accumulo/tasks/jobs/ExtCEnv.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.accumulo.compactor; +package org.apache.accumulo.tasks.jobs; import org.apache.accumulo.core.conf.AccumuloConfiguration; import org.apache.accumulo.core.data.Key; diff --git a/server/task-runner/src/main/java/org/apache/accumulo/tasks/jobs/Job.java b/server/task-runner/src/main/java/org/apache/accumulo/tasks/jobs/Job.java new file mode 100644 index 00000000000..db1c33d86a7 --- /dev/null +++ b/server/task-runner/src/main/java/org/apache/accumulo/tasks/jobs/Job.java @@ -0,0 +1,44 @@ +/* + * 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.tasks.jobs; + +import org.apache.accumulo.core.tasks.TaskMessage; +import org.apache.accumulo.tasks.TaskRunnerProcess; +import org.apache.thrift.TException; + +public abstract class Job { + + private final TaskRunnerProcess worker; + protected final T msg; + + public Job(TaskRunnerProcess worker, T msg) { + this.worker = worker; + this.msg = msg; + } + + public TaskRunnerProcess getTaskWorker() { + return this.worker; + } + + public abstract Runnable createJob() throws Exception; + + public abstract void executeJob(Thread executionThread) throws InterruptedException; + + public abstract void cancel(String id) throws TException; +} diff --git a/server/compactor/src/test/java/org/apache/accumulo/compactor/CompactorTest.java b/server/task-runner/src/test/java/org/apache/accumulo/tasks/TaskRunnerTest.java similarity index 70% rename from server/compactor/src/test/java/org/apache/accumulo/compactor/CompactorTest.java rename to server/task-runner/src/test/java/org/apache/accumulo/tasks/TaskRunnerTest.java index 1be73d0fc3e..0fa5819b7f0 100644 --- a/server/compactor/src/test/java/org/apache/accumulo/compactor/CompactorTest.java +++ b/server/task-runner/src/test/java/org/apache/accumulo/tasks/TaskRunnerTest.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.accumulo.compactor; +package org.apache.accumulo.tasks; import static org.easymock.EasyMock.expect; import static org.junit.Assert.assertEquals; @@ -27,9 +27,7 @@ import java.util.Timer; import java.util.TimerTask; import java.util.UUID; -import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicReference; -import java.util.concurrent.atomic.LongAdder; import java.util.function.Supplier; import org.apache.accumulo.core.cli.ConfigOpts; @@ -39,11 +37,15 @@ import org.apache.accumulo.core.conf.ConfigurationCopy; import org.apache.accumulo.core.conf.DefaultConfiguration; 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.dataImpl.thrift.TKeyExtent; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.metadata.schema.ExternalCompactionId; import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats; import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob; +import org.apache.accumulo.core.tasks.compaction.CompactionTask; +import org.apache.accumulo.core.tasks.thrift.WorkerType; import org.apache.accumulo.core.util.Halt; import org.apache.accumulo.core.util.UtilWaitThread; import org.apache.accumulo.server.AbstractServer; @@ -52,6 +54,10 @@ import org.apache.accumulo.server.fs.VolumeManagerImpl; import org.apache.accumulo.server.mem.LowMemoryDetector; import org.apache.accumulo.server.rpc.ServerAddress; +import org.apache.accumulo.tasks.jobs.CompactionJob; +import org.apache.accumulo.tasks.jobs.Job; +import org.apache.hadoop.io.Text; +import org.apache.thrift.TException; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.ZooKeeper; import org.junit.Test; @@ -67,115 +73,145 @@ import com.google.common.net.HostAndPort; @RunWith(PowerMockRunner.class) -@PrepareForTest({Compactor.class}) +@PrepareForTest({TaskRunner.class}) @SuppressStaticInitializationFor({"org.apache.log4j.LogManager"}) @PowerMockIgnore({"org.slf4j.*", "org.apache.logging.*", "org.apache.log4j.*", "org.apache.commons.logging.*", "org.xml.*", "javax.xml.*", "org.w3c.dom.*", "com.sun.org.apache.xerces.*"}) -public class CompactorTest { +public class TaskRunnerTest { - public class SuccessfulCompaction implements Runnable { + public class SuccessfulCompaction extends CompactionJob { protected final Logger LOG = LoggerFactory.getLogger(this.getClass()); - protected final LongAdder totalInputEntries; - protected final LongAdder totalInputBytes; - protected final CountDownLatch started; - protected final CountDownLatch stopped; - protected final AtomicReference err; - - public SuccessfulCompaction(LongAdder totalInputEntries, LongAdder totalInputBytes, - CountDownLatch started, CountDownLatch stopped, AtomicReference err) { - this.totalInputEntries = totalInputEntries; - this.totalInputBytes = totalInputBytes; - this.err = err; - this.started = started; - this.stopped = stopped; + private volatile boolean completedCalled = false; + private volatile boolean failedCalled = false; + private TCompactionStatusUpdate latestState = null; + + public SuccessfulCompaction(TaskRunnerProcess worker, CompactionTask msg, + AtomicReference currentCompactionId) throws TException { + super(worker, msg, currentCompactionId); } @Override - public void run() { - try { - started.countDown(); - UtilWaitThread.sleep(1000); - } catch (Exception e) { - err.set(e); - } finally { - stopped.countDown(); - } + public Runnable createJob() throws TException { + return () -> { + try { + started.countDown(); + UtilWaitThread.sleep(1000); + } catch (Exception e) { + errorRef.set(e); + } finally { + stopped.countDown(); + } + }; + } + + @Override + protected void updateCompactionState(TExternalCompactionJob job, TCompactionStatusUpdate update) + throws RetriesExceededException { + latestState = update; + } + + @Override + protected void updateCompactionFailed(TExternalCompactionJob job) + throws RetriesExceededException { + failedCalled = true; } + + @Override + protected void updateCompactionCompleted(TExternalCompactionJob job, TCompactionStats stats) + throws RetriesExceededException { + completedCalled = true; + } + + public boolean isCompletedCalled() { + return completedCalled; + } + + public boolean isFailedCalled() { + return failedCalled; + } + + public TCompactionStatusUpdate getLatestState() { + return latestState; + } + } public class FailedCompaction extends SuccessfulCompaction { - public FailedCompaction(LongAdder totalInputEntries, LongAdder totalInputBytes, - CountDownLatch started, CountDownLatch stopped, AtomicReference err) { - super(totalInputEntries, totalInputBytes, started, stopped, err); + public FailedCompaction(TaskRunnerProcess worker, CompactionTask msg, + AtomicReference currentCompactionId) throws TException { + super(worker, msg, currentCompactionId); } @Override - public void run() { - try { - started.countDown(); - UtilWaitThread.sleep(1000); - throw new RuntimeException(); - } catch (Exception e) { - err.set(e); - } finally { - stopped.countDown(); - } + public Runnable createJob() throws TException { + return () -> { + try { + started.countDown(); + UtilWaitThread.sleep(1000); + throw new RuntimeException(); + } catch (Exception e) { + errorRef.set(e); + } finally { + stopped.countDown(); + } + }; } } public class InterruptedCompaction extends SuccessfulCompaction { - public InterruptedCompaction(LongAdder totalInputEntries, LongAdder totalInputBytes, - CountDownLatch started, CountDownLatch stopped, AtomicReference err) { - super(totalInputEntries, totalInputBytes, started, stopped, err); + public InterruptedCompaction(TaskRunnerProcess worker, CompactionTask msg, + AtomicReference currentCompactionId) throws TException { + super(worker, msg, currentCompactionId); } @Override - public void run() { - try { - started.countDown(); - final Thread thread = Thread.currentThread(); - Timer t = new Timer(); - TimerTask task = new TimerTask() { - @Override - public void run() { - thread.interrupt(); - } - }; - t.schedule(task, 250); - Thread.sleep(1000); - } catch (Exception e) { - LOG.error("Compaction failed: {}", e.getMessage()); - err.set(e); - throw new RuntimeException("Compaction failed", e); - } finally { - stopped.countDown(); - } + public Runnable createJob() throws TException { + return () -> { + try { + started.countDown(); + final Thread thread = Thread.currentThread(); + Timer t = new Timer(); + TimerTask task = new TimerTask() { + @Override + public void run() { + thread.interrupt(); + } + }; + t.schedule(task, 250); + Thread.sleep(1000); + } catch (Exception e) { + LOG.error("Compaction failed: {}", e.getMessage()); + errorRef.set(e); + throw new RuntimeException("Compaction failed", e); + } finally { + stopped.countDown(); + } + }; } } - public class SuccessfulCompactor extends Compactor { + public class SuccessfulCompactor extends TaskRunner { private final Logger LOG = LoggerFactory.getLogger(SuccessfulCompactor.class); private final Supplier uuid; private final ServerAddress address; - private final TExternalCompactionJob job; + protected final TExternalCompactionJob job; private final ServerContext context; private final ExternalCompactionId eci; - private volatile boolean completedCalled = false; - private volatile boolean failedCalled = false; - private TCompactionStatusUpdate latestState = null; + private SuccessfulCompaction compactionJob; SuccessfulCompactor(Supplier uuid, ServerAddress address, TExternalCompactionJob job, ServerContext context, ExternalCompactionId eci) { super(new ConfigOpts(), - new String[] {"-o", Property.COMPACTOR_GROUP_NAME.getKey() + "=testQ"}, + new String[] {"-o", Property.TASK_RUNNER_GROUP_NAME.getKey() + "=testQ", "-o", + Property.TASK_RUNNER_WORKER_TYPE.getKey() + "=COMPACTION"}, context.getConfiguration()); this.uuid = uuid; this.address = address; @@ -184,6 +220,11 @@ public class SuccessfulCompactor extends Compactor { this.eci = eci; } + @Override + protected String getTaskWorkerTypePropertyValue() { + return WorkerType.COMPACTION.toString(); + } + @Override public AccumuloConfiguration getConfiguration() { return context.getConfiguration(); @@ -205,27 +246,28 @@ protected void announceExistence(HostAndPort clientAddress) throws KeeperException, InterruptedException {} @Override - protected ServerAddress startCompactorClientService() throws UnknownHostException { + protected ServerAddress startThriftClientService() throws UnknownHostException { return this.address; } @Override - protected TExternalCompactionJob getNextJob(Supplier uuid) - throws RetriesExceededException { - LOG.info("Attempting to get next job, eci = {}", eci); - currentCompactionId.set(eci); - this.shutdown = true; - return job; + protected Job getNextJob(Supplier uuid) throws RetriesExceededException { + try { + LOG.info("Attempting to get next job, eci = {}", eci); + currentTaskId.set(eci); + compactionJob = createJobForTest(); + return compactionJob; + } catch (TException e) { + throw new RuntimeException("Error creating CompactionJob", e); + } finally { + this.shutdown = true; + } } - @Override - protected synchronized void checkIfCanceled() {} - - @Override - protected Runnable createCompactionJob(TExternalCompactionJob job, LongAdder totalInputEntries, - LongAdder totalInputBytes, CountDownLatch started, CountDownLatch stopped, - AtomicReference err) { - return new SuccessfulCompaction(totalInputEntries, totalInputBytes, started, stopped, err); + protected SuccessfulCompaction createJobForTest() throws TException { + CompactionTask task = new CompactionTask(); + task.setCompactionJob(job); + return new SuccessfulCompaction(this, task, currentTaskId); } @Override @@ -233,34 +275,16 @@ protected Supplier getNextId() { return uuid; } - @Override - protected void updateCompactionState(TExternalCompactionJob job, TCompactionStatusUpdate update) - throws RetriesExceededException { - latestState = update; - } - - @Override - protected void updateCompactionFailed(TExternalCompactionJob job) - throws RetriesExceededException { - failedCalled = true; - } - - @Override - protected void updateCompactionCompleted(TExternalCompactionJob job, TCompactionStats stats) - throws RetriesExceededException { - completedCalled = true; - } - public TCompactionState getLatestState() { - return latestState.getState(); + return compactionJob.getLatestState().getState(); } public boolean isCompletedCalled() { - return completedCalled; + return compactionJob.isCompletedCalled(); } public boolean isFailedCalled() { - return failedCalled; + return compactionJob.isFailedCalled(); } } @@ -273,10 +297,10 @@ public class FailedCompactor extends SuccessfulCompactor { } @Override - protected Runnable createCompactionJob(TExternalCompactionJob job, LongAdder totalInputEntries, - LongAdder totalInputBytes, CountDownLatch started, CountDownLatch stopped, - AtomicReference err) { - return new FailedCompaction(totalInputEntries, totalInputBytes, started, stopped, err); + protected SuccessfulCompaction createJobForTest() throws TException { + CompactionTask task = new CompactionTask(); + task.setCompactionJob(job); + return new FailedCompaction(this, task, currentTaskId); } } @@ -288,21 +312,21 @@ public class InterruptedCompactor extends SuccessfulCompactor { } @Override - protected Runnable createCompactionJob(TExternalCompactionJob job, LongAdder totalInputEntries, - LongAdder totalInputBytes, CountDownLatch started, CountDownLatch stopped, - AtomicReference err) { - return new InterruptedCompaction(totalInputEntries, totalInputBytes, started, stopped, err); + protected SuccessfulCompaction createJobForTest() throws TException { + CompactionTask task = new CompactionTask(); + task.setCompactionJob(job); + return new InterruptedCompaction(this, task, currentTaskId); } } @Test public void testCheckTime() throws Exception { - assertEquals(1, Compactor.calculateProgressCheckTime(1024)); - assertEquals(1, Compactor.calculateProgressCheckTime(1048576)); - assertEquals(1, Compactor.calculateProgressCheckTime(10485760)); - assertEquals(10, Compactor.calculateProgressCheckTime(104857600)); - assertEquals(102, Compactor.calculateProgressCheckTime(1024 * 1024 * 1024)); + assertEquals(1, CompactionJob.calculateProgressCheckTime(1024)); + assertEquals(1, CompactionJob.calculateProgressCheckTime(1048576)); + assertEquals(1, CompactionJob.calculateProgressCheckTime(10485760)); + assertEquals(10, CompactionJob.calculateProgressCheckTime(104857600)); + assertEquals(102, CompactionJob.calculateProgressCheckTime(1024 * 1024 * 1024)); } @Test @@ -321,11 +345,11 @@ public void testCompactionSucceeds() throws Exception { expect(client.getAddress()).andReturn(address); TExternalCompactionJob job = PowerMock.createNiceMock(TExternalCompactionJob.class); - TKeyExtent extent = PowerMock.createNiceMock(TKeyExtent.class); + KeyExtent ke = new KeyExtent(TableId.of("1"), new Text("b"), new Text("a")); + TKeyExtent extent = ke.toThrift(); expect(job.isSetExternalCompactionId()).andReturn(true).anyTimes(); expect(job.getExternalCompactionId()).andReturn(eci.toString()).anyTimes(); expect(job.getExtent()).andReturn(extent).anyTimes(); - expect(extent.getTable()).andReturn("testTable".getBytes()).anyTimes(); var conf = new ConfigurationCopy(DefaultConfiguration.getInstance()); conf.set(Property.INSTANCE_ZK_TIMEOUT, "1d"); @@ -369,9 +393,8 @@ public void testCompactionFails() throws Exception { expect(client.getAddress()).andReturn(address); TExternalCompactionJob job = PowerMock.createNiceMock(TExternalCompactionJob.class); - TKeyExtent extent = PowerMock.createNiceMock(TKeyExtent.class); - expect(extent.getTable()).andReturn("testTable".getBytes()).anyTimes(); - + KeyExtent ke = new KeyExtent(TableId.of("1"), new Text("b"), new Text("a")); + TKeyExtent extent = ke.toThrift(); expect(job.isSetExternalCompactionId()).andReturn(true).anyTimes(); expect(job.getExternalCompactionId()).andReturn(eci.toString()).anyTimes(); expect(job.getExtent()).andReturn(extent).anyTimes(); @@ -419,11 +442,11 @@ public void testCompactionInterrupted() throws Exception { expect(client.getAddress()).andReturn(address); TExternalCompactionJob job = PowerMock.createNiceMock(TExternalCompactionJob.class); - TKeyExtent extent = PowerMock.createNiceMock(TKeyExtent.class); + KeyExtent ke = new KeyExtent(TableId.of("1"), new Text("b"), new Text("a")); + TKeyExtent extent = ke.toThrift(); expect(job.isSetExternalCompactionId()).andReturn(true).anyTimes(); expect(job.getExternalCompactionId()).andReturn(eci.toString()).anyTimes(); expect(job.getExtent()).andReturn(extent).anyTimes(); - expect(extent.getTable()).andReturn("testTable".getBytes()).anyTimes(); var conf = new ConfigurationCopy(DefaultConfiguration.getInstance()); conf.set(Property.INSTANCE_ZK_TIMEOUT, "1d"); diff --git a/server/compactor/src/test/resources/log4j2-test.properties b/server/task-runner/src/test/resources/log4j2-test.properties similarity index 97% rename from server/compactor/src/test/resources/log4j2-test.properties rename to server/task-runner/src/test/resources/log4j2-test.properties index f3095a1452d..ed4655ddae8 100644 --- a/server/compactor/src/test/resources/log4j2-test.properties +++ b/server/task-runner/src/test/resources/log4j2-test.properties @@ -30,6 +30,6 @@ appender.console.layout.pattern = [%t] %-5p %c %x - %m%n #logger.01.name = org.apache.accumulo.core.iterators.system.VisibilityFilter #logger.01.level = fatal -rootLogger.level = info +rootLogger.level = debug rootLogger.appenderRef.console.ref = STDOUT diff --git a/src/build/ci/find-unapproved-junit.sh b/src/build/ci/find-unapproved-junit.sh index 39e6102c67f..8801b72d72c 100755 --- a/src/build/ci/find-unapproved-junit.sh +++ b/src/build/ci/find-unapproved-junit.sh @@ -23,8 +23,8 @@ # APIs to new jupiter tests. NUM_EXPECTED=0 ALLOWED=( - server/compaction-coordinator/src/test/java/org/apache/accumulo/coordinator/CompactionCoordinatorTest.java - server/compactor/src/test/java/org/apache/accumulo/compactor/CompactorTest.java + server/manager/src/test/java/org/apache/accumulo/compaction/CompactionCoordinatorTest.java + server/task-runner/src/test/java/org/apache/accumulo/tasks/TaskRunnerTest.java ) ALLOWED_PIPE_SEP=$({ for x in "${ALLOWED[@]}"; do echo "$x"; done; } | paste -sd'|') diff --git a/test/pom.xml b/test/pom.xml index 2ffd6b1af49..944e7503c1a 100644 --- a/test/pom.xml +++ b/test/pom.xml @@ -78,10 +78,6 @@ io.opentelemetry opentelemetry-context - - org.apache.accumulo - accumulo-compactor - org.apache.accumulo accumulo-core @@ -119,6 +115,10 @@ org.apache.accumulo accumulo-start + + org.apache.accumulo + accumulo-task-runner + org.apache.accumulo accumulo-tserver diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionProgressIT.java b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionProgressIT.java index b77ec14a8ea..7bdeff1d16b 100644 --- a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionProgressIT.java +++ b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionProgressIT.java @@ -58,7 +58,7 @@ /** * Tests that external compactions report progress from start to finish. To prevent flaky test * failures, we only measure progress in quarter segments: STARTED, QUARTER, HALF, THREE_QUARTERS. - * We can detect if the compaction finished without errors but the coordinator will never report + * We can detect if the compaction finished without errors but the TaskManager will never report * 100% progress since it will remove the ECID upon completion. The {@link SlowIterator} is used to * control the length of time it takes to complete the compaction. */ @@ -132,12 +132,12 @@ public Thread startChecker() { private void checkRunning() throws TException { ServerContext ctx = getCluster().getServerContext(); - Optional coordinatorHost = ExternalCompactionUtil.findCompactionCoordinator(ctx); - if (coordinatorHost.isEmpty()) { - throw new TTransportException("Unable to get CompactionCoordinator address from ZooKeeper"); + Optional taskManagerHost = ExternalCompactionUtil.findTaskManager(ctx); + if (taskManagerHost.isEmpty()) { + throw new TTransportException("Unable to get TaskManager address from ZooKeeper"); } - var ecList = getRunningCompactions(ctx, coordinatorHost); + var ecList = getRunningCompactions(ctx, taskManagerHost); var ecMap = ecList.getCompactions(); if (ecMap != null) { ecMap.forEach((ecid, ec) -> { diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionTestUtils.java b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionTestUtils.java index 6b014c0f21d..4f15a45a078 100644 --- a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionTestUtils.java +++ b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionTestUtils.java @@ -46,7 +46,6 @@ import org.apache.accumulo.core.client.admin.CompactionConfig; import org.apache.accumulo.core.client.admin.NewTableConfiguration; import org.apache.accumulo.core.clientImpl.ClientContext; -import org.apache.accumulo.core.compaction.thrift.CompactionCoordinatorService; import org.apache.accumulo.core.compaction.thrift.TCompactionState; import org.apache.accumulo.core.compaction.thrift.TExternalCompaction; import org.apache.accumulo.core.compaction.thrift.TExternalCompactionList; @@ -64,6 +63,12 @@ import org.apache.accumulo.core.rpc.clients.ThriftClientTypes; import org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner; import org.apache.accumulo.core.spi.compaction.SimpleCompactionDispatcher; +import org.apache.accumulo.core.tasks.TaskMessage; +import org.apache.accumulo.core.tasks.TaskMessageType; +import org.apache.accumulo.core.tasks.compaction.CompactionTasksCompleted; +import org.apache.accumulo.core.tasks.compaction.CompactionTasksRunning; +import org.apache.accumulo.core.tasks.thrift.Task; +import org.apache.accumulo.core.tasks.thrift.TaskManager; import org.apache.accumulo.core.trace.TraceUtil; import org.apache.accumulo.core.util.UtilWaitThread; import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil; @@ -227,12 +232,11 @@ public static void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuratio DefaultCompactionPlanner.class.getName()); cfg.setProperty("tserver.compaction.major.service.cs8.planner.opts.executors", "[{'name':'all', 'type': 'external','group': '" + GROUP8 + "'}]"); - cfg.setProperty(Property.COMPACTION_COORDINATOR_FINALIZER_COMPLETION_CHECK_INTERVAL, "5s"); - cfg.setProperty(Property.COMPACTION_COORDINATOR_DEAD_COMPACTOR_CHECK_INTERVAL, "5s"); - cfg.setProperty(Property.COMPACTION_COORDINATOR_TSERVER_COMPACTION_CHECK_INTERVAL, "3s"); - cfg.setProperty(Property.COMPACTOR_PORTSEARCH, "true"); - cfg.setProperty(Property.COMPACTOR_MIN_JOB_WAIT_TIME, "100ms"); - cfg.setProperty(Property.COMPACTOR_MAX_JOB_WAIT_TIME, "1s"); + cfg.setProperty(Property.TASK_MANAGER_DEAD_COMPACTOR_CHECK_INTERVAL, "5s"); + cfg.setProperty(Property.TASK_MANAGER_TSERVER_COMPACTION_CHECK_INTERVAL, "3s"); + cfg.setProperty(Property.TASK_RUNNER_PORTSEARCH, "true"); + cfg.setProperty(Property.TASK_RUNNER_MIN_JOB_WAIT_TIME, "100ms"); + cfg.setProperty(Property.TASK_RUNNER_MAX_JOB_WAIT_TIME, "1s"); cfg.setProperty(Property.GENERAL_THREADPOOL_SIZE, "10"); cfg.setProperty(Property.MANAGER_FATE_THREADPOOL_SIZE, "10"); cfg.setProperty(Property.MANAGER_TABLET_GROUP_WATCHER_INTERVAL, "1s"); @@ -241,26 +245,28 @@ public static void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuratio } public static TExternalCompactionList getRunningCompactions(ClientContext context, - Optional coordinatorHost) throws TException { - CompactionCoordinatorService.Client client = - ThriftUtil.getClient(ThriftClientTypes.COORDINATOR, coordinatorHost.orElseThrow(), context); + Optional taskManagerHost) throws TException { + TaskManager.Client client = ThriftUtil.getClient(ThriftClientTypes.TASK_MANAGER, + taskManagerHost.orElseThrow(), context); try { - TExternalCompactionList running = - client.getRunningCompactions(TraceUtil.traceInfo(), context.rpcCreds()); - return running; + Task task = client.getRunningTasks(TraceUtil.traceInfo(), context.rpcCreds()); + final CompactionTasksRunning list = + TaskMessage.fromThiftTask(task, TaskMessageType.COMPACTION_TASKS_RUNNING); + return list.getRunning(); } finally { ThriftUtil.returnClient(client, context); } } private static TExternalCompactionList getCompletedCompactions(ClientContext context, - Optional coordinatorHost) throws Exception { - CompactionCoordinatorService.Client client = - ThriftUtil.getClient(ThriftClientTypes.COORDINATOR, coordinatorHost.orElseThrow(), context); + Optional taskManagerHost) throws Exception { + TaskManager.Client client = ThriftUtil.getClient(ThriftClientTypes.TASK_MANAGER, + taskManagerHost.orElseThrow(), context); try { - TExternalCompactionList completed = - client.getCompletedCompactions(TraceUtil.traceInfo(), context.rpcCreds()); - return completed; + Task task = client.getCompletedTasks(TraceUtil.traceInfo(), context.rpcCreds()); + final CompactionTasksCompleted list = + TaskMessage.fromThiftTask(task, TaskMessageType.COMPACTION_TASKS_COMPLETED); + return list.getCompleted(); } finally { ThriftUtil.returnClient(client, context); } @@ -314,13 +320,13 @@ public static void waitForRunningCompactions(ServerContext ctx, TableId tid, public static int confirmCompactionRunning(ServerContext ctx, Set ecids) throws Exception { int matches = 0; - Optional coordinatorHost = ExternalCompactionUtil.findCompactionCoordinator(ctx); - if (coordinatorHost.isEmpty()) { - throw new TTransportException("Unable to get CompactionCoordinator address from ZooKeeper"); + Optional taskManagerHost = ExternalCompactionUtil.findTaskManager(ctx); + if (taskManagerHost.isEmpty()) { + throw new TTransportException("Unable to get TaskManager address from ZooKeeper"); } while (matches == 0) { TExternalCompactionList running = - ExternalCompactionTestUtils.getRunningCompactions(ctx, coordinatorHost); + ExternalCompactionTestUtils.getRunningCompactions(ctx, taskManagerHost); if (running.getCompactions() != null) { for (ExternalCompactionId ecid : ecids) { TExternalCompaction tec = running.getCompactions().get(ecid.canonical()); @@ -339,24 +345,24 @@ public static int confirmCompactionRunning(ServerContext ctx, Set ecids, TCompactionState expectedState) throws Exception { - Optional coordinatorHost = ExternalCompactionUtil.findCompactionCoordinator(ctx); - if (coordinatorHost.isEmpty()) { - throw new TTransportException("Unable to get CompactionCoordinator address from ZooKeeper"); + Optional taskManagerHost = ExternalCompactionUtil.findTaskManager(ctx); + if (taskManagerHost.isEmpty()) { + throw new TTransportException("Unable to get TaskManager address from ZooKeeper"); } // The running compaction should be removed TExternalCompactionList running = - ExternalCompactionTestUtils.getRunningCompactions(ctx, coordinatorHost); + ExternalCompactionTestUtils.getRunningCompactions(ctx, taskManagerHost); while (running.getCompactions() != null && running.getCompactions().keySet().stream() .anyMatch((e) -> ecids.contains(ExternalCompactionId.of(e)))) { - running = ExternalCompactionTestUtils.getRunningCompactions(ctx, coordinatorHost); + running = ExternalCompactionTestUtils.getRunningCompactions(ctx, taskManagerHost); } // The compaction should be in the completed list with the expected state TExternalCompactionList completed = - ExternalCompactionTestUtils.getCompletedCompactions(ctx, coordinatorHost); + ExternalCompactionTestUtils.getCompletedCompactions(ctx, taskManagerHost); while (completed.getCompactions() == null) { UtilWaitThread.sleep(50); - completed = ExternalCompactionTestUtils.getCompletedCompactions(ctx, coordinatorHost); + completed = ExternalCompactionTestUtils.getCompletedCompactions(ctx, taskManagerHost); } for (ExternalCompactionId e : ecids) { TExternalCompaction tec = completed.getCompactions().get(e.canonical()); diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_1_IT.java b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_1_IT.java index af13e5e576b..afca8b0c85f 100644 --- a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_1_IT.java +++ b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_1_IT.java @@ -47,7 +47,6 @@ import java.util.TreeSet; import java.util.stream.Collectors; -import org.apache.accumulo.compactor.ExtCEnv.CompactorIterEnv; import org.apache.accumulo.core.client.Accumulo; import org.apache.accumulo.core.client.AccumuloClient; import org.apache.accumulo.core.client.AccumuloException; @@ -75,6 +74,7 @@ import org.apache.accumulo.harness.SharedMiniClusterBase; import org.apache.accumulo.minicluster.ServerType; import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl; +import org.apache.accumulo.tasks.jobs.ExtCEnv.CompactorIterEnv; import org.apache.accumulo.test.functional.CompactionIT.ErrorThrowingSelector; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.Text; @@ -226,7 +226,7 @@ public void testCompactionAndCompactorDies() throws Exception { // Kill the compactor getCluster().getClusterControl().stop(ServerType.COMPACTOR); - // DeadCompactionDetector in the CompactionCoordinator should fail the compaction and delete + // DeadCompactionDetector in the TaskManager should fail the compaction and delete // it from the tablet. ExternalCompactionTestUtils.waitForRunningCompactions(getCluster().getServerContext(), tid, ecids); diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_3_IT.java b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_3_IT.java index 52da17e0095..e92d6b4125a 100644 --- a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_3_IT.java +++ b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_3_IT.java @@ -139,7 +139,7 @@ public void testMergeCancelsExternalCompaction() throws Exception { } @Test - public void testCoordinatorRestartsDuringCompaction() throws Exception { + public void testTaskManagerRestartsDuringCompaction() throws Exception { String table1 = this.getUniqueNames(1)[0]; try (AccumuloClient client = @@ -155,7 +155,7 @@ public void testCoordinatorRestartsDuringCompaction() throws Exception { Set ecids = waitForCompactionStartAndReturnEcids(getCluster().getServerContext(), tid); - // Stop the Manager (Coordinator) + // Stop the Manager (TaskManager) getCluster().getClusterControl().stop(ServerType.MANAGER); // Restart the Manager while the compaction is running @@ -169,13 +169,11 @@ public void testCoordinatorRestartsDuringCompaction() throws Exception { TExternalCompactionList running = null; while (running == null) { try { - Optional coordinatorHost = - ExternalCompactionUtil.findCompactionCoordinator(ctx); - if (coordinatorHost.isEmpty()) { - throw new TTransportException( - "Unable to get CompactionCoordinator address from ZooKeeper"); + Optional taskManagerHost = ExternalCompactionUtil.findTaskManager(ctx); + if (taskManagerHost.isEmpty()) { + throw new TTransportException("Unable to get TaskManager address from ZooKeeper"); } - running = getRunningCompactions(ctx, coordinatorHost); + running = getRunningCompactions(ctx, taskManagerHost); } catch (TException t) { running = null; Thread.sleep(2000); diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalDoNothingCompactor.java b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalDoNothingCompactor.java index 14b3329faae..2b8653a3454 100644 --- a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalDoNothingCompactor.java +++ b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalDoNothingCompactor.java @@ -18,25 +18,71 @@ */ package org.apache.accumulo.test.compaction; -import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static java.util.concurrent.TimeUnit.SECONDS; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.UUID; import java.util.concurrent.atomic.AtomicReference; -import java.util.concurrent.atomic.LongAdder; +import java.util.function.Supplier; -import org.apache.accumulo.compactor.Compactor; import org.apache.accumulo.core.cli.ConfigOpts; -import org.apache.accumulo.core.compaction.thrift.CompactorService.Iface; import org.apache.accumulo.core.compaction.thrift.TCompactionState; import org.apache.accumulo.core.compaction.thrift.TCompactionStatusUpdate; -import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob; +import org.apache.accumulo.core.metadata.schema.ExternalCompactionId; +import org.apache.accumulo.core.tasks.compaction.CompactionTask; +import org.apache.accumulo.core.tasks.thrift.TaskRunner.Iface; import org.apache.accumulo.core.util.UtilWaitThread; import org.apache.accumulo.server.compaction.FileCompactor.CompactionCanceledException; +import org.apache.accumulo.server.compaction.RetryableThriftCall.RetriesExceededException; +import org.apache.accumulo.tasks.TaskRunner; +import org.apache.accumulo.tasks.TaskRunnerProcess; +import org.apache.accumulo.tasks.jobs.CompactionJob; +import org.apache.accumulo.tasks.jobs.Job; +import org.apache.thrift.TException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class ExternalDoNothingCompactor extends Compactor implements Iface { +public class ExternalDoNothingCompactor extends TaskRunner implements Iface { + + public static class DoNothingCompactionJob extends CompactionJob { + + public DoNothingCompactionJob(TaskRunnerProcess worker, CompactionTask msg, + AtomicReference currentCompactionId) throws TException { + super(worker, msg, currentCompactionId); + } + + @Override + public Runnable createJob() throws TException { + // Set this to true so that only 1 external compaction is run + getTaskWorker().shutdown(); + + return () -> { + try { + LOG.info("Starting up compaction runnable for job: {}", this.getJobDetails()); + TCompactionStatusUpdate update = new TCompactionStatusUpdate(); + update.setState(TCompactionState.STARTED); + update.setMessage("Compaction started"); + updateCompactionState(this.getJobDetails(), update); + + LOG.info("Starting compactor"); + started.countDown(); + + while (!JOB_HOLDER.isCancelled()) { + LOG.info("Sleeping while job is not cancelled"); + UtilWaitThread.sleep(1000); + } + // Compactor throws this exception when cancelled + throw new CompactionCanceledException(); + + } catch (Exception e) { + LOG.error("Compaction failed", e); + errorRef.set(e); + } finally { + stopped.countDown(); + } + }; + } + + } private static final Logger LOG = LoggerFactory.getLogger(ExternalDoNothingCompactor.class); @@ -45,46 +91,29 @@ public class ExternalDoNothingCompactor extends Compactor implements Iface { } @Override - protected void startCancelChecker(ScheduledThreadPoolExecutor schedExecutor, - long timeBetweenChecks) { - @SuppressWarnings("unused") - var future = schedExecutor.scheduleWithFixedDelay(this::checkIfCanceled, 0, 5000, MILLISECONDS); + protected long getTimeBetweenCancelChecks() { + return SECONDS.toMillis(5); } @Override - protected Runnable createCompactionJob(TExternalCompactionJob job, LongAdder totalInputEntries, - LongAdder totalInputBytes, CountDownLatch started, CountDownLatch stopped, - AtomicReference err) { - - // Set this to true so that only 1 external compaction is run - this.shutdown = true; - - return () -> { - try { - LOG.info("Starting up compaction runnable for job: {}", job); - TCompactionStatusUpdate update = new TCompactionStatusUpdate(); - update.setState(TCompactionState.STARTED); - update.setMessage("Compaction started"); - updateCompactionState(job, update); - - LOG.info("Starting compactor"); - started.countDown(); - - while (!JOB_HOLDER.isCancelled()) { - LOG.info("Sleeping while job is not cancelled"); - UtilWaitThread.sleep(1000); - } - // Compactor throws this exception when cancelled - throw new CompactionCanceledException(); - - } catch (Exception e) { - LOG.error("Compaction failed", e); - err.set(e); - } finally { - stopped.countDown(); - } - }; - + protected Job getNextJob(Supplier uuid) throws RetriesExceededException { + + // Get the next job, use the parent class for this + CompactionJob cjob = null; + do { + cjob = (CompactionJob) super.getNextJob(uuid); + UtilWaitThread.sleep(1000); + } while (!cjob.getJobDetails().isSetExternalCompactionId()); + + // Take the job details and return a DoNothingCompactionJob + try { + CompactionTask task = new CompactionTask(); + task.setCompactionJob(cjob.getJobDetails()); + return new DoNothingCompactionJob(this, task, new AtomicReference( + ExternalCompactionId.from(cjob.getJobDetails().getExternalCompactionId()))); + } catch (TException e) { + throw new RuntimeException("Error creating job", e); + } } public static void main(String[] args) throws Exception { diff --git a/test/src/main/java/org/apache/accumulo/test/functional/CompactionIT.java b/test/src/main/java/org/apache/accumulo/test/functional/CompactionIT.java index b45c0337bd2..61e8a88638a 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/CompactionIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/CompactionIT.java @@ -164,8 +164,8 @@ public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoo cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "15s"); cfg.setProperty(Property.TSERV_MAJC_DELAY, "1"); cfg.setProperty(Property.MANAGER_TABLET_GROUP_WATCHER_INTERVAL, "1s"); - cfg.setProperty(Property.COMPACTOR_MIN_JOB_WAIT_TIME, "100ms"); - cfg.setProperty(Property.COMPACTOR_MAX_JOB_WAIT_TIME, "1s"); + cfg.setProperty(Property.TASK_RUNNER_MIN_JOB_WAIT_TIME, "100ms"); + cfg.setProperty(Property.TASK_RUNNER_MAX_JOB_WAIT_TIME, "1s"); // use raw local file system so walogs sync and flush will work hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName()); } diff --git a/test/src/main/java/org/apache/accumulo/test/start/KeywordStartIT.java b/test/src/main/java/org/apache/accumulo/test/start/KeywordStartIT.java index 01298502d45..959b962fc4f 100644 --- a/test/src/main/java/org/apache/accumulo/test/start/KeywordStartIT.java +++ b/test/src/main/java/org/apache/accumulo/test/start/KeywordStartIT.java @@ -37,7 +37,6 @@ import java.util.Map.Entry; import java.util.TreeMap; -import org.apache.accumulo.compactor.CompactorExecutable; import org.apache.accumulo.core.file.rfile.CreateEmpty; import org.apache.accumulo.core.file.rfile.GenerateSplits; import org.apache.accumulo.core.file.rfile.PrintInfo; @@ -67,6 +66,7 @@ import org.apache.accumulo.shell.Shell; import org.apache.accumulo.start.Main; import org.apache.accumulo.start.spi.KeywordExecutable; +import org.apache.accumulo.tasks.TaskRunnerExecutable; import org.apache.accumulo.tserver.ScanServerExecutable; import org.apache.accumulo.tserver.TServerExecutable; import org.apache.accumulo.tserver.TabletServer; @@ -128,7 +128,7 @@ public void testExpectedClasses() { expectSet.put("admin", Admin.class); expectSet.put("check-compaction-config", CheckCompactionConfig.class); expectSet.put("check-server-config", CheckServerConfig.class); - expectSet.put("compactor", CompactorExecutable.class); + expectSet.put("compactor", TaskRunnerExecutable.class); expectSet.put("create-empty", CreateEmpty.class); expectSet.put("create-token", CreateToken.class); expectSet.put("dump-zoo", DumpZookeeper.class);