diff --git a/modules/api/src/main/java/org/apache/ignite/compute/AllNodesBroadcastJobTarget.java b/modules/api/src/main/java/org/apache/ignite/compute/AllNodesBroadcastJobTarget.java new file mode 100644 index 00000000000..2c7a87e5f61 --- /dev/null +++ b/modules/api/src/main/java/org/apache/ignite/compute/AllNodesBroadcastJobTarget.java @@ -0,0 +1,48 @@ +/* + * 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 + * + * http://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.ignite.compute; + +import java.util.Objects; +import java.util.Set; +import org.apache.ignite.network.ClusterNode; + +/** + * All nodes broadcast execution target. Indicates that the job will be executed on all nodes from the provided set. + */ +public class AllNodesBroadcastJobTarget implements BroadcastJobTarget { + private final Set nodes; + + AllNodesBroadcastJobTarget(Set nodes) { + Objects.requireNonNull(nodes); + + if (nodes.isEmpty()) { + throw new IllegalArgumentException("Nodes collection must not be empty."); + } + + this.nodes = nodes; + } + + /** + * Returns a set of nodes to execute the jobs on. + * + * @return A set of nodes to execute the jobs on. + */ + public Set nodes() { + return nodes; + } +} diff --git a/modules/api/src/main/java/org/apache/ignite/compute/BroadcastExecution.java b/modules/api/src/main/java/org/apache/ignite/compute/BroadcastExecution.java new file mode 100644 index 00000000000..4ac82e94de3 --- /dev/null +++ b/modules/api/src/main/java/org/apache/ignite/compute/BroadcastExecution.java @@ -0,0 +1,42 @@ +/* + * 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 + * + * http://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.ignite.compute; + +import java.util.Collection; +import java.util.concurrent.CompletableFuture; + +/** + * Broadcast job control object, provides information about the job executions and results. + * + * @param Job result type. + */ +public interface BroadcastExecution { + /** + * Returns a collection of individual job executions. + * + * @return A collection of individual job executions. + */ + Collection> executions(); + + /** + * Returns all jobs execution results. + * + * @return Future which will be completed with a collection of all jobs results when all jobs have completed successfully. + */ + CompletableFuture> resultsAsync(); +} diff --git a/modules/api/src/main/java/org/apache/ignite/compute/BroadcastJobTarget.java b/modules/api/src/main/java/org/apache/ignite/compute/BroadcastJobTarget.java new file mode 100644 index 00000000000..8d841f14296 --- /dev/null +++ b/modules/api/src/main/java/org/apache/ignite/compute/BroadcastJobTarget.java @@ -0,0 +1,66 @@ +/* + * 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 + * + * http://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.ignite.compute; + +import java.util.Collection; +import java.util.HashSet; +import java.util.Set; +import org.apache.ignite.network.ClusterNode; + +/** + * Broadcast job execution target. + * + *

Determines the rules for selecting nodes to execute a job. + */ +public interface BroadcastJobTarget { + /** + * Creates a job target for all nodes from the provided collection. + * + *

This target determines that a job will be executed on all nodes from a given nodes. + * + * @param nodes Collection of nodes. + * @return Job target. + */ + static BroadcastJobTarget nodes(ClusterNode... nodes) { + return new AllNodesBroadcastJobTarget(Set.of(nodes)); + } + + /** + * Creates a job target for all nodes from the provided collection. + * + *

This target determines that a job will be executed on all nodes from a given collection. + * + * @param nodes Collection of nodes. + * @return Job target. + */ + static BroadcastJobTarget nodes(Collection nodes) { + return new AllNodesBroadcastJobTarget(new HashSet<>(nodes)); + } + + /** + * Creates a job target for all nodes from the provided collection. + * + *

This target determines that a job will be executed on all nodes from a given set. + * + * @param nodes Collection of nodes. + * @return Job target. + */ + static BroadcastJobTarget nodes(Set nodes) { + return new AllNodesBroadcastJobTarget(nodes); + } +} diff --git a/modules/api/src/main/java/org/apache/ignite/compute/IgniteCompute.java b/modules/api/src/main/java/org/apache/ignite/compute/IgniteCompute.java index cdb5da87889..08ec0029a85 100644 --- a/modules/api/src/main/java/org/apache/ignite/compute/IgniteCompute.java +++ b/modules/api/src/main/java/org/apache/ignite/compute/IgniteCompute.java @@ -17,19 +17,11 @@ package org.apache.ignite.compute; -import static java.util.concurrent.CompletableFuture.allOf; -import static java.util.function.Function.identity; -import static java.util.stream.Collectors.toMap; - -import java.util.HashMap; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Set; +import java.util.Collection; import java.util.concurrent.CompletableFuture; import org.apache.ignite.compute.task.MapReduceTask; import org.apache.ignite.compute.task.TaskExecution; import org.apache.ignite.lang.CancellationToken; -import org.apache.ignite.network.ClusterNode; import org.jetbrains.annotations.Nullable; /** @@ -47,174 +39,203 @@ public interface IgniteCompute { * @param target Execution target. * @param descriptor Job descriptor. * @param arg Argument of the job. - * @return Job execution object. + * @return Future of the job execution object which will be completed when the job is submitted. */ - JobExecution submit( + default CompletableFuture> submitAsync( JobTarget target, JobDescriptor descriptor, @Nullable T arg - ); + ) { + return submitAsync(target, descriptor, arg, null); + } /** - * Submits a {@link ComputeJob} of the given class for an execution on a single node from a set of candidate nodes. A shortcut for - * {@code submit(...).resultAsync()}. + * Submits a {@link ComputeJob} of the given class for an execution on a single node from a set of candidate nodes. * * @param Job argument (T)ype. * @param Job (R)esult type. * @param target Execution target. * @param descriptor Job descriptor. * @param arg Argument of the job. - * @return Job result future. + * @param cancellationToken Cancellation token or {@code null}. + * @return Future of the job execution object which will be completed when the job is submitted. */ - default CompletableFuture executeAsync( + CompletableFuture> submitAsync( JobTarget target, JobDescriptor descriptor, + @Nullable T arg, + @Nullable CancellationToken cancellationToken + ); + + /** + * Submits a {@link ComputeJob} of the given class for an execution on all nodes in the given node set. + * + * @param Job argument (T)ype. + * @param Job (R)esult type. + * @param target Broadcast execution target. + * @param descriptor Job descriptor. + * @param arg Argument of the job. + * @return Future of the broadcast job execution object which will be completed when all the jobs are submitted. + */ + default CompletableFuture> submitAsync( + BroadcastJobTarget target, + JobDescriptor descriptor, @Nullable T arg ) { - return executeAsync(target, descriptor, null, arg); + return submitAsync(target, descriptor, arg, null); } /** - * Submits a {@link ComputeJob} of the given class for an execution on a single node from a set of candidate nodes. A shortcut for - * {@code submit(...).resultAsync()}. + * Submits a {@link ComputeJob} of the given class for an execution on all nodes in the given node set. * * @param Job argument (T)ype. * @param Job (R)esult type. - * @param target Execution target. + * @param target Broadcast execution target. * @param descriptor Job descriptor. - * @param cancellationToken Cancellation token or {@code null}. * @param arg Argument of the job. - * @return Job result future. + * @param cancellationToken Cancellation token or {@code null}. + * @return Future of the broadcast job execution object which will be completed when all the jobs are submitted. */ - CompletableFuture executeAsync( - JobTarget target, + CompletableFuture> submitAsync( + BroadcastJobTarget target, JobDescriptor descriptor, - @Nullable CancellationToken cancellationToken, - @Nullable T arg + @Nullable T arg, + @Nullable CancellationToken cancellationToken ); /** - * Executes a {@link ComputeJob} of the given class on a single node from a set of candidate nodes. + * Submits a {@link ComputeJob} of the given class for an execution on a single node from a set of candidate nodes. A shortcut for + * {@code submit(...).resultAsync()}. * * @param Job argument (T)ype. * @param Job (R)esult type. * @param target Execution target. * @param descriptor Job descriptor. * @param arg Argument of the job. - * @return Job result. - * @throws ComputeException If there is any problem executing the job. + * @return Job result future. */ - default R execute( + default CompletableFuture executeAsync( JobTarget target, JobDescriptor descriptor, @Nullable T arg ) { - return execute(target, descriptor, null, arg); + return executeAsync(target, descriptor, arg, null); } /** - * Executes a {@link ComputeJob} of the given class on a single node from a set of candidate nodes. + * Submits a {@link ComputeJob} of the given class for an execution on a single node from a set of candidate nodes. A shortcut for + * {@code submit(...).resultAsync()}. * * @param Job argument (T)ype. * @param Job (R)esult type. * @param target Execution target. * @param descriptor Job descriptor. - * @param cancellationToken Cancellation token or {@code null}. * @param arg Argument of the job. - * @return Job result. - * @throws ComputeException If there is any problem executing the job. + * @param cancellationToken Cancellation token or {@code null}. + * @return Job result future. */ - R execute( + default CompletableFuture executeAsync( JobTarget target, JobDescriptor descriptor, - @Nullable CancellationToken cancellationToken, - @Nullable T arg - ); + @Nullable T arg, + @Nullable CancellationToken cancellationToken + ) { + return submitAsync(target, descriptor, arg, cancellationToken).thenCompose(JobExecution::resultAsync); + } /** - * Submits a {@link ComputeJob} of the given class for an execution on all nodes in the given node set. + * Executes a {@link ComputeJob} of the given class on all nodes in the given node set. * * @param Job argument (T)ype. * @param Job (R)esult type. - * @param nodes Nodes to execute the job on. + * @param target Broadcast execution target. * @param descriptor Job descriptor. * @param arg Argument of the job. - * @return Map from node to job execution object. + * @return Job results future. */ - Map> submitBroadcast( - Set nodes, + default CompletableFuture> executeAsync( + BroadcastJobTarget target, JobDescriptor descriptor, @Nullable T arg - ); + ) { + return executeAsync(target, descriptor, arg, null); + } /** * Executes a {@link ComputeJob} of the given class on all nodes in the given node set. * * @param Job argument (T)ype. * @param Job (R)esult type. - * @param nodes Nodes to execute the job on. + * @param target Broadcast execution target. * @param descriptor Job descriptor. * @param arg Argument of the job. - * @return Map from node to job result. + * @param cancellationToken Cancellation token or {@code null}. + * @return Job results future. */ - default CompletableFuture> executeBroadcastAsync( - Set nodes, + default CompletableFuture> executeAsync( + BroadcastJobTarget target, JobDescriptor descriptor, - @Nullable T arg + @Nullable T arg, + @Nullable CancellationToken cancellationToken ) { - return executeBroadcastAsync(nodes, descriptor, null, arg); + return submitAsync(target, descriptor, arg, cancellationToken).thenCompose(BroadcastExecution::resultsAsync); } /** - * Executes a {@link ComputeJob} of the given class on all nodes in the given node set. + * Executes a {@link ComputeJob} of the given class on a single node from a set of candidate nodes. * * @param Job argument (T)ype. * @param Job (R)esult type. - * @param nodes Nodes to execute the job on. + * @param target Execution target. * @param descriptor Job descriptor. - * @param cancellationToken Cancellation token or {@code null}. * @param arg Argument of the job. - * @return Map from node to job result. + * @return Job result. + * @throws ComputeException If there is any problem executing the job. */ - default CompletableFuture> executeBroadcastAsync( - Set nodes, + default R execute( + JobTarget target, JobDescriptor descriptor, - @Nullable CancellationToken cancellationToken, @Nullable T arg ) { - Map> futures = nodes.stream() - .collect(toMap(identity(), node -> executeAsync(JobTarget.node(node), descriptor, cancellationToken, arg))); - - return allOf(futures.values().toArray(CompletableFuture[]::new)) - .thenApply(ignored -> { - Map map = new HashMap<>(); - - for (Entry> entry : futures.entrySet()) { - map.put(entry.getKey(), entry.getValue().join()); - } - - return map; - } - ); + return execute(target, descriptor, arg, null); } + /** + * Executes a {@link ComputeJob} of the given class on a single node from a set of candidate nodes. + * + * @param Job argument (T)ype. + * @param Job (R)esult type. + * @param target Execution target. + * @param descriptor Job descriptor. + * @param arg Argument of the job. + * @param cancellationToken Cancellation token or {@code null}. + * @return Job result. + * @throws ComputeException If there is any problem executing the job. + */ + R execute( + JobTarget target, + JobDescriptor descriptor, + @Nullable T arg, + @Nullable CancellationToken cancellationToken + ); + /** * Executes a {@link ComputeJob} of the given class on all nodes in the given node set. * * @param Job argument (T)ype. * @param Job (R)esult type. - * @param nodes Nodes to execute the job on. + * @param target Broadcast execution target. * @param descriptor Job descriptor. * @param arg Argument of the job. - * @return Map from node to job result. + * @return Collection of results. * @throws ComputeException If there is any problem executing the job. */ - default Map executeBroadcast( - Set nodes, + default Collection execute( + BroadcastJobTarget target, JobDescriptor descriptor, @Nullable T arg ) { - return executeBroadcast(nodes, descriptor, null, arg); + return execute(target, descriptor, arg, null); } /** @@ -222,26 +243,31 @@ default Map executeBroadcast( * * @param Job argument (T)ype. * @param Job (R)esult type. - * @param nodes Nodes to execute the job on. + * @param target Broadcast execution target. * @param descriptor Job descriptor. - * @param cancellationToken Cancellation token or {@code null}. * @param arg Argument of the job. - * @return Map from node to job result. + * @param cancellationToken Cancellation token or {@code null}. + * @return Collection of results. * @throws ComputeException If there is any problem executing the job. */ - default Map executeBroadcast( - Set nodes, + Collection execute( + BroadcastJobTarget target, JobDescriptor descriptor, - @Nullable CancellationToken cancellationToken, - @Nullable T arg - ) { - Map map = new HashMap<>(); - - for (ClusterNode node : nodes) { - map.put(node, execute(JobTarget.node(node), descriptor, cancellationToken, arg)); - } + @Nullable T arg, + @Nullable CancellationToken cancellationToken + ); - return map; + /** + * Submits a {@link MapReduceTask} of the given class for an execution. + * + * @param Job argument (T)ype. + * @param Job (R)esult type. + * @param taskDescriptor Map reduce task descriptor. + * @param arg Task argument. + * @return Task execution interface. + */ + default TaskExecution submitMapReduce(TaskDescriptor taskDescriptor, @Nullable T arg) { + return submitMapReduce(taskDescriptor, arg, null); } /** @@ -251,9 +277,14 @@ default Map executeBroadcast( * @param Job (R)esult type. * @param taskDescriptor Map reduce task descriptor. * @param arg Task argument. + * @param cancellationToken Cancellation token or {@code null}. * @return Task execution interface. */ - TaskExecution submitMapReduce(TaskDescriptor taskDescriptor, @Nullable T arg); + TaskExecution submitMapReduce( + TaskDescriptor taskDescriptor, + @Nullable T arg, + @Nullable CancellationToken cancellationToken + ); /** * Submits a {@link MapReduceTask} of the given class for an execution. A shortcut for {@code submitMapReduce(...).resultAsync()}. @@ -265,7 +296,7 @@ default Map executeBroadcast( * @return Task result future. */ default CompletableFuture executeMapReduceAsync(TaskDescriptor taskDescriptor, @Nullable T arg) { - return executeMapReduceAsync(taskDescriptor, null, arg); + return executeMapReduceAsync(taskDescriptor, arg, null); } /** @@ -274,15 +305,17 @@ default CompletableFuture executeMapReduceAsync(TaskDescriptor t * @param Job argument (T)ype. * @param Job (R)esult type. * @param taskDescriptor Map reduce task descriptor. - * @param cancellationToken Cancellation token or {@code null}. * @param arg Task argument. + * @param cancellationToken Cancellation token or {@code null}. * @return Task result future. */ - CompletableFuture executeMapReduceAsync( + default CompletableFuture executeMapReduceAsync( TaskDescriptor taskDescriptor, - @Nullable CancellationToken cancellationToken, - @Nullable T arg - ); + @Nullable T arg, + @Nullable CancellationToken cancellationToken + ) { + return submitMapReduce(taskDescriptor, arg, cancellationToken).resultAsync(); + } /** * Executes a {@link MapReduceTask} of the given class. @@ -295,7 +328,7 @@ CompletableFuture executeMapReduceAsync( * @throws ComputeException If there is any problem executing the task. */ default R executeMapReduce(TaskDescriptor taskDescriptor, @Nullable T arg) { - return executeMapReduce(taskDescriptor, null, arg); + return executeMapReduce(taskDescriptor, arg, null); } /** @@ -304,11 +337,10 @@ default R executeMapReduce(TaskDescriptor taskDescriptor, @Nullable * @param Job argument (T)ype. * @param Job (R)esult type. * @param taskDescriptor Map reduce task descriptor. - * @param cancellationToken Cancellation token or {@code null}. * @param arg Task argument. + * @param cancellationToken Cancellation token or {@code null}. * @return Task result. * @throws ComputeException If there is any problem executing the task. */ - R executeMapReduce(TaskDescriptor taskDescriptor, @Nullable CancellationToken cancellationToken, @Nullable T arg); - + R executeMapReduce(TaskDescriptor taskDescriptor, @Nullable T arg, @Nullable CancellationToken cancellationToken); } diff --git a/modules/api/src/main/java/org/apache/ignite/compute/JobExecution.java b/modules/api/src/main/java/org/apache/ignite/compute/JobExecution.java index 485b24cf6d4..740f1167193 100644 --- a/modules/api/src/main/java/org/apache/ignite/compute/JobExecution.java +++ b/modules/api/src/main/java/org/apache/ignite/compute/JobExecution.java @@ -19,6 +19,7 @@ import java.util.UUID; import java.util.concurrent.CompletableFuture; +import org.apache.ignite.network.ClusterNode; import org.jetbrains.annotations.Nullable; /** @@ -52,15 +53,6 @@ public interface JobExecution { return stateAsync().thenApply(state -> state != null ? state.id() : null); } - /** - * Cancels the job. - * - * @return The future which will be completed with {@code true} when the job is cancelled, {@code false} when the job couldn't be - * cancelled (if it's already completed or in the process of cancelling), or {@code null} if the job no longer exists due to - * exceeding the retention time limit. - */ - CompletableFuture<@Nullable Boolean> cancelAsync(); - /** * Changes job priority. After priority change job will be the last in the queue of jobs with the same priority. * @@ -70,4 +62,11 @@ public interface JobExecution { * retention time limit. */ CompletableFuture<@Nullable Boolean> changePriorityAsync(int newPriority); + + /** + * A cluster node on which this job is executing. + * + * @return Node instance. + */ + ClusterNode node(); } diff --git a/modules/api/src/main/java/org/apache/ignite/compute/task/MapReduceJob.java b/modules/api/src/main/java/org/apache/ignite/compute/task/MapReduceJob.java index 3819bc24580..957b947b8ad 100644 --- a/modules/api/src/main/java/org/apache/ignite/compute/task/MapReduceJob.java +++ b/modules/api/src/main/java/org/apache/ignite/compute/task/MapReduceJob.java @@ -27,7 +27,7 @@ /** * A description of the job to be submitted as a result of the split step of the {@link MapReduceTask}. Reflects the parameters of the - * {@link org.apache.ignite.compute.IgniteCompute#submit(JobTarget, JobDescriptor, Object)} method. + * {@link org.apache.ignite.compute.IgniteCompute#submitAsync(JobTarget, JobDescriptor, Object)} method. */ public class MapReduceJob { private final Set nodes; diff --git a/modules/api/src/main/java/org/apache/ignite/compute/task/TaskExecution.java b/modules/api/src/main/java/org/apache/ignite/compute/task/TaskExecution.java index a51805de74f..5d232c0fb6a 100644 --- a/modules/api/src/main/java/org/apache/ignite/compute/task/TaskExecution.java +++ b/modules/api/src/main/java/org/apache/ignite/compute/task/TaskExecution.java @@ -77,15 +77,6 @@ public interface TaskExecution { return stateAsync().thenApply(state -> state != null ? state.id() : null); } - /** - * Cancels the task. - * - * @return The future which will be completed with {@code true} when the task is cancelled, {@code false} when the task couldn't be - * cancelled (if it's already completed or in the process of cancelling), or {@code null} if the task no longer exists due to - * exceeding the retention time limit. - */ - CompletableFuture<@Nullable Boolean> cancelAsync(); - /** * Changes task priority. After priority change task will be the last in the queue of tasks with the same priority. * diff --git a/modules/client-common/src/main/java/org/apache/ignite/internal/compute/BroadcastJobExecutionImpl.java b/modules/client-common/src/main/java/org/apache/ignite/internal/compute/BroadcastJobExecutionImpl.java new file mode 100644 index 00000000000..565b48d197a --- /dev/null +++ b/modules/client-common/src/main/java/org/apache/ignite/internal/compute/BroadcastJobExecutionImpl.java @@ -0,0 +1,62 @@ +/* + * 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 + * + * http://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.ignite.internal.compute; + +import static java.util.concurrent.CompletableFuture.allOf; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import org.apache.ignite.compute.BroadcastExecution; +import org.apache.ignite.compute.JobExecution; + +/** + * {@link BroadcastExecution} implementation. Contains a collection of individual executions. + * + * @param Result type. + */ +public class BroadcastJobExecutionImpl implements BroadcastExecution { + private final Collection> executions; + + public BroadcastJobExecutionImpl(Collection> executions) { + this.executions = executions; + } + + @Override + public Collection> executions() { + return List.copyOf(executions); + } + + @Override + public CompletableFuture> resultsAsync() { + CompletableFuture[] futures = executions.stream() + .map(JobExecution::resultAsync) + .toArray(CompletableFuture[]::new); + + return allOf(futures).thenApply(ignored -> { + ArrayList result = new ArrayList<>(futures.length); + + for (CompletableFuture future : futures) { + result.add(future.join()); + } + + return result; + }); + } +} diff --git a/modules/compute/src/main/java/org/apache/ignite/internal/compute/FailedExecution.java b/modules/client-common/src/main/java/org/apache/ignite/internal/compute/FailedExecution.java similarity index 88% rename from modules/compute/src/main/java/org/apache/ignite/internal/compute/FailedExecution.java rename to modules/client-common/src/main/java/org/apache/ignite/internal/compute/FailedExecution.java index ec20b042aa4..36a6e7ea388 100644 --- a/modules/compute/src/main/java/org/apache/ignite/internal/compute/FailedExecution.java +++ b/modules/client-common/src/main/java/org/apache/ignite/internal/compute/FailedExecution.java @@ -18,10 +18,12 @@ package org.apache.ignite.internal.compute; import static java.util.concurrent.CompletableFuture.failedFuture; +import static org.apache.ignite.internal.util.ExceptionUtils.sneakyThrow; import java.util.concurrent.CompletableFuture; import org.apache.ignite.compute.JobExecution; import org.apache.ignite.compute.JobState; +import org.apache.ignite.network.ClusterNode; import org.jetbrains.annotations.Nullable; /** @@ -33,7 +35,7 @@ public class FailedExecution implements JobExecution { private final Throwable error; - FailedExecution(Throwable error) { + public FailedExecution(Throwable error) { this.error = error; } @@ -48,12 +50,12 @@ public CompletableFuture resultAsync() { } @Override - public CompletableFuture<@Nullable Boolean> cancelAsync() { + public CompletableFuture<@Nullable Boolean> changePriorityAsync(int newPriority) { return failedFuture(error); } @Override - public CompletableFuture<@Nullable Boolean> changePriorityAsync(int newPriority) { - return failedFuture(error); + public ClusterNode node() { + throw sneakyThrow(error); } } diff --git a/modules/client-common/src/main/java/org/apache/ignite/internal/compute/JobTaskStatusMapper.java b/modules/client-common/src/main/java/org/apache/ignite/internal/compute/JobTaskStatusMapper.java index a2b89e38891..af688084278 100644 --- a/modules/client-common/src/main/java/org/apache/ignite/internal/compute/JobTaskStatusMapper.java +++ b/modules/client-common/src/main/java/org/apache/ignite/internal/compute/JobTaskStatusMapper.java @@ -25,20 +25,42 @@ import static org.apache.ignite.compute.TaskStatus.QUEUED; import org.apache.ignite.compute.JobStatus; +import org.apache.ignite.compute.TaskState; import org.apache.ignite.compute.TaskStatus; +import org.jetbrains.annotations.Nullable; /** * Mapper for job status from\to task status. */ public class JobTaskStatusMapper { + /** + * Map task state to job state. + * + * @param taskState Task state. + * @return Mapped job state. + */ + public static @Nullable JobStateImpl toJobState(@Nullable TaskState taskState) { + if (taskState == null) { + return null; + + } + return JobStateImpl.builder() + .id(taskState.id()) + .createTime(taskState.createTime()) + .startTime(taskState.startTime()) + .finishTime(taskState.finishTime()) + .status(toJobStatus(taskState.status())) + .build(); + } + /** * Map task status to job status. * * @param taskStatus Task status. * @return Mapped job status. */ - public static JobStatus toJobStatus(TaskStatus taskStatus) { + private static JobStatus toJobStatus(TaskStatus taskStatus) { switch (taskStatus) { case QUEUED: return JobStatus.QUEUED; @@ -63,7 +85,7 @@ public static JobStatus toJobStatus(TaskStatus taskStatus) { * @param jobStatus Job status. * @return Mapped task status. */ - public static TaskStatus toTaskStatus(JobStatus jobStatus) { + static TaskStatus toTaskStatus(JobStatus jobStatus) { switch (jobStatus) { case QUEUED: return QUEUED; diff --git a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/compute/ClientComputeExecuteColocatedRequest.java b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/compute/ClientComputeExecuteColocatedRequest.java index caeba69cfd0..23564247a9a 100644 --- a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/compute/ClientComputeExecuteColocatedRequest.java +++ b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/compute/ClientComputeExecuteColocatedRequest.java @@ -17,6 +17,7 @@ package org.apache.ignite.client.handler.requests.compute; +import static org.apache.ignite.client.handler.requests.compute.ClientComputeExecuteRequest.packSubmitResult; import static org.apache.ignite.client.handler.requests.compute.ClientComputeExecuteRequest.sendResultAndState; import static org.apache.ignite.client.handler.requests.table.ClientTableCommon.readTableAsync; import static org.apache.ignite.client.handler.requests.table.ClientTableCommon.readTuple; @@ -73,12 +74,12 @@ public static CompletableFuture process( null, args); - JobExecution jobExecution = compute.wrapJobExecutionFuture(jobExecutionFut); - - sendResultAndState(jobExecution, notificationSender); + sendResultAndState(jobExecutionFut, notificationSender); //noinspection DataFlowIssue - return jobExecution.idAsync().thenAccept(out::packUuid); + return jobExecutionFut.thenCompose(execution -> + execution.idAsync().thenAccept(jobId -> packSubmitResult(out, jobId, execution.node())) + ); })); } } diff --git a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/compute/ClientComputeExecuteRequest.java b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/compute/ClientComputeExecuteRequest.java index 8342398fbaf..7d6dc7ac17a 100644 --- a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/compute/ClientComputeExecuteRequest.java +++ b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/compute/ClientComputeExecuteRequest.java @@ -17,12 +17,15 @@ package org.apache.ignite.client.handler.requests.compute; +import static java.util.concurrent.CompletableFuture.completedFuture; +import static org.apache.ignite.client.handler.requests.cluster.ClientClusterGetNodesRequest.packClusterNode; import static org.apache.ignite.client.handler.requests.compute.ClientComputeGetStateRequest.packJobState; import static org.apache.ignite.internal.client.proto.ClientComputeJobUnpacker.unpackJobArgumentWithoutMarshaller; import java.util.HashSet; import java.util.List; import java.util.Set; +import java.util.UUID; import java.util.concurrent.CompletableFuture; import org.apache.ignite.client.handler.NotificationSender; import org.apache.ignite.compute.JobExecution; @@ -71,10 +74,11 @@ public static CompletableFuture process( JobExecution execution = compute.executeAsyncWithFailover( candidates, deploymentUnits, jobClassName, options, null, arg ); - sendResultAndState(execution, notificationSender); + // TODO https://issues.apache.org/jira/browse/IGNITE-24184 + sendResultAndState(completedFuture(execution), notificationSender); //noinspection DataFlowIssue - return execution.idAsync().thenAccept(out::packUuid); + return execution.idAsync().thenAccept(jobId -> packSubmitResult(out, jobId, execution.node())); } private static Set unpackCandidateNodes(ClientMessageUnpacker in, ClusterService cluster) { @@ -104,16 +108,22 @@ private static Set unpackCandidateNodes(ClientMessageUnpacker in, C } static CompletableFuture sendResultAndState( - JobExecution execution, + CompletableFuture> executionFut, NotificationSender notificationSender ) { - return execution.resultAsync().whenComplete((val, err) -> - execution.stateAsync().whenComplete((state, errState) -> - notificationSender.sendNotification(w -> { - Marshaller marshaller = extractMarshaller(execution); - ClientComputeJobPacker.packJobResult(val, marshaller, w); - packJobState(w, state); - }, err))); + return executionFut.thenCompose(execution -> + execution.resultAsync().whenComplete((val, err) -> + execution.stateAsync().whenComplete((state, errState) -> + notificationSender.sendNotification(w -> { + Marshaller marshaller = extractMarshaller(execution); + ClientComputeJobPacker.packJobResult(val, marshaller, w); + packJobState(w, state); + }, err)))); + } + + static void packSubmitResult(ClientMessagePacker out, UUID jobId, ClusterNode node) { + out.packUuid(jobId); + packClusterNode(node, out); } private static @Nullable Marshaller extractMarshaller(JobExecution e) { diff --git a/modules/client/src/main/java/org/apache/ignite/client/ClientOperationType.java b/modules/client/src/main/java/org/apache/ignite/client/ClientOperationType.java index 58fb6ba3e82..b2a2d2f0820 100644 --- a/modules/client/src/main/java/org/apache/ignite/client/ClientOperationType.java +++ b/modules/client/src/main/java/org/apache/ignite/client/ClientOperationType.java @@ -136,7 +136,7 @@ public enum ClientOperationType { TUPLE_CONTAINS_ALL_KEYS, /** - * Compute Execute ({@link org.apache.ignite.compute.IgniteCompute#submit(JobTarget, JobDescriptor, Object)}). + * Compute Execute ({@link org.apache.ignite.compute.IgniteCompute#submitAsync(JobTarget, JobDescriptor, Object)}). */ COMPUTE_EXECUTE, @@ -151,7 +151,7 @@ public enum ClientOperationType { COMPUTE_GET_STATE, /** - * Cancel compute job ({@link org.apache.ignite.compute.JobExecution#cancelAsync()}). + * Cancel compute job. */ COMPUTE_CANCEL, diff --git a/modules/client/src/main/java/org/apache/ignite/internal/client/compute/ClientCompute.java b/modules/client/src/main/java/org/apache/ignite/internal/client/compute/ClientCompute.java index c35f502d90d..6db88b04ee0 100644 --- a/modules/client/src/main/java/org/apache/ignite/internal/client/compute/ClientCompute.java +++ b/modules/client/src/main/java/org/apache/ignite/internal/client/compute/ClientCompute.java @@ -17,14 +17,16 @@ package org.apache.ignite.internal.client.compute; +import static java.util.concurrent.CompletableFuture.allOf; import static java.util.concurrent.CompletableFuture.completedFuture; +import static org.apache.ignite.internal.client.TcpIgniteClient.unpackClusterNode; import static org.apache.ignite.lang.ErrorGroups.Client.TABLE_ID_NOT_FOUND_ERR; import java.util.ArrayList; -import java.util.HashMap; +import java.util.Arrays; +import java.util.Collection; import java.util.Iterator; import java.util.List; -import java.util.Map; import java.util.Objects; import java.util.Set; import java.util.UUID; @@ -35,7 +37,11 @@ import java.util.function.BiConsumer; import java.util.function.Function; import java.util.function.Supplier; +import java.util.stream.Collectors; +import org.apache.ignite.compute.AllNodesBroadcastJobTarget; import org.apache.ignite.compute.AnyNodeJobTarget; +import org.apache.ignite.compute.BroadcastExecution; +import org.apache.ignite.compute.BroadcastJobTarget; import org.apache.ignite.compute.ColocatedJobTarget; import org.apache.ignite.compute.IgniteCompute; import org.apache.ignite.compute.JobDescriptor; @@ -56,6 +62,8 @@ import org.apache.ignite.internal.client.table.ClientTables; import org.apache.ignite.internal.client.table.ClientTupleSerializer; import org.apache.ignite.internal.client.table.PartitionAwarenessProvider; +import org.apache.ignite.internal.compute.BroadcastJobExecutionImpl; +import org.apache.ignite.internal.compute.FailedExecution; import org.apache.ignite.internal.sql.SqlCommon; import org.apache.ignite.internal.util.ExceptionUtils; import org.apache.ignite.internal.util.ViewUtils; @@ -92,24 +100,82 @@ public ClientCompute(ReliableChannel ch, ClientTables tables) { this.tables = tables; } - private JobExecution submit(JobTarget target, JobDescriptor descriptor, @Nullable CancellationToken cancellationToken, - @Nullable T arg) { + @Override + public CompletableFuture> submitAsync( + JobTarget target, + JobDescriptor descriptor, + @Nullable T arg, + @Nullable CancellationToken cancellationToken + ) { Objects.requireNonNull(target); Objects.requireNonNull(descriptor); - ClientJobExecution execution = new ClientJobExecution<>(ch, submit0(target, descriptor, arg), descriptor.resultMarshaller(), - descriptor.resultClass()); + return submit0(target, descriptor, arg).thenApply(submitResult -> { + ClientJobExecution execution = new ClientJobExecution<>( + ch, + submitResult, + descriptor.resultMarshaller(), + descriptor.resultClass() + ); - if (cancellationToken != null) { - CancelHandleHelper.addCancelAction(cancellationToken, execution::cancelAsync, execution.resultAsync()); - } + if (cancellationToken != null) { + CancelHandleHelper.addCancelAction(cancellationToken, execution::cancelAsync, execution.resultAsync()); + } - return execution; + return execution; + }); } @Override - public JobExecution submit(JobTarget target, JobDescriptor descriptor, @Nullable T arg) { - return submit(target, descriptor, null, arg); + public CompletableFuture> submitAsync( + BroadcastJobTarget target, + JobDescriptor descriptor, + @Nullable T arg, + @Nullable CancellationToken cancellationToken + ) { + Objects.requireNonNull(target); + Objects.requireNonNull(descriptor); + + if (target instanceof AllNodesBroadcastJobTarget) { + AllNodesBroadcastJobTarget allNodesBroadcastTarget = (AllNodesBroadcastJobTarget) target; + Set nodes = allNodesBroadcastTarget.nodes(); + + CompletableFuture[] futures = nodes.stream() + .map(node -> executeOnAnyNodeAsync(Set.of(node), descriptor, arg)) + .toArray(CompletableFuture[]::new); + + // Wait for all the futures but don't fail resulting future, keep individual futures in executions. + return allOf(futures).handle((unused, throwable) -> new BroadcastJobExecutionImpl<>( + Arrays.stream(futures) + .map(fut -> mapSubmitResult(descriptor, cancellationToken, fut)) + .collect(Collectors.toList()) + )); + } + + throw new IllegalArgumentException("Unsupported job target: " + target); + } + + private JobExecution mapSubmitResult( + JobDescriptor descriptor, + @Nullable CancellationToken cancellationToken, + CompletableFuture submitResultFut + ) { + SubmitResult submitResult; + try { + submitResult = submitResultFut.join(); + } catch (Exception e) { + return new FailedExecution<>(ExceptionUtils.unwrapCause(e)); + } + ClientJobExecution execution = new ClientJobExecution<>( + ch, + submitResult, + descriptor.resultMarshaller(), + descriptor.resultClass() + ); + if (cancellationToken != null) { + CancelHandleHelper.addCancelAction(cancellationToken, execution::cancelAsync, execution.resultAsync()); + } + return execution; } private CompletableFuture submit0(JobTarget target, JobDescriptor descriptor, T arg) { @@ -134,15 +200,23 @@ private CompletableFuture submit0(JobTarget target, JobDesc } @Override - public CompletableFuture executeAsync(JobTarget target, JobDescriptor descriptor, - @Nullable CancellationToken cancellationToken, @Nullable T arg) { - return submit(target, descriptor, cancellationToken, arg).resultAsync(); + public R execute( + JobTarget target, + JobDescriptor descriptor, + @Nullable T arg, + @Nullable CancellationToken cancellationToken + ) { + return sync(executeAsync(target, descriptor, arg, cancellationToken)); } @Override - public R execute(JobTarget target, JobDescriptor descriptor, @Nullable CancellationToken cancellationToken, - @Nullable T arg) { - return sync(executeAsync(target, descriptor, cancellationToken, arg)); + public Collection execute( + BroadcastJobTarget target, + JobDescriptor descriptor, + @Nullable T arg, + @Nullable CancellationToken cancellationToken + ) { + return sync(executeAsync(target, descriptor, arg, cancellationToken)); } private CompletableFuture doExecuteColocatedAsync( @@ -181,41 +255,11 @@ private CompletableFuture doExecuteColocatedAsync( } @Override - public Map> submitBroadcast(Set nodes, JobDescriptor descriptor, - @Nullable T arg) { - Objects.requireNonNull(nodes); - Objects.requireNonNull(descriptor); - - Map> map = new HashMap<>(nodes.size()); - - for (ClusterNode node : nodes) { - JobExecution execution = new ClientJobExecution<>( - ch, - executeOnAnyNodeAsync(Set.of(node), descriptor, arg), - descriptor.resultMarshaller(), - descriptor.resultClass() - ); - if (map.put(node, execution) != null) { - throw new IllegalStateException("Node can't be specified more than once: " + node); - } - } - - return map; - } - - @Override - public CompletableFuture executeMapReduceAsync(TaskDescriptor taskDescriptor, - @Nullable CancellationToken cancellationToken, @Nullable T arg) { - return submitMapReduce(taskDescriptor, cancellationToken, arg).resultAsync(); - } - - @Override - public TaskExecution submitMapReduce(TaskDescriptor taskDescriptor, @Nullable T arg) { - return submitMapReduce(taskDescriptor, null, arg); - } - - private TaskExecution submitMapReduce(TaskDescriptor taskDescriptor, @Nullable CancellationToken cancellationToken, - @Nullable T arg) { + public TaskExecution submitMapReduce( + TaskDescriptor taskDescriptor, + @Nullable T arg, + @Nullable CancellationToken cancellationToken + ) { Objects.requireNonNull(taskDescriptor); ClientTaskExecution clientExecution = new ClientTaskExecution<>(ch, @@ -232,8 +276,12 @@ private TaskExecution submitMapReduce(TaskDescriptor taskDescrip } @Override - public R executeMapReduce(TaskDescriptor taskDescriptor, @Nullable CancellationToken cancellationToken, @Nullable T arg) { - return sync(executeMapReduceAsync(taskDescriptor, cancellationToken, arg)); + public R executeMapReduce( + TaskDescriptor taskDescriptor, + @Nullable T arg, + @Nullable CancellationToken cancellationToken + ) { + return sync(executeMapReduceAsync(taskDescriptor, arg, cancellationToken)); } private CompletableFuture doExecuteMapReduceAsync(TaskDescriptor taskDescriptor, @Nullable T arg) { @@ -413,7 +461,11 @@ private static void packTask(ClientMessagePacker w, TaskDescriptor */ private static SubmitResult unpackSubmitResult(PayloadInputChannel ch) { //noinspection DataFlowIssue (reviewed) - return new SubmitResult(ch.in().unpackUuid(), ch.notificationFuture()); + return new SubmitResult( + ch.in().unpackUuid(), + unpackClusterNode(ch), + ch.notificationFuture() + ); } /** @@ -434,7 +486,12 @@ private static SubmitTaskResult unpackSubmitTaskResult(PayloadInputChannel ch) { } //noinspection DataFlowIssue (reviewed) - return new SubmitTaskResult(jobId, jobIds, ch.notificationFuture()); + return new SubmitTaskResult( + jobId, + jobIds, + ch.clientChannel().protocolContext().clusterNode(), // Task is always executed on a client handler node + ch.notificationFuture() + ); } private static R sync(CompletableFuture future) { diff --git a/modules/client/src/main/java/org/apache/ignite/internal/client/compute/ClientJobExecution.java b/modules/client/src/main/java/org/apache/ignite/internal/client/compute/ClientJobExecution.java index 01657da217a..c78e3f31796 100644 --- a/modules/client/src/main/java/org/apache/ignite/internal/client/compute/ClientJobExecution.java +++ b/modules/client/src/main/java/org/apache/ignite/internal/client/compute/ClientJobExecution.java @@ -36,6 +36,7 @@ import org.apache.ignite.internal.compute.TaskStateImpl; import org.apache.ignite.lang.IgniteException; import org.apache.ignite.marshalling.Marshaller; +import org.apache.ignite.network.ClusterNode; import org.jetbrains.annotations.Nullable; @@ -49,7 +50,9 @@ class ClientJobExecution implements JobExecution { private final ReliableChannel ch; - private final CompletableFuture jobIdFuture; + private final UUID jobId; + + private final ClusterNode node; private final CompletableFuture resultAsync; @@ -58,24 +61,22 @@ class ClientJobExecution implements JobExecution { ClientJobExecution( ReliableChannel ch, - CompletableFuture reqFuture, + SubmitResult submitResult, @Nullable Marshaller marshaller, @Nullable Class resultClass ) { this.ch = ch; - - jobIdFuture = reqFuture.thenApply(SubmitResult::jobId); - - resultAsync = reqFuture - .thenCompose(SubmitResult::notificationFuture) - .thenApply(r -> { - // Notifications require explicit input close. - try (r) { - Object result = ClientComputeJobUnpacker.unpackJobResult(r.in(), marshaller, resultClass); - stateFuture.complete(unpackJobState(r)); - return (R) result; - } - }); + this.jobId = submitResult.jobId(); + node = submitResult.clusterNode(); + + resultAsync = submitResult.notificationFuture().thenApply(r -> { + // Notifications require explicit input close. + try (r) { + Object result = ClientComputeJobUnpacker.unpackJobResult(r.in(), marshaller, resultClass); + stateFuture.complete(unpackJobState(r)); + return (R) result; + } + }); } @Override @@ -88,15 +89,14 @@ public CompletableFuture resultAsync() { if (stateFuture.isDone()) { return stateFuture; } - return jobIdFuture.thenCompose(jobId -> getJobState(ch, jobId)); + return getJobState(ch, jobId); } - @Override public CompletableFuture<@Nullable Boolean> cancelAsync() { if (stateFuture.isDone()) { return falseCompletedFuture(); } - return jobIdFuture.thenCompose(jobId -> cancelJob(ch, jobId)); + return cancelJob(ch, jobId); } @Override @@ -104,7 +104,12 @@ public CompletableFuture resultAsync() { if (stateFuture.isDone()) { return falseCompletedFuture(); } - return jobIdFuture.thenCompose(jobId -> changePriority(ch, jobId, newPriority)); + return changePriority(ch, jobId, newPriority); + } + + @Override + public ClusterNode node() { + return node; } static CompletableFuture<@Nullable JobState> getJobState(ReliableChannel ch, UUID jobId) { diff --git a/modules/client/src/main/java/org/apache/ignite/internal/client/compute/ClientTaskExecution.java b/modules/client/src/main/java/org/apache/ignite/internal/client/compute/ClientTaskExecution.java index f17fb77fa9f..a31969e2dad 100644 --- a/modules/client/src/main/java/org/apache/ignite/internal/client/compute/ClientTaskExecution.java +++ b/modules/client/src/main/java/org/apache/ignite/internal/client/compute/ClientTaskExecution.java @@ -98,7 +98,6 @@ public CompletableFuture resultAsync() { return jobIdFuture.thenCompose(jobId -> getTaskState(ch, jobId)); } - @Override public CompletableFuture<@Nullable Boolean> cancelAsync() { if (stateFuture.isDone()) { return falseCompletedFuture(); diff --git a/modules/client/src/main/java/org/apache/ignite/internal/client/compute/SubmitResult.java b/modules/client/src/main/java/org/apache/ignite/internal/client/compute/SubmitResult.java index 577526094ee..c30e054c549 100644 --- a/modules/client/src/main/java/org/apache/ignite/internal/client/compute/SubmitResult.java +++ b/modules/client/src/main/java/org/apache/ignite/internal/client/compute/SubmitResult.java @@ -20,16 +20,19 @@ import java.util.UUID; import java.util.concurrent.CompletableFuture; import org.apache.ignite.internal.client.PayloadInputChannel; +import org.apache.ignite.network.ClusterNode; /** * Result of the job submission. Contains unpacked job id and notification future. */ class SubmitResult { private final UUID jobId; + private final ClusterNode clusterNode; private final CompletableFuture notificationFuture; - SubmitResult(UUID jobId, CompletableFuture notificationFuture) { + SubmitResult(UUID jobId, ClusterNode clusterNode, CompletableFuture notificationFuture) { this.jobId = jobId; + this.clusterNode = clusterNode; this.notificationFuture = notificationFuture; } @@ -40,4 +43,8 @@ UUID jobId() { CompletableFuture notificationFuture() { return notificationFuture; } + + public ClusterNode clusterNode() { + return clusterNode; + } } diff --git a/modules/client/src/main/java/org/apache/ignite/internal/client/compute/SubmitTaskResult.java b/modules/client/src/main/java/org/apache/ignite/internal/client/compute/SubmitTaskResult.java index 71ef4942016..7e247bb65fd 100644 --- a/modules/client/src/main/java/org/apache/ignite/internal/client/compute/SubmitTaskResult.java +++ b/modules/client/src/main/java/org/apache/ignite/internal/client/compute/SubmitTaskResult.java @@ -21,6 +21,7 @@ import java.util.UUID; import java.util.concurrent.CompletableFuture; import org.apache.ignite.internal.client.PayloadInputChannel; +import org.apache.ignite.network.ClusterNode; import org.jetbrains.annotations.Nullable; /** @@ -30,8 +31,8 @@ class SubmitTaskResult extends SubmitResult { private final List jobIds; - SubmitTaskResult(UUID jobId, List jobIds, CompletableFuture notificationFuture) { - super(jobId, notificationFuture); + SubmitTaskResult(UUID jobId, List jobIds, ClusterNode node, CompletableFuture notificationFuture) { + super(jobId, node, notificationFuture); this.jobIds = jobIds; } diff --git a/modules/client/src/test/java/org/apache/ignite/client/ClientComputeTest.java b/modules/client/src/test/java/org/apache/ignite/client/ClientComputeTest.java index 136cb908d21..1dc746ea3f5 100644 --- a/modules/client/src/test/java/org/apache/ignite/client/ClientComputeTest.java +++ b/modules/client/src/test/java/org/apache/ignite/client/ClientComputeTest.java @@ -23,6 +23,8 @@ import static org.apache.ignite.compute.JobStatus.FAILED; import static org.apache.ignite.internal.testframework.matchers.CompletableFutureExceptionMatcher.willThrowFast; import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willBe; +import static org.apache.ignite.internal.testframework.matchers.JobExecutionMatcher.jobExecutionWithResultAndStateFuture; +import static org.apache.ignite.internal.testframework.matchers.JobExecutionMatcher.jobExecutionWithResultAndStatus; import static org.apache.ignite.internal.testframework.matchers.JobStateMatcher.jobStateWithStatus; import static org.apache.ignite.internal.testframework.matchers.TaskStateMatcher.taskStateWithStatus; import static org.apache.ignite.internal.util.IgniteUtils.closeAll; @@ -98,19 +100,15 @@ public void testClientSendsComputeJobToTargetNodeWhenDirectConnectionExists() th try (var client = getClient(server1, server2, server3, server1, server2)) { assertTrue(IgniteTestUtils.waitForCondition(() -> client.connections().size() == 3, 3000)); - JobDescriptor job = JobDescriptor.builder("job").build(); - - JobExecution execution1 = client.compute().submit(getClusterNodes("s1"), job, null); - JobExecution execution2 = client.compute().submit(getClusterNodes("s2"), job, null); - JobExecution execution3 = client.compute().submit(getClusterNodes("s3"), job, null); + JobDescriptor job = JobDescriptor.builder("job").build(); - assertThat(execution1.resultAsync(), willBe("s1")); - assertThat(execution2.resultAsync(), willBe("s2")); - assertThat(execution3.resultAsync(), willBe("s3")); + CompletableFuture> executionFut1 = client.compute().submitAsync(getClusterNodes("s1"), job, null); + CompletableFuture> executionFut2 = client.compute().submitAsync(getClusterNodes("s2"), job, null); + CompletableFuture> executionFut3 = client.compute().submitAsync(getClusterNodes("s3"), job, null); - assertThat(execution1.stateAsync(), willBe(jobStateWithStatus(COMPLETED))); - assertThat(execution2.stateAsync(), willBe(jobStateWithStatus(COMPLETED))); - assertThat(execution3.stateAsync(), willBe(jobStateWithStatus(COMPLETED))); + assertThat(executionFut1, willBe(jobExecutionWithResultAndStatus("s1", COMPLETED))); + assertThat(executionFut2, willBe(jobExecutionWithResultAndStatus("s2", COMPLETED))); + assertThat(executionFut3, willBe(jobExecutionWithResultAndStatus("s3", COMPLETED))); } } @@ -119,19 +117,15 @@ public void testClientSendsComputeJobToDefaultNodeWhenDirectConnectionToTargetDo initServers(reqId -> false); try (var client = getClient(server3)) { - JobDescriptor job = JobDescriptor.builder("job").build(); - - JobExecution execution1 = client.compute().submit(getClusterNodes("s1"), job, null); - JobExecution execution2 = client.compute().submit(getClusterNodes("s2"), job, null); - JobExecution execution3 = client.compute().submit(getClusterNodes("s3"), job, null); + JobDescriptor job = JobDescriptor.builder("job").build(); - assertThat(execution1.resultAsync(), willBe("s3")); - assertThat(execution2.resultAsync(), willBe("s3")); - assertThat(execution3.resultAsync(), willBe("s3")); + CompletableFuture> executionFut1 = client.compute().submitAsync(getClusterNodes("s1"), job, null); + CompletableFuture> executionFut2 = client.compute().submitAsync(getClusterNodes("s2"), job, null); + CompletableFuture> executionFut3 = client.compute().submitAsync(getClusterNodes("s3"), job, null); - assertThat(execution1.stateAsync(), willBe(jobStateWithStatus(COMPLETED))); - assertThat(execution2.stateAsync(), willBe(jobStateWithStatus(COMPLETED))); - assertThat(execution3.stateAsync(), willBe(jobStateWithStatus(COMPLETED))); + assertThat(executionFut1, willBe(jobExecutionWithResultAndStatus("s3", COMPLETED))); + assertThat(executionFut2, willBe(jobExecutionWithResultAndStatus("s3", COMPLETED))); + assertThat(executionFut3, willBe(jobExecutionWithResultAndStatus("s3", COMPLETED))); } } @@ -144,7 +138,7 @@ public void testClientRetriesComputeJobOnPrimaryAndDefaultNodes() { var nodeId = i % 3 + 1; var nodeName = "s" + nodeId; - JobDescriptor job = JobDescriptor.builder("job").build(); + JobDescriptor job = JobDescriptor.builder("job").build(); CompletableFuture fut = client.compute().executeAsync(getClusterNodes(nodeName), job, null); assertThat(fut, willBe("s3")); @@ -172,15 +166,15 @@ public void testExecuteColocatedAsync() { initServers(reqId -> false); try (var client = getClient(server2)) { - JobDescriptor job = JobDescriptor.builder("job").build(); + JobDescriptor job = JobDescriptor.builder("job").build(); - JobExecution execution1 = client.compute().submit(JobTarget.colocated( + CompletableFuture> executionFut1 = client.compute().submitAsync(JobTarget.colocated( TABLE_NAME, Tuple.create().set("key", "k")), job, null ); - JobExecution execution2 = client.compute().submit(JobTarget.colocated( + CompletableFuture> executionFut2 = client.compute().submitAsync(JobTarget.colocated( TABLE_NAME, 1L, Mapper.of(Long.class)), @@ -188,11 +182,8 @@ public void testExecuteColocatedAsync() { null ); - assertThat(execution1.resultAsync(), willBe("s2")); - assertThat(execution2.resultAsync(), willBe("s2")); - - assertThat(execution1.stateAsync(), willBe(jobStateWithStatus(COMPLETED))); - assertThat(execution2.stateAsync(), willBe(jobStateWithStatus(COMPLETED))); + assertThat(executionFut1, willBe(jobExecutionWithResultAndStatus("s2", COMPLETED))); + assertThat(executionFut2, willBe(jobExecutionWithResultAndStatus("s2", COMPLETED))); } } @@ -329,10 +320,13 @@ void testExceptionInJob() { IgniteCompute igniteCompute = client.compute(); var jobTarget = getClusterNodes("s1"); - JobExecution execution = igniteCompute.submit(jobTarget, JobDescriptor.builder("job").build(), null); + JobDescriptor jobDescriptor = JobDescriptor.builder("job").build(); + CompletableFuture> executionFut = igniteCompute.submitAsync(jobTarget, jobDescriptor, null); - assertThat(execution.resultAsync(), willThrowFast(IgniteException.class)); - assertThat(execution.stateAsync(), willBe(jobStateWithStatus(FAILED))); + assertThat(executionFut, willBe(jobExecutionWithResultAndStateFuture( + willThrowFast(IgniteException.class), + willBe(jobStateWithStatus(FAILED)) + ))); } } @@ -346,13 +340,16 @@ void testRequestCompletesOnAsyncContinuationExecutorThread() { IgniteCompute igniteCompute = client.compute(); var jobTarget = getClusterNodes("s1"); - JobExecution execution = igniteCompute.submit(jobTarget, JobDescriptor.builder("job").build(), null); + JobDescriptor jobDescriptor = JobDescriptor.builder("job").build(); + CompletableFuture> executionFut = igniteCompute.submitAsync(jobTarget, jobDescriptor, null); CompletableFuture threadNameFut = new CompletableFuture<>(); - execution.resultAsync().thenAccept(unused -> threadNameFut.complete(Thread.currentThread().getName())); + + CompletableFuture resultFut = executionFut.thenCompose(JobExecution::resultAsync); + resultFut.thenAccept(unused -> threadNameFut.complete(Thread.currentThread().getName())); // Wait for the job to start on the server. - execution.idAsync().join(); + executionFut.thenCompose(JobExecution::idAsync).join(); // Complete job future to trigger server -> client notification. jobFut.complete("res"); diff --git a/modules/client/src/test/java/org/apache/ignite/client/ServerMetricsTest.java b/modules/client/src/test/java/org/apache/ignite/client/ServerMetricsTest.java index b57065032d2..d49cfea36ca 100644 --- a/modules/client/src/test/java/org/apache/ignite/client/ServerMetricsTest.java +++ b/modules/client/src/test/java/org/apache/ignite/client/ServerMetricsTest.java @@ -93,7 +93,7 @@ public void testRequestsActive() throws Exception { FakeCompute.latch = new CountDownLatch(1); - client.compute().submit(getClusterNodes("s1"), JobDescriptor.builder("job").build(), null); + client.compute().submitAsync(getClusterNodes("s1"), JobDescriptor.builder("job").build(), null); assertTrue( IgniteTestUtils.waitForCondition(() -> testServer.metrics().requestsActive() == 1, 1000), @@ -110,7 +110,7 @@ public void testRequestsActive() throws Exception { public void testRequestsProcessed() throws Exception { long processed = testServer.metrics().requestsProcessed(); - client.compute().submit(getClusterNodes("s1"), JobDescriptor.builder("job").build(), null); + client.compute().submitAsync(getClusterNodes("s1"), JobDescriptor.builder("job").build(), null); assertTrue( IgniteTestUtils.waitForCondition(() -> testServer.metrics().requestsProcessed() == processed + 1, 1000), @@ -123,7 +123,7 @@ public void testRequestsFailed() throws Exception { FakeCompute.err = new RuntimeException("test"); - client.compute().submit(getClusterNodes("s1"), JobDescriptor.builder("job").build(), null); + client.compute().submitAsync(getClusterNodes("s1"), JobDescriptor.builder("job").build(), null); assertTrue( IgniteTestUtils.waitForCondition(() -> testServer.metrics().requestsFailed() == 1, 1000), diff --git a/modules/client/src/test/java/org/apache/ignite/client/fakes/FakeCompute.java b/modules/client/src/test/java/org/apache/ignite/client/fakes/FakeCompute.java index 39b9cbeeedd..b97615ae4c6 100644 --- a/modules/client/src/test/java/org/apache/ignite/client/fakes/FakeCompute.java +++ b/modules/client/src/test/java/org/apache/ignite/client/fakes/FakeCompute.java @@ -41,6 +41,8 @@ import java.util.stream.Collectors; import org.apache.ignite.Ignite; import org.apache.ignite.compute.AnyNodeJobTarget; +import org.apache.ignite.compute.BroadcastExecution; +import org.apache.ignite.compute.BroadcastJobTarget; import org.apache.ignite.compute.ColocatedJobTarget; import org.apache.ignite.compute.ComputeJob; import org.apache.ignite.compute.IgniteCompute; @@ -63,11 +65,13 @@ import org.apache.ignite.internal.compute.SharedComputeUtils; import org.apache.ignite.internal.compute.TaskStateImpl; import org.apache.ignite.internal.compute.loader.JobClassLoader; +import org.apache.ignite.internal.network.ClusterNodeImpl; import org.apache.ignite.internal.table.TableViewInternal; import org.apache.ignite.internal.util.ExceptionUtils; import org.apache.ignite.lang.CancellationToken; import org.apache.ignite.marshalling.Marshaller; import org.apache.ignite.network.ClusterNode; +import org.apache.ignite.network.NetworkAddress; import org.apache.ignite.table.Tuple; import org.jetbrains.annotations.Nullable; @@ -102,7 +106,7 @@ public JobExecution executeAsyncWithFailover( String jobClassName, JobExecutionOptions options, @Nullable CancellationToken cancellationToken, - @Nullable ComputeJobDataHolder args) { + @Nullable ComputeJobDataHolder arg) { if (Objects.equals(jobClassName, GET_UNITS)) { String unitString = units.stream().map(DeploymentUnit::render).collect(Collectors.joining(",")); return completedExecution(unitString); @@ -125,7 +129,7 @@ public JobExecution executeAsyncWithFailover( ComputeJob job = ComputeUtils.instantiateJob(jobClass); CompletableFuture jobFut = job.executeAsync( new JobExecutionContextImpl(ignite, new AtomicBoolean(), this.getClass().getClassLoader()), - SharedComputeUtils.unmarshalArgOrResult(args, null, null)); + SharedComputeUtils.unmarshalArgOrResult(arg, null, null)); return jobExecution(jobFut != null ? jobFut : nullCompletedFuture()); } @@ -150,11 +154,13 @@ public CompletableFuture> submitColocatedInte : completedFuture(SharedComputeUtils.marshalArgOrResult(nodeName, null)))); } - private JobExecution submit( + @Override + public CompletableFuture> submitAsync( JobTarget target, JobDescriptor descriptor, - @Nullable CancellationToken cancellationToken, - @Nullable T args) { + @Nullable T arg, + @Nullable CancellationToken cancellationToken + ) { if (target instanceof AnyNodeJobTarget) { Set nodes = ((AnyNodeJobTarget) target).nodes(); @@ -164,10 +170,10 @@ private JobExecution submit( descriptor.jobClassName(), descriptor.options(), cancellationToken, - SharedComputeUtils.marshalArgOrResult(args, null) + SharedComputeUtils.marshalArgOrResult(arg, null) ); - return new JobExecution<>() { + return completedFuture(new JobExecution<>() { @Override public CompletableFuture resultAsync() { return internalExecution.resultAsync() @@ -181,62 +187,68 @@ public CompletableFuture resultAsync() { } @Override - public CompletableFuture<@Nullable Boolean> cancelAsync() { - return internalExecution.cancelAsync(); + public CompletableFuture<@Nullable Boolean> changePriorityAsync(int newPriority) { + return internalExecution.changePriorityAsync(newPriority); } @Override - public CompletableFuture<@Nullable Boolean> changePriorityAsync(int newPriority) { - return internalExecution.cancelAsync(); + public ClusterNode node() { + return internalExecution.node(); } - }; + }); } else if (target instanceof ColocatedJobTarget) { - return jobExecution(future != null ? future : completedFuture((R) nodeName)); + return completedFuture(jobExecution(future != null ? future : completedFuture((R) nodeName))); } else { throw new IllegalArgumentException("Unsupported job target: " + target); } } @Override - public JobExecution submit(JobTarget target, JobDescriptor descriptor, @Nullable T arg) { - return submit(target, descriptor, null, arg); - } - - @Override - public CompletableFuture executeAsync(JobTarget target, JobDescriptor descriptor, - @Nullable CancellationToken cancellationToken, @Nullable T arg) { - return submit(target, descriptor, cancellationToken, arg).resultAsync(); + public CompletableFuture> submitAsync( + BroadcastJobTarget target, + JobDescriptor descriptor, + @Nullable T arg, + @Nullable CancellationToken cancellationToken + ) { + return nullCompletedFuture(); } @Override - public R execute(JobTarget target, JobDescriptor descriptor, @Nullable CancellationToken cancellationToken, - @Nullable T args) { - return sync(executeAsync(target, descriptor, cancellationToken, args)); + public R execute( + JobTarget target, + JobDescriptor descriptor, + @Nullable T args, + @Nullable CancellationToken cancellationToken + ) { + return sync(executeAsync(target, descriptor, args, cancellationToken)); } @Override - public Map> submitBroadcast( - Set nodes, + public Collection execute( + BroadcastJobTarget target, JobDescriptor descriptor, - T args + @Nullable T arg, + @Nullable CancellationToken cancellationToken ) { - return null; + return sync(executeAsync(target, descriptor, arg, cancellationToken)); } @Override - public TaskExecution submitMapReduce(TaskDescriptor taskDescriptor, @Nullable T arg) { + public TaskExecution submitMapReduce( + TaskDescriptor taskDescriptor, + @Nullable T arg, + @Nullable CancellationToken cancellationToken + ) { return taskExecution(future != null ? future : completedFuture((R) nodeName)); } @Override - public CompletableFuture executeMapReduceAsync(TaskDescriptor taskDescriptor, - @Nullable CancellationToken cancellationToken, @Nullable T arg) { - return submitMapReduce(taskDescriptor, arg).resultAsync(); - } - - @Override - public R executeMapReduce(TaskDescriptor taskDescriptor, @Nullable CancellationToken cancellationToken, @Nullable T arg) { - return sync(executeMapReduceAsync(taskDescriptor, cancellationToken, arg)); + public R executeMapReduce( + TaskDescriptor taskDescriptor, + @Nullable T arg, + @Nullable CancellationToken cancellationToken + ) { + return sync(executeMapReduceAsync(taskDescriptor, arg, cancellationToken)); } private JobExecution completedExecution(R result) { @@ -285,13 +297,13 @@ public CompletableFuture resultAsync() { } @Override - public CompletableFuture<@Nullable Boolean> cancelAsync() { + public CompletableFuture<@Nullable Boolean> changePriorityAsync(int newPriority) { return trueCompletedFuture(); } @Override - public CompletableFuture<@Nullable Boolean> changePriorityAsync(int newPriority) { - return trueCompletedFuture(); + public ClusterNode node() { + return new ClusterNodeImpl(UUID.randomUUID(), nodeName, new NetworkAddress("local-host", 1)); } @@ -365,11 +377,6 @@ public CompletableFuture resultAsync() { return completedFuture(List.of(jobStates.get(subJobId1), jobStates.get(subJobId2))); } - @Override - public CompletableFuture<@Nullable Boolean> cancelAsync() { - return trueCompletedFuture(); - } - @Override public CompletableFuture<@Nullable Boolean> changePriorityAsync(int newPriority) { return trueCompletedFuture(); diff --git a/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/ItComputeBaseTest.java b/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/ItComputeBaseTest.java index 3736855d094..93eaa645a57 100644 --- a/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/ItComputeBaseTest.java +++ b/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/ItComputeBaseTest.java @@ -24,15 +24,21 @@ import static org.apache.ignite.compute.JobStatus.FAILED; import static org.apache.ignite.compute.JobStatus.QUEUED; import static org.apache.ignite.internal.IgniteExceptionTestUtils.assertTraceableException; +import static org.apache.ignite.internal.testframework.matchers.CompletableFutureExceptionMatcher.willThrow; +import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.will; import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willBe; +import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully; +import static org.apache.ignite.internal.testframework.matchers.JobExecutionMatcher.jobExecutionWithResultStatusAndNode; import static org.apache.ignite.internal.testframework.matchers.JobStateMatcher.jobStateWithStatus; import static org.apache.ignite.lang.ErrorGroups.Compute.CLASS_INITIALIZATION_ERR; import static org.apache.ignite.lang.ErrorGroups.Compute.COMPUTE_JOB_FAILED_ERR; import static org.awaitility.Awaitility.await; import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.aMapWithSize; import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.everyItem; +import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.in; import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.is; @@ -55,6 +61,8 @@ import java.util.stream.IntStream; import java.util.stream.Stream; import org.apache.ignite.Ignite; +import org.apache.ignite.compute.BroadcastExecution; +import org.apache.ignite.compute.BroadcastJobTarget; import org.apache.ignite.compute.ComputeException; import org.apache.ignite.compute.IgniteCompute; import org.apache.ignite.compute.JobDescriptor; @@ -68,6 +76,7 @@ import org.apache.ignite.internal.testframework.IgniteTestUtils; import org.apache.ignite.internal.util.ExceptionUtils; import org.apache.ignite.lang.CancelHandle; +import org.apache.ignite.lang.CancellationToken; import org.apache.ignite.lang.IgniteException; import org.apache.ignite.lang.TableNotFoundException; import org.apache.ignite.network.ClusterNode; @@ -92,6 +101,45 @@ protected IgniteCompute compute() { return node(0).compute(); } + /** + * Submits the job for execution, verifies that the execution future completes successfully and returns an execution object. + * + * @param Job argument (T)ype. + * @param Job (R)esult type. + * @param target Execution target. + * @param descriptor Job descriptor. + * @param arg Argument of the job. + * @return Job execution object. + */ + protected JobExecution submit( + JobTarget target, + JobDescriptor descriptor, + @Nullable T arg + ) { + return submit(target, descriptor, null, arg); + } + + protected JobExecution submit( + JobTarget target, + JobDescriptor descriptor, + @Nullable CancellationToken cancellationToken, + @Nullable T arg + ) { + CompletableFuture> executionFut = compute().submitAsync(target, descriptor, arg, cancellationToken); + assertThat(executionFut, willCompleteSuccessfully()); + return executionFut.join(); + } + + protected BroadcastExecution submit( + Set nodes, + JobDescriptor descriptor, + @Nullable T arg + ) { + CompletableFuture> executionFut = compute().submitAsync(BroadcastJobTarget.nodes(nodes), descriptor, arg); + assertThat(executionFut, willCompleteSuccessfully()); + return executionFut.join(); + } + private static List wrongJobClassArguments() { return List.of( Arguments.of("org.example.NonExistentJob", CLASS_INITIALIZATION_ERR, "Cannot load job class by name"), @@ -167,14 +215,14 @@ void executesJobLocally() { void executesJobLocallyAsync() { Ignite entryNode = node(0); - JobExecution execution = compute().submit( + JobExecution execution = submit( JobTarget.node(clusterNode(entryNode)), JobDescriptor.builder(toStringJobClass()).units(units()).build(), - 42); + 42 + ); assertThat(execution.resultAsync(), willBe("42")); assertThat(execution.stateAsync(), willBe(jobStateWithStatus(COMPLETED))); - assertThat(execution.cancelAsync(), willBe(false)); } @Test @@ -189,14 +237,14 @@ void executesJobOnRemoteNodes() { @Test void executesJobOnRemoteNodesAsync() { - JobExecution execution = compute().submit( + JobExecution execution = submit( JobTarget.anyNode(clusterNode(node(1)), clusterNode(node(2))), JobDescriptor.builder(toStringJobClass()).units(units()).build(), - 42); + 42 + ); assertThat(execution.resultAsync(), willBe("42")); assertThat(execution.stateAsync(), willBe(jobStateWithStatus(COMPLETED))); - assertThat(execution.cancelAsync(), willBe(false)); } @Test @@ -236,16 +284,17 @@ void executesFailingJobLocally() { void executesFailingJobLocallyAsync() { Ignite entryNode = node(0); - JobExecution execution = compute().submit( + JobExecution execution = submit( JobTarget.node(clusterNode(entryNode)), - JobDescriptor.builder(failingJobClassName()).units(units()).build(), null); + JobDescriptor.builder(failingJobClassName()).units(units()).build(), + null + ); ExecutionException ex = assertThrows(ExecutionException.class, () -> execution.resultAsync().get(1, TimeUnit.SECONDS)); assertComputeException(ex, "JobException", "Oops"); assertThat(execution.stateAsync(), willBe(jobStateWithStatus(FAILED))); - assertThat(execution.cancelAsync(), willBe(false)); } @Test @@ -272,76 +321,76 @@ void executesFailingJobOnRemoteNodesWithOptions() { @Test void executesFailingJobOnRemoteNodesAsync() { - JobExecution execution = compute().submit( + JobExecution execution = submit( JobTarget.anyNode(clusterNode(node(1)), clusterNode(node(2))), - JobDescriptor.builder(failingJobClassName()).units(units()).build(), null); + JobDescriptor.builder(failingJobClassName()).units(units()).build(), + null + ); ExecutionException ex = assertThrows(ExecutionException.class, () -> execution.resultAsync().get(1, TimeUnit.SECONDS)); assertComputeException(ex, "JobException", "Oops"); assertThat(execution.stateAsync(), willBe(jobStateWithStatus(FAILED))); - assertThat(execution.cancelAsync(), willBe(false)); } @Test void broadcastsJobWithArgumentsAsync() { - Ignite entryNode = node(0); - - Map> results = compute().submitBroadcast( - Set.of(clusterNode(entryNode), clusterNode(node(1)), clusterNode(node(2))), + BroadcastExecution broadcastExecution = submit( + Set.of(clusterNode(node(0)), clusterNode(node(1)), clusterNode(node(2))), JobDescriptor.builder(toStringJobClass()).units(units()).build(), - 42); + 42 + ); - assertThat(results, is(aMapWithSize(3))); - for (int i = 0; i < 3; i++) { - ClusterNode node = clusterNode(node(i)); - JobExecution execution = results.get(node); - assertThat(execution.resultAsync(), willBe("42")); - assertThat(execution.stateAsync(), willBe(jobStateWithStatus(COMPLETED))); - assertThat(execution.cancelAsync(), willBe(false)); - } + Collection> executions = broadcastExecution.executions(); + assertThat(executions, containsInAnyOrder( + jobExecutionWithResultStatusAndNode("42", COMPLETED, clusterNode(0)), + jobExecutionWithResultStatusAndNode("42", COMPLETED, clusterNode(1)), + jobExecutionWithResultStatusAndNode("42", COMPLETED, clusterNode(2)) + )); + + assertThat(broadcastExecution.resultsAsync(), will(hasSize(3))); + assertThat(broadcastExecution.resultsAsync(), will(everyItem(is("42")))); } @Test void broadcastExecutesJobOnRespectiveNodes() { - Ignite entryNode = node(0); + BroadcastExecution broadcastExecution = submit( + Set.of(clusterNode(node(0)), clusterNode(node(1)), clusterNode(node(2))), + JobDescriptor.builder(getNodeNameJobClass()).units(units()).build(), + null + ); - Map> results = compute().submitBroadcast( - Set.of(clusterNode(entryNode), clusterNode(node(1)), clusterNode(node(2))), - JobDescriptor.builder(getNodeNameJobClass()).units(units()).build(), null); + Collection> executions = broadcastExecution.executions(); + assertThat(executions, containsInAnyOrder( + jobExecutionWithResultStatusAndNode(clusterNode(0).name(), COMPLETED, clusterNode(0)), + jobExecutionWithResultStatusAndNode(clusterNode(1).name(), COMPLETED, clusterNode(1)), + jobExecutionWithResultStatusAndNode(clusterNode(2).name(), COMPLETED, clusterNode(2)) + )); - assertThat(results, is(aMapWithSize(3))); - for (int i = 0; i < 3; i++) { - ClusterNode node = clusterNode(node(i)); - JobExecution execution = results.get(node); - assertThat(execution.resultAsync(), willBe(node.name())); - assertThat(execution.stateAsync(), willBe(jobStateWithStatus(COMPLETED))); - assertThat(execution.cancelAsync(), willBe(false)); - } + assertThat(broadcastExecution.resultsAsync(), will(hasSize(3))); + assertThat(broadcastExecution.resultsAsync(), will(containsInAnyOrder(allNodeNames().toArray()))); } @Test - void broadcastsFailingJob() throws Exception { - Ignite entryNode = node(0); - - Map> results = compute().submitBroadcast( - Set.of(clusterNode(entryNode), clusterNode(node(1)), clusterNode(node(2))), - JobDescriptor.builder(failingJobClassName()).units(units()).build(), null); - - assertThat(results, is(aMapWithSize(3))); - for (int i = 0; i < 3; i++) { - JobExecution execution = results.get(clusterNode(node(i))); - Exception result = (Exception) execution.resultAsync() - .handle((res, ex) -> ex != null ? ex : res) - .get(1, TimeUnit.SECONDS); + void broadcastsFailingJob() { + BroadcastExecution broadcastExecution = submit( + Set.of(clusterNode(node(0)), clusterNode(node(1)), clusterNode(node(2))), + JobDescriptor.builder(failingJobClassName()).units(units()).build(), + null + ); - assertThat(result, is(instanceOf(CompletionException.class))); - assertComputeException(result, "JobException", "Oops"); + Collection> executions = broadcastExecution.executions(); + assertThat(executions, hasSize(3)); + for (JobExecution execution : executions) { + ExecutionException ex = assertThrows(ExecutionException.class, () -> execution.resultAsync().get(1, TimeUnit.SECONDS)); + assertComputeException(ex, "JobException", "Oops"); assertThat(execution.stateAsync(), willBe(jobStateWithStatus(FAILED))); - assertThat(execution.cancelAsync(), willBe(false)); } + + ExecutionException ex = assertThrows(ExecutionException.class, () -> broadcastExecution.resultsAsync().get(1, TimeUnit.SECONDS)); + assertComputeException(ex, "JobException", "Oops"); } @Test @@ -359,13 +408,14 @@ void executesColocatedWithTupleKey() { void executesColocatedWithTupleKeyAsync() { createTestTableWithOneRow(); - JobExecution execution = compute().submit( + JobExecution execution = submit( JobTarget.colocated("test", Tuple.create(Map.of("k", 1))), - JobDescriptor.builder(getNodeNameJobClass()).units(units()).build(), null); + JobDescriptor.builder(getNodeNameJobClass()).units(units()).build(), + null + ); assertThat(execution.resultAsync(), willBe(in(allNodeNames()))); assertThat(execution.stateAsync(), willBe(jobStateWithStatus(COMPLETED))); - assertThat(execution.cancelAsync(), willBe(false)); } @Test @@ -383,9 +433,12 @@ public void executesColocatedWithNonConsecutiveKeyColumnOrder() { @Test void executeColocatedThrowsTableNotFoundExceptionWhenTableDoesNotExist() { var ex = assertThrows(CompletionException.class, - () -> compute().submit( + () -> compute().submitAsync( JobTarget.colocated("BAD_TABLE", Tuple.create(Map.of("k", 1))), - JobDescriptor.builder(getNodeNameJobClassName()).units(units()).build(), null).resultAsync().join()); + JobDescriptor.builder(getNodeNameJobClassName()).units(units()).build(), + null + ).join() + ); assertInstanceOf(TableNotFoundException.class, ex.getCause()); assertThat(ex.getCause().getMessage(), containsString("The table does not exist [name=\"PUBLIC\".\"BAD_TABLE\"]")); @@ -401,7 +454,7 @@ void cancelComputeExecuteAsyncWithCancelHandle(boolean local) { JobDescriptor job = JobDescriptor.builder(SilentSleepJob.class).units(units()).build(); CompletableFuture execution = compute() - .executeAsync(JobTarget.node(clusterNode(executeNode)), job, cancelHandle.token(), 100L); + .executeAsync(JobTarget.node(clusterNode(executeNode)), job, 100L, cancelHandle.token()); cancelHandle.cancel(); @@ -418,7 +471,7 @@ void cancelComputeExecuteWithCancelHandle(boolean local) { JobDescriptor job = JobDescriptor.builder(SilentSleepJob.class).units(units()).build(); CompletableFuture runFut = IgniteTestUtils.runAsync(() -> compute() - .execute(JobTarget.node(clusterNode(executeNode)), job, cancelHandle.token(), 100L)); + .execute(JobTarget.node(clusterNode(executeNode)), job, 100L, cancelHandle.token())); cancelHandle.cancel(); @@ -434,14 +487,14 @@ void cancelComputeExecuteBroadcastAsyncWithCancelHandle(boolean local) { CancelHandle cancelHandle = CancelHandle.create(); - CompletableFuture> executions = compute().executeBroadcastAsync( - executeNodes, - JobDescriptor.builder(SilentSleepJob.class).units(units()).build(), cancelHandle.token(), 100L + CompletableFuture> resultsFut = compute().executeAsync( + BroadcastJobTarget.nodes(executeNodes), + JobDescriptor.builder(SilentSleepJob.class).units(units()).build(), 100L, cancelHandle.token() ); cancelHandle.cancel(); - assertThrows(ExecutionException.class, () -> executions.get(10, TimeUnit.SECONDS)); + assertThat(resultsFut, willThrow(ComputeException.class)); } @ParameterizedTest(name = "local: {0}") @@ -453,14 +506,14 @@ void cancelComputeExecuteBroadcastWithCancelHandle(boolean local) { CancelHandle cancelHandle = CancelHandle.create(); - CompletableFuture> runFut = IgniteTestUtils.runAsync(() -> compute().executeBroadcast( - executeNodes, - JobDescriptor.builder(SilentSleepJob.class).units(units()).build(), cancelHandle.token(), 100L + CompletableFuture> runFut = IgniteTestUtils.runAsync(() -> compute().execute( + BroadcastJobTarget.nodes(executeNodes), + JobDescriptor.builder(SilentSleepJob.class).units(units()).build(), 100L, cancelHandle.token() )); cancelHandle.cancel(); - assertThrows(ExecutionException.class, () -> runFut.get(10, TimeUnit.SECONDS)); + assertThat(runFut, willThrow(ComputeException.class)); } @Test @@ -468,7 +521,7 @@ void cancelComputeExecuteMapReduceAsyncWithCancelHandle() { CancelHandle cancelHandle = CancelHandle.create(); CompletableFuture execution = compute() - .executeMapReduceAsync(TaskDescriptor.builder(InfiniteMapReduceTask.class).build(), cancelHandle.token(), null); + .executeMapReduceAsync(TaskDescriptor.builder(InfiniteMapReduceTask.class).build(), null, cancelHandle.token()); cancelHandle.cancel(); @@ -503,13 +556,14 @@ void executesColocatedWithMappedKey() { void executesColocatedWithMappedKeyAsync() { createTestTableWithOneRow(); - JobExecution execution = compute().submit( + JobExecution execution = submit( JobTarget.colocated("test", 1, Mapper.of(Integer.class)), - JobDescriptor.builder(getNodeNameJobClass()).units(units()).build(), null); + JobDescriptor.builder(getNodeNameJobClass()).units(units()).build(), + null + ); assertThat(execution.resultAsync(), willBe(in(allNodeNames()))); assertThat(execution.stateAsync(), willBe(jobStateWithStatus(COMPLETED))); - assertThat(execution.cancelAsync(), willBe(false)); } @Test @@ -555,13 +609,15 @@ void executeMapReduce() { void cancelsJob(boolean local) { Ignite executeNode = local ? node(0) : node(1); + CancelHandle cancelHandle = CancelHandle.create(); + // This job catches the interruption and throws a RuntimeException JobDescriptor job = JobDescriptor.builder(SleepJob.class).units(units()).build(); - JobExecution execution = compute().submit(JobTarget.node(clusterNode(executeNode)), job, Long.MAX_VALUE); + JobExecution execution = submit(JobTarget.node(clusterNode(executeNode)), job, cancelHandle.token(), Long.MAX_VALUE); await().until(execution::stateAsync, willBe(jobStateWithStatus(EXECUTING))); - assertThat(execution.cancelAsync(), willBe(true)); + assertThat(cancelHandle.cancelAsync(), willCompleteSuccessfully()); CompletionException completionException = assertThrows(CompletionException.class, () -> execution.resultAsync().join()); @@ -584,13 +640,15 @@ void cancelsJob(boolean local) { void cancelsNotCancellableJob(boolean local) { Ignite executeNode = local ? node(0) : node(1); + CancelHandle cancelHandle = CancelHandle.create(); + // This job catches the interruption and returns normally JobDescriptor job = JobDescriptor.builder(SilentSleepJob.class).units(units()).build(); - JobExecution execution = compute().submit(JobTarget.node(clusterNode(executeNode)), job, Long.MAX_VALUE); + JobExecution execution = submit(JobTarget.node(clusterNode(executeNode)), job, cancelHandle.token(), Long.MAX_VALUE); await().until(execution::stateAsync, willBe(jobStateWithStatus(EXECUTING))); - assertThat(execution.cancelAsync(), willBe(true)); + assertThat(cancelHandle.cancelAsync(), willCompleteSuccessfully()); CompletionException completionException = assertThrows(CompletionException.class, () -> execution.resultAsync().join()); @@ -615,23 +673,25 @@ void cancelsQueuedJob(boolean local) { JobDescriptor job = JobDescriptor.builder(SleepJob.class).units(units()).build(); + CancelHandle cancelHandle1 = CancelHandle.create(); // Start 1 task in executor with 1 thread - JobExecution execution1 = compute().submit(nodes, job, Long.MAX_VALUE); + JobExecution execution1 = submit(nodes, job, cancelHandle1.token(), Long.MAX_VALUE); await().until(execution1::stateAsync, willBe(jobStateWithStatus(EXECUTING))); + CancelHandle cancelHandle2 = CancelHandle.create(); // Start one more task - JobExecution execution2 = compute().submit(nodes, job, Long.MAX_VALUE); + JobExecution execution2 = submit(nodes, job, cancelHandle2.token(), Long.MAX_VALUE); await().until(execution2::stateAsync, willBe(jobStateWithStatus(QUEUED))); // Task 2 is not complete, in queued state assertThat(execution2.resultAsync().isDone(), is(false)); // Cancel queued task - assertThat(execution2.cancelAsync(), willBe(true)); + assertThat(cancelHandle2.cancelAsync(), willCompleteSuccessfully()); await().until(execution2::stateAsync, willBe(jobStateWithStatus(CANCELED))); // Cancel running task - assertThat(execution1.cancelAsync(), willBe(true)); + assertThat(cancelHandle1.cancelAsync(), willCompleteSuccessfully()); await().until(execution1::stateAsync, willBe(jobStateWithStatus(CANCELED))); } @@ -640,12 +700,13 @@ void cancelsQueuedJob(boolean local) { void changeExecutingJobPriority(boolean local) { Ignite executeNode = local ? node(0) : node(1); + CancelHandle cancelHandle = CancelHandle.create(); JobDescriptor job = JobDescriptor.builder(SleepJob.class).units(units()).build(); - JobExecution execution = compute().submit(JobTarget.node(clusterNode(executeNode)), job, Long.MAX_VALUE); + JobExecution execution = submit(JobTarget.node(clusterNode(executeNode)), job, cancelHandle.token(), Long.MAX_VALUE); await().until(execution::stateAsync, willBe(jobStateWithStatus(EXECUTING))); assertThat(execution.changePriorityAsync(2), willBe(false)); - assertThat(execution.cancelAsync(), willBe(true)); + assertThat(cancelHandle.cancelAsync(), willCompleteSuccessfully()); } @Test diff --git a/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/ItComputeErrorsBaseTest.java b/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/ItComputeErrorsBaseTest.java index b36a6630ccb..321dd1e5b74 100644 --- a/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/ItComputeErrorsBaseTest.java +++ b/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/ItComputeErrorsBaseTest.java @@ -22,15 +22,17 @@ import static org.apache.ignite.internal.compute.utils.InteractiveJobs.Signal.RETURN_WORKER_NAME; import static org.apache.ignite.internal.testframework.IgniteTestUtils.assertThrows; import static org.apache.ignite.internal.testframework.matchers.CompletableFutureExceptionMatcher.willThrow; +import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.is; -import java.util.Map; import java.util.Set; +import java.util.concurrent.CompletableFuture; import java.util.stream.Collectors; +import org.apache.ignite.compute.BroadcastExecution; +import org.apache.ignite.compute.BroadcastJobTarget; import org.apache.ignite.compute.IgniteCompute; import org.apache.ignite.compute.JobDescriptor; -import org.apache.ignite.compute.JobExecution; import org.apache.ignite.compute.JobTarget; import org.apache.ignite.compute.NodeNotFoundException; import org.apache.ignite.internal.ClusterPerClassIntegrationTest; @@ -123,29 +125,29 @@ void broadcastAsync() { InteractiveJobs.initChannels(nodes.stream().map(ClusterNode::name).collect(Collectors.toList())); // When broadcast a job - Map> executions = compute().submitBroadcast( - nodes, JobDescriptor.builder(InteractiveJobs.interactiveJobName()).build(), null); - - // Then one job is alive - assertThat(executions.size(), is(2)); - new TestingJobExecution<>(executions.get(existingNode)).assertExecuting(); + CompletableFuture> executionFut = compute().submitAsync( + BroadcastJobTarget.nodes(nodes), + JobDescriptor.builder(InteractiveJobs.interactiveJobName()).build(), + null + ); - // And second job failed - String errorMessageFragment = "None of the specified nodes are present in the cluster: [" + nonExistingNode.name() + "]"; - assertThat(executions.get(nonExistingNode).resultAsync(), willThrow(NodeNotFoundException.class, errorMessageFragment)); + assertThat(executionFut, willCompleteSuccessfully()); + BroadcastExecution execution = executionFut.join(); - // Cleanup + // Finish running job so that the results could be retrieved. InteractiveJobs.all().finish(); + + String errorMessageFragment = "None of the specified nodes are present in the cluster: [" + nonExistingNode.name() + "]"; + assertThat(execution.resultsAsync(), willThrow(NodeNotFoundException.class, errorMessageFragment)); } protected abstract IgniteCompute compute(); private TestingJobExecution executeGlobalInteractiveJob(Set nodes) { - return new TestingJobExecution<>( - compute().submit( - JobTarget.anyNode(nodes), - JobDescriptor.builder(InteractiveJobs.globalJob().name()).build(), - "") - ); + return new TestingJobExecution<>(compute().submitAsync( + JobTarget.anyNode(nodes), + JobDescriptor.builder(InteractiveJobs.globalJob().name()).build(), + "" + )); } } diff --git a/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/ItComputeTestEmbedded.java b/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/ItComputeTestEmbedded.java index b10c1b945c4..d0257f5d866 100644 --- a/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/ItComputeTestEmbedded.java +++ b/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/ItComputeTestEmbedded.java @@ -17,37 +17,34 @@ package org.apache.ignite.internal.compute; -import static java.util.concurrent.CompletableFuture.allOf; -import static java.util.stream.Collectors.toList; -import static java.util.stream.Collectors.toSet; import static org.apache.ignite.compute.JobStatus.EXECUTING; import static org.apache.ignite.compute.JobStatus.QUEUED; import static org.apache.ignite.internal.IgniteExceptionTestUtils.assertPublicCheckedException; import static org.apache.ignite.internal.IgniteExceptionTestUtils.assertPublicException; +import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.will; import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willBe; import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully; import static org.apache.ignite.internal.testframework.matchers.JobStateMatcher.jobStateWithStatus; import static org.apache.ignite.lang.ErrorGroups.Common.INTERNAL_ERR; import static org.awaitility.Awaitility.await; import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.contains; -import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.everyItem; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.nullValue; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; -import java.util.ArrayList; -import java.util.HashSet; +import java.util.Collection; import java.util.List; -import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.IntStream; import java.util.stream.Stream; import org.apache.ignite.Ignite; +import org.apache.ignite.compute.BroadcastExecution; +import org.apache.ignite.compute.BroadcastJobTarget; import org.apache.ignite.compute.ComputeJob; import org.apache.ignite.compute.JobDescriptor; import org.apache.ignite.compute.JobExecution; @@ -58,9 +55,9 @@ import org.apache.ignite.internal.lang.IgniteInternalCheckedException; import org.apache.ignite.internal.lang.IgniteInternalException; import org.apache.ignite.internal.util.ExceptionUtils; +import org.apache.ignite.lang.CancelHandle; import org.apache.ignite.lang.IgniteCheckedException; import org.apache.ignite.lang.IgniteException; -import org.apache.ignite.network.ClusterNode; import org.apache.ignite.table.KeyValueView; import org.apache.ignite.table.Table; import org.jetbrains.annotations.Nullable; @@ -82,8 +79,7 @@ protected List units() { @SuppressWarnings("AssignmentToStaticFieldFromInstanceMethod") @Test void changeJobPriorityLocally() { - Ignite entryNode = node(0); - JobTarget jobTarget = JobTarget.node(clusterNode(entryNode)); + JobTarget jobTarget = JobTarget.node(clusterNode(0)); CountDownLatch countDownLatch = new CountDownLatch(1); WaitLatchJob.latches = new CountDownLatch[]{countDownLatch, new CountDownLatch(1)}; @@ -91,15 +87,16 @@ void changeJobPriorityLocally() { JobDescriptor job = JobDescriptor.builder(WaitLatchJob.class).units(units()).build(); // Start 1 task in executor with 1 thread - JobExecution execution1 = entryNode.compute().submit(jobTarget, job, 0); + JobExecution execution1 = submit(jobTarget, job, 0); await().until(execution1::stateAsync, willBe(jobStateWithStatus(EXECUTING))); // Start one more task - JobExecution execution2 = entryNode.compute().submit(jobTarget, job, 1); + CancelHandle cancelHandle = CancelHandle.create(); + JobExecution execution2 = submit(jobTarget, job, cancelHandle.token(), 1); await().until(execution2::stateAsync, willBe(jobStateWithStatus(QUEUED))); // Start third task - JobExecution execution3 = entryNode.compute().submit(jobTarget, job, 0); + JobExecution execution3 = submit(jobTarget, job, 0); await().until(execution3::stateAsync, willBe(jobStateWithStatus(QUEUED))); // Task 2 and 3 are not completed, in queued state @@ -120,14 +117,13 @@ void changeJobPriorityLocally() { assertThat(execution2.resultAsync().isDone(), is(false)); // Finish task 2 - assertThat(execution2.cancelAsync(), willBe(true)); + assertThat(cancelHandle.cancelAsync(), willCompleteSuccessfully()); } @SuppressWarnings("AssignmentToStaticFieldFromInstanceMethod") @Test void executesJobLocallyWithOptions() { - Ignite entryNode = node(0); - JobTarget jobTarget = JobTarget.node(clusterNode(entryNode)); + JobTarget jobTarget = JobTarget.node(clusterNode(0)); CountDownLatch countDownLatch = new CountDownLatch(1); WaitLatchJob.latches = new CountDownLatch[]{countDownLatch, new CountDownLatch(1)}; @@ -135,25 +131,27 @@ void executesJobLocallyWithOptions() { JobDescriptor job = JobDescriptor.builder(WaitLatchJob.class).units(units()).build(); // Start 1 task in executor with 1 thread - JobExecution execution1 = entryNode.compute().submit(jobTarget, job, 0); + JobExecution execution1 = submit(jobTarget, job, 0); await().until(execution1::stateAsync, willBe(jobStateWithStatus(EXECUTING))); // Start one more task - JobExecution execution2 = entryNode.compute().submit(jobTarget, job, 1); + CancelHandle cancelHandle = CancelHandle.create(); + JobExecution execution2 = submit(jobTarget, job, cancelHandle.token(), 1); await().until(execution2::stateAsync, willBe(jobStateWithStatus(QUEUED))); // Start third task it should be before task2 in the queue due to higher priority in options JobExecutionOptions options = JobExecutionOptions.builder().priority(1).maxRetries(2).build(); WaitLatchThrowExceptionOnFirstExecutionJob.latch = countDownLatch; - JobExecution execution3 = entryNode.compute().submit( + JobExecution execution3 = submit( jobTarget, JobDescriptor.builder(WaitLatchThrowExceptionOnFirstExecutionJob.class) .units(units()) .options(options) .build(), - null); + null + ); await().until(execution3::stateAsync, willBe(jobStateWithStatus(QUEUED))); // Task 1 and 2 are not competed, in queue state @@ -177,7 +175,7 @@ void executesJobLocallyWithOptions() { assertThat(execution2.resultAsync().isDone(), is(false)); // Cancel task2 - assertThat(execution2.cancelAsync(), willBe(true)); + assertThat(cancelHandle.cancelAsync(), willCompleteSuccessfully()); } @SuppressWarnings("AssignmentToStaticFieldFromInstanceMethod") @@ -256,43 +254,37 @@ void executesSyncKvGetPutFromJob(int targetNodeIndex) { void executesNullReturningJobViaSyncBroadcast() { Ignite entryNode = node(0); - Map results = entryNode.compute() - .executeBroadcast(new HashSet<>(entryNode.clusterNodes()), JobDescriptor.builder(NullReturningJob.class).build(), null); + Collection results = entryNode.compute() + .execute(BroadcastJobTarget.nodes(entryNode.clusterNodes()), JobDescriptor.builder(NullReturningJob.class).build(), null); - assertThat(results.keySet(), equalTo(new HashSet<>(entryNode.clusterNodes()))); - assertThat(new HashSet<>(results.values()), contains(nullValue())); + assertThat(results, everyItem(nullValue())); } @Test void executesNullReturningJobViaAsyncBroadcast() { Ignite entryNode = node(0); - CompletableFuture> resultsFuture = entryNode.compute().executeBroadcastAsync( - new HashSet<>(entryNode.clusterNodes()), JobDescriptor.builder(NullReturningJob.class).build(), null + CompletableFuture> resultsFuture = entryNode.compute().executeAsync( + BroadcastJobTarget.nodes(entryNode.clusterNodes()), + JobDescriptor.builder(NullReturningJob.class).build(), + null ); - assertThat(resultsFuture, willCompleteSuccessfully()); - Map results = resultsFuture.join(); - - assertThat(results.keySet(), equalTo(new HashSet<>(entryNode.clusterNodes()))); - assertThat(new HashSet<>(results.values()), contains(nullValue())); + assertThat(resultsFuture, will(everyItem(nullValue()))); } @Test void executesNullReturningJobViaSubmitBroadcast() { Ignite entryNode = node(0); - Map> executionsMap = entryNode.compute().submitBroadcast( - new HashSet<>(entryNode.clusterNodes()), - JobDescriptor.builder(NullReturningJob.class.getName()).build(), null); - assertThat(executionsMap.keySet(), equalTo(new HashSet<>(entryNode.clusterNodes()))); - - List> executions = new ArrayList<>(executionsMap.values()); - List> futures = executions.stream() - .map(JobExecution::resultAsync) - .collect(toList()); - assertThat(allOf(futures.toArray(CompletableFuture[]::new)), willCompleteSuccessfully()); + CompletableFuture> executionFut = entryNode.compute().submitAsync( + BroadcastJobTarget.nodes(entryNode.clusterNodes()), + JobDescriptor.builder(NullReturningJob.class).build(), + null + ); + assertThat(executionFut, willCompleteSuccessfully()); + BroadcastExecution execution = executionFut.join(); - assertThat(futures.stream().map(CompletableFuture::join).collect(toSet()), contains(nullValue())); + assertThat(execution.resultsAsync(), will(everyItem(nullValue()))); } private Stream targetNodeIndexes() { diff --git a/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/ItExecutionsCleanerTest.java b/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/ItExecutionsCleanerTest.java index 5badd065d43..65c7e2c9ce6 100644 --- a/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/ItExecutionsCleanerTest.java +++ b/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/ItExecutionsCleanerTest.java @@ -37,7 +37,10 @@ import org.apache.ignite.internal.compute.utils.InteractiveJobs; import org.apache.ignite.internal.compute.utils.TestingJobExecution; import org.apache.ignite.internal.wrapper.Wrappers; +import org.apache.ignite.lang.CancelHandle; +import org.apache.ignite.lang.CancellationToken; import org.apache.ignite.network.ClusterNode; +import org.jetbrains.annotations.Nullable; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -116,21 +119,23 @@ void localCompleted() throws Exception { @Test void localCancelled() throws Exception { // Start first task - TestingJobExecution runningExecution = submit(localNodes); + CancelHandle runningCancelHandle = CancelHandle.create(); + TestingJobExecution runningExecution = submit(localNodes, runningCancelHandle.token()); UUID runningJobId = runningExecution.idSync(); // Start second task - TestingJobExecution queuedExecution = submit(localNodes); + CancelHandle queuedCancelHandle = CancelHandle.create(); + TestingJobExecution queuedExecution = submit(localNodes, queuedCancelHandle.token()); UUID queuedJobId = queuedExecution.idSync(); // Second task is queued, cancel it queuedExecution.assertQueued(); - queuedExecution.cancelAsync(); + queuedCancelHandle.cancel(); queuedExecution.assertCancelled(); // First task is executing, cancel it runningExecution.assertExecuting(); - runningExecution.cancelAsync(); + runningCancelHandle.cancel(); runningExecution.assertCancelled(); // All executions are retained @@ -173,21 +178,23 @@ void remoteCompleted() throws Exception { @Test void remoteCancelled() throws Exception { // Start first task - TestingJobExecution runningExecution = submit(remoteNodes); + CancelHandle runningCancelHandle = CancelHandle.create(); + TestingJobExecution runningExecution = submit(remoteNodes, runningCancelHandle.token()); UUID runningJobId = runningExecution.idSync(); // Start second task - TestingJobExecution queuedExecution = submit(remoteNodes); + CancelHandle queuedCancelHandle = CancelHandle.create(); + TestingJobExecution queuedExecution = submit(remoteNodes, queuedCancelHandle.token()); UUID queuedJobId = queuedExecution.idSync(); // Second task is queued, cancel it queuedExecution.assertQueued(); - queuedExecution.cancelAsync(); + queuedCancelHandle.cancel(); queuedExecution.assertCancelled(); // First task is executing, cancel it runningExecution.assertExecuting(); - runningExecution.cancelAsync(); + runningCancelHandle.cancel(); runningExecution.assertCancelled(); // All executions are retained @@ -249,9 +256,12 @@ void failover() throws Exception { } private static TestingJobExecution submit(Set nodes) { - IgniteCompute igniteCompute = CLUSTER.node(0).compute(); - return new TestingJobExecution<>(igniteCompute.submit( - JobTarget.anyNode(nodes), JobDescriptor.builder(InteractiveJobs.globalJob().name()).build(), null + return submit(nodes, null); + } + + private static TestingJobExecution submit(Set nodes, @Nullable CancellationToken cancellationToken) { + return new TestingJobExecution<>(CLUSTER.node(0).compute().submitAsync( + JobTarget.anyNode(nodes), JobDescriptor.builder(InteractiveJobs.globalJob().name()).build(), null, cancellationToken )); } } diff --git a/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/ItFailoverCandidateNotFoundTest.java b/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/ItFailoverCandidateNotFoundTest.java index 852b29eb235..a9c61370fb9 100644 --- a/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/ItFailoverCandidateNotFoundTest.java +++ b/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/ItFailoverCandidateNotFoundTest.java @@ -97,7 +97,10 @@ private void failoverCandidateLeavesCluster(IgniteCompute compute) throws Except } private static TestingJobExecution executeGlobalInteractiveJob(IgniteCompute compute, Set nodes) { - return new TestingJobExecution<>( - compute.submit(JobTarget.anyNode(nodes), JobDescriptor.builder(InteractiveJobs.globalJob().jobClass()).build(), null)); + return new TestingJobExecution<>(compute.submitAsync( + JobTarget.anyNode(nodes), + JobDescriptor.builder(InteractiveJobs.globalJob().jobClass()).build(), + null + )); } } diff --git a/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/ItMapReduceTest.java b/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/ItMapReduceTest.java index fd7fcf8afa6..a82335c7194 100644 --- a/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/ItMapReduceTest.java +++ b/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/ItMapReduceTest.java @@ -24,12 +24,14 @@ import static org.apache.ignite.compute.TaskStatus.FAILED; import static org.apache.ignite.internal.testframework.matchers.CompletableFutureExceptionMatcher.willThrow; import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willBe; +import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully; import static org.apache.ignite.internal.testframework.matchers.JobStateMatcher.jobStateWithStatus; import static org.apache.ignite.internal.testframework.matchers.TaskStateMatcher.taskStateWithStatusAndCreateTimeStartTimeFinishTime; import static org.awaitility.Awaitility.await; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.nullValue; @@ -46,9 +48,11 @@ import org.apache.ignite.internal.ClusterPerClassIntegrationTest; import org.apache.ignite.internal.compute.utils.InteractiveJobs; import org.apache.ignite.internal.compute.utils.InteractiveTasks; -import org.apache.ignite.internal.compute.utils.TestingJobExecution; +import org.apache.ignite.lang.CancelHandle; +import org.apache.ignite.lang.CancellationToken; import org.apache.ignite.lang.IgniteException; import org.hamcrest.Matcher; +import org.jetbrains.annotations.Nullable; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; @@ -73,8 +77,7 @@ void taskMaintainsState() throws Exception { IgniteCompute igniteCompute = entryNode.compute(); TaskExecution> taskExecution = igniteCompute.submitMapReduce( TaskDescriptor.>builder(InteractiveTasks.GlobalApi.name()).build(), null); - TestingJobExecution> testExecution = new TestingJobExecution<>(new TaskToJobExecutionWrapper<>(taskExecution)); - testExecution.assertExecuting(); + assertTaskExecuting(taskExecution); InteractiveTasks.GlobalApi.assertAlive(); // Save state before split. @@ -87,7 +90,7 @@ void taskMaintainsState() throws Exception { InteractiveTasks.GlobalApi.finishSplit(); // Then the task is still executing while waiting for the jobs to finish. - testExecution.assertExecuting(); + assertTaskExecuting(taskExecution); assertTaskStateIs(taskExecution, EXECUTING, stateBeforeSplit, nullValue(Instant.class)); // And states list contains states for 3 running nodes. @@ -97,7 +100,7 @@ void taskMaintainsState() throws Exception { InteractiveJobs.all().finishReturnWorkerNames(); // Then the task is still executing while waiting for the reduce to finish. - testExecution.assertExecuting(); + assertTaskExecuting(taskExecution); assertTaskStateIs(taskExecution, EXECUTING, stateBeforeSplit, nullValue(Instant.class)); // When finish the reduce job. @@ -140,22 +143,20 @@ void cancelSplit(boolean cooperativeCancel) throws Exception { Ignite entryNode = CLUSTER.node(0); // Given running task. - TaskExecution> taskExecution = startTask(entryNode, cooperativeCancel ? "NO_INTERRUPT" : ""); + CancelHandle cancelHandle = CancelHandle.create(); + TaskExecution> taskExecution = startTask(entryNode, cancelHandle.token(), cooperativeCancel ? "NO_INTERRUPT" : ""); // Save state before split. TaskState stateBeforeSplit = taskExecution.stateAsync().join(); // When cancel the task. - assertThat(taskExecution.cancelAsync(), willBe(true)); + assertThat(cancelHandle.cancelAsync(), willCompleteSuccessfully()); // Then the task is cancelled. assertTaskFailed(taskExecution, CANCELED, stateBeforeSplit); // And states list will fail. assertThat(taskExecution.statesAsync(), willThrow(RuntimeException.class)); - - // And second cancel will fail. - assertThat(taskExecution.cancelAsync(), willBe(false)); } @Test @@ -183,12 +184,8 @@ void cancelJobs() throws Exception { Ignite entryNode = CLUSTER.node(0); // Given running task. - IgniteCompute igniteCompute = entryNode.compute(); - TaskExecution> taskExecution = igniteCompute.submitMapReduce( - TaskDescriptor.>builder(InteractiveTasks.GlobalApi.name()).build(), null); - TestingJobExecution> testExecution = new TestingJobExecution<>(new TaskToJobExecutionWrapper<>(taskExecution)); - testExecution.assertExecuting(); - InteractiveTasks.GlobalApi.assertAlive(); + CancelHandle cancelHandle = CancelHandle.create(); + TaskExecution> taskExecution = startTask(entryNode, cancelHandle.token(), null); // Save state before split. TaskState stateBeforeSplit = taskExecution.stateAsync().join(); @@ -197,16 +194,13 @@ void cancelJobs() throws Exception { finishSplit(taskExecution); // When cancel the task. - assertThat(taskExecution.cancelAsync(), willBe(true)); + assertThat(cancelHandle.cancelAsync(), willCompleteSuccessfully()); // Then the task is cancelled. assertTaskFailed(taskExecution, FAILED, stateBeforeSplit); // And states list contains canceled states. assertJobStates(taskExecution, JobStatus.CANCELED); - - // And second cancel will fail. - assertThat(taskExecution.cancelAsync(), willBe(false)); } @Test @@ -241,13 +235,9 @@ void cancelReduce(boolean cooperativeCancel) throws Exception { Ignite entryNode = CLUSTER.node(0); // Given running task. + CancelHandle cancelHandle = CancelHandle.create(); String arg = cooperativeCancel ? "NO_INTERRUPT" : null; - IgniteCompute igniteCompute = entryNode.compute(); - TaskExecution> taskExecution = igniteCompute.submitMapReduce( - TaskDescriptor.>builder(InteractiveTasks.GlobalApi.name()).build(), arg); - TestingJobExecution> testExecution = new TestingJobExecution<>(new TaskToJobExecutionWrapper<>(taskExecution)); - testExecution.assertExecuting(); - InteractiveTasks.GlobalApi.assertAlive(); + TaskExecution> taskExecution = startTask(entryNode, cancelHandle.token(), arg); // Save state before split. TaskState stateBeforeSplit = taskExecution.stateAsync().join(); @@ -262,27 +252,43 @@ void cancelReduce(boolean cooperativeCancel) throws Exception { InteractiveTasks.GlobalApi.assertAlive(); // When cancel the task. - assertThat(taskExecution.cancelAsync(), willBe(true)); + assertThat(cancelHandle.cancelAsync(), willCompleteSuccessfully()); // Then the task is cancelled. assertTaskFailed(taskExecution, CANCELED, stateBeforeSplit); // And states list contains completed states. assertJobStates(taskExecution, JobStatus.COMPLETED); + } - // And second cancel will fail. - assertThat(taskExecution.cancelAsync(), willBe(false)); + private static TaskExecution> startTask(Ignite entryNode, @Nullable String arg) throws InterruptedException { + return startTask(entryNode, null, arg); } - private static TaskExecution> startTask(Ignite entryNode, String args) throws InterruptedException { - IgniteCompute igniteCompute = entryNode.compute(); - TaskExecution> taskExecution = igniteCompute.submitMapReduce( - TaskDescriptor.>builder(InteractiveTasks.GlobalApi.name()).build(), args); - new TestingJobExecution<>(new TaskToJobExecutionWrapper<>(taskExecution)).assertExecuting(); + private static TaskExecution> startTask( + Ignite entryNode, + @Nullable CancellationToken cancellationToken, + @Nullable String arg + ) throws InterruptedException { + TaskExecution> taskExecution = entryNode.compute().submitMapReduce( + TaskDescriptor.>builder(InteractiveTasks.GlobalApi.name()).build(), arg, cancellationToken + ); + assertTaskExecuting(taskExecution); InteractiveTasks.GlobalApi.assertAlive(); return taskExecution; } + private static void assertTaskExecuting(TaskExecution> taskExecution) { + await().until( + () -> taskExecution.stateAsync().thenApply(JobTaskStatusMapper::toJobState), + willBe(jobStateWithStatus(JobStatus.EXECUTING)) + ); + + assertThat(taskExecution.resultAsync().isDone(), equalTo(false)); + + assertThat(taskExecution.idAsync(), willBe(notNullValue())); + } + private static void finishSplit(TaskExecution> taskExecution) { // Finish the split job. InteractiveTasks.GlobalApi.finishSplit(); diff --git a/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/ItWorkerShutdownTest.java b/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/ItWorkerShutdownTest.java index c18dda27e16..6f28ba3ccc0 100644 --- a/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/ItWorkerShutdownTest.java +++ b/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/ItWorkerShutdownTest.java @@ -21,23 +21,30 @@ import static org.apache.ignite.internal.TestWrappers.unwrapIgniteImpl; import static org.apache.ignite.internal.TestWrappers.unwrapTableImpl; import static org.apache.ignite.internal.catalog.CatalogService.DEFAULT_STORAGE_PROFILE; +import static org.apache.ignite.internal.testframework.matchers.CompletableFutureExceptionMatcher.willThrow; +import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.hasItem; +import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.in; -import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.not; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; import java.util.UUID; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import org.apache.ignite.Ignite; +import org.apache.ignite.compute.BroadcastExecution; +import org.apache.ignite.compute.BroadcastJobTarget; +import org.apache.ignite.compute.ComputeException; import org.apache.ignite.compute.IgniteCompute; import org.apache.ignite.compute.JobDescriptor; import org.apache.ignite.compute.JobExecution; @@ -52,6 +59,8 @@ import org.apache.ignite.internal.placementdriver.ReplicaMeta; import org.apache.ignite.internal.replicator.TablePartitionId; import org.apache.ignite.internal.table.TableImpl; +import org.apache.ignite.lang.CancelHandle; +import org.apache.ignite.lang.CancellationToken; import org.apache.ignite.network.ClusterNode; import org.apache.ignite.table.Tuple; import org.junit.jupiter.api.BeforeEach; @@ -217,14 +226,20 @@ void broadcastExecutionWorkerShutdown() { InteractiveJobs.initChannels(allNodeNames()); // When start broadcast job. - Map> executions = compute(entryNode).submitBroadcast( - clusterNodesByNames(workerCandidates(node(0), node(1), node(2))), - JobDescriptor.builder(InteractiveJobs.interactiveJobName()).build(), null); + CompletableFuture> executionFut = compute(entryNode).submitAsync( + BroadcastJobTarget.nodes(clusterNode(0), clusterNode(1), clusterNode(2)), + JobDescriptor.builder(InteractiveJobs.interactiveJobName()).build(), + null + ); + + assertThat(executionFut, willCompleteSuccessfully()); + BroadcastExecution broadcastExecution = executionFut.join(); + Collection> executions = broadcastExecution.executions(); // Then all three jobs are alive. - assertThat(executions.size(), is(3)); - executions.forEach((node, execution) -> { - InteractiveJobs.byNode(node).assertAlive(); + assertThat(executions, hasSize(3)); + executions.forEach(execution -> { + InteractiveJobs.byNode(execution.node()).assertAlive(); new TestingJobExecution<>(execution).assertExecuting(); }); @@ -233,11 +248,11 @@ void broadcastExecutionWorkerShutdown() { stopNode(node(1)); // Then two jobs are alive. - executions.forEach((node, execution) -> { - if (node.name().equals(stoppedNodeName)) { + executions.forEach(execution -> { + if (execution.node().name().equals(stoppedNodeName)) { new TestingJobExecution<>(execution).assertFailed(); } else { - InteractiveJobs.byNode(node).assertAlive(); + InteractiveJobs.byNode(execution.node()).assertAlive(); new TestingJobExecution<>(execution).assertExecuting(); } }); @@ -245,6 +260,8 @@ void broadcastExecutionWorkerShutdown() { // When. InteractiveJobs.all().finish(); + assertThat(broadcastExecution.resultsAsync(), willThrow(ComputeException.class)); + // Then every job ran once because broadcast execution does not require failover. AllInteractiveJobsApi.assertEachCalledOnce(); } @@ -257,7 +274,8 @@ void cancelRemoteExecutionOnRestartedJob() throws Exception { Set remoteWorkerCandidates = workerCandidates(node(1), node(2)); // When execute job. - TestingJobExecution execution = executeGlobalInteractiveJob(entryNode, remoteWorkerCandidates); + CancelHandle cancelHandle = CancelHandle.create(); + TestingJobExecution execution = executeGlobalInteractiveJob(entryNode, remoteWorkerCandidates, cancelHandle.token()); // Then one of candidates became a worker and run the job. String workerNodeName = InteractiveJobs.globalJob().currentWorkerName(); @@ -278,7 +296,7 @@ void cancelRemoteExecutionOnRestartedJob() throws Exception { assertThat(remoteWorkerCandidates, hasItem(failoverWorker)); // When cancel job. - execution.cancelSync(); + cancelHandle.cancel(); // Then it is cancelled. execution.assertCancelled(); @@ -294,7 +312,7 @@ void colocatedExecutionWorkerShutdown() throws Exception { // When start colocated job on node that is not primary replica. Ignite entryNode = anyNodeExcept(primaryReplica); TestingJobExecution execution = new TestingJobExecution<>( - compute(entryNode).submit( + compute(entryNode).submitAsync( JobTarget.colocated(TABLE_NAME, Tuple.create(1).set("K", 1)), JobDescriptor.builder(InteractiveJobs.globalJob().name()).build(), null)); @@ -367,11 +385,16 @@ private Ignite nodeByName(String candidateName) { } private TestingJobExecution executeGlobalInteractiveJob(Ignite entryNode, Set nodes) { - return new TestingJobExecution<>( - compute(entryNode).submit( - JobTarget.anyNode(clusterNodesByNames(nodes)), - JobDescriptor.builder(InteractiveJobs.globalJob().jobClass()).build(), null) - ); + return executeGlobalInteractiveJob(entryNode, nodes, null); + } + + private TestingJobExecution executeGlobalInteractiveJob(Ignite entryNode, Set nodes, CancellationToken token) { + return new TestingJobExecution<>(compute(entryNode).submitAsync( + JobTarget.anyNode(clusterNodesByNames(nodes)), + JobDescriptor.builder(InteractiveJobs.globalJob().jobClass()).build(), + null, + token + )); } abstract IgniteCompute compute(Ignite entryNode); diff --git a/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/threading/ItComputeApiThreadingTest.java b/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/threading/ItComputeApiThreadingTest.java index 8edd9d955a8..b35ac1d73c6 100644 --- a/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/threading/ItComputeApiThreadingTest.java +++ b/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/threading/ItComputeApiThreadingTest.java @@ -23,6 +23,7 @@ import static org.apache.ignite.internal.PublicApiThreadingTests.asyncContinuationPool; import static org.apache.ignite.internal.TestWrappers.unwrapIgniteImpl; import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willBe; +import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.either; import static org.hamcrest.Matchers.is; @@ -33,6 +34,7 @@ import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.function.Function; +import org.apache.ignite.compute.BroadcastJobTarget; import org.apache.ignite.compute.ComputeJob; import org.apache.ignite.compute.IgniteCompute; import org.apache.ignite.compute.JobDescriptor; @@ -121,7 +123,10 @@ void jobExecutionFuturesCompleteInContinuationsPool( @Enum ComputeSubmitOperation submitOperation, @Enum JobExecutionAsyncOperation executionOperation ) { - JobExecution execution = submitOperation.executeOn(computeForPublicUse()); + CompletableFuture> executionFut = submitOperation.executeOn(computeForPublicUse()); + + assertThat(executionFut, willCompleteSuccessfully()); + JobExecution execution = executionFut.join(); CompletableFuture completerFuture = executionOperation.executeOn(execution) .thenApply(unused -> currentThread()); @@ -136,7 +141,10 @@ void jobExecutionFuturesFromInternalCallsAreNotResubmittedToContinuationsPool( @Enum ComputeSubmitOperation submitOperation, @Enum JobExecutionAsyncOperation executionOperation ) { - JobExecution execution = submitOperation.executeOn(computeForInternalUse()); + CompletableFuture> executionFut = submitOperation.executeOn(computeForInternalUse()); + + assertThat(executionFut, willCompleteSuccessfully()); + JobExecution execution = executionFut.join(); CompletableFuture completerFuture = executionOperation.executeOn(execution) .thenApply(unused -> currentThread()); @@ -217,10 +225,12 @@ private enum ComputeAsyncOperation { compute.executeAsync( JobTarget.colocated(TABLE_NAME, KEY, Mapper.of(Integer.class)), JobDescriptor.builder(NoOpJob.class).build(), - null)), - EXECUTE_BROADCAST_ASYNC(compute -> compute.executeBroadcastAsync(justNonEntryNode(), JobDescriptor.builder(NoOpJob.class).build(), - null)), + EXECUTE_BROADCAST_ASYNC(compute -> + compute.executeAsync( + BroadcastJobTarget.nodes(justNonEntryNode()), + JobDescriptor.builder(NoOpJob.class).build(), + null)), EXECUTE_MAP_REDUCE_ASYNC(compute -> compute .executeMapReduceAsync(TaskDescriptor.builder(NoOpMapReduceTask.class).build(), null) ); @@ -237,29 +247,28 @@ CompletableFuture executeOn(IgniteCompute compute) { } private enum ComputeSubmitOperation { - SUBMIT(compute -> compute.submit(JobTarget.anyNode(justNonEntryNode()), JobDescriptor.builder(NoOpJob.class).build(), null)), + SUBMIT(compute -> compute.submitAsync(JobTarget.anyNode(justNonEntryNode()), JobDescriptor.builder(NoOpJob.class).build(), null)), - SUBMIT_COLOCATED_BY_TUPLE(compute -> compute.submit( + SUBMIT_COLOCATED_BY_TUPLE(compute -> compute.submitAsync( JobTarget.colocated(TABLE_NAME, KEY_TUPLE), JobDescriptor.builder(NoOpJob.class).build(), null)), - SUBMIT_COLOCATED_BY_KEY(compute -> compute.submit( + SUBMIT_COLOCATED_BY_KEY(compute -> compute.submitAsync( JobTarget.colocated(TABLE_NAME, KEY, Mapper.of(Integer.class)), JobDescriptor.builder(NoOpJob.class).build(), null) ), - SUBMIT_BROADCAST(compute -> compute - .submitBroadcast(justNonEntryNode(), JobDescriptor.builder(NoOpJob.class).build(), null) - .values().iterator().next() + .submitAsync(BroadcastJobTarget.nodes(justNonEntryNode()), JobDescriptor.builder(NoOpJob.class).build(), null) + .thenApply(broadcastExecution -> broadcastExecution.executions().iterator().next()) ); - private final Function> action; + private final Function>> action; - ComputeSubmitOperation(Function> action) { + ComputeSubmitOperation(Function>> action) { this.action = action; } - JobExecution executeOn(IgniteCompute compute) { + CompletableFuture> executeOn(IgniteCompute compute) { return action.apply(compute); } } @@ -268,7 +277,6 @@ private enum JobExecutionAsyncOperation { RESULT_ASYNC(execution -> execution.resultAsync()), STATE_ASYNC(execution -> execution.stateAsync()), ID_ASYNC(execution -> execution.idAsync()), - CANCEL_ASYNC(execution -> execution.cancelAsync()), CHANGE_PRIORITY_ASYNC(execution -> execution.changePriorityAsync(1)); private final Function, CompletableFuture> action; @@ -304,7 +312,6 @@ private enum TaskExecutionAsyncOperation { RESULT_ASYNC(execution -> execution.resultAsync()), STATE_ASYNC(execution -> execution.stateAsync()), ID_ASYNC(execution -> execution.idAsync()), - CANCEL_ASYNC(execution -> execution.cancelAsync()), CHANGE_PRIORITY_ASYNC(execution -> execution.changePriorityAsync(1)); private final Function, CompletableFuture> action; diff --git a/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/utils/TestingJobExecution.java b/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/utils/TestingJobExecution.java index 5262538b6fd..0ed5220988f 100644 --- a/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/utils/TestingJobExecution.java +++ b/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/utils/TestingJobExecution.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.compute.utils; +import static java.util.concurrent.CompletableFuture.completedFuture; import static org.apache.ignite.compute.JobStatus.CANCELED; import static org.apache.ignite.compute.JobStatus.COMPLETED; import static org.apache.ignite.compute.JobStatus.EXECUTING; @@ -37,6 +38,7 @@ import org.apache.ignite.compute.JobExecution; import org.apache.ignite.compute.JobState; import org.apache.ignite.lang.IgniteException; +import org.apache.ignite.network.ClusterNode; /** * Testing instance of {@link JobExecution}. Adds useful assertions on job's state and sync methods. @@ -44,27 +46,31 @@ * @param Job result type. */ public class TestingJobExecution implements JobExecution { - private final JobExecution jobExecution; + private final CompletableFuture> jobExecution; /** * Constructor. * * @param jobExecution job execution to wrap. */ - public TestingJobExecution(JobExecution jobExecution) { + public TestingJobExecution(CompletableFuture> jobExecution) { this.jobExecution = jobExecution; } + public TestingJobExecution(JobExecution jobExecution) { + this.jobExecution = completedFuture(jobExecution); + } + private JobState stateSync() throws InterruptedException, ExecutionException, TimeoutException { - return jobExecution.stateAsync().get(10, TimeUnit.SECONDS); + return jobExecution.thenCompose(JobExecution::stateAsync).get(10, TimeUnit.SECONDS); } public UUID idSync() throws InterruptedException, ExecutionException, TimeoutException { - return jobExecution.idAsync().get(10, TimeUnit.SECONDS); + return jobExecution.thenCompose(JobExecution::idAsync).get(10, TimeUnit.SECONDS); } private R resultSync() throws ExecutionException, InterruptedException, TimeoutException { - return jobExecution.resultAsync().get(10, TimeUnit.SECONDS); + return jobExecution.thenCompose(JobExecution::resultAsync).get(10, TimeUnit.SECONDS); } public long createTimeMillis() throws ExecutionException, InterruptedException, TimeoutException { @@ -79,35 +85,31 @@ public long finishTimeMillis() throws ExecutionException, InterruptedException, return stateSync().finishTime().toEpochMilli(); } - public void cancelSync() throws ExecutionException, InterruptedException, TimeoutException { - jobExecution.cancelAsync().get(10, TimeUnit.SECONDS); - } - @Override public CompletableFuture resultAsync() { - return jobExecution.resultAsync(); + return jobExecution.thenCompose(JobExecution::resultAsync); } @Override public CompletableFuture stateAsync() { - return jobExecution.stateAsync(); + return jobExecution.thenCompose(JobExecution::stateAsync); } @Override - public CompletableFuture cancelAsync() { - return jobExecution.cancelAsync(); + public CompletableFuture changePriorityAsync(int newPriority) { + return jobExecution.thenCompose(execution -> execution.changePriorityAsync(newPriority)); } @Override - public CompletableFuture changePriorityAsync(int newPriority) { - return jobExecution.changePriorityAsync(newPriority); + public ClusterNode node() { + return jobExecution.join().node(); } /** * Checks that the job execution object has EXECUTING state. */ public void assertQueued() { - await().until(jobExecution::stateAsync, willBe(jobStateWithStatus(QUEUED))); + await().until(this::stateAsync, willBe(jobStateWithStatus(QUEUED))); assertThat(resultAsync().isDone(), equalTo(false)); @@ -118,7 +120,7 @@ public void assertQueued() { * Checks that the job execution object has EXECUTING state. */ public void assertExecuting() { - await().until(jobExecution::stateAsync, willBe(jobStateWithStatus(EXECUTING))); + await().until(this::stateAsync, willBe(jobStateWithStatus(EXECUTING))); assertThat(resultAsync().isDone(), equalTo(false)); @@ -129,7 +131,7 @@ public void assertExecuting() { * Checks that the job execution object is cancelled. */ public void assertCancelled() { - await().until(jobExecution::stateAsync, willBe(jobStateWithStatus(CANCELED))); + await().until(this::stateAsync, willBe(jobStateWithStatus(CANCELED))); assertThat(resultAsync(), willThrow(IgniteException.class)); } @@ -138,7 +140,7 @@ public void assertCancelled() { * Checks that the job execution object is completed successfully. */ public void assertCompleted() { - await().until(jobExecution::stateAsync, willBe(jobStateWithStatus(COMPLETED))); + await().until(this::stateAsync, willBe(jobStateWithStatus(COMPLETED))); assertThat(resultAsync(), willBe("Done")); } @@ -147,11 +149,7 @@ public void assertCompleted() { * Checks that the job execution has failed. */ public void assertFailed() { - await().untilAsserted(() -> { - assertThat(jobExecution.resultAsync(), willThrow(IgniteException.class)); - }); - await().untilAsserted(() -> { - assertThat(jobExecution.stateAsync(), willThrow(IgniteException.class)); - }); + await().until(this::resultAsync, willThrow(IgniteException.class)); + await().until(this::stateAsync, willThrow(IgniteException.class)); } } diff --git a/modules/compute/src/main/java/org/apache/ignite/internal/compute/AntiHijackBroadcastExecution.java b/modules/compute/src/main/java/org/apache/ignite/internal/compute/AntiHijackBroadcastExecution.java new file mode 100644 index 00000000000..c8430f1aa55 --- /dev/null +++ b/modules/compute/src/main/java/org/apache/ignite/internal/compute/AntiHijackBroadcastExecution.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://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.ignite.internal.compute; + +import java.util.Collection; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; +import java.util.stream.Collectors; +import org.apache.ignite.compute.BroadcastExecution; +import org.apache.ignite.compute.JobExecution; +import org.apache.ignite.internal.thread.PublicApiThreading; + +/** + * Wrapper around {@link JobExecution} that adds protection against thread hijacking by users. + */ +class AntiHijackBroadcastExecution implements BroadcastExecution { + private final BroadcastExecution execution; + private final Executor asyncContinuationExecutor; + + /** + * Constructor. + */ + AntiHijackBroadcastExecution(BroadcastExecution execution, Executor asyncContinuationExecutor) { + this.execution = execution; + this.asyncContinuationExecutor = asyncContinuationExecutor; + } + + @Override + public Collection> executions() { + return execution.executions().stream() + .map(execution -> new AntiHijackJobExecution<>(execution, asyncContinuationExecutor)) + .collect(Collectors.toList()); + } + + @Override + public CompletableFuture> resultsAsync() { + return preventThreadHijack(execution.resultsAsync()); + } + + private CompletableFuture preventThreadHijack(CompletableFuture originalFuture) { + return PublicApiThreading.preventThreadHijack(originalFuture, asyncContinuationExecutor); + } +} diff --git a/modules/compute/src/main/java/org/apache/ignite/internal/compute/AntiHijackIgniteCompute.java b/modules/compute/src/main/java/org/apache/ignite/internal/compute/AntiHijackIgniteCompute.java index 995329093a0..e56ab70723c 100644 --- a/modules/compute/src/main/java/org/apache/ignite/internal/compute/AntiHijackIgniteCompute.java +++ b/modules/compute/src/main/java/org/apache/ignite/internal/compute/AntiHijackIgniteCompute.java @@ -17,13 +17,11 @@ package org.apache.ignite.internal.compute; -import static java.util.stream.Collectors.toMap; - -import java.util.Map; -import java.util.Map.Entry; -import java.util.Set; +import java.util.Collection; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; +import org.apache.ignite.compute.BroadcastExecution; +import org.apache.ignite.compute.BroadcastJobTarget; import org.apache.ignite.compute.IgniteCompute; import org.apache.ignite.compute.JobDescriptor; import org.apache.ignite.compute.JobExecution; @@ -33,7 +31,6 @@ import org.apache.ignite.internal.compute.task.AntiHijackTaskExecution; import org.apache.ignite.internal.wrapper.Wrapper; import org.apache.ignite.lang.CancellationToken; -import org.apache.ignite.network.ClusterNode; import org.jetbrains.annotations.Nullable; /** @@ -52,69 +49,71 @@ public AntiHijackIgniteCompute(IgniteCompute compute, Executor asyncContinuation } @Override - public JobExecution submit(JobTarget target, JobDescriptor descriptor, - @Nullable T arg) { - return preventThreadHijack(compute.submit(target, descriptor, arg)); - } - - private JobExecution submit(JobTarget target, JobDescriptor descriptor, - @Nullable CancellationToken cancellationToken, @Nullable T arg) { - IgniteComputeImpl compute0 = unwrap(IgniteComputeImpl.class); - - return preventThreadHijack(compute0.submit(target, descriptor, cancellationToken, arg)); - } - - @Override - public CompletableFuture executeAsync(JobTarget target, JobDescriptor descriptor, - @Nullable CancellationToken cancellationToken, @Nullable T arg) { - return submit(target, descriptor, cancellationToken, arg).resultAsync(); + public CompletableFuture> submitAsync( + JobTarget target, + JobDescriptor descriptor, + @Nullable T arg, + @Nullable CancellationToken cancellationToken + ) { + return compute.submitAsync(target, descriptor, arg, cancellationToken).thenApply(this::preventThreadHijack); } @Override - public R execute(JobTarget target, JobDescriptor descriptor, @Nullable CancellationToken cancellationToken, - @Nullable T arg) { - return compute.execute(target, descriptor, cancellationToken, arg); + public CompletableFuture> submitAsync( + BroadcastJobTarget target, + JobDescriptor descriptor, + @Nullable T arg, + @Nullable CancellationToken cancellationToken + ) { + return compute.submitAsync(target, descriptor, arg, cancellationToken).thenApply(this::preventThreadHijack); } @Override - public Map> submitBroadcast( - Set nodes, + public R execute( + JobTarget target, JobDescriptor descriptor, - T args + @Nullable T arg, + @Nullable CancellationToken cancellationToken ) { - Map> results = compute.submitBroadcast(nodes, descriptor, args); - - return results.entrySet().stream() - .collect(toMap(Entry::getKey, entry -> preventThreadHijack(entry.getValue()))); - } - - private TaskExecution submitMapReduce(TaskDescriptor taskDescriptor, @Nullable CancellationToken cancellationToken, - @Nullable T arg) { - IgniteComputeImpl compute0 = unwrap(IgniteComputeImpl.class); - - return new AntiHijackTaskExecution<>(compute0.submitMapReduce(taskDescriptor, cancellationToken, arg), asyncContinuationExecutor); + return compute.execute(target, descriptor, arg, cancellationToken); } @Override - public TaskExecution submitMapReduce(TaskDescriptor taskDescriptor, @Nullable T arg) { - return new AntiHijackTaskExecution<>(compute.submitMapReduce(taskDescriptor, arg), asyncContinuationExecutor); + public Collection execute( + BroadcastJobTarget target, + JobDescriptor descriptor, + @Nullable T arg, + @Nullable CancellationToken cancellationToken + ) { + return compute.execute(target, descriptor, arg, cancellationToken); } @Override - public CompletableFuture executeMapReduceAsync(TaskDescriptor taskDescriptor, - @Nullable CancellationToken cancellationToken, @Nullable T arg) { - return submitMapReduce(taskDescriptor, cancellationToken, arg).resultAsync(); + public TaskExecution submitMapReduce( + TaskDescriptor taskDescriptor, + @Nullable T arg, + @Nullable CancellationToken cancellationToken + ) { + return new AntiHijackTaskExecution<>(compute.submitMapReduce(taskDescriptor, arg, cancellationToken), asyncContinuationExecutor); } @Override - public R executeMapReduce(TaskDescriptor taskDescriptor, @Nullable CancellationToken cancellationToken, @Nullable T arg) { - return compute.executeMapReduce(taskDescriptor, cancellationToken, arg); + public R executeMapReduce( + TaskDescriptor taskDescriptor, + @Nullable T arg, + @Nullable CancellationToken cancellationToken + ) { + return compute.executeMapReduce(taskDescriptor, arg, cancellationToken); } private JobExecution preventThreadHijack(JobExecution execution) { return new AntiHijackJobExecution<>(execution, asyncContinuationExecutor); } + private BroadcastExecution preventThreadHijack(BroadcastExecution execution) { + return new AntiHijackBroadcastExecution<>(execution, asyncContinuationExecutor); + } + @Override public T unwrap(Class classToUnwrap) { return classToUnwrap.cast(compute); diff --git a/modules/compute/src/main/java/org/apache/ignite/internal/compute/AntiHijackJobExecution.java b/modules/compute/src/main/java/org/apache/ignite/internal/compute/AntiHijackJobExecution.java index d68af92a791..9f3fa418f4c 100644 --- a/modules/compute/src/main/java/org/apache/ignite/internal/compute/AntiHijackJobExecution.java +++ b/modules/compute/src/main/java/org/apache/ignite/internal/compute/AntiHijackJobExecution.java @@ -22,6 +22,7 @@ import org.apache.ignite.compute.JobExecution; import org.apache.ignite.compute.JobState; import org.apache.ignite.internal.thread.PublicApiThreading; +import org.apache.ignite.network.ClusterNode; import org.jetbrains.annotations.Nullable; /** @@ -50,13 +51,13 @@ public CompletableFuture resultAsync() { } @Override - public CompletableFuture<@Nullable Boolean> cancelAsync() { - return preventThreadHijack(execution.cancelAsync()); + public CompletableFuture<@Nullable Boolean> changePriorityAsync(int newPriority) { + return preventThreadHijack(execution.changePriorityAsync(newPriority)); } @Override - public CompletableFuture<@Nullable Boolean> changePriorityAsync(int newPriority) { - return preventThreadHijack(execution.changePriorityAsync(newPriority)); + public ClusterNode node() { + return execution.node(); } private CompletableFuture preventThreadHijack(CompletableFuture originalFuture) { diff --git a/modules/compute/src/main/java/org/apache/ignite/internal/compute/CancellableJobExecution.java b/modules/compute/src/main/java/org/apache/ignite/internal/compute/CancellableJobExecution.java new file mode 100644 index 00000000000..3dd56a3345f --- /dev/null +++ b/modules/compute/src/main/java/org/apache/ignite/internal/compute/CancellableJobExecution.java @@ -0,0 +1,38 @@ +/* + * 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 + * + * http://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.ignite.internal.compute; + +import java.util.concurrent.CompletableFuture; +import org.apache.ignite.compute.JobExecution; +import org.jetbrains.annotations.Nullable; + +/** + * {@link JobExecution} that can be cancelled. + * + * @param Result type. + */ +public interface CancellableJobExecution extends JobExecution { + /** + * Cancels the job. + * + * @return The future which will be completed with {@code true} when the job is cancelled, {@code false} when the job couldn't be + * cancelled (if it's already completed or in the process of cancelling), or {@code null} if the job no longer exists due to + * exceeding the retention time limit. + */ + CompletableFuture<@Nullable Boolean> cancelAsync(); +} diff --git a/modules/compute/src/main/java/org/apache/ignite/internal/compute/CancellableTaskExecution.java b/modules/compute/src/main/java/org/apache/ignite/internal/compute/CancellableTaskExecution.java new file mode 100644 index 00000000000..dfa09cb2061 --- /dev/null +++ b/modules/compute/src/main/java/org/apache/ignite/internal/compute/CancellableTaskExecution.java @@ -0,0 +1,38 @@ +/* + * 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 + * + * http://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.ignite.internal.compute; + +import java.util.concurrent.CompletableFuture; +import org.apache.ignite.compute.task.TaskExecution; +import org.jetbrains.annotations.Nullable; + +/** + * {@link TaskExecution} that can be cancelled. + * + * @param Result type. + */ +public interface CancellableTaskExecution extends TaskExecution { + /** + * Cancels the task. + * + * @return The future which will be completed with {@code true} when the task is cancelled, {@code false} when the task couldn't be + * cancelled (if it's already completed or in the process of cancelling), or {@code null} if the task no longer exists due to + * exceeding the retention time limit. + */ + CompletableFuture<@Nullable Boolean> cancelAsync(); +} diff --git a/modules/compute/src/main/java/org/apache/ignite/internal/compute/ComputeComponent.java b/modules/compute/src/main/java/org/apache/ignite/internal/compute/ComputeComponent.java index 60ba436efed..b6ce58c448f 100644 --- a/modules/compute/src/main/java/org/apache/ignite/internal/compute/ComputeComponent.java +++ b/modules/compute/src/main/java/org/apache/ignite/internal/compute/ComputeComponent.java @@ -23,7 +23,6 @@ import java.util.concurrent.CompletableFuture; import org.apache.ignite.compute.JobExecution; import org.apache.ignite.compute.JobState; -import org.apache.ignite.compute.task.TaskExecution; import org.apache.ignite.deployment.DeploymentUnit; import org.apache.ignite.internal.compute.task.JobSubmitter; import org.apache.ignite.internal.manager.IgniteComponent; @@ -38,7 +37,6 @@ public interface ComputeComponent extends IgniteComponent { /** * Executes a job of the given class on the current node. * - * * @param options Job execution options. * @param units Deployment units which will be loaded for execution. * @param jobClassName Name of the job class. @@ -46,7 +44,7 @@ public interface ComputeComponent extends IgniteComponent { * @param arg Job args. * @return Job execution object. */ - JobExecution executeLocally( + CancellableJobExecution executeLocally( ExecutionOptions options, List units, String jobClassName, @@ -65,7 +63,7 @@ JobExecution executeLocally( * @param arg Job args. * @return Job execution object. */ - JobExecution executeRemotely( + CancellableJobExecution executeRemotely( ExecutionOptions options, ClusterNode remoteNode, List units, @@ -107,7 +105,7 @@ JobExecution executeRemotelyWithFailover( * @param Task result type. * @return Task execution object. */ - TaskExecution executeTask( + CancellableTaskExecution executeTask( JobSubmitter jobSubmitter, List units, String taskClassName, diff --git a/modules/compute/src/main/java/org/apache/ignite/internal/compute/ComputeComponentImpl.java b/modules/compute/src/main/java/org/apache/ignite/internal/compute/ComputeComponentImpl.java index f2c613d4a1f..8e14d99e309 100644 --- a/modules/compute/src/main/java/org/apache/ignite/internal/compute/ComputeComponentImpl.java +++ b/modules/compute/src/main/java/org/apache/ignite/internal/compute/ComputeComponentImpl.java @@ -35,7 +35,6 @@ import java.util.concurrent.atomic.AtomicBoolean; import org.apache.ignite.compute.JobExecution; import org.apache.ignite.compute.JobState; -import org.apache.ignite.compute.task.TaskExecution; import org.apache.ignite.deployment.DeploymentUnit; import org.apache.ignite.internal.cluster.management.topology.api.LogicalTopologyService; import org.apache.ignite.internal.compute.configuration.ComputeConfiguration; @@ -122,7 +121,7 @@ public ComputeComponentImpl( /** {@inheritDoc} */ @Override - public JobExecution executeLocally( + public CancellableJobExecution executeLocally( ExecutionOptions options, List units, String jobClassName, @@ -150,7 +149,7 @@ public JobExecution executeLocally( inFlightFutures.registerFuture(future); inFlightFutures.registerFuture(classLoaderFut); - JobExecution result = new DelegatingJobExecution(future); + DelegatingJobExecution result = new DelegatingJobExecution(future); if (cancellationToken != null) { CancelHandleHelper.addCancelAction(cancellationToken, classLoaderFut); @@ -166,7 +165,7 @@ public JobExecution executeLocally( } @Override - public TaskExecution executeTask( + public CancellableTaskExecution executeTask( JobSubmitter jobSubmitter, List units, String taskClassName, @@ -192,7 +191,10 @@ public TaskExecution executeTask( DelegatingTaskExecution result = new DelegatingTaskExecution<>(taskFuture); - result.idAsync().thenAccept(jobId -> executionManager.addExecution(jobId, new TaskToJobExecutionWrapper<>(result))); + result.idAsync().thenAccept(jobId -> executionManager.addExecution( + jobId, + new TaskToJobExecutionWrapper<>(result, topologyService.localMember()) + )); return result; } finally { busyLock.leaveBusy(); @@ -201,7 +203,7 @@ public TaskExecution executeTask( /** {@inheritDoc} */ @Override - public JobExecution executeRemotely( + public CancellableJobExecution executeRemotely( ExecutionOptions options, ClusterNode remoteNode, List units, @@ -223,8 +225,9 @@ public JobExecution executeRemotely( inFlightFutures.registerFuture(jobIdFuture); inFlightFutures.registerFuture(resultFuture); - JobExecution result = new RemoteJobExecution<>( - remoteNode, jobIdFuture, resultFuture, inFlightFutures, messaging); + RemoteJobExecution result = new RemoteJobExecution<>( + remoteNode, jobIdFuture, resultFuture, inFlightFutures, messaging + ); if (cancellationToken != null) { CancelHandleHelper.addCancelAction(cancellationToken, result::cancelAsync, result.resultAsync()); @@ -247,12 +250,17 @@ public JobExecution executeRemotelyWithFailover( @Nullable CancellationToken cancellationToken, @Nullable ComputeJobDataHolder arg ) { - JobExecution result = new ComputeJobFailover( + CancellableJobExecution result = new ComputeJobFailover( this, logicalTopologyService, topologyService, remoteNode, nextWorkerSelector, failoverExecutor, units, - jobClassName, options, cancellationToken, arg + jobClassName, options, arg ).failSafeExecute(); + // Do not add cancel action to the underlying jobs, let the FailSafeJobExecution handle it. + if (cancellationToken != null) { + CancelHandleHelper.addCancelAction(cancellationToken, result::cancelAsync, result.resultAsync()); + } + result.idAsync().thenAccept(jobId -> executionManager.addExecution(jobId, result)); return result; } diff --git a/modules/compute/src/main/java/org/apache/ignite/internal/compute/ComputeJobFailover.java b/modules/compute/src/main/java/org/apache/ignite/internal/compute/ComputeJobFailover.java index 87f36789736..ae42f21c596 100644 --- a/modules/compute/src/main/java/org/apache/ignite/internal/compute/ComputeJobFailover.java +++ b/modules/compute/src/main/java/org/apache/ignite/internal/compute/ComputeJobFailover.java @@ -20,7 +20,6 @@ import java.util.List; import java.util.concurrent.Executor; import java.util.concurrent.atomic.AtomicReference; -import org.apache.ignite.compute.JobExecution; import org.apache.ignite.deployment.DeploymentUnit; import org.apache.ignite.internal.cluster.management.topology.api.LogicalNode; import org.apache.ignite.internal.cluster.management.topology.api.LogicalTopologyEventListener; @@ -30,7 +29,6 @@ import org.apache.ignite.internal.logger.IgniteLogger; import org.apache.ignite.internal.logger.Loggers; import org.apache.ignite.internal.network.TopologyService; -import org.apache.ignite.lang.CancellationToken; import org.apache.ignite.lang.ErrorGroups.Compute; import org.apache.ignite.network.ClusterNode; import org.jetbrains.annotations.Nullable; @@ -84,9 +82,6 @@ class ComputeJobFailover { */ private final RemoteExecutionContext jobContext; - /** Cancellation token. */ - @Nullable private final CancellationToken cancellationToken; - /** * Creates a per-job instance. * @@ -99,7 +94,6 @@ class ComputeJobFailover { * @param units deployment units. * @param jobClassName the name of the job class. * @param executionOptions execution options like priority or max retries. - * @param cancellationToken Cancellation token or {@code null}. * @param arg the arguments of the job. */ ComputeJobFailover( @@ -112,7 +106,6 @@ class ComputeJobFailover { List units, String jobClassName, ExecutionOptions executionOptions, - @Nullable CancellationToken cancellationToken, @Nullable ComputeJobDataHolder arg ) { this.computeComponent = computeComponent; @@ -122,7 +115,6 @@ class ComputeJobFailover { this.nextWorkerSelector = nextWorkerSelector; this.jobContext = new RemoteExecutionContext(units, jobClassName, executionOptions, arg); this.executor = executor; - this.cancellationToken = cancellationToken; } /** @@ -130,8 +122,8 @@ class ComputeJobFailover { * * @return JobExecution with the result of the job and the status of the job. */ - JobExecution failSafeExecute() { - JobExecution jobExecution = launchJobOn(runningWorkerNode.get()); + CancellableJobExecution failSafeExecute() { + CancellableJobExecution jobExecution = launchJobOn(runningWorkerNode.get()); jobContext.initJobExecution(new FailSafeJobExecution<>(jobExecution)); LogicalTopologyEventListener nodeLeftEventListener = new OnNodeLeft(); @@ -141,16 +133,14 @@ JobExecution failSafeExecute() { return jobContext.failSafeJobExecution(); } - private JobExecution launchJobOn(ClusterNode runningWorkerNode) { + private CancellableJobExecution launchJobOn(ClusterNode runningWorkerNode) { if (runningWorkerNode.name().equals(topologyService.localMember().name())) { return computeComponent.executeLocally( - jobContext.executionOptions(), jobContext.units(), jobContext.jobClassName(), cancellationToken, - jobContext.arg() + jobContext.executionOptions(), jobContext.units(), jobContext.jobClassName(), null, jobContext.arg() ); } else { return computeComponent.executeRemotely( - jobContext.executionOptions(), runningWorkerNode, jobContext.units(), jobContext.jobClassName(), cancellationToken, - jobContext.arg() + jobContext.executionOptions(), runningWorkerNode, jobContext.units(), jobContext.jobClassName(), null, jobContext.arg() ); } } @@ -189,7 +179,7 @@ private void selectNewWorker() { LOG.info("Restarting the job {} on node {}.", jobContext.jobClassName(), nextWorker.name()); runningWorkerNode.set(nextWorker); - JobExecution jobExecution = launchJobOn(runningWorkerNode.get()); + CancellableJobExecution jobExecution = launchJobOn(runningWorkerNode.get()); jobContext.updateJobExecution(jobExecution); }); } diff --git a/modules/compute/src/main/java/org/apache/ignite/internal/compute/DelegatingJobExecution.java b/modules/compute/src/main/java/org/apache/ignite/internal/compute/DelegatingJobExecution.java index 00c4d18cd6c..d7db5b9db76 100644 --- a/modules/compute/src/main/java/org/apache/ignite/internal/compute/DelegatingJobExecution.java +++ b/modules/compute/src/main/java/org/apache/ignite/internal/compute/DelegatingJobExecution.java @@ -18,15 +18,17 @@ package org.apache.ignite.internal.compute; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; import org.apache.ignite.compute.JobExecution; import org.apache.ignite.compute.JobState; import org.apache.ignite.internal.compute.executor.JobExecutionInternal; +import org.apache.ignite.network.ClusterNode; import org.jetbrains.annotations.Nullable; /** * Delegates {@link JobExecution} to the future of {@link JobExecutionInternal}. */ -class DelegatingJobExecution implements JobExecution { +class DelegatingJobExecution implements CancellableJobExecution { private final CompletableFuture> delegate; DelegatingJobExecution(CompletableFuture> delegate) { @@ -52,4 +54,14 @@ public CompletableFuture resultAsync() { public CompletableFuture<@Nullable Boolean> changePriorityAsync(int newPriority) { return delegate.thenApply(jobExecutionInternal -> jobExecutionInternal.changePriority(newPriority)); } + + @Override + public ClusterNode node() { + try { + // TODO https://issues.apache.org/jira/browse/IGNITE-24184 + return delegate.thenApply(JobExecutionInternal::node).get(); + } catch (InterruptedException | ExecutionException e) { + throw new RuntimeException(e); + } + } } diff --git a/modules/compute/src/main/java/org/apache/ignite/internal/compute/ExecutionManager.java b/modules/compute/src/main/java/org/apache/ignite/internal/compute/ExecutionManager.java index 9287a7b7aab..413dfefa105 100644 --- a/modules/compute/src/main/java/org/apache/ignite/internal/compute/ExecutionManager.java +++ b/modules/compute/src/main/java/org/apache/ignite/internal/compute/ExecutionManager.java @@ -49,7 +49,7 @@ public class ExecutionManager { private final Cleaner> cleaner = new Cleaner<>(); - private final Map> executions = new ConcurrentHashMap<>(); + private final Map> executions = new ConcurrentHashMap<>(); ExecutionManager(ComputeConfiguration computeConfiguration, TopologyService topologyService) { this.computeConfiguration = computeConfiguration; @@ -62,7 +62,7 @@ public class ExecutionManager { * @param jobId Job id. * @param execution Job execution. */ - void addExecution(UUID jobId, JobExecution execution) { + void addExecution(UUID jobId, CancellableJobExecution execution) { executions.put(jobId, execution); execution.resultAsync().whenComplete((r, throwable) -> cleaner.scheduleRemove(jobId)); } @@ -137,7 +137,7 @@ public CompletableFuture> localStatesAsync() { * cancelled (either it's not yet started, or it's already completed), or {@code null} if there's no job with the specified id. */ public CompletableFuture<@Nullable Boolean> cancelAsync(UUID jobId) { - JobExecution execution = executions.get(jobId); + CancellableJobExecution execution = executions.get(jobId); if (execution != null) { return execution.cancelAsync(); } diff --git a/modules/compute/src/main/java/org/apache/ignite/internal/compute/FailSafeJobExecution.java b/modules/compute/src/main/java/org/apache/ignite/internal/compute/FailSafeJobExecution.java index 669e6e66071..5638168792b 100644 --- a/modules/compute/src/main/java/org/apache/ignite/internal/compute/FailSafeJobExecution.java +++ b/modules/compute/src/main/java/org/apache/ignite/internal/compute/FailSafeJobExecution.java @@ -28,6 +28,7 @@ import org.apache.ignite.internal.logger.IgniteLogger; import org.apache.ignite.internal.logger.Loggers; import org.apache.ignite.marshalling.Marshaller; +import org.apache.ignite.network.ClusterNode; import org.jetbrains.annotations.Nullable; /** @@ -41,7 +42,7 @@ * * @param the type of the job result. */ -class FailSafeJobExecution implements JobExecution, MarshallerProvider { +class FailSafeJobExecution implements CancellableJobExecution, MarshallerProvider { private static final IgniteLogger LOG = Loggers.forClass(FailSafeJobExecution.class); /** @@ -62,9 +63,9 @@ class FailSafeJobExecution implements JobExecution, MarshallerProvider /** * Link to the current job execution object. It can be updated when the job is restarted on another node. */ - private final AtomicReference> runningJobExecution; + private final AtomicReference> runningJobExecution; - FailSafeJobExecution(JobExecution runningJobExecution) throws RuntimeException { + FailSafeJobExecution(CancellableJobExecution runningJobExecution) throws RuntimeException { this.resultFuture = new CompletableFuture<>(); this.runningJobExecution = new AtomicReference<>(runningJobExecution); @@ -103,7 +104,7 @@ private void registerCompleteHook() { }); } - void updateJobExecution(JobExecution jobExecution) { + void updateJobExecution(CancellableJobExecution jobExecution) { LOG.debug("Updating job execution: {}", jobExecution); runningJobExecution.set(jobExecution); @@ -166,6 +167,11 @@ public CompletableFuture resultAsync() { return runningJobExecution.get().changePriorityAsync(newPriority); } + @Override + public ClusterNode node() { + return runningJobExecution.get().node(); + } + /** * Completes the future with the exception. This method can be called only once. * diff --git a/modules/compute/src/main/java/org/apache/ignite/internal/compute/IgniteComputeImpl.java b/modules/compute/src/main/java/org/apache/ignite/internal/compute/IgniteComputeImpl.java index 04e7daca294..2d32f539c07 100644 --- a/modules/compute/src/main/java/org/apache/ignite/internal/compute/IgniteComputeImpl.java +++ b/modules/compute/src/main/java/org/apache/ignite/internal/compute/IgniteComputeImpl.java @@ -18,9 +18,9 @@ package org.apache.ignite.internal.compute; import static java.util.concurrent.CompletableFuture.completedFuture; -import static java.util.function.Function.identity; -import static java.util.stream.Collectors.toUnmodifiableMap; +import static java.util.stream.Collectors.toList; import static org.apache.ignite.internal.lang.IgniteExceptionMapperUtil.mapToPublicException; +import static org.apache.ignite.internal.util.CompletableFutures.allOfToList; import static org.apache.ignite.internal.util.CompletableFutures.nullCompletedFuture; import static org.apache.ignite.internal.util.ExceptionUtils.unwrapCause; import static org.apache.ignite.lang.ErrorGroups.Compute.COMPUTE_JOB_FAILED_ERR; @@ -29,7 +29,6 @@ import java.util.HashSet; import java.util.Iterator; import java.util.List; -import java.util.Map; import java.util.NoSuchElementException; import java.util.Objects; import java.util.Set; @@ -40,7 +39,10 @@ import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; +import org.apache.ignite.compute.AllNodesBroadcastJobTarget; import org.apache.ignite.compute.AnyNodeJobTarget; +import org.apache.ignite.compute.BroadcastExecution; +import org.apache.ignite.compute.BroadcastJobTarget; import org.apache.ignite.compute.ColocatedJobTarget; import org.apache.ignite.compute.ComputeException; import org.apache.ignite.compute.IgniteCompute; @@ -111,8 +113,13 @@ public IgniteComputeImpl(PlacementDriver placementDriver, TopologyService topolo tables.setStreamerReceiverRunner(this); } - JobExecution submit(JobTarget target, JobDescriptor descriptor, @Nullable CancellationToken cancellationToken, - @Nullable T args) { + @Override + public CompletableFuture> submitAsync( + JobTarget target, + JobDescriptor descriptor, + @Nullable T args, + @Nullable CancellationToken cancellationToken + ) { Objects.requireNonNull(target); Objects.requireNonNull(descriptor); @@ -122,14 +129,14 @@ JobExecution submit(JobTarget target, JobDescriptor descriptor, if (target instanceof AnyNodeJobTarget) { Set nodes = ((AnyNodeJobTarget) target).nodes(); - return new ResultUnmarshallingJobExecution<>( + return completedFuture(new ResultUnmarshallingJobExecution<>( executeAsyncWithFailover( nodes, descriptor.units(), descriptor.jobClassName(), descriptor.options(), cancellationToken, SharedComputeUtils.marshalArgOrResult(args, argumentMarshaller) ), resultMarshaller, descriptor.resultClass() - ); + )); } if (target instanceof ColocatedJobTarget) { @@ -163,28 +170,88 @@ JobExecution submit(JobTarget target, JobDescriptor descriptor, SharedComputeUtils.marshalArgOrResult(args, argumentMarshaller))); } - return new ResultUnmarshallingJobExecution<>( - new JobExecutionFutureWrapper<>(jobFut), resultMarshaller, descriptor.resultClass()); + return jobFut.thenApply(execution -> new ResultUnmarshallingJobExecution<>( + execution, + resultMarshaller, + descriptor.resultClass() + )); } throw new IllegalArgumentException("Unsupported job target: " + target); } @Override - public JobExecution submit(JobTarget target, JobDescriptor descriptor, @Nullable T arg) { - return submit(target, descriptor, null, arg); + public CompletableFuture> submitAsync( + BroadcastJobTarget target, + JobDescriptor descriptor, + @Nullable T arg, + @Nullable CancellationToken cancellationToken + ) { + Objects.requireNonNull(target); + Objects.requireNonNull(descriptor); + + if (target instanceof AllNodesBroadcastJobTarget) { + AllNodesBroadcastJobTarget allNodesBroadcastTarget = (AllNodesBroadcastJobTarget) target; + Set nodes = allNodesBroadcastTarget.nodes(); + Marshaller argumentMarshaller = descriptor.argumentMarshaller(); + Marshaller resultMarshaller = descriptor.resultMarshaller(); + ComputeJobDataHolder argHolder = SharedComputeUtils.marshalArgOrResult(arg, argumentMarshaller); + + return completedFuture(new BroadcastJobExecutionImpl<>(nodes.stream() + .map(node -> submitForBroadcast(descriptor, cancellationToken, node, resultMarshaller, argHolder)) + .collect(toList())) + ); + } + + throw new IllegalArgumentException("Unsupported job target: " + target); + } + + private JobExecution submitForBroadcast( + JobDescriptor descriptor, + @Nullable CancellationToken cancellationToken, + ClusterNode node, + @Nullable Marshaller resultMarshaller, + @Nullable ComputeJobDataHolder argHolder + ) { + // No failover nodes for broadcast. We use failover here in order to complete futures with exceptions + // if worker node has left the cluster. + if (topologyService.getByConsistentId(node.name()) == null) { + return new FailedExecution<>(new NodeNotFoundException(Set.of(node.name()))); + } + + return new ResultUnmarshallingJobExecution<>( + executeOnOneNodeWithFailover( + node, + CompletableFutures::nullCompletedFuture, + descriptor.units(), + descriptor.jobClassName(), + descriptor.options(), + cancellationToken, + argHolder + ), + resultMarshaller, + descriptor.resultClass() + ); } @Override - public CompletableFuture executeAsync(JobTarget target, JobDescriptor descriptor, - @Nullable CancellationToken cancellationToken, @Nullable T arg) { - return submit(target, descriptor, cancellationToken, arg).resultAsync(); + public R execute( + JobTarget target, + JobDescriptor descriptor, + @Nullable T arg, + @Nullable CancellationToken cancellationToken + ) { + return sync(executeAsync(target, descriptor, arg, cancellationToken)); } @Override - public R execute(JobTarget target, JobDescriptor descriptor, @Nullable CancellationToken cancellationToken, - @Nullable T args) { - return sync(executeAsync(target, descriptor, cancellationToken, args)); + public Collection execute( + BroadcastJobTarget target, + JobDescriptor descriptor, + @Nullable T arg, + @Nullable CancellationToken cancellationToken + ) { + return sync(executeAsync(target, descriptor, arg, cancellationToken)); } @Override @@ -212,16 +279,15 @@ public JobExecution executeAsyncWithFailover( NextWorkerSelector selector = new DeqNextWorkerSelector(new ConcurrentLinkedDeque<>(candidates)); - return new JobExecutionWrapper<>( - executeOnOneNodeWithFailover( - targetNode, - selector, - units, - jobClassName, - options, - cancellationToken, - arg - )); + return executeOnOneNodeWithFailover( + targetNode, + selector, + units, + jobClassName, + options, + cancellationToken, + arg + ); } private static ClusterNode randomNode(Set nodes) { @@ -235,6 +301,7 @@ private static ClusterNode randomNode(Set nodes) { return iterator.next(); } + // TODO https://issues.apache.org/jira/browse/IGNITE-24184 private JobExecution executeOnOneNodeWithFailover( ClusterNode targetNode, NextWorkerSelector nextWorkerSelector, @@ -247,10 +314,10 @@ private JobExecution executeOnOneNodeWithFailover( ExecutionOptions options = ExecutionOptions.from(jobExecutionOptions); if (isLocal(targetNode)) { - return computeComponent.executeLocally(options, units, jobClassName, cancellationToken, arg); + return new JobExecutionWrapper<>(computeComponent.executeLocally(options, units, jobClassName, cancellationToken, arg)); } else { - return computeComponent.executeRemotelyWithFailover( - targetNode, nextWorkerSelector, units, jobClassName, options, cancellationToken, arg); + return new JobExecutionWrapper<>(computeComponent.executeRemotelyWithFailover( + targetNode, nextWorkerSelector, units, jobClassName, options, cancellationToken, arg)); } } @@ -330,74 +397,50 @@ private CompletableFuture primaryReplicaForPartition(TableViewInter } @Override - public Map> submitBroadcast( - Set nodes, - JobDescriptor descriptor, - T args + public TaskExecution submitMapReduce( + TaskDescriptor taskDescriptor, + @Nullable T arg, + @Nullable CancellationToken cancellationToken ) { - Objects.requireNonNull(nodes); - Objects.requireNonNull(descriptor); - - Marshaller argumentMarshaller = descriptor.argumentMarshaller(); - Marshaller resultMarshaller = descriptor.resultMarshaller(); - ComputeJobDataHolder argHolder = SharedComputeUtils.marshalArgOrResult(args, argumentMarshaller); - - return nodes.stream() - .collect(toUnmodifiableMap(identity(), - // No failover nodes for broadcast. We use failover here in order to complete futures with exceptions - // if worker node has left the cluster. - node -> { - if (topologyService.getByConsistentId(node.name()) == null) { - return new FailedExecution<>(new NodeNotFoundException(Set.of(node.name()))); - } - - return new ResultUnmarshallingJobExecution<>( - new JobExecutionWrapper<>( - executeOnOneNodeWithFailover( - node, - CompletableFutures::nullCompletedFuture, - descriptor.units(), - descriptor.jobClassName(), - descriptor.options(), - null, - argHolder)), - resultMarshaller, - descriptor.resultClass()); - })); - } - - @Override - public CompletableFuture executeMapReduceAsync(TaskDescriptor taskDescriptor, - @Nullable CancellationToken cancellationToken, @Nullable T arg) { - return submitMapReduce(taskDescriptor, cancellationToken, arg).resultAsync(); - } - - @Override - public TaskExecution submitMapReduce(TaskDescriptor taskDescriptor, @Nullable T arg) { - return submitMapReduce(taskDescriptor, null, arg); - } - - TaskExecution submitMapReduce(TaskDescriptor taskDescriptor, @Nullable CancellationToken cancellationToken, - @Nullable T arg) { Objects.requireNonNull(taskDescriptor); - TaskExecutionWrapper execution = new TaskExecutionWrapper<>( - computeComponent.executeTask(this::submitJob, taskDescriptor.units(), taskDescriptor.taskClassName(), arg)); + CancellableTaskExecution taskExecution = computeComponent.executeTask( + this::submitJobs, + taskDescriptor.units(), + taskDescriptor.taskClassName(), + arg + ); if (cancellationToken != null) { - CancelHandleHelper.addCancelAction(cancellationToken, execution::cancelAsync, execution.resultAsync()); + CancelHandleHelper.addCancelAction(cancellationToken, taskExecution::cancelAsync, taskExecution.resultAsync()); } - return execution; + return new TaskExecutionWrapper<>(taskExecution); } @Override - public R executeMapReduce(TaskDescriptor taskDescriptor, @Nullable CancellationToken cancellationToken, @Nullable T arg) { - return sync(executeMapReduceAsync(taskDescriptor, cancellationToken, arg)); + public R executeMapReduce( + TaskDescriptor taskDescriptor, + @Nullable T arg, + @Nullable CancellationToken cancellationToken + ) { + return sync(executeMapReduceAsync(taskDescriptor, arg, cancellationToken)); } - private JobExecution submitJob(MapReduceJob runner) { - return submit(JobTarget.anyNode(runner.nodes()), runner.jobDescriptor(), runner.arg()); + private CompletableFuture>> submitJobs( + List> runners, + CancellationToken cancellationToken + ) { + return allOfToList( + runners.stream() + .map(runner -> submitAsync( + JobTarget.anyNode(runner.nodes()), + runner.jobDescriptor(), + runner.arg(), + cancellationToken + )) + .toArray(CompletableFuture[]::new) + ); } @Override diff --git a/modules/compute/src/main/java/org/apache/ignite/internal/compute/IgniteComputeInternal.java b/modules/compute/src/main/java/org/apache/ignite/internal/compute/IgniteComputeInternal.java index db3453e2db3..0f2e419dba1 100644 --- a/modules/compute/src/main/java/org/apache/ignite/internal/compute/IgniteComputeInternal.java +++ b/modules/compute/src/main/java/org/apache/ignite/internal/compute/IgniteComputeInternal.java @@ -50,6 +50,7 @@ public interface IgniteComputeInternal extends IgniteCompute { * @param payload Arguments of the job. * @return CompletableFuture Job result. */ + // TODO https://issues.apache.org/jira/browse/IGNITE-24184 JobExecution executeAsyncWithFailover( Set nodes, List units, @@ -81,17 +82,6 @@ CompletableFuture> submitColocatedInternal( @Nullable CancellationToken cancellationToken, @Nullable ComputeJobDataHolder payload); - /** - * Wraps the given future into a job execution object. - * - * @param fut Future to wrap. - * @param Job result type. - * @return Job execution object. - */ - default JobExecution wrapJobExecutionFuture(CompletableFuture> fut) { - return new JobExecutionFutureWrapper<>(fut); - } - /** * Retrieves the current state of all jobs on all nodes in the cluster. * diff --git a/modules/compute/src/main/java/org/apache/ignite/internal/compute/JobExecutionFutureWrapper.java b/modules/compute/src/main/java/org/apache/ignite/internal/compute/JobExecutionFutureWrapper.java deleted file mode 100644 index 04766d87384..00000000000 --- a/modules/compute/src/main/java/org/apache/ignite/internal/compute/JobExecutionFutureWrapper.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://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.ignite.internal.compute; - -import static org.apache.ignite.internal.compute.ComputeUtils.convertToComputeFuture; -import static org.apache.ignite.internal.lang.IgniteExceptionMapperUtil.convertToPublicFuture; - -import java.util.concurrent.CompletableFuture; -import org.apache.ignite.compute.JobExecution; -import org.apache.ignite.compute.JobState; -import org.jetbrains.annotations.Nullable; - -/** - * Wraps the future of {@link JobExecution} converting exceptions thrown by the delegate to public. - * - * @param Result type. - */ -class JobExecutionFutureWrapper implements JobExecution { - private final CompletableFuture> delegate; - - JobExecutionFutureWrapper(CompletableFuture> delegate) { - this.delegate = delegate; - } - - @Override - public CompletableFuture resultAsync() { - return convertToComputeFuture(delegate.thenCompose(JobExecution::resultAsync)); - } - - @Override - public CompletableFuture<@Nullable JobState> stateAsync() { - return convertToPublicFuture(delegate.thenCompose(JobExecution::stateAsync)); - } - - @Override - public CompletableFuture<@Nullable Boolean> cancelAsync() { - return convertToPublicFuture(delegate.thenCompose(JobExecution::cancelAsync)); - } - - @Override - public CompletableFuture<@Nullable Boolean> changePriorityAsync(int newPriority) { - return convertToPublicFuture(delegate.thenCompose(jobExecution -> jobExecution.changePriorityAsync(newPriority))); - } -} diff --git a/modules/compute/src/main/java/org/apache/ignite/internal/compute/JobExecutionWrapper.java b/modules/compute/src/main/java/org/apache/ignite/internal/compute/JobExecutionWrapper.java index 76d49b62616..2ccf97a4c1d 100644 --- a/modules/compute/src/main/java/org/apache/ignite/internal/compute/JobExecutionWrapper.java +++ b/modules/compute/src/main/java/org/apache/ignite/internal/compute/JobExecutionWrapper.java @@ -24,6 +24,7 @@ import org.apache.ignite.compute.JobExecution; import org.apache.ignite.compute.JobState; import org.apache.ignite.marshalling.Marshaller; +import org.apache.ignite.network.ClusterNode; import org.jetbrains.annotations.Nullable; /** @@ -49,13 +50,13 @@ public CompletableFuture resultAsync() { } @Override - public CompletableFuture<@Nullable Boolean> cancelAsync() { - return convertToPublicFuture(delegate.cancelAsync()); + public CompletableFuture<@Nullable Boolean> changePriorityAsync(int newPriority) { + return convertToPublicFuture(delegate.changePriorityAsync(newPriority)); } @Override - public CompletableFuture<@Nullable Boolean> changePriorityAsync(int newPriority) { - return convertToPublicFuture(delegate.changePriorityAsync(newPriority)); + public ClusterNode node() { + return delegate.node(); } @Override diff --git a/modules/compute/src/main/java/org/apache/ignite/internal/compute/RemoteExecutionContext.java b/modules/compute/src/main/java/org/apache/ignite/internal/compute/RemoteExecutionContext.java index bab48411d4b..010f0d0f243 100644 --- a/modules/compute/src/main/java/org/apache/ignite/internal/compute/RemoteExecutionContext.java +++ b/modules/compute/src/main/java/org/apache/ignite/internal/compute/RemoteExecutionContext.java @@ -19,7 +19,6 @@ import java.util.List; import java.util.concurrent.atomic.AtomicReference; -import org.apache.ignite.compute.JobExecution; import org.apache.ignite.deployment.DeploymentUnit; /** @@ -76,7 +75,7 @@ FailSafeJobExecution failSafeJobExecution() { * * @param jobExecution the new job execution object (supposed to be a restarted job but in another worker node). */ - void updateJobExecution(JobExecution jobExecution) { + void updateJobExecution(CancellableJobExecution jobExecution) { failSafeJobExecution().updateJobExecution(jobExecution); } diff --git a/modules/compute/src/main/java/org/apache/ignite/internal/compute/ResultUnmarshallingJobExecution.java b/modules/compute/src/main/java/org/apache/ignite/internal/compute/ResultUnmarshallingJobExecution.java index 54f8099b9cd..8f237657fe5 100644 --- a/modules/compute/src/main/java/org/apache/ignite/internal/compute/ResultUnmarshallingJobExecution.java +++ b/modules/compute/src/main/java/org/apache/ignite/internal/compute/ResultUnmarshallingJobExecution.java @@ -21,6 +21,7 @@ import org.apache.ignite.compute.JobExecution; import org.apache.ignite.compute.JobState; import org.apache.ignite.marshalling.Marshaller; +import org.apache.ignite.network.ClusterNode; import org.jetbrains.annotations.Nullable; /** @@ -54,12 +55,12 @@ public CompletableFuture resultAsync() { } @Override - public CompletableFuture<@Nullable Boolean> cancelAsync() { - return delegate.cancelAsync(); + public CompletableFuture<@Nullable Boolean> changePriorityAsync(int newPriority) { + return delegate.changePriorityAsync(newPriority); } @Override - public CompletableFuture<@Nullable Boolean> changePriorityAsync(int newPriority) { - return delegate.changePriorityAsync(newPriority); + public ClusterNode node() { + return delegate.node(); } } diff --git a/modules/compute/src/main/java/org/apache/ignite/internal/compute/TaskExecutionWrapper.java b/modules/compute/src/main/java/org/apache/ignite/internal/compute/TaskExecutionWrapper.java index 3ce81b8a621..61250e99e9b 100644 --- a/modules/compute/src/main/java/org/apache/ignite/internal/compute/TaskExecutionWrapper.java +++ b/modules/compute/src/main/java/org/apache/ignite/internal/compute/TaskExecutionWrapper.java @@ -55,11 +55,6 @@ public CompletableFuture resultAsync() { return convertToPublicFuture(delegate.stateAsync()); } - @Override - public CompletableFuture<@Nullable Boolean> cancelAsync() { - return convertToPublicFuture(delegate.cancelAsync()); - } - @Override public CompletableFuture<@Nullable Boolean> changePriorityAsync(int newPriority) { return convertToPublicFuture(delegate.changePriorityAsync(newPriority)); diff --git a/modules/compute/src/main/java/org/apache/ignite/internal/compute/TaskToJobExecutionWrapper.java b/modules/compute/src/main/java/org/apache/ignite/internal/compute/TaskToJobExecutionWrapper.java index a04fa8cc128..52b45d81113 100644 --- a/modules/compute/src/main/java/org/apache/ignite/internal/compute/TaskToJobExecutionWrapper.java +++ b/modules/compute/src/main/java/org/apache/ignite/internal/compute/TaskToJobExecutionWrapper.java @@ -21,6 +21,7 @@ import org.apache.ignite.compute.JobExecution; import org.apache.ignite.compute.JobState; import org.apache.ignite.compute.task.TaskExecution; +import org.apache.ignite.network.ClusterNode; import org.jetbrains.annotations.Nullable; /** @@ -28,11 +29,13 @@ * * @param Job result type. */ -public class TaskToJobExecutionWrapper implements JobExecution { - private final TaskExecution taskExecution; +class TaskToJobExecutionWrapper implements CancellableJobExecution { + private final CancellableTaskExecution taskExecution; + private final ClusterNode localNode; - public TaskToJobExecutionWrapper(TaskExecution taskExecution) { + TaskToJobExecutionWrapper(CancellableTaskExecution taskExecution, ClusterNode localNode) { this.taskExecution = taskExecution; + this.localNode = localNode; } @Override @@ -42,19 +45,7 @@ public CompletableFuture resultAsync() { @Override public CompletableFuture<@Nullable JobState> stateAsync() { - return taskExecution.stateAsync().thenApply(state -> { - if (state == null) { - return null; - - } - return JobStateImpl.builder() - .id(state.id()) - .createTime(state.createTime()) - .startTime(state.startTime()) - .finishTime(state.finishTime()) - .status(JobTaskStatusMapper.toJobStatus(state.status())) - .build(); - }); + return taskExecution.stateAsync().thenApply(JobTaskStatusMapper::toJobState); } @Override @@ -66,4 +57,9 @@ public CompletableFuture resultAsync() { public CompletableFuture<@Nullable Boolean> changePriorityAsync(int newPriority) { return taskExecution.changePriorityAsync(newPriority); } + + @Override + public ClusterNode node() { + return localNode; + } } diff --git a/modules/compute/src/main/java/org/apache/ignite/internal/compute/executor/ComputeExecutorImpl.java b/modules/compute/src/main/java/org/apache/ignite/internal/compute/executor/ComputeExecutorImpl.java index 06e99b80a55..1aa5050c316 100644 --- a/modules/compute/src/main/java/org/apache/ignite/internal/compute/executor/ComputeExecutorImpl.java +++ b/modules/compute/src/main/java/org/apache/ignite/internal/compute/executor/ComputeExecutorImpl.java @@ -45,6 +45,7 @@ import org.apache.ignite.internal.compute.task.TaskExecutionInternal; import org.apache.ignite.internal.logger.IgniteLogger; import org.apache.ignite.internal.logger.Loggers; +import org.apache.ignite.internal.network.TopologyService; import org.apache.ignite.internal.thread.IgniteThreadFactory; import org.apache.ignite.marshalling.Marshaller; import org.jetbrains.annotations.Nullable; @@ -61,6 +62,8 @@ public class ComputeExecutorImpl implements ComputeExecutor { private final ComputeStateMachine stateMachine; + private final TopologyService topologyService; + private PriorityQueueExecutor executorService; /** @@ -69,15 +72,18 @@ public class ComputeExecutorImpl implements ComputeExecutor { * @param ignite Ignite instance for public API access. * @param stateMachine Compute jobs state machine. * @param configuration Compute configuration. + * @param topologyService Topology service. */ public ComputeExecutorImpl( Ignite ignite, ComputeStateMachine stateMachine, - ComputeConfiguration configuration + ComputeConfiguration configuration, + TopologyService topologyService ) { this.ignite = ignite; this.configuration = configuration; this.stateMachine = stateMachine; + this.topologyService = topologyService; } @Override @@ -101,7 +107,7 @@ public JobExecutionInternal executeJob( options.maxRetries() ); - return new JobExecutionInternal<>(execution, isInterrupted, null, false); + return new JobExecutionInternal<>(execution, isInterrupted, null, false, topologyService.localMember()); } private static Callable> unmarshalExecMarshal( diff --git a/modules/compute/src/main/java/org/apache/ignite/internal/compute/executor/JobExecutionInternal.java b/modules/compute/src/main/java/org/apache/ignite/internal/compute/executor/JobExecutionInternal.java index 84953fcd840..dca750649eb 100644 --- a/modules/compute/src/main/java/org/apache/ignite/internal/compute/executor/JobExecutionInternal.java +++ b/modules/compute/src/main/java/org/apache/ignite/internal/compute/executor/JobExecutionInternal.java @@ -24,6 +24,7 @@ import org.apache.ignite.internal.compute.MarshallerProvider; import org.apache.ignite.internal.compute.queue.QueueExecution; import org.apache.ignite.marshalling.Marshaller; +import org.apache.ignite.network.ClusterNode; import org.jetbrains.annotations.Nullable; /** @@ -40,6 +41,8 @@ public class JobExecutionInternal implements MarshallerProvider { private final boolean marshalResult; + private final ClusterNode localNode; + /** * Constructor. * @@ -47,17 +50,20 @@ public class JobExecutionInternal implements MarshallerProvider { * @param isInterrupted Flag which is passed to the execution context so that the job can check it for cancellation request. * @param marshaller Result marshaller. * @param marshalResult Flag indicating whether the marshalling of the result will be needed. + * @param localNode Local cluster node. */ JobExecutionInternal( QueueExecution execution, AtomicBoolean isInterrupted, @Nullable Marshaller marshaller, - boolean marshalResult + boolean marshalResult, + ClusterNode localNode ) { this.execution = execution; this.isInterrupted = isInterrupted; this.marshaller = marshaller; this.marshalResult = marshalResult; + this.localNode = localNode; } public CompletableFuture resultAsync() { @@ -98,4 +104,8 @@ public boolean changePriority(int newPriority) { public boolean marshalResult() { return marshalResult; } + + public ClusterNode node() { + return localNode; + } } diff --git a/modules/compute/src/main/java/org/apache/ignite/internal/compute/messaging/RemoteJobExecution.java b/modules/compute/src/main/java/org/apache/ignite/internal/compute/messaging/RemoteJobExecution.java index 4097a6db39c..67243516079 100644 --- a/modules/compute/src/main/java/org/apache/ignite/internal/compute/messaging/RemoteJobExecution.java +++ b/modules/compute/src/main/java/org/apache/ignite/internal/compute/messaging/RemoteJobExecution.java @@ -19,8 +19,8 @@ import java.util.UUID; import java.util.concurrent.CompletableFuture; -import org.apache.ignite.compute.JobExecution; import org.apache.ignite.compute.JobState; +import org.apache.ignite.internal.compute.CancellableJobExecution; import org.apache.ignite.internal.future.InFlightFutures; import org.apache.ignite.network.ClusterNode; import org.jetbrains.annotations.Nullable; @@ -30,7 +30,7 @@ * * @param Job result type. */ -public class RemoteJobExecution implements JobExecution { +public class RemoteJobExecution implements CancellableJobExecution { private final ClusterNode remoteNode; private final CompletableFuture jobIdFuture; @@ -89,4 +89,9 @@ public CompletableFuture resultAsync() { jobIdFuture.thenCompose(jobId -> messaging.remoteChangePriorityAsync(remoteNode, jobId, newPriority)) ); } + + @Override + public ClusterNode node() { + return remoteNode; + } } diff --git a/modules/compute/src/main/java/org/apache/ignite/internal/compute/task/AntiHijackTaskExecution.java b/modules/compute/src/main/java/org/apache/ignite/internal/compute/task/AntiHijackTaskExecution.java index a3ce63c7f47..71e4a741368 100644 --- a/modules/compute/src/main/java/org/apache/ignite/internal/compute/task/AntiHijackTaskExecution.java +++ b/modules/compute/src/main/java/org/apache/ignite/internal/compute/task/AntiHijackTaskExecution.java @@ -60,11 +60,6 @@ public CompletableFuture resultAsync() { return preventThreadHijack(execution.stateAsync()); } - @Override - public CompletableFuture<@Nullable Boolean> cancelAsync() { - return preventThreadHijack(execution.cancelAsync()); - } - @Override public CompletableFuture<@Nullable Boolean> changePriorityAsync(int newPriority) { return preventThreadHijack(execution.changePriorityAsync(newPriority)); diff --git a/modules/compute/src/main/java/org/apache/ignite/internal/compute/task/DelegatingTaskExecution.java b/modules/compute/src/main/java/org/apache/ignite/internal/compute/task/DelegatingTaskExecution.java index 2d008fec739..873f2bf2f5f 100644 --- a/modules/compute/src/main/java/org/apache/ignite/internal/compute/task/DelegatingTaskExecution.java +++ b/modules/compute/src/main/java/org/apache/ignite/internal/compute/task/DelegatingTaskExecution.java @@ -22,6 +22,7 @@ import org.apache.ignite.compute.JobState; import org.apache.ignite.compute.TaskState; import org.apache.ignite.compute.task.TaskExecution; +import org.apache.ignite.internal.compute.CancellableTaskExecution; import org.apache.ignite.internal.compute.MarshallerProvider; import org.apache.ignite.marshalling.Marshaller; import org.jetbrains.annotations.Nullable; @@ -31,7 +32,7 @@ * * @param Result type. */ -public class DelegatingTaskExecution implements TaskExecution, MarshallerProvider { +public class DelegatingTaskExecution implements CancellableTaskExecution, MarshallerProvider { private final CompletableFuture> delegate; public DelegatingTaskExecution(CompletableFuture> delegate) { diff --git a/modules/compute/src/main/java/org/apache/ignite/internal/compute/task/JobSubmitter.java b/modules/compute/src/main/java/org/apache/ignite/internal/compute/task/JobSubmitter.java index df5350664ea..8532554ab19 100644 --- a/modules/compute/src/main/java/org/apache/ignite/internal/compute/task/JobSubmitter.java +++ b/modules/compute/src/main/java/org/apache/ignite/internal/compute/task/JobSubmitter.java @@ -17,8 +17,11 @@ package org.apache.ignite.internal.compute.task; +import java.util.List; +import java.util.concurrent.CompletableFuture; import org.apache.ignite.compute.JobExecution; import org.apache.ignite.compute.task.MapReduceJob; +import org.apache.ignite.lang.CancellationToken; /** * Compute job submitter. @@ -26,9 +29,10 @@ @FunctionalInterface public interface JobSubmitter { /** - * Submits compute job for an execution. + * Submits compute jobs for an execution. * - * @param computeJobRunner Computer job start parameters. + * @param computeJobRunners List of the compute job start parameters. + * @param cancellationToken Cancellation token. */ - JobExecution submit(MapReduceJob computeJobRunner); + CompletableFuture>> submit(List> computeJobRunners, CancellationToken cancellationToken); } diff --git a/modules/compute/src/main/java/org/apache/ignite/internal/compute/task/TaskExecutionInternal.java b/modules/compute/src/main/java/org/apache/ignite/internal/compute/task/TaskExecutionInternal.java index cd8d654c125..5e9dbc2e6e7 100644 --- a/modules/compute/src/main/java/org/apache/ignite/internal/compute/task/TaskExecutionInternal.java +++ b/modules/compute/src/main/java/org/apache/ignite/internal/compute/task/TaskExecutionInternal.java @@ -50,14 +50,15 @@ import org.apache.ignite.compute.TaskStatus; import org.apache.ignite.compute.task.MapReduceJob; import org.apache.ignite.compute.task.MapReduceTask; -import org.apache.ignite.compute.task.TaskExecution; import org.apache.ignite.compute.task.TaskExecutionContext; +import org.apache.ignite.internal.compute.CancellableTaskExecution; import org.apache.ignite.internal.compute.MarshallerProvider; import org.apache.ignite.internal.compute.TaskStateImpl; import org.apache.ignite.internal.compute.queue.PriorityQueueExecutor; import org.apache.ignite.internal.compute.queue.QueueExecution; import org.apache.ignite.internal.logger.IgniteLogger; import org.apache.ignite.internal.logger.Loggers; +import org.apache.ignite.lang.CancelHandle; import org.apache.ignite.marshalling.Marshaller; import org.jetbrains.annotations.Nullable; @@ -70,7 +71,7 @@ * @param Task result type. */ @SuppressWarnings("unchecked") -public class TaskExecutionInternal implements TaskExecution, MarshallerProvider { +public class TaskExecutionInternal implements CancellableTaskExecution, MarshallerProvider { private static final IgniteLogger LOG = Loggers.forClass(TaskExecutionInternal.class); private final QueueExecution> splitExecution; @@ -83,6 +84,8 @@ public class TaskExecutionInternal implements TaskExecution, Mars private final AtomicReference reduceFailedState = new AtomicReference<>(); + private final CancelHandle cancelHandle = CancelHandle.create(); + private final AtomicBoolean isCancelled; private volatile @Nullable Marshaller reduceResultMarshallerRef; @@ -122,10 +125,10 @@ public TaskExecutionInternal( 0 ); - executionsFuture = splitExecution.resultAsync().thenApply(splitResult -> { + executionsFuture = splitExecution.resultAsync().thenCompose(splitResult -> { List> runners = splitResult.runners(); LOG.debug("Submitting {} jobs for {}", runners.size(), taskClass.getName()); - return submit(runners, jobSubmitter); + return jobSubmitter.submit(runners, cancelHandle.token()); }); resultsFuture = executionsFuture.thenCompose(TaskExecutionInternal::resultsAsync); @@ -223,13 +226,8 @@ public CompletableFuture resultAsync() { // Split job was complete, results future was running, but not complete yet. if (resultsFuture.cancel(true)) { - return executionsFuture.thenCompose(executions -> { - CompletableFuture[] cancelFutures = executions.stream() - .map(JobExecution::cancelAsync) - .toArray(CompletableFuture[]::new); - - return allOf(cancelFutures); - }).thenApply(unused -> true); + return executionsFuture.thenCompose(unused -> cancelHandle.cancelAsync()) + .thenApply(unused -> true); } // Results arrived but reduce is not yet submitted @@ -303,12 +301,6 @@ private static CompletableFuture> resultsAsync(List List> submit(List> runners, JobSubmitter jobSubmitter) { - return runners.stream() - .map(jobSubmitter::submit) - .collect(toList()); - } - @Override public @Nullable Marshaller resultMarshaller() { return reduceResultMarshallerRef; diff --git a/modules/compute/src/test/java/org/apache/ignite/internal/compute/ComputeComponentImplTest.java b/modules/compute/src/test/java/org/apache/ignite/internal/compute/ComputeComponentImplTest.java index 08bbe54d854..619a7fce30a 100644 --- a/modules/compute/src/test/java/org/apache/ignite/internal/compute/ComputeComponentImplTest.java +++ b/modules/compute/src/test/java/org/apache/ignite/internal/compute/ComputeComponentImplTest.java @@ -166,7 +166,7 @@ void setUp() { }).when(messagingService).addMessageHandler(eq(ComputeMessageTypes.class), any()); InMemoryComputeStateMachine stateMachine = new InMemoryComputeStateMachine(computeConfiguration, INSTANCE_NAME); - ComputeExecutor computeExecutor = new ComputeExecutorImpl(ignite, stateMachine, computeConfiguration); + ComputeExecutor computeExecutor = new ComputeExecutorImpl(ignite, stateMachine, computeConfiguration, topologyService); computeComponent = new ComputeComponentImpl( INSTANCE_NAME, @@ -189,12 +189,15 @@ void cleanup() { @Test void executesLocally() { + CancelHandle cancelHandle = CancelHandle.create(); + JobExecution execution = computeComponent.executeLocally( - DEFAULT, List.of(), SimpleJob.class.getName(), null, SharedComputeUtils.marshalArgOrResult("", null)); + DEFAULT, List.of(), SimpleJob.class.getName(), cancelHandle.token(), SharedComputeUtils.marshalArgOrResult("", null) + ); assertThat(unwrapResult(execution), willBe("jobResponse")); assertThat(execution.stateAsync(), willBe(jobStateWithStatus(COMPLETED))); - assertThat(execution.cancelAsync(), willBe(false)); + assertThat(cancelHandle.cancelAsync(), willCompleteSuccessfully()); assertThat(execution.changePriorityAsync(1), willBe(false)); assertThatNoRequestsWereSent(); @@ -225,12 +228,15 @@ void testLongPreExecutionInitialization() { @Test void getsStateAndCancelsLocally() { + CancelHandle cancelHandle = CancelHandle.create(); + JobExecution execution = computeComponent.executeLocally( - DEFAULT, List.of(), LongJob.class.getName(), null, null); + DEFAULT, List.of(), LongJob.class.getName(), cancelHandle.token(), null + ); await().until(execution::stateAsync, willBe(jobStateWithStatus(EXECUTING))); - assertThat(execution.cancelAsync(), willBe(true)); + assertThat(cancelHandle.cancelAsync(), willCompleteSuccessfully()); await().until(execution::stateAsync, willBe(jobStateWithStatus(CANCELED))); @@ -282,8 +288,10 @@ void executesRemotelyUsingNetworkCommunication() { respondWithJobStateResponseWhenJobStateRequestIsSent(jobId, COMPLETED); respondWithJobCancelResponseWhenJobCancelRequestIsSent(jobId, false); + CancelHandle cancelHandle = CancelHandle.create(); JobExecution execution = computeComponent.executeRemotely( - DEFAULT, remoteNode, List.of(), SimpleJob.class.getName(), null, SharedComputeUtils.marshalArgOrResult("a", null) + DEFAULT, remoteNode, List.of(), SimpleJob.class.getName(), cancelHandle.token(), + SharedComputeUtils.marshalArgOrResult("a", null) ); assertThat(unwrapResult(execution), willBe("remoteResponse")); @@ -291,7 +299,7 @@ void executesRemotelyUsingNetworkCommunication() { assertThat(unwrapResult(execution), willBe("remoteResponse")); assertThat(execution.stateAsync(), willBe(jobStateWithStatus(COMPLETED))); - assertThat(execution.cancelAsync(), willBe(false)); + assertThat(cancelHandle.cancelAsync(), willCompleteSuccessfully()); assertThatExecuteRequestWasSent(SimpleJob.class.getName(), "a"); assertThatJobResultRequestWasSent(jobId); @@ -307,12 +315,13 @@ void getsStateAndCancelsRemotelyUsingNetworkCommunication() { respondWithJobStateResponseWhenJobStateRequestIsSent(jobId, EXECUTING); respondWithJobCancelResponseWhenJobCancelRequestIsSent(jobId, true); + CancelHandle cancelHandle = CancelHandle.create(); JobExecution execution = computeComponent.executeRemotely( - DEFAULT, remoteNode, List.of(), LongJob.class.getName(), null, null); + DEFAULT, remoteNode, List.of(), LongJob.class.getName(), cancelHandle.token(), null); assertThat(execution.stateAsync(), willBe(jobStateWithStatus(EXECUTING))); assertThat(unwrapResult(execution), willBe("remoteResponse")); - assertThat(execution.cancelAsync(), willBe(true)); + assertThat(cancelHandle.cancelAsync(), willCompleteSuccessfully()); assertThatExecuteRequestWasSent(LongJob.class.getName(), null); assertThatJobResultRequestWasSent(jobId); diff --git a/modules/compute/src/test/java/org/apache/ignite/internal/compute/IgniteComputeImplTest.java b/modules/compute/src/test/java/org/apache/ignite/internal/compute/IgniteComputeImplTest.java index 88de0e19ee6..7f79cb3a3b8 100644 --- a/modules/compute/src/test/java/org/apache/ignite/internal/compute/IgniteComputeImplTest.java +++ b/modules/compute/src/test/java/org/apache/ignite/internal/compute/IgniteComputeImplTest.java @@ -20,12 +20,13 @@ import static java.util.UUID.randomUUID; import static java.util.concurrent.CompletableFuture.completedFuture; import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willBe; +import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully; import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willSucceedFast; +import static org.apache.ignite.internal.testframework.matchers.JobExecutionMatcher.jobExecutionWithResultAndNode; import static org.apache.ignite.internal.util.CompletableFutures.nullCompletedFuture; import static org.apache.ignite.internal.util.CompletableFutures.trueCompletedFuture; import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.aMapWithSize; -import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.containsInAnyOrder; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyLong; @@ -37,10 +38,12 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import java.util.Collection; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.concurrent.CompletableFuture; +import org.apache.ignite.compute.BroadcastExecution; +import org.apache.ignite.compute.BroadcastJobTarget; import org.apache.ignite.compute.JobDescriptor; import org.apache.ignite.compute.JobExecution; import org.apache.ignite.compute.JobExecutionOptions; @@ -136,8 +139,9 @@ void safeCallCancelHandleAfterJobProcessing() { compute.executeAsync( JobTarget.node(localNode), JobDescriptor.builder(JOB_CLASS_NAME).units(testDeploymentUnits).build(), - cancelHandle.token(), - null), + null, + cancelHandle.token() + ), willBe("jobResponse") ); @@ -248,14 +252,18 @@ void executeBroadcastAsync() { respondWhenExecutingSimpleJobLocally(ExecutionOptions.DEFAULT); respondWhenExecutingSimpleJobRemotely(ExecutionOptions.DEFAULT); - CompletableFuture> future = compute.executeBroadcastAsync( - Set.of(localNode, remoteNode), JobDescriptor.builder(JOB_CLASS_NAME).units(testDeploymentUnits).build(), + CompletableFuture> future = compute.submitAsync( + BroadcastJobTarget.nodes(localNode, remoteNode), + JobDescriptor.builder(JOB_CLASS_NAME).units(testDeploymentUnits).build(), null ); + assertThat(future, willCompleteSuccessfully()); + Collection> executions = future.join().executions(); - assertThat(future, willBe(aMapWithSize(2))); - assertThat(future.join().keySet(), contains(localNode, remoteNode)); - assertThat(future.join().values(), contains("jobResponse", "remoteResponse")); + assertThat(executions, containsInAnyOrder( + jobExecutionWithResultAndNode("jobResponse", localNode), + jobExecutionWithResultAndNode("remoteResponse", remoteNode) + )); } private void respondWhenAskForPrimaryReplica() { @@ -269,18 +277,18 @@ private void respondWhenAskForPrimaryReplica() { private void respondWhenExecutingSimpleJobLocally(ExecutionOptions executionOptions) { when(computeComponent.executeLocally(executionOptions, testDeploymentUnits, JOB_CLASS_NAME, null, null)) - .thenReturn(completedExecution(SharedComputeUtils.marshalArgOrResult("jobResponse", null))); + .thenReturn(completedExecution(SharedComputeUtils.marshalArgOrResult("jobResponse", null), localNode)); } private void respondWhenExecutingSimpleJobLocally(ExecutionOptions executionOptions, CancellationToken token) { when(computeComponent.executeLocally(executionOptions, testDeploymentUnits, JOB_CLASS_NAME, token, null)) - .thenReturn(completedExecution(SharedComputeUtils.marshalArgOrResult("jobResponse", null))); + .thenReturn(completedExecution(SharedComputeUtils.marshalArgOrResult("jobResponse", null), localNode)); } private void respondWhenExecutingSimpleJobRemotely(ExecutionOptions options) { when(computeComponent.executeRemotelyWithFailover( eq(remoteNode), any(), eq(testDeploymentUnits), eq(JOB_CLASS_NAME), eq(options), isNull(), any() - )).thenReturn(completedExecution(SharedComputeUtils.marshalArgOrResult("remoteResponse", null))); + )).thenReturn(completedExecution(SharedComputeUtils.marshalArgOrResult("remoteResponse", null), remoteNode)); } private void verifyExecuteRemotelyWithFailover(ExecutionOptions options) { @@ -289,8 +297,8 @@ private void verifyExecuteRemotelyWithFailover(ExecutionOptions options) { ); } - private static JobExecution completedExecution(R result) { - return new JobExecution<>() { + private static CancellableJobExecution completedExecution(@Nullable R result, ClusterNode node) { + return new CancellableJobExecution<>() { @Override public CompletableFuture resultAsync() { return completedFuture(result); @@ -310,6 +318,11 @@ public CompletableFuture resultAsync() { public CompletableFuture<@Nullable Boolean> changePriorityAsync(int newPriority) { return nullCompletedFuture(); } + + @Override + public ClusterNode node() { + return node; + } }; } } diff --git a/modules/compute/src/test/java/org/apache/ignite/internal/compute/executor/ComputeExecutorTest.java b/modules/compute/src/test/java/org/apache/ignite/internal/compute/executor/ComputeExecutorTest.java index 2aac5713266..2f75ab81e7b 100644 --- a/modules/compute/src/test/java/org/apache/ignite/internal/compute/executor/ComputeExecutorTest.java +++ b/modules/compute/src/test/java/org/apache/ignite/internal/compute/executor/ComputeExecutorTest.java @@ -30,6 +30,7 @@ import static org.awaitility.Awaitility.await; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.is; +import static org.mockito.Answers.RETURNS_DEEP_STUBS; import java.util.List; import java.util.Map; @@ -49,6 +50,7 @@ import org.apache.ignite.internal.compute.state.InMemoryComputeStateMachine; import org.apache.ignite.internal.configuration.testframework.ConfigurationExtension; import org.apache.ignite.internal.configuration.testframework.InjectConfiguration; +import org.apache.ignite.internal.network.TopologyService; import org.apache.ignite.internal.testframework.BaseIgniteAbstractTest; import org.jetbrains.annotations.Nullable; import org.junit.jupiter.api.AfterEach; @@ -66,12 +68,15 @@ class ComputeExecutorTest extends BaseIgniteAbstractTest { @InjectConfiguration private ComputeConfiguration computeConfiguration; + @Mock(answer = RETURNS_DEEP_STUBS) + private TopologyService topologyService; + private ComputeExecutor computeExecutor; @BeforeEach void setUp() { InMemoryComputeStateMachine stateMachine = new InMemoryComputeStateMachine(computeConfiguration, "testNode"); - computeExecutor = new ComputeExecutorImpl(ignite, stateMachine, computeConfiguration); + computeExecutor = new ComputeExecutorImpl(ignite, stateMachine, computeConfiguration, topologyService); computeExecutor.start(); } diff --git a/modules/core/src/testFixtures/java/org/apache/ignite/internal/testframework/matchers/BroadcastExecutionMatcher.java b/modules/core/src/testFixtures/java/org/apache/ignite/internal/testframework/matchers/BroadcastExecutionMatcher.java new file mode 100644 index 00000000000..34f7851c6c9 --- /dev/null +++ b/modules/core/src/testFixtures/java/org/apache/ignite/internal/testframework/matchers/BroadcastExecutionMatcher.java @@ -0,0 +1,73 @@ +/* + * 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 + * + * http://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.ignite.internal.testframework.matchers; + +import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.will; +import static org.hamcrest.Matchers.contains; + +import java.util.Collection; +import java.util.concurrent.CompletableFuture; +import org.apache.ignite.compute.BroadcastExecution; +import org.apache.ignite.compute.JobExecution; +import org.hamcrest.Description; +import org.hamcrest.Matcher; +import org.hamcrest.TypeSafeMatcher; + +/** + * Matcher for {@link JobExecution}. + */ +public class BroadcastExecutionMatcher extends TypeSafeMatcher> { + private final Matcher>> resultsMatcher; + + private BroadcastExecutionMatcher( + Matcher>> resultsMatcher + ) { + this.resultsMatcher = resultsMatcher; + } + + public static BroadcastExecutionMatcher broadcastExecutionWithResults(R... results) { + return broadcastExecutionWithResultsFuture(will(contains(results))); + } + + public static BroadcastExecutionMatcher broadcastExecutionWithResults(Matcher... resultMatchers) { + return broadcastExecutionWithResultsFuture(will(contains(resultMatchers))); + } + + public static BroadcastExecutionMatcher broadcastExecutionWithResultsFuture( + Matcher>> resultsMatcher + ) { + return new BroadcastExecutionMatcher<>(resultsMatcher); + } + + @Override + protected boolean matchesSafely(BroadcastExecution execution) { + return resultsMatcher.matches(execution.resultsAsync()); + } + + @Override + protected void describeMismatchSafely(BroadcastExecution execution, Description mismatchDescription) { + mismatchDescription.appendText("results "); + resultsMatcher.describeMismatch(execution.resultsAsync(), mismatchDescription); + } + + @Override + public void describeTo(Description description) { + description.appendText("a BroadcastExecution with results ") + .appendDescriptionOf(resultsMatcher); + } +} diff --git a/modules/core/src/testFixtures/java/org/apache/ignite/internal/testframework/matchers/JobExecutionMatcher.java b/modules/core/src/testFixtures/java/org/apache/ignite/internal/testframework/matchers/JobExecutionMatcher.java new file mode 100644 index 00000000000..e73727ca240 --- /dev/null +++ b/modules/core/src/testFixtures/java/org/apache/ignite/internal/testframework/matchers/JobExecutionMatcher.java @@ -0,0 +1,135 @@ +/* + * 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 + * + * http://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.ignite.internal.testframework.matchers; + +import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.will; +import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willBe; +import static org.apache.ignite.internal.testframework.matchers.JobStateMatcher.jobStateWithStatus; +import static org.hamcrest.Matchers.is; + +import java.util.concurrent.CompletableFuture; +import org.apache.ignite.compute.JobExecution; +import org.apache.ignite.compute.JobState; +import org.apache.ignite.compute.JobStatus; +import org.apache.ignite.network.ClusterNode; +import org.hamcrest.Description; +import org.hamcrest.Matcher; +import org.hamcrest.TypeSafeMatcher; +import org.jetbrains.annotations.Nullable; + +/** + * Matcher for {@link JobExecution}. + */ +public class JobExecutionMatcher extends TypeSafeMatcher> { + private final Matcher> resultMatcher; + private final Matcher> stateMatcher; + private final Matcher nodeNameMatcher; + + private JobExecutionMatcher( + @Nullable Matcher> resultMatcher, + @Nullable Matcher> stateMatcher, + @Nullable Matcher nodeNameMatcher + ) { + this.resultMatcher = resultMatcher; + this.stateMatcher = stateMatcher; + this.nodeNameMatcher = nodeNameMatcher; + } + + public static JobExecutionMatcher jobExecutionWithStatus(JobStatus status) { + return new JobExecutionMatcher<>(null, will(jobStateWithStatus(status)), null); + } + + public static JobExecutionMatcher jobExecutionWithResultAndStatus(R result, JobStatus status) { + return new JobExecutionMatcher<>(willBe(result), will(jobStateWithStatus(status)), null); + } + + public static JobExecutionMatcher jobExecutionWithResultAndStatus( + Matcher resultMatcher, + Matcher statusMatcher + ) { + return new JobExecutionMatcher<>(will(resultMatcher), will(jobStateWithStatus(statusMatcher)), null); + } + + public static JobExecutionMatcher jobExecutionWithResultAndNode(R result, ClusterNode node) { + return new JobExecutionMatcher<>(willBe(result), null, is(node.name())); + } + + public static JobExecutionMatcher jobExecutionWithResultStatusAndNode(R result, JobStatus status, ClusterNode node) { + return new JobExecutionMatcher<>(willBe(result), will(jobStateWithStatus(status)), is(node.name())); + } + + public static JobExecutionMatcher jobExecutionWithResultAndStateFuture( + Matcher> resultMatcher, + Matcher> stateMatcher + ) { + return new JobExecutionMatcher<>(resultMatcher, stateMatcher, null); + } + + @Override + protected boolean matchesSafely(JobExecution execution) { + if (resultMatcher != null && !resultMatcher.matches(execution.resultAsync())) { + return false; + } + if (stateMatcher != null && !stateMatcher.matches(execution.stateAsync())) { + return false; + } + return nodeNameMatcher == null || nodeNameMatcher.matches(execution.node().name()); + } + + @Override + protected void describeMismatchSafely(JobExecution execution, Description mismatchDescription) { + if (resultMatcher != null) { + mismatchDescription.appendText("result "); + resultMatcher.describeMismatch(execution.resultAsync(), mismatchDescription); + } + if (stateMatcher != null) { + if (resultMatcher != null) { + mismatchDescription.appendText(", "); + } + mismatchDescription.appendText("state "); + stateMatcher.describeMismatch(execution.stateAsync(), mismatchDescription); + } + if (nodeNameMatcher != null) { + if (resultMatcher != null || stateMatcher != null) { + mismatchDescription.appendText(", "); + } + mismatchDescription.appendText("node "); + nodeNameMatcher.describeMismatch(execution.node().name(), mismatchDescription); + } + } + + @Override + public void describeTo(Description description) { + description.appendText("a JobExecution with "); + if (resultMatcher != null) { + description.appendText("result ").appendDescriptionOf(resultMatcher); + } + if (stateMatcher != null) { + if (resultMatcher != null) { + description.appendText(" and "); + } + description.appendText("state ").appendDescriptionOf(stateMatcher); + } + if (nodeNameMatcher != null) { + if (resultMatcher != null || stateMatcher != null) { + description.appendText(" and "); + } + description.appendText("node ").appendDescriptionOf(nodeNameMatcher); + } + } +} diff --git a/modules/distribution-zones/src/integrationTest/java/org/apache/ignite/internal/rebalance/ItRebalanceTest.java b/modules/distribution-zones/src/integrationTest/java/org/apache/ignite/internal/rebalance/ItRebalanceTest.java index 6dbcd8a32ed..1d312c3795b 100644 --- a/modules/distribution-zones/src/integrationTest/java/org/apache/ignite/internal/rebalance/ItRebalanceTest.java +++ b/modules/distribution-zones/src/integrationTest/java/org/apache/ignite/internal/rebalance/ItRebalanceTest.java @@ -36,7 +36,6 @@ import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; -import org.apache.ignite.Ignite; import org.apache.ignite.InitParametersBuilder; import org.apache.ignite.internal.ClusterConfiguration.Builder; import org.apache.ignite.internal.ClusterPerTestIntegrationTest; @@ -52,7 +51,6 @@ import org.apache.ignite.internal.schema.marshaller.reflection.KvMarshallerImpl; import org.apache.ignite.internal.schema.row.Row; import org.apache.ignite.internal.table.TableViewInternal; -import org.apache.ignite.network.ClusterNode; import org.apache.ignite.table.Tuple; import org.apache.ignite.table.mapper.Mapper; import org.junit.jupiter.api.Test; @@ -101,18 +99,18 @@ void assignmentsChangingOnNodeLeaveNodeJoin() throws Exception { BinaryRowEx row = marshalTuple(table, Tuple.create().set("id", 1).set("val", "value1")); BinaryRowEx key = marshalKey(table, 1, Integer.class); - assertThat(table.internalTable().get(key, clock.now(), clusterNode(cluster.node(0))), willBe(nullValue())); - assertThat(table.internalTable().get(key, clock.now(), clusterNode(cluster.node(1))), willBe(nullValue())); - assertThat(table.internalTable().get(key, clock.now(), clusterNode(cluster.node(2))), willBe(nullValue())); + assertThat(table.internalTable().get(key, clock.now(), clusterNode(0)), willBe(nullValue())); + assertThat(table.internalTable().get(key, clock.now(), clusterNode(1)), willBe(nullValue())); + assertThat(table.internalTable().get(key, clock.now(), clusterNode(2)), willBe(nullValue())); assertThat(table.internalTable().insert(row, null), willCompleteSuccessfully()); - assertThat(table.internalTable().get(key, clock.now(), clusterNode(cluster.node(0))), willBe(notNullValue())); - assertThat(table.internalTable().get(key, clock.now(), clusterNode(cluster.node(1))), willBe(notNullValue())); - assertThat(table.internalTable().get(key, clock.now(), clusterNode(cluster.node(2))), willBe(notNullValue())); + assertThat(table.internalTable().get(key, clock.now(), clusterNode(0)), willBe(notNullValue())); + assertThat(table.internalTable().get(key, clock.now(), clusterNode(1)), willBe(notNullValue())); + assertThat(table.internalTable().get(key, clock.now(), clusterNode(2)), willBe(notNullValue())); assertThat( - table.internalTable().get(key, clock.now(), clusterNode(cluster.node(3))), + table.internalTable().get(key, clock.now(), clusterNode(3)), willThrow(ReplicationException.class, 10, TimeUnit.SECONDS) ); @@ -124,9 +122,9 @@ void assignmentsChangingOnNodeLeaveNodeJoin() throws Exception { nodeName(3) ), table.tableId()); - assertThat(table.internalTable().get(key, clock.now(), clusterNode(cluster.node(0))), willBe(notNullValue())); - assertThat(table.internalTable().get(key, clock.now(), clusterNode(cluster.node(1))), willBe(notNullValue())); - assertThat(table.internalTable().get(key, clock.now(), clusterNode(cluster.node(3))), willBe(notNullValue())); + assertThat(table.internalTable().get(key, clock.now(), clusterNode(0)), willBe(notNullValue())); + assertThat(table.internalTable().get(key, clock.now(), clusterNode(1)), willBe(notNullValue())); + assertThat(table.internalTable().get(key, clock.now(), clusterNode(3)), willBe(notNullValue())); cluster.startNode(2); @@ -136,20 +134,16 @@ void assignmentsChangingOnNodeLeaveNodeJoin() throws Exception { nodeName(2) ), table.tableId()); - assertThat(table.internalTable().get(key, clock.now(), clusterNode(cluster.node(0))), willBe(notNullValue())); - assertThat(table.internalTable().get(key, clock.now(), clusterNode(cluster.node(1))), willBe(notNullValue())); - assertThat(table.internalTable().get(key, clock.now(), clusterNode(cluster.node(2))), willBe(notNullValue())); + assertThat(table.internalTable().get(key, clock.now(), clusterNode(0)), willBe(notNullValue())); + assertThat(table.internalTable().get(key, clock.now(), clusterNode(1)), willBe(notNullValue())); + assertThat(table.internalTable().get(key, clock.now(), clusterNode(2)), willBe(notNullValue())); assertThat( - table.internalTable().get(key, clock.now(), clusterNode(cluster.node(3))), + table.internalTable().get(key, clock.now(), clusterNode(3)), willThrow(ReplicationException.class, 10, TimeUnit.SECONDS) ); } - private static ClusterNode clusterNode(Ignite ignite) { - return unwrapIgniteImpl(ignite).node(); - } - private static Row marshalTuple(TableViewInternal table, Tuple tuple) { SchemaRegistry schemaReg = table.schemaView(); var marshaller = new TupleMarshallerImpl(schemaReg.lastKnownSchema()); diff --git a/modules/rest/src/integrationTest/java/org/apache/ignite/internal/rest/compute/ItComputeControllerTest.java b/modules/rest/src/integrationTest/java/org/apache/ignite/internal/rest/compute/ItComputeControllerTest.java index f5546d185bf..13e65ad5c33 100644 --- a/modules/rest/src/integrationTest/java/org/apache/ignite/internal/rest/compute/ItComputeControllerTest.java +++ b/modules/rest/src/integrationTest/java/org/apache/ignite/internal/rest/compute/ItComputeControllerTest.java @@ -27,6 +27,7 @@ import static org.apache.ignite.internal.rest.matcher.RestJobStateMatcher.completed; import static org.apache.ignite.internal.rest.matcher.RestJobStateMatcher.executing; import static org.apache.ignite.internal.rest.matcher.RestJobStateMatcher.queued; +import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully; import static org.awaitility.Awaitility.await; import static org.hamcrest.MatcherAssert.assertThat; @@ -309,7 +310,10 @@ void shouldReturnFalseIfUpdatePriorityOfCompletedJob() { } private static JobExecution runBlockingJob(Ignite entryNode, Set nodes) { - return entryNode.compute().submit(JobTarget.anyNode(nodes), JobDescriptor.builder(BlockingJob.class).build(), null); + CompletableFuture> executionFut = entryNode.compute() + .submitAsync(JobTarget.anyNode(nodes), JobDescriptor.builder(BlockingJob.class).build(), null); + assertThat(executionFut, willCompleteSuccessfully()); + return executionFut.join(); } private static void unblockJob() { diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/app/AsyncApiOperation.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/app/AsyncApiOperation.java index bbe191793f7..f66143fd171 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/app/AsyncApiOperation.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/app/AsyncApiOperation.java @@ -31,9 +31,9 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; -import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.function.Function; +import org.apache.ignite.compute.BroadcastJobTarget; import org.apache.ignite.compute.JobDescriptor; import org.apache.ignite.compute.TaskDescriptor; import org.apache.ignite.internal.streamer.SimplePublisher; @@ -136,11 +136,19 @@ enum AsyncApiOperation { SQL_EXECUTE_BATCH_STATEMENT(refs -> refs.sql.executeBatchAsync(null, refs.updateStatement, BatchedArguments.of(999))), SQL_EXECUTE_SCRIPT(refs -> refs.sql.executeScriptAsync(SELECT_IDS_QUERY)), + COMPUTE_SUBMIT(refs -> refs.compute.submitAsync( + anyNode(refs.clusterNodes), JobDescriptor.builder(NoOpJob.class).build(), null + )), + COMPUTE_SUBMIT_BROADCAST(refs -> refs.compute.submitAsync( + BroadcastJobTarget.nodes(refs.clusterNodes), + JobDescriptor.builder(NoOpJob.class).build(), + null + )), COMPUTE_EXECUTE(refs -> refs.compute.executeAsync( anyNode(refs.clusterNodes), JobDescriptor.builder(NoOpJob.class).build(), null )), - COMPUTE_EXECUTE_BROADCAST(refs -> refs.compute.executeBroadcastAsync( - Set.copyOf(refs.clusterNodes), + COMPUTE_EXECUTE_BROADCAST(refs -> refs.compute.executeAsync( + BroadcastJobTarget.nodes(refs.clusterNodes), JobDescriptor.builder(NoOpJob.class).build(), null )), diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/app/ItInProcessRestartApiReferencesTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/app/ItInProcessRestartApiReferencesTest.java index 4ed21020efe..b2a9180a9f9 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/app/ItInProcessRestartApiReferencesTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/app/ItInProcessRestartApiReferencesTest.java @@ -22,6 +22,7 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import java.util.concurrent.CompletableFuture; import org.apache.ignite.compute.JobDescriptor; import org.apache.ignite.compute.JobExecution; import org.apache.ignite.internal.ClusterPerClassIntegrationTest; @@ -63,11 +64,13 @@ static void init() throws Exception { @Test void submitStaysLocalAfterRestart() { - JobExecution execution = beforeRestart.compute.submit( + CompletableFuture> executionFut = beforeRestart.compute.submitAsync( anyNode(beforeRestart.clusterNodes), JobDescriptor.builder(NoOpJob.class).build(), null ); + assertThat(executionFut, willCompleteSuccessfully()); + JobExecution execution = executionFut.join(); assertThat(execution.stateAsync(), willCompleteSuccessfully()); } diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/app/SyncApiOperation.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/app/SyncApiOperation.java index 14888912852..57dcbec76ac 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/app/SyncApiOperation.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/app/SyncApiOperation.java @@ -29,8 +29,8 @@ import java.util.List; import java.util.Map; -import java.util.Set; import java.util.function.Consumer; +import org.apache.ignite.compute.BroadcastJobTarget; import org.apache.ignite.compute.JobDescriptor; import org.apache.ignite.compute.TaskDescriptor; import org.apache.ignite.sql.BatchedArguments; @@ -132,15 +132,9 @@ enum SyncApiOperation { SQL_EXECUTE_BATCH_STATEMENT(refs -> refs.sql.executeBatch(null, refs.updateStatement, BatchedArguments.of(999))), SQL_EXECUTE_SCRIPT(refs -> refs.sql.executeScript(SELECT_IDS_QUERY)), - COMPUTE_SUBMIT(refs -> refs.compute.submit(anyNode(refs.clusterNodes), JobDescriptor.builder(NoOpJob.class).build(), null)), COMPUTE_EXECUTE(refs -> refs.compute.execute(anyNode(refs.clusterNodes), JobDescriptor.builder(NoOpJob.class).build(), null)), - COMPUTE_SUBMIT_BROADCAST(refs -> refs.compute.submitBroadcast( - Set.copyOf(refs.clusterNodes), - JobDescriptor.builder(NoOpJob.class).build(), - null - )), - COMPUTE_EXECUTE_BROADCAST(refs -> refs.compute.executeBroadcast( - Set.copyOf(refs.clusterNodes), + COMPUTE_EXECUTE_BROADCAST(refs -> refs.compute.execute( + BroadcastJobTarget.nodes(refs.clusterNodes), JobDescriptor.builder(NoOpJob.class).build(), null )), diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/client/ItAbstractThinClientTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/client/ItAbstractThinClientTest.java index 47154304c42..fd58888024c 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/client/ItAbstractThinClientTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/client/ItAbstractThinClientTest.java @@ -32,18 +32,27 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteServer; import org.apache.ignite.InitParameters; import org.apache.ignite.client.IgniteClient; +import org.apache.ignite.compute.BroadcastExecution; +import org.apache.ignite.compute.BroadcastJobTarget; +import org.apache.ignite.compute.JobDescriptor; +import org.apache.ignite.compute.JobExecution; +import org.apache.ignite.compute.JobTarget; import org.apache.ignite.internal.testframework.BaseIgniteAbstractTest; import org.apache.ignite.internal.testframework.IgniteTestUtils; import org.apache.ignite.internal.testframework.TestIgnitionManager; import org.apache.ignite.internal.testframework.WorkDirectory; import org.apache.ignite.internal.testframework.WorkDirectoryExtension; import org.apache.ignite.internal.util.IgniteUtils; +import org.apache.ignite.lang.CancellationToken; import org.apache.ignite.network.ClusterNode; import org.apache.ignite.sql.IgniteSql; +import org.jetbrains.annotations.Nullable; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.TestInfo; @@ -205,6 +214,61 @@ protected List sortedNodes() { .collect(toList()); } + /** + * Submits the job for execution, verifies that the execution future completes successfully and returns an execution object. + * + * @param Job argument (T)ype. + * @param Job (R)esult type. + * @param target Execution target. + * @param descriptor Job descriptor. + * @param arg Argument of the job. + * @return Job execution object. + */ + protected JobExecution submit( + JobTarget target, + JobDescriptor descriptor, + @Nullable T arg + ) { + return submit(target, descriptor, null, arg); + } + + protected JobExecution submit( + JobTarget target, + JobDescriptor descriptor, + @Nullable CancellationToken cancellationToken, + @Nullable T arg + ) { + //noinspection resource (closed in afterAll) + CompletableFuture> executionFut = client().compute().submitAsync(target, descriptor, arg, cancellationToken); + assertThat(executionFut, willCompleteSuccessfully()); + return executionFut.join(); + } + + protected BroadcastExecution submit( + Set nodes, + JobDescriptor descriptor, + @Nullable T arg + ) { + return submit(nodes, descriptor, null, arg); + } + + protected BroadcastExecution submit( + Set nodes, + JobDescriptor descriptor, + @Nullable CancellationToken cancellationToken, + @Nullable T arg + ) { + //noinspection resource (closed in afterAll) + CompletableFuture> executionFut = client().compute().submitAsync( + BroadcastJobTarget.nodes(nodes), + descriptor, + arg, + cancellationToken + ); + assertThat(executionFut, willCompleteSuccessfully()); + return executionFut.join(); + } + /** * Test class. */ diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/client/ItThinClientComputeMarshallingTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/client/ItThinClientComputeMarshallingTest.java index 41836de82b6..da5f387a52e 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/client/ItThinClientComputeMarshallingTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/client/ItThinClientComputeMarshallingTest.java @@ -24,15 +24,15 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; +import java.util.Collection; import java.util.List; import java.util.Map; -import java.util.Map.Entry; -import java.util.Set; import java.util.concurrent.ExecutionException; import java.util.stream.Collectors; import org.apache.ignite.catalog.ColumnType; import org.apache.ignite.catalog.definitions.TableDefinition; import org.apache.ignite.client.IgniteClient; +import org.apache.ignite.compute.BroadcastJobTarget; import org.apache.ignite.compute.JobDescriptor; import org.apache.ignite.compute.JobExecution; import org.apache.ignite.compute.JobTarget; @@ -213,8 +213,8 @@ void pojoJobWithMarshallers(int targetNodeIdx) { @Test void executeBroadcast() { // When. - Map result = client().compute().executeBroadcast( - Set.of(node(0), node(1)), + Collection result = client().compute().execute( + BroadcastJobTarget.nodes(node(0), node(1)), JobDescriptor.builder(ArgumentAndResultMarshallingJob.class) .argumentMarshaller(new ArgumentStringMarshaller()) .resultMarshaller(new ResultStringUnMarshaller()) @@ -223,9 +223,9 @@ void executeBroadcast() { ); // Then. - Map resultExpected = Map.of( - node(0), "Input:marshalledOnClient:unmarshalledOnServer:processedOnServer:marshalledOnServer:unmarshalledOnClient", - node(1), "Input:marshalledOnClient:unmarshalledOnServer:processedOnServer:marshalledOnServer:unmarshalledOnClient" + List resultExpected = List.of( + "Input:marshalledOnClient:unmarshalledOnServer:processedOnServer:marshalledOnServer:unmarshalledOnClient", + "Input:marshalledOnClient:unmarshalledOnServer:processedOnServer:marshalledOnServer:unmarshalledOnClient" ); assertEquals(resultExpected, result); @@ -235,29 +235,29 @@ void executeBroadcast() { @Test void submitBroadcast() { // When. - Map result = client().compute().submitBroadcast( - Set.of(node(0), node(1)), + Map result = client().compute().submitAsync( + BroadcastJobTarget.nodes(node(0), node(1)), JobDescriptor.builder(ArgumentAndResultMarshallingJob.class) .argumentMarshaller(new ArgumentStringMarshaller()) .resultMarshaller(new ResultStringUnMarshaller()) .build(), "Input" - ).entrySet().stream().collect( - Collectors.toMap(Entry::getKey, ItThinClientComputeMarshallingTest::extractResult, (v, i) -> v) - ); + ).thenApply(broadcastExecution -> broadcastExecution.executions().stream().collect( + Collectors.toMap(execution -> execution.node().name(), ItThinClientComputeMarshallingTest::extractResult, (v, i) -> v) + )).join(); // Then. - Map resultExpected = Map.of( - node(0), "Input:marshalledOnClient:unmarshalledOnServer:processedOnServer:marshalledOnServer:unmarshalledOnClient", - node(1), "Input:marshalledOnClient:unmarshalledOnServer:processedOnServer:marshalledOnServer:unmarshalledOnClient" + Map resultExpected = Map.of( + node(0).name(), "Input:marshalledOnClient:unmarshalledOnServer:processedOnServer:marshalledOnServer:unmarshalledOnClient", + node(1).name(), "Input:marshalledOnClient:unmarshalledOnServer:processedOnServer:marshalledOnServer:unmarshalledOnClient" ); assertEquals(resultExpected, result); } - private static String extractResult(Entry> e) { + private static String extractResult(JobExecution e) { try { - return e.getValue().resultAsync().get(); + return e.resultAsync().get(); } catch (InterruptedException | ExecutionException ex) { throw new RuntimeException(ex); } diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/client/ItThinClientComputeTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/client/ItThinClientComputeTest.java index fb994373d76..6ef5de0ec9f 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/client/ItThinClientComputeTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/client/ItThinClientComputeTest.java @@ -25,9 +25,13 @@ import static org.apache.ignite.compute.JobStatus.FAILED; import static org.apache.ignite.compute.JobStatus.QUEUED; import static org.apache.ignite.internal.IgniteExceptionTestUtils.assertTraceableException; +import static org.apache.ignite.internal.testframework.matchers.CompletableFutureExceptionMatcher.willThrow; import static org.apache.ignite.internal.testframework.matchers.CompletableFutureExceptionMatcher.willThrowFast; import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.will; import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willBe; +import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully; +import static org.apache.ignite.internal.testframework.matchers.JobExecutionMatcher.jobExecutionWithResultStatusAndNode; +import static org.apache.ignite.internal.testframework.matchers.JobExecutionMatcher.jobExecutionWithStatus; import static org.apache.ignite.internal.testframework.matchers.JobStateMatcher.jobStateWithStatus; import static org.apache.ignite.internal.testframework.matchers.TaskStateMatcher.taskStateWithStatus; import static org.apache.ignite.internal.util.CompletableFutures.nullCompletedFuture; @@ -37,6 +41,8 @@ import static org.awaitility.Awaitility.await; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.allOf; +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.everyItem; import static org.hamcrest.Matchers.hasItem; @@ -57,6 +63,7 @@ import java.time.LocalDateTime; import java.time.LocalTime; import java.util.Arrays; +import java.util.Collection; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -67,11 +74,14 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; +import java.util.function.Function; import java.util.function.Supplier; import java.util.stream.Collectors; import org.apache.ignite.client.IgniteClient; import org.apache.ignite.client.IgniteClient.Builder; import org.apache.ignite.client.IgniteClientConnectionException; +import org.apache.ignite.compute.BroadcastExecution; +import org.apache.ignite.compute.BroadcastJobTarget; import org.apache.ignite.compute.ComputeException; import org.apache.ignite.compute.ComputeJob; import org.apache.ignite.compute.IgniteCompute; @@ -86,7 +96,7 @@ import org.apache.ignite.compute.task.TaskExecution; import org.apache.ignite.compute.task.TaskExecutionContext; import org.apache.ignite.deployment.DeploymentUnit; -import org.apache.ignite.internal.compute.TaskToJobExecutionWrapper; +import org.apache.ignite.internal.compute.JobTaskStatusMapper; import org.apache.ignite.internal.testframework.IgniteTestUtils; import org.apache.ignite.lang.CancelHandle; import org.apache.ignite.lang.IgniteException; @@ -145,7 +155,7 @@ void computeExecuteAsyncWithCancelHandle() { CancelHandle cancelHandle = CancelHandle.create(); JobDescriptor job = JobDescriptor.builder(InfiniteJob.class).units(List.of()).build(); - CompletableFuture execution = entryNode.compute().executeAsync(JobTarget.node(executeNode), job, cancelHandle.token(), null); + CompletableFuture execution = entryNode.compute().executeAsync(JobTarget.node(executeNode), job, null, cancelHandle.token()); cancelHandle.cancel(); @@ -161,7 +171,7 @@ void computeExecuteWithCancelHandle() { JobDescriptor job = JobDescriptor.builder(InfiniteJob.class).units(List.of()).build(); CompletableFuture runFut = IgniteTestUtils.runAsync(() -> entryNode.compute() - .execute(JobTarget.node(executeNode), job, cancelHandle.token(), null)); + .execute(JobTarget.node(executeNode), job, null, cancelHandle.token())); cancelHandle.cancel(); @@ -170,35 +180,34 @@ void computeExecuteWithCancelHandle() { @Test void computeExecuteBroadcastAsyncWithCancelHandle() { - IgniteClient entryNode = client(); - Set executeNodes = Set.of(node(0), node(1)); - CancelHandle cancelHandle = CancelHandle.create(); - CompletableFuture> executions = entryNode.compute().executeBroadcastAsync( - executeNodes, - JobDescriptor.builder(InfiniteJob.class.getName()).build(), cancelHandle.token(), 100L); + CompletableFuture> results = client().compute().executeAsync( + BroadcastJobTarget.nodes(node(0), node(1)), + JobDescriptor.builder(InfiniteJob.class).build(), + 100L, + cancelHandle.token() + ); cancelHandle.cancel(); - assertThrows(ExecutionException.class, () -> executions.get(10, TimeUnit.SECONDS)); + assertThat(results, willThrow(ComputeException.class)); } @Test void computeExecuteBroadcastWithCancelHandle() { - IgniteClient entryNode = client(); - Set executeNodes = Set.of(node(0), node(1)); - CancelHandle cancelHandle = CancelHandle.create(); - CompletableFuture> runFut = IgniteTestUtils.runAsync(() -> entryNode.compute().executeBroadcast( - executeNodes, - JobDescriptor.builder(InfiniteJob.class.getName()).build(), cancelHandle.token(), 100L) - ); + CompletableFuture> runFut = IgniteTestUtils.runAsync(() -> client().compute().execute( + BroadcastJobTarget.nodes(node(0), node(1)), + JobDescriptor.builder(InfiniteJob.class).build(), + 100L, + cancelHandle.token() + )); cancelHandle.cancel(); - assertThrows(ExecutionException.class, () -> runFut.get(10, TimeUnit.SECONDS)); + assertThat(runFut, willThrow(ComputeException.class)); } @Test @@ -208,7 +217,7 @@ void cancelComputeExecuteMapReduceAsyncWithCancelHandle() { CancelHandle cancelHandle = CancelHandle.create(); CompletableFuture execution = entryNode.compute() - .executeMapReduceAsync(TaskDescriptor.builder(InfiniteMapReduceTask.class).build(), cancelHandle.token(), null); + .executeMapReduceAsync(TaskDescriptor.builder(InfiniteMapReduceTask.class).build(), null, cancelHandle.token()); cancelHandle.cancel(); @@ -217,10 +226,10 @@ void cancelComputeExecuteMapReduceAsyncWithCancelHandle() { @Test void testExecuteOnSpecificNodeAsync() { - JobExecution execution1 = client().compute().submit( + JobExecution execution1 = submit( JobTarget.node(node(0)), JobDescriptor.builder(NodeNameJob.class).build(), null ); - JobExecution execution2 = client().compute().submit( + JobExecution execution2 = submit( JobTarget.node(node(1)), JobDescriptor.builder(NodeNameJob.class).build(), null ); @@ -233,21 +242,24 @@ void testExecuteOnSpecificNodeAsync() { @Test void testCancellingCompletedJob() { - JobExecution execution = client().compute().submit( + CancelHandle cancelHandle = CancelHandle.create(); + JobExecution execution = submit( JobTarget.node(node(0)), - JobDescriptor.builder(NodeNameJob.class).build(), null + JobDescriptor.builder(NodeNameJob.class).build(), + cancelHandle.token(), + null ); assertThat(execution.resultAsync(), willBe("itcct_n_3344")); assertThat(execution.stateAsync(), willBe(jobStateWithStatus(COMPLETED))); - assertThat(execution.cancelAsync(), willBe(false)); + assertThat(cancelHandle.cancelAsync(), willCompleteSuccessfully()); } @Test void testChangingPriorityCompletedJob() { - JobExecution execution = client().compute().submit( + JobExecution execution = submit( JobTarget.node(node(0)), JobDescriptor.builder(NodeNameJob.class).build(), null ); @@ -267,14 +279,14 @@ void testCancelOnSpecificNodeAsync(boolean asyncJob) { .builder(asyncJob ? AsyncSleepJob.class : SleepJob.class) .build(); - JobExecution execution1 = client().compute().submit(JobTarget.node(node(0)), sleepJob, sleepMs); - JobExecution execution2 = client().compute().submit(JobTarget.node(node(1)), sleepJob, sleepMs); + CancelHandle cancelHandle = CancelHandle.create(); + JobExecution execution1 = submit(JobTarget.node(node(0)), sleepJob, cancelHandle.token(), sleepMs); + JobExecution execution2 = submit(JobTarget.node(node(1)), sleepJob, cancelHandle.token(), sleepMs); await().until(execution1::stateAsync, willBe(jobStateWithStatus(EXECUTING))); await().until(execution2::stateAsync, willBe(jobStateWithStatus(EXECUTING))); - assertThat(execution1.cancelAsync(), willBe(true)); - assertThat(execution2.cancelAsync(), willBe(true)); + assertThat(cancelHandle.cancelAsync(), willCompleteSuccessfully()); await().until(execution1::stateAsync, willBe(jobStateWithStatus(CANCELED))); await().until(execution2::stateAsync, willBe(jobStateWithStatus(CANCELED))); @@ -287,15 +299,18 @@ void changeJobPriority() { JobTarget target = JobTarget.node(node(0)); // Start 1 task in executor with 1 thread - JobExecution execution1 = client().compute().submit(target, sleepJob, sleepMs); + CancelHandle cancelHandle1 = CancelHandle.create(); + JobExecution execution1 = submit(target, sleepJob, cancelHandle1.token(), sleepMs); await().until(execution1::stateAsync, willBe(jobStateWithStatus(EXECUTING))); // Start one more long lasting task - JobExecution execution2 = client().compute().submit(target, sleepJob, sleepMs); + CancelHandle cancelHandle2 = CancelHandle.create(); + JobExecution execution2 = submit(target, sleepJob, cancelHandle2.token(), sleepMs); await().until(execution2::stateAsync, willBe(jobStateWithStatus(QUEUED))); // Start third task - JobExecution execution3 = client().compute().submit(target, sleepJob, sleepMs); + CancelHandle cancelHandle3 = CancelHandle.create(); + JobExecution execution3 = submit(target, sleepJob, cancelHandle3.token(), sleepMs); await().until(execution3::stateAsync, willBe(jobStateWithStatus(QUEUED))); // Task 2 and 3 are not completed, in queue state @@ -306,7 +321,7 @@ void changeJobPriority() { assertThat(execution3.changePriorityAsync(2), willBe(true)); // Cancel task 1, task 3 should start executing - assertThat(execution1.cancelAsync(), willBe(true)); + assertThat(cancelHandle1.cancelAsync(), willCompleteSuccessfully()); await().until(execution1::stateAsync, willBe(jobStateWithStatus(CANCELED))); await().until(execution3::stateAsync, willBe(jobStateWithStatus(EXECUTING))); @@ -314,8 +329,8 @@ void changeJobPriority() { assertThat(execution2.stateAsync(), willBe(jobStateWithStatus(QUEUED))); // Cleanup - assertThat(execution2.cancelAsync(), willBe(true)); - assertThat(execution3.cancelAsync(), willBe(true)); + assertThat(cancelHandle2.cancelAsync(), willCompleteSuccessfully()); + assertThat(cancelHandle3.cancelAsync(), willCompleteSuccessfully()); } @Test @@ -327,7 +342,7 @@ void testExecuteOnRandomNode() { @Test void testExecuteOnRandomNodeAsync() { - JobExecution execution = client().compute().submit( + JobExecution execution = submit( JobTarget.anyNode(sortedNodes()), JobDescriptor.builder(NodeNameJob.class).build(), null); assertThat( @@ -339,65 +354,63 @@ void testExecuteOnRandomNodeAsync() { @Test void testBroadcastOneNode() { - Map> executionsPerNode = client().compute().submitBroadcast( + BroadcastExecution broadcastExecution = submit( Set.of(node(1)), JobDescriptor.builder(NodeNameJob.class).build(), - null); - - assertEquals(1, executionsPerNode.size()); - - JobExecution execution = executionsPerNode.get(node(1)); + null + ); - assertThat(execution.resultAsync(), willBe("itcct_n_3345")); - assertThat(execution.stateAsync(), willBe(jobStateWithStatus(COMPLETED))); + assertThat(broadcastExecution.executions(), contains( + jobExecutionWithResultStatusAndNode("itcct_n_3345", COMPLETED, node(1)) + )); } @Test void testBroadcastAllNodes() { - Map> executionsPerNode = client().compute().submitBroadcast( + BroadcastExecution broadcastExecution = submit( new HashSet<>(sortedNodes()), JobDescriptor.builder(NodeNameJob.class).build(), null ); - assertEquals(2, executionsPerNode.size()); - - JobExecution execution1 = executionsPerNode.get(node(0)); - JobExecution execution2 = executionsPerNode.get(node(1)); - - assertThat(execution1.resultAsync(), willBe("itcct_n_3344")); - assertThat(execution2.resultAsync(), willBe("itcct_n_3345")); - - assertThat(execution1.stateAsync(), willBe(jobStateWithStatus(COMPLETED))); - assertThat(execution2.stateAsync(), willBe(jobStateWithStatus(COMPLETED))); + assertThat(broadcastExecution.executions(), containsInAnyOrder( + jobExecutionWithResultStatusAndNode("itcct_n_3344", COMPLETED, node(0)), + jobExecutionWithResultStatusAndNode("itcct_n_3345", COMPLETED, node(1)) + )); } @Test void testCancelBroadcastAllNodes() { int sleepMs = 1_000_000; - Map> executionsPerNode = client().compute().submitBroadcast( + CancelHandle cancelHandle = CancelHandle.create(); + + BroadcastExecution broadcastExecution = submit( new HashSet<>(sortedNodes()), JobDescriptor.builder(SleepJob.class).build(), - sleepMs); + cancelHandle.token(), + sleepMs + ); - assertEquals(2, executionsPerNode.size()); + Collection> executions = broadcastExecution.executions(); - JobExecution execution1 = executionsPerNode.get(node(0)); - JobExecution execution2 = executionsPerNode.get(node(1)); + await().until(() -> executions, contains( + jobExecutionWithStatus(EXECUTING), + jobExecutionWithStatus(EXECUTING) + )); - await().until(execution1::stateAsync, willBe(jobStateWithStatus(EXECUTING))); - await().until(execution2::stateAsync, willBe(jobStateWithStatus(EXECUTING))); + cancelHandle.cancel(); - assertThat(execution1.cancelAsync(), willBe(true)); - assertThat(execution2.cancelAsync(), willBe(true)); + await().until(() -> executions, contains( + jobExecutionWithStatus(CANCELED), + jobExecutionWithStatus(CANCELED) + )); - await().until(execution1::stateAsync, willBe(jobStateWithStatus(CANCELED))); - await().until(execution2::stateAsync, willBe(jobStateWithStatus(CANCELED))); + assertThat(broadcastExecution.resultsAsync(), willThrow(ComputeException.class)); } @Test void testExecuteWithArgs() { - JobExecution execution = client().compute().submit( + JobExecution execution = submit( JobTarget.anyNode(client().clusterNodes()), JobDescriptor.builder(ConcatJob.class).build(), "1:2:3.3" ); @@ -409,7 +422,7 @@ void testExecuteWithArgs() { @Test void testIgniteExceptionInJobPropagatesToClientWithMessageAndCodeAndTraceIdAsync() { IgniteException cause = getExceptionInJobExecutionAsync( - client().compute().submit(JobTarget.node(node(0)), JobDescriptor.builder(IgniteExceptionJob.class).build(), null) + submit(JobTarget.node(node(0)), JobDescriptor.builder(IgniteExceptionJob.class).build(), null) ); assertThat(cause.getMessage(), containsString("Custom job error")); @@ -442,7 +455,7 @@ void testIgniteExceptionInJobPropagatesToClientWithMessageAndCodeAndTraceIdSync( @ValueSource(booleans = {true, false}) void testExceptionInJobPropagatesToClientWithClassAndMessageAsync(boolean asyncJob) { IgniteException cause = getExceptionInJobExecutionAsync( - client().compute().submit(JobTarget.node(node(0)), JobDescriptor.builder(ExceptionJob.class).build(), asyncJob) + submit(JobTarget.node(node(0)), JobDescriptor.builder(ExceptionJob.class).build(), asyncJob) ); assertComputeExceptionWithClassAndMessage(cause); @@ -462,7 +475,7 @@ void testExceptionInJobPropagatesToClientWithClassAndMessageSync(boolean asyncJo void testExceptionInJobWithSendServerExceptionStackTraceToClientPropagatesToClientWithStackTraceAsync() { // Second node has sendServerExceptionStackTraceToClient enabled. IgniteException cause = getExceptionInJobExecutionAsync( - client().compute().submit( + submit( JobTarget.node(node(1)), JobDescriptor.builder(ExceptionJob.class).build(), null ) ); @@ -482,14 +495,19 @@ void testExceptionInJobWithSendServerExceptionStackTraceToClientPropagatesToClie @Test void testExceptionInBroadcastJobPropagatesToClient() { - Map> executions = client().compute().submitBroadcast( + BroadcastExecution broadcastExecution = submit( Set.of(node(0), node(1)), - JobDescriptor.builder(ExceptionJob.class).build(), null); + JobDescriptor.builder(ExceptionJob.class).build(), + null + ); - assertComputeExceptionWithClassAndMessage(getExceptionInJobExecutionAsync(executions.get(node(0)))); + Map> executions = broadcastExecution.executions().stream() + .collect(Collectors.toMap(execution -> execution.node().name(), Function.identity())); + + assertComputeExceptionWithClassAndMessage(getExceptionInJobExecutionAsync(executions.get(node(0).name()))); // Second node has sendServerExceptionStackTraceToClient enabled. - assertComputeExceptionWithStackTrace(getExceptionInJobExecutionAsync(executions.get(node(1)))); + assertComputeExceptionWithStackTrace(getExceptionInJobExecutionAsync(executions.get(node(1).name()))); } @Test @@ -497,7 +515,7 @@ void testExceptionInColocatedTupleJobPropagatesToClientWithClassAndMessageAsync( var key = Tuple.create().set(COLUMN_KEY, 1); IgniteException cause = getExceptionInJobExecutionAsync( - client().compute().submit(JobTarget.colocated(TABLE_NAME, key), JobDescriptor.builder(ExceptionJob.class).build(), null)); + submit(JobTarget.colocated(TABLE_NAME, key), JobDescriptor.builder(ExceptionJob.class).build(), null)); assertComputeExceptionWithClassAndMessage(cause); } @@ -522,9 +540,8 @@ void testExceptionInColocatedTupleJobWithSendServerExceptionStackTraceToClientPr // Second node has sendServerExceptionStackTraceToClient enabled. var key = Tuple.create().set(COLUMN_KEY, 2); - IgniteCompute igniteCompute = client().compute(); IgniteException cause = getExceptionInJobExecutionAsync( - igniteCompute.submit(JobTarget.colocated(TABLE_NAME, key), JobDescriptor.builder(ExceptionJob.class).build(), null) + submit(JobTarget.colocated(TABLE_NAME, key), JobDescriptor.builder(ExceptionJob.class).build(), null) ); assertComputeExceptionWithStackTrace(cause); @@ -551,9 +568,8 @@ void testExceptionInColocatedPojoJobPropagatesToClientWithClassAndMessageAsync() var key = new TestPojo(1); Mapper mapper = Mapper.of(TestPojo.class); - IgniteCompute igniteCompute = client().compute(); IgniteException cause = getExceptionInJobExecutionAsync( - igniteCompute.submit(JobTarget.colocated( + submit(JobTarget.colocated( TABLE_NAME, key, mapper), JobDescriptor.builder(ExceptionJob.class).build(), null ) ); @@ -580,9 +596,8 @@ void testExceptionInColocatedPojoJobWithSendServerExceptionStackTraceToClientPro var key = new TestPojo(2); Mapper mapper = Mapper.of(TestPojo.class); - IgniteCompute igniteCompute = client().compute(); IgniteException cause = getExceptionInJobExecutionAsync( - igniteCompute.submit(JobTarget.colocated(TABLE_NAME, key, mapper), JobDescriptor.builder(ExceptionJob.class).build(), null) + submit(JobTarget.colocated(TABLE_NAME, key, mapper), JobDescriptor.builder(ExceptionJob.class).build(), null) ); assertComputeExceptionWithStackTrace(cause); @@ -613,6 +628,17 @@ private static IgniteException getExceptionInJobExecutionAsync(JobExecution execution) { + CompletionException ex = assertThrows( + CompletionException.class, + () -> execution.resultAsync().join() + ); + + assertThat(execution.stateAsync().thenApply(JobTaskStatusMapper::toJobState), willBe(jobStateWithStatus(FAILED))); + + return (IgniteException) ex.getCause(); + } + private static IgniteException getExceptionInJobExecutionSync(Supplier execution) { IgniteException ex = assertThrows(IgniteException.class, execution::get); @@ -647,8 +673,7 @@ private static void assertComputeExceptionWithStackTrace(IgniteException cause) void testExecuteColocatedTupleRunsComputeJobOnKeyNode(int key, int port) { var keyTuple = Tuple.create().set(COLUMN_KEY, key); - IgniteCompute igniteCompute = client().compute(); - JobExecution tupleExecution = igniteCompute.submit( + JobExecution tupleExecution = submit( JobTarget.colocated(TABLE_NAME, keyTuple), JobDescriptor.builder(NodeNameJob.class).build(), null ); @@ -663,9 +688,8 @@ void testExecuteColocatedTupleRunsComputeJobOnKeyNode(int key, int port) { void testExecuteColocatedPojoRunsComputeJobOnKeyNode(int key, int port) { var keyPojo = new TestPojo(key); - IgniteCompute igniteCompute = client().compute(); Mapper keyMapper = Mapper.of(TestPojo.class); - JobExecution pojoExecution = igniteCompute.submit( + JobExecution pojoExecution = submit( JobTarget.colocated(TABLE_NAME, keyPojo, keyMapper), JobDescriptor.builder(NodeNameJob.class).build(), null); String expectedNode = "itcct_n_" + port; @@ -680,13 +704,17 @@ void testCancelColocatedTuple(int key, int port) { var keyTuple = Tuple.create().set(COLUMN_KEY, key); int sleepMs = 1_000_000; - IgniteCompute igniteCompute = client().compute(); - JobExecution tupleExecution = igniteCompute.submit( - JobTarget.colocated(TABLE_NAME, keyTuple), JobDescriptor.builder(SleepJob.class).build(), sleepMs); + CancelHandle cancelHandle = CancelHandle.create(); + JobExecution tupleExecution = submit( + JobTarget.colocated(TABLE_NAME, keyTuple), + JobDescriptor.builder(SleepJob.class).build(), + cancelHandle.token(), + sleepMs + ); await().until(tupleExecution::stateAsync, willBe(jobStateWithStatus(EXECUTING))); - assertThat(tupleExecution.cancelAsync(), willBe(true)); + assertThat(cancelHandle.cancelAsync(), willCompleteSuccessfully()); await().until(tupleExecution::stateAsync, willBe(jobStateWithStatus(CANCELED))); } @@ -697,14 +725,18 @@ void testCancelColocatedPojo(int key, int port) { var keyPojo = new TestPojo(key); int sleepMs = 1_000_000; - IgniteCompute igniteCompute = client().compute(); Mapper keyMapper = Mapper.of(TestPojo.class); - JobExecution pojoExecution = igniteCompute.submit( - JobTarget.colocated(TABLE_NAME, keyPojo, keyMapper), JobDescriptor.builder(SleepJob.class).build(), sleepMs); + CancelHandle cancelHandle = CancelHandle.create(); + JobExecution pojoExecution = submit( + JobTarget.colocated(TABLE_NAME, keyPojo, keyMapper), + JobDescriptor.builder(SleepJob.class).build(), + cancelHandle.token(), + sleepMs + ); await().until(pojoExecution::stateAsync, willBe(jobStateWithStatus(EXECUTING))); - assertThat(pojoExecution.cancelAsync(), willBe(true)); + assertThat(cancelHandle.cancelAsync(), willCompleteSuccessfully()); await().until(pojoExecution::stateAsync, willBe(jobStateWithStatus(CANCELED))); } @@ -839,11 +871,8 @@ void testExecuteMapReduceWithArgs() { @ParameterizedTest @ValueSource(classes = {MapReduceExceptionOnSplitTask.class, MapReduceExceptionOnReduceTask.class}) void testExecuteMapReduceExceptionPropagation(Class> taskClass) { - IgniteCompute igniteCompute = client().compute(); TaskDescriptor taskDescriptor = TaskDescriptor.builder(taskClass).build(); - IgniteException cause = getExceptionInJobExecutionAsync(new TaskToJobExecutionWrapper<>( - igniteCompute.submitMapReduce(taskDescriptor, null)) - ); + IgniteException cause = getExceptionInTaskExecutionAsync(client().compute().submitMapReduce(taskDescriptor, null)); assertThat(cause.getMessage(), containsString("Custom job error")); assertEquals(TRACE_ID, cause.traceId()); diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/client/ItThinClientComputeTypeCheckMarshallingTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/client/ItThinClientComputeTypeCheckMarshallingTest.java index 2c504d64bc7..9cb5feb8bac 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/client/ItThinClientComputeTypeCheckMarshallingTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/client/ItThinClientComputeTypeCheckMarshallingTest.java @@ -56,7 +56,7 @@ public class ItThinClientComputeTypeCheckMarshallingTest extends ItAbstractThinC void argumentMarshallerDefinedOnlyInJob() { // When submit job with custom marshaller that is defined in job but // client JobDescriptor does not declare the argument marshaller. - JobExecution result = client().compute().submit( + JobExecution result = submit( JobTarget.node(node(1)), JobDescriptor.builder(ArgMarshallingJob.class).build(), "Input" @@ -70,7 +70,7 @@ void argumentMarshallerDefinedOnlyInJob() { void resultMarshallerDefinedOnlyInJob() { // When submit job with custom marshaller that is defined in job but // client JobDescriptor does not declare the result marshaller. - JobExecution result = client().compute().submit( + JobExecution result = submit( JobTarget.node(node(1)), JobDescriptor.builder(ResultMarshallingJob.class).build(), "Input" @@ -84,7 +84,7 @@ void resultMarshallerDefinedOnlyInJob() { void argumentMarshallerDoesNotMatch() { // When submit job with custom marshaller that is defined in job but // client JobDescriptor does not declare the result marshaller. - JobExecution result = client().compute().submit( + JobExecution result = submit( JobTarget.node(node(1)), // The descriptor does not match actual job arguments. JobDescriptor.builder(ArgumentTypeCheckingmarshallingJob.class.getName()) @@ -101,7 +101,7 @@ void argumentMarshallerDoesNotMatch() { void resultMarshallerDoesNotMatch() { // When submit job with custom marshaller that is defined in job but the client JobDescriptor // declares the result marshaller which is not compatible with the marshaller in the job. - JobExecution result = client().compute().submit( + JobExecution result = submit( JobTarget.node(node(1)), // The descriptor does not match actual result. JobDescriptor.builder(ResultMarshallingJob.class.getName()) diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/compute/ItEmbeddedMarshallingTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/compute/ItEmbeddedMarshallingTest.java index 090c426a5ab..e2ae84852fa 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/compute/ItEmbeddedMarshallingTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/compute/ItEmbeddedMarshallingTest.java @@ -18,15 +18,17 @@ package org.apache.ignite.internal.runner.app.compute; import static org.apache.ignite.catalog.definitions.ColumnDefinition.column; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsInAnyOrder; import static org.junit.jupiter.api.Assertions.assertEquals; +import java.util.Collection; import java.util.Map; -import java.util.Map.Entry; -import java.util.Set; import java.util.concurrent.ExecutionException; import java.util.stream.Collectors; import org.apache.ignite.catalog.ColumnType; import org.apache.ignite.catalog.definitions.TableDefinition; +import org.apache.ignite.compute.BroadcastJobTarget; import org.apache.ignite.compute.JobDescriptor; import org.apache.ignite.compute.JobExecution; import org.apache.ignite.compute.JobTarget; @@ -40,7 +42,6 @@ import org.apache.ignite.internal.runner.app.Jobs.PojoResult; import org.apache.ignite.internal.runner.app.Jobs.ResultStringUnMarshaller; import org.apache.ignite.internal.runner.app.client.ItAbstractThinClientTest; -import org.apache.ignite.network.ClusterNode; import org.apache.ignite.table.Tuple; import org.junit.jupiter.api.Test; @@ -155,8 +156,8 @@ void broadcastExecute() { var node = server(0); // When. - Map result = node.compute().executeBroadcast( - Set.of(node(0), node(1)), + Collection result = node.compute().execute( + BroadcastJobTarget.nodes(node(0), node(1)), JobDescriptor.builder(ArgumentAndResultMarshallingJob.class) .argumentMarshaller(new ArgumentStringMarshaller()) .resultMarshaller(new ResultStringUnMarshaller()) @@ -166,13 +167,11 @@ void broadcastExecute() { // Then. // TODO IGNITE-24183 Avoid job argument and result marshalling on local execution - Map resultExpected = Map.of( + assertThat(result, containsInAnyOrder( // todo: "https://issues.apache.org/jira/browse/IGNITE-23024" - node(0), "Input:marshalledOnClient:unmarshalledOnServer:processedOnServer:marshalledOnServer:unmarshalledOnClient", - node(1), "Input:marshalledOnClient:unmarshalledOnServer:processedOnServer:marshalledOnServer:unmarshalledOnClient" - ); - - assertEquals(resultExpected, result); + "Input:marshalledOnClient:unmarshalledOnServer:processedOnServer:marshalledOnServer:unmarshalledOnClient", + "Input:marshalledOnClient:unmarshalledOnServer:processedOnServer:marshalledOnServer:unmarshalledOnClient" + )); } @Test @@ -181,31 +180,31 @@ void broadcastSubmit() { var node = server(0); // When. - Map result = node.compute().submitBroadcast( - Set.of(node(0), node(1)), + Map result = node.compute().submitAsync( + BroadcastJobTarget.nodes(node(0), node(1)), JobDescriptor.builder(ArgumentAndResultMarshallingJob.class) .argumentMarshaller(new ArgumentStringMarshaller()) .resultMarshaller(new ResultStringUnMarshaller()) .build(), "Input" - ).entrySet().stream().collect( - Collectors.toMap(Entry::getKey, ItEmbeddedMarshallingTest::extractResult, (v, i) -> v) - ); + ).thenApply(broadcastExecution -> broadcastExecution.executions().stream().collect( + Collectors.toMap(execution -> execution.node().name(), ItEmbeddedMarshallingTest::extractResult, (v, i) -> v) + )).join(); // Then. // TODO IGNITE-24183 Avoid job argument and result marshalling on local execution - Map resultExpected = Map.of( + Map resultExpected = Map.of( // todo: "https://issues.apache.org/jira/browse/IGNITE-23024" - node(0), "Input:marshalledOnClient:unmarshalledOnServer:processedOnServer:marshalledOnServer:unmarshalledOnClient", - node(1), "Input:marshalledOnClient:unmarshalledOnServer:processedOnServer:marshalledOnServer:unmarshalledOnClient" + node(0).name(), "Input:marshalledOnClient:unmarshalledOnServer:processedOnServer:marshalledOnServer:unmarshalledOnClient", + node(1).name(), "Input:marshalledOnClient:unmarshalledOnServer:processedOnServer:marshalledOnServer:unmarshalledOnClient" ); assertEquals(resultExpected, result); } - private static String extractResult(Entry> e) { + private static String extractResult(JobExecution e) { try { - return e.getValue().resultAsync().get(); + return e.resultAsync().get(); } catch (InterruptedException | ExecutionException ex) { throw new RuntimeException(ex); } diff --git a/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java b/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java index c109175b9d2..90395eca4e7 100644 --- a/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java +++ b/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java @@ -1130,7 +1130,7 @@ public class IgniteImpl implements Ignite { clusterSvc.topologyService(), logicalTopologyService, new JobContextManager(deploymentManagerImpl, deploymentManagerImpl.deploymentUnitAccessor(), new JobClassLoaderFactory()), - new ComputeExecutorImpl(this, stateMachine, computeCfg), + new ComputeExecutorImpl(this, stateMachine, computeCfg, clusterSvc.topologyService()), computeCfg ); diff --git a/modules/runner/src/main/java/org/apache/ignite/internal/restart/RestartProofIgniteCompute.java b/modules/runner/src/main/java/org/apache/ignite/internal/restart/RestartProofIgniteCompute.java index a830a15bbfc..4f46d716b74 100644 --- a/modules/runner/src/main/java/org/apache/ignite/internal/restart/RestartProofIgniteCompute.java +++ b/modules/runner/src/main/java/org/apache/ignite/internal/restart/RestartProofIgniteCompute.java @@ -17,10 +17,11 @@ package org.apache.ignite.internal.restart; -import java.util.Map; -import java.util.Set; +import java.util.Collection; import java.util.concurrent.CompletableFuture; import org.apache.ignite.Ignite; +import org.apache.ignite.compute.BroadcastExecution; +import org.apache.ignite.compute.BroadcastJobTarget; import org.apache.ignite.compute.IgniteCompute; import org.apache.ignite.compute.JobDescriptor; import org.apache.ignite.compute.JobExecution; @@ -30,7 +31,6 @@ import org.apache.ignite.internal.wrapper.Wrapper; import org.apache.ignite.internal.wrapper.Wrappers; import org.apache.ignite.lang.CancellationToken; -import org.apache.ignite.network.ClusterNode; import org.jetbrains.annotations.Nullable; /** @@ -49,45 +49,70 @@ class RestartProofIgniteCompute implements IgniteCompute, Wrapper { } @Override - public JobExecution submit(JobTarget target, JobDescriptor descriptor, @Nullable T arg) { - return attachmentLock.attached(ignite -> ignite.compute().submit(target, descriptor, arg)); + public CompletableFuture> submitAsync( + JobTarget target, + JobDescriptor descriptor, + @Nullable T arg, + @Nullable CancellationToken cancellationToken + ) { + return attachmentLock.attachedAsync(ignite -> ignite.compute().submitAsync(target, descriptor, arg, cancellationToken)); } @Override - public R execute(JobTarget target, JobDescriptor descriptor, @Nullable CancellationToken cancellationToken, - @Nullable T arg) { - return attachmentLock.attached(ignite -> ignite.compute().execute(target, descriptor, cancellationToken, arg)); + public CompletableFuture> submitAsync( + BroadcastJobTarget target, + JobDescriptor descriptor, + @Nullable T arg, + @Nullable CancellationToken cancellationToken + ) { + return attachmentLock.attachedAsync(ignite -> ignite.compute().submitAsync(target, descriptor, arg, cancellationToken)); } @Override - public CompletableFuture executeAsync(JobTarget target, JobDescriptor descriptor, - @Nullable CancellationToken cancellationToken, @Nullable T arg) { - return attachmentLock.attachedAsync(ignite -> ignite.compute().executeAsync(target, descriptor, cancellationToken, arg)); + public R execute( + JobTarget target, + JobDescriptor descriptor, + @Nullable T arg, + @Nullable CancellationToken cancellationToken + ) { + return attachmentLock.attached(ignite -> ignite.compute().execute(target, descriptor, arg, cancellationToken)); } @Override - public Map> submitBroadcast( - Set nodes, + public Collection execute( + BroadcastJobTarget target, JobDescriptor descriptor, - @Nullable T arg + @Nullable T arg, + @Nullable CancellationToken cancellationToken ) { - return attachmentLock.attached(ignite -> ignite.compute().submitBroadcast(nodes, descriptor, arg)); + return attachmentLock.attached(ignite -> ignite.compute().execute(target, descriptor, arg, cancellationToken)); } @Override - public TaskExecution submitMapReduce(TaskDescriptor taskDescriptor, @Nullable T arg) { - return attachmentLock.attached(ignite -> ignite.compute().submitMapReduce(taskDescriptor, arg)); + public TaskExecution submitMapReduce( + TaskDescriptor taskDescriptor, + @Nullable T arg, + @Nullable CancellationToken cancellationToken + ) { + return attachmentLock.attached(ignite -> ignite.compute().submitMapReduce(taskDescriptor, arg, cancellationToken)); } @Override - public R executeMapReduce(TaskDescriptor taskDescriptor, @Nullable CancellationToken cancellationToken, @Nullable T arg) { - return attachmentLock.attached(ignite -> ignite.compute().executeMapReduce(taskDescriptor, cancellationToken, arg)); + public CompletableFuture executeMapReduceAsync( + TaskDescriptor taskDescriptor, + @Nullable T arg, + @Nullable CancellationToken cancellationToken + ) { + return attachmentLock.attachedAsync(ignite -> ignite.compute().executeMapReduceAsync(taskDescriptor, arg, cancellationToken)); } @Override - public CompletableFuture executeMapReduceAsync(TaskDescriptor taskDescriptor, - @Nullable CancellationToken cancellationToken, @Nullable T arg) { - return attachmentLock.attachedAsync(ignite -> ignite.compute().executeMapReduceAsync(taskDescriptor, cancellationToken, arg)); + public R executeMapReduce( + TaskDescriptor taskDescriptor, + @Nullable T arg, + @Nullable CancellationToken cancellationToken + ) { + return attachmentLock.attached(ignite -> ignite.compute().executeMapReduce(taskDescriptor, arg, cancellationToken)); } @Override diff --git a/modules/runner/src/testFixtures/java/org/apache/ignite/internal/ClusterPerTestIntegrationTest.java b/modules/runner/src/testFixtures/java/org/apache/ignite/internal/ClusterPerTestIntegrationTest.java index 0b40c1b193a..5e9617723ea 100644 --- a/modules/runner/src/testFixtures/java/org/apache/ignite/internal/ClusterPerTestIntegrationTest.java +++ b/modules/runner/src/testFixtures/java/org/apache/ignite/internal/ClusterPerTestIntegrationTest.java @@ -34,6 +34,7 @@ import org.apache.ignite.internal.testframework.BaseIgniteAbstractTest; import org.apache.ignite.internal.testframework.WorkDirectory; import org.apache.ignite.internal.testframework.WorkDirectoryExtension; +import org.apache.ignite.network.ClusterNode; import org.jetbrains.annotations.Nullable; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; @@ -256,4 +257,12 @@ protected final List> executeSql(int nodeIndex, String sql, Object. return ClusterPerClassIntegrationTest.sql(ignite, null, null, null, sql, args); } + + protected ClusterNode clusterNode(int index) { + return clusterNode(node(index)); + } + + protected static ClusterNode clusterNode(Ignite node) { + return unwrapIgniteImpl(node).node(); + } } diff --git a/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItComputeSystemViewTest.java b/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItComputeSystemViewTest.java index 4fd98817c6f..d701b380715 100644 --- a/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItComputeSystemViewTest.java +++ b/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItComputeSystemViewTest.java @@ -23,6 +23,7 @@ import static org.apache.ignite.internal.TestWrappers.unwrapIgniteImpl; import static org.apache.ignite.internal.sql.engine.util.Commons.closeQuiet; import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willBe; +import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully; import static org.apache.ignite.internal.testframework.matchers.JobStateMatcher.jobStateWithStatus; import static org.apache.ignite.internal.testframework.matchers.TaskStateMatcher.taskStateWithStatus; import static org.awaitility.Awaitility.await; @@ -32,16 +33,18 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.hasLength; +import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.lessThanOrEqualTo; import java.time.Instant; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.UUID; import java.util.concurrent.CompletableFuture; import org.apache.ignite.Ignite; import org.apache.ignite.client.IgniteClient; +import org.apache.ignite.compute.BroadcastExecution; +import org.apache.ignite.compute.BroadcastJobTarget; import org.apache.ignite.compute.ComputeJob; import org.apache.ignite.compute.JobDescriptor; import org.apache.ignite.compute.JobExecution; @@ -56,8 +59,7 @@ import org.apache.ignite.internal.hlc.ClockService; import org.apache.ignite.internal.sql.BaseSqlIntegrationTest; import org.apache.ignite.internal.sql.engine.util.MetadataMatcher; -import org.apache.ignite.internal.testframework.IgniteTestUtils; -import org.apache.ignite.network.ClusterNode; +import org.apache.ignite.lang.CancelHandle; import org.apache.ignite.sql.ColumnType; import org.hamcrest.Matchers; import org.jetbrains.annotations.Nullable; @@ -109,8 +111,13 @@ void viewRunningJobs(boolean isClient) { long tsBefore = clockService.now().getPhysical(); - JobDescriptor job = JobDescriptor.builder(InfiniteJob.class).units(List.of()).build(); - JobExecution execution = entryNode.compute().submit(JobTarget.node(clusterNode(targetNode)), job, null); + JobDescriptor job = JobDescriptor.builder(InfiniteJob.class).build(); + CancelHandle cancelHandle = CancelHandle.create(); + CompletableFuture> executionFut = entryNode.compute().submitAsync( + JobTarget.node(clusterNode(targetNode)), job, null, cancelHandle.token() + ); + assertThat(executionFut, willCompleteSuccessfully()); + JobExecution execution = executionFut.join(); await().until(execution::stateAsync, willBe(jobStateWithStatus(EXECUTING))); @@ -120,22 +127,23 @@ void viewRunningJobs(boolean isClient) { List> res = sql(0, query, EXECUTING.name()); - assertThat(res, Matchers.hasSize(1)); + assertThat(res, hasSize(1)); verifyComputeJobState(res.get(0), List.of(targetNode.name()), EXECUTING.name(), tsBefore, tsAfter); - IgniteTestUtils.await(execution.cancelAsync()); + assertThat(cancelHandle.cancelAsync(), willCompleteSuccessfully()); await().until(execution::stateAsync, willBe(jobStateWithStatus(CANCELED))); // Second Job call on different node. - job = JobDescriptor.builder(InfiniteJob.class).units(List.of()).build(); - tsBefore = clockService.now().getPhysical(); targetNode = CLUSTER.node(1); - execution = entryNode.compute().submit(JobTarget.node(clusterNode(targetNode)), job, null); + cancelHandle = CancelHandle.create(); + executionFut = entryNode.compute().submitAsync(JobTarget.node(clusterNode(targetNode)), job, null, cancelHandle.token()); + assertThat(executionFut, willCompleteSuccessfully()); + execution = executionFut.join(); await().until(execution::stateAsync, willBe(jobStateWithStatus(EXECUTING))); @@ -147,7 +155,7 @@ void viewRunningJobs(boolean isClient) { verifyComputeJobState(res.get(0), List.of(targetNode.name()), EXECUTING.name(), tsBefore, tsAfter); - IgniteTestUtils.await(execution.cancelAsync()); + assertThat(cancelHandle.cancelAsync(), willCompleteSuccessfully()); await().until(execution::stateAsync, willBe(jobStateWithStatus(CANCELED))); } finally { @@ -165,16 +173,22 @@ void viewRunningBroadcasts(boolean isClient) { long tsBefore = clockService.now().getPhysical(); - JobDescriptor job = JobDescriptor.builder(InfiniteJob.class).units(List.of()).build(); - Map> execution = entryNode.compute().submitBroadcast( - Set.of(clusterNode(CLUSTER.node(0)), clusterNode(CLUSTER.node(1))), job, null); - - execution.forEach((k, exec) -> await().until(exec::stateAsync, willBe(jobStateWithStatus(EXECUTING)))); + CancelHandle cancelHandle = CancelHandle.create(); + CompletableFuture> executionFut = entryNode.compute().submitAsync( + BroadcastJobTarget.nodes(clusterNode(0), clusterNode(1)), + JobDescriptor.builder(InfiniteJob.class).build(), + null, + cancelHandle.token() + ); - long tsAfter = clockService.now().getPhysical(); + assertThat(executionFut, willCompleteSuccessfully()); String query = "SELECT * FROM SYSTEM.COMPUTE_TASKS WHERE STATUS = ?"; + await().until(() -> sql(0, query, EXECUTING.name()), hasSize(2)); + + long tsAfter = clockService.now().getPhysical(); + List> res = sql(0, query, EXECUTING.name()); assertThat(res.size(), is(2)); @@ -183,7 +197,7 @@ void viewRunningBroadcasts(boolean isClient) { verifyComputeJobState(res.get(0), execNodes, EXECUTING.name(), tsBefore, tsAfter); verifyComputeJobState(res.get(1), execNodes, EXECUTING.name(), tsBefore, tsAfter); - execution.forEach((k, exec) -> IgniteTestUtils.await(exec.cancelAsync())); + cancelHandle.cancel(); } finally { closeQuiet(entryNode); } @@ -200,8 +214,9 @@ void viewRunningMapReduceTask(boolean isClient) { long tsBefore = clockService.now().getPhysical(); + CancelHandle cancelHandle = CancelHandle.create(); TaskExecution execution = entryNode.compute() - .submitMapReduce(TaskDescriptor.builder(MapReduceTaskCustom.class).build(), null); + .submitMapReduce(TaskDescriptor.builder(MapReduceTaskCustom.class).build(), null, cancelHandle.token()); await().until(execution::stateAsync, willBe(taskStateWithStatus(TaskStatus.EXECUTING))); @@ -217,7 +232,7 @@ void viewRunningMapReduceTask(boolean isClient) { verifyComputeJobState(res.get(0), execNodes, EXECUTING.name(), tsBefore, tsAfter); verifyComputeJobState(res.get(1), execNodes, EXECUTING.name(), tsBefore, tsAfter); - IgniteTestUtils.await(execution.cancelAsync()); + assertThat(cancelHandle.cancelAsync(), willCompleteSuccessfully()); } finally { closeQuiet(entryNode); } diff --git a/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/kill/ItSqlKillCommandTest.java b/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/kill/ItSqlKillCommandTest.java index 2bbb7c452b5..9b1b66ed0aa 100644 --- a/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/kill/ItSqlKillCommandTest.java +++ b/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/kill/ItSqlKillCommandTest.java @@ -26,6 +26,7 @@ import static org.apache.ignite.internal.sql.engine.util.SqlTestUtils.expectQueryCancelled; import static org.apache.ignite.internal.testframework.IgniteTestUtils.await; import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willBe; +import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully; import static org.apache.ignite.internal.testframework.matchers.JobStateMatcher.jobStateWithStatus; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; @@ -36,14 +37,16 @@ import static org.hamcrest.Matchers.nullValue; import static org.junit.jupiter.api.Assumptions.assumeTrue; +import java.util.Collection; import java.util.List; -import java.util.Map; import java.util.Set; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.function.Consumer; import java.util.stream.Collectors; import org.apache.ignite.Ignite; +import org.apache.ignite.compute.BroadcastExecution; +import org.apache.ignite.compute.BroadcastJobTarget; import org.apache.ignite.compute.JobDescriptor; import org.apache.ignite.compute.JobExecution; import org.apache.ignite.compute.JobTarget; @@ -154,8 +157,7 @@ public void killQueryFromLocal() { @Test public void killComputeJobFromLocal() { Ignite node = CLUSTER.aliveNode(); - JobDescriptor job = JobDescriptor.builder(InfiniteJob.class).units(List.of()).build(); - JobExecution execution = node.compute().submit(JobTarget.node(clusterNode(node)), job, null); + JobExecution execution = submit(node, JobDescriptor.builder(InfiniteJob.class).build()); UUID jobId = await(execution.idAsync()); assertThat(jobId, not(nullValue())); @@ -242,8 +244,7 @@ public void killComputeJobFromRemote() { // Single execution. { - JobDescriptor job = JobDescriptor.builder(InfiniteJob.class).units(List.of()).build(); - JobExecution execution = local.compute().submit(JobTarget.node(clusterNode(local)), job, null); + JobExecution execution = submit(local, JobDescriptor.builder(InfiniteJob.class).build()); UUID jobId = await(execution.idAsync()); assertThat(jobId, not(nullValue())); @@ -257,8 +258,7 @@ public void killComputeJobFromRemote() { // Single execution with nowait. { - JobDescriptor job = JobDescriptor.builder(InfiniteJob.class).units(List.of()).build(); - JobExecution execution = local.compute().submit(JobTarget.node(clusterNode(local)), job, null); + JobExecution execution = submit(local, JobDescriptor.builder(InfiniteJob.class).build()); UUID jobId = await(execution.idAsync()); assertThat(jobId, not(nullValue())); @@ -272,10 +272,10 @@ public void killComputeJobFromRemote() { // Multiple executions. { JobDescriptor job = JobDescriptor.builder(InfiniteJob.class).units(List.of()).build(); - Map> executions = local.compute().submitBroadcast( - Set.of(clusterNode(CLUSTER.node(0)), clusterNode(CLUSTER.node(1))), job, null); + Collection> executions = submit(local, Set.of(clusterNode(0), clusterNode(1)), job); - executions.forEach((node, execution) -> { + executions.forEach(execution -> { + ClusterNode node = execution.node(); UUID jobId = await(execution.idAsync()); assertThat(jobId, not(nullValue())); assertThat("Node=" + node.name(), executeKillJob(remote, jobId), is(true)); @@ -331,4 +331,16 @@ private static void waitUntilRunningQueriesCountInCluster(Matcher match SqlTestUtils.waitUntilRunningQueriesCount(queryProcessor, matcher); }); } + + private static JobExecution submit(Ignite node, JobDescriptor job) { + CompletableFuture> executionFut = node.compute().submitAsync(JobTarget.node(clusterNode(node)), job, null); + assertThat(executionFut, willCompleteSuccessfully()); + return executionFut.join(); + } + + private static Collection> submit(Ignite node, Set nodes, JobDescriptor job) { + CompletableFuture> executionFut = node.compute().submitAsync(BroadcastJobTarget.nodes(nodes), job, null); + assertThat(executionFut, willCompleteSuccessfully()); + return executionFut.join().executions(); + } }