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 extends TaskMessage> taskClass;
+
+ TaskMessageType(Class extends TaskMessage> taskClass) {
+ this.taskClass = taskClass;
+ }
+
+ public Class extends TaskMessage> 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