Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

IGNITE-24174 Unify compute API #5016

Draft
wants to merge 1 commit into
base: main
Choose a base branch
from
Draft
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,43 @@
/*
* 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;

/**
* Any node execution target. Indicates any node from the provided set.
*/
public class AllNodesBroadcastJobTarget implements BroadcastJobTarget {
private final Set<ClusterNode> nodes;

AllNodesBroadcastJobTarget(Set<ClusterNode> nodes) {
Objects.requireNonNull(nodes);

if (nodes.isEmpty()) {
throw new IllegalArgumentException("Nodes collection must not be empty.");
}

this.nodes = nodes;
}

public Set<ClusterNode> nodes() {
return nodes;
}
}
Original file line number Diff line number Diff line change
@@ -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.List;
import java.util.Map;
import java.util.UUID;
import java.util.concurrent.CompletableFuture;
import java.util.stream.Collectors;
import org.apache.ignite.network.ClusterNode;
import org.jetbrains.annotations.Nullable;

/**
* Job control object, provides information about the job execution process and result, allows cancelling the job.
*
* @param <R> Job result type.
*/
public interface BroadcastExecution<R> {
/**
* Returns job's execution result.
*
* @return Job's execution result future.
*/
CompletableFuture<Map<ClusterNode, R>> resultsAsync();

CompletableFuture<Map<ClusterNode, JobState>> statesAsync();

default CompletableFuture<List<@Nullable UUID>> idsAsync() {
return statesAsync().thenApply(states -> states.values().stream()
.map(state -> state != null ? state.id() : null)
.collect(Collectors.toList()));
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,82 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* 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;

/**
* Job execution target.
*
* <p>Determines the rules for selecting nodes to execute a job.
*/
public interface BroadcastJobTarget {
/**
* Creates a job target for any node from the provided collection.
*
* <p>This target determines that a job can be executed on any node in a given collection, but only one of them.
* Which node is chosen is implementation defined.
*
* @param nodes Collection of nodes.
* @return Job target.
*/
static BroadcastJobTarget nodes(ClusterNode... nodes) {
return new AllNodesBroadcastJobTarget(Set.of(nodes));
}

/**
* Creates a job target for any node from the provided collection.
*
* <p>This target determines that a job can be executed on any node in a given collection, but only one of them.
* Which node is chosen is implementation defined.
*
* @param nodes Collection of nodes.
* @return Job target.
*/
static BroadcastJobTarget nodes(Collection<ClusterNode> nodes) {
return new AllNodesBroadcastJobTarget(new HashSet<>(nodes));
}

/**
* Creates a job target for any node from the provided collection.
*
* <p>This target determines that a job can be executed on any node in a given collection, but only one of them.
* Which node is chosen is implementation defined.
*
* @param nodes Collection of nodes.
* @return Job target.
*/
static BroadcastJobTarget nodes(Set<ClusterNode> nodes) {
return new AllNodesBroadcastJobTarget(nodes);
}

/**
* Creates a colocated job target for a specific table and key.
*
* <p>This target determines that a job should be executed on the same node that hosts the data for a given key of provided table.
*
* @param tableName Table name.
* @param key Key.
* @return Job target.
*/
static BroadcastJobTarget partitioned(String tableName) {
return new TableJobTarget(tableName);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,13 +17,7 @@

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.concurrent.CompletableFuture;
import org.apache.ignite.compute.task.MapReduceTask;
Expand All @@ -49,10 +43,30 @@ public interface IgniteCompute {
* @param arg Argument of the job.
* @return Job execution object.
*/
<T, R> JobExecution<R> submit(
default <T, R> CompletableFuture<JobExecution<R>> submitAsync(
JobTarget target,
JobDescriptor<T, R> descriptor,
@Nullable T arg
) {
return submitAsync(target, descriptor, null, arg);
}

/**
* Submits a {@link ComputeJob} of the given class for an execution on a single node from a set of candidate nodes.
*
* @param <T> Job argument (T)ype.
* @param <R> 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 execution object.
*/
<T, R> CompletableFuture<JobExecution<R>> submitAsync(
JobTarget target,
JobDescriptor<T, R> descriptor,
@Nullable CancellationToken cancellationToken,
@Nullable T arg
);

/**
Expand Down Expand Up @@ -86,12 +100,14 @@ default <T, R> CompletableFuture<R> executeAsync(
* @param arg Argument of the job.
* @return Job result future.
*/
<T, R> CompletableFuture<R> executeAsync(
default <T, R> CompletableFuture<R> executeAsync(
JobTarget target,
JobDescriptor<T, R> descriptor,
@Nullable CancellationToken cancellationToken,
@Nullable T arg
);
) {
return submitAsync(target, descriptor, cancellationToken, arg).thenCompose(JobExecution::resultAsync);
}

/**
* Executes a {@link ComputeJob} of the given class on a single node from a set of candidate nodes.
Expand Down Expand Up @@ -141,9 +157,29 @@ <T, R> R execute(
* @param arg Argument of the job.
* @return Map from node to job execution object.
*/
<T, R> Map<ClusterNode, JobExecution<R>> submitBroadcast(
default <T, R> CompletableFuture<BroadcastExecution<R>> submitAsync(
Set<ClusterNode> nodes,
JobDescriptor<T, R> descriptor,
@Nullable T arg
) {
return submitAsync(nodes, descriptor, null, arg);
}

/**
* Submits a {@link ComputeJob} of the given class for an execution on all nodes in the given node set.
*
* @param <T> Job argument (T)ype.
* @param <R> Job (R)esult type.
* @param nodes Nodes to execute the job on.
* @param descriptor Job descriptor.
* @param cancellationToken Cancellation token or {@code null}.
* @param arg Argument of the job.
* @return Map from node to job execution object.
*/
<T, R> CompletableFuture<BroadcastExecution<R>> submitAsync(
Set<ClusterNode> nodes,
JobDescriptor<T, R> descriptor,
@Nullable CancellationToken cancellationToken,
@Nullable T arg
);

Expand Down Expand Up @@ -182,20 +218,8 @@ default <T, R> CompletableFuture<Map<ClusterNode, R>> executeBroadcastAsync(
@Nullable CancellationToken cancellationToken,
@Nullable T arg
) {
Map<ClusterNode, CompletableFuture<R>> futures = nodes.stream()
.collect(toMap(identity(), node -> executeAsync(JobTarget.node(node), descriptor, cancellationToken, arg)));

return allOf(futures.values().toArray(CompletableFuture[]::new))
.thenApply(ignored -> {
Map<ClusterNode, R> map = new HashMap<>();

for (Entry<ClusterNode, CompletableFuture<R>> entry : futures.entrySet()) {
map.put(entry.getKey(), entry.getValue().join());
}

return map;
}
);
return submitAsync(nodes, descriptor, cancellationToken, arg)
.thenCompose(BroadcastExecution::resultsAsync);
}

/**
Expand Down Expand Up @@ -229,20 +253,12 @@ default <T, R> Map<ClusterNode, R> executeBroadcast(
* @return Map from node to job result.
* @throws ComputeException If there is any problem executing the job.
*/
default <T, R> Map<ClusterNode, R> executeBroadcast(
<T, R> Map<ClusterNode, R> executeBroadcast(
Set<ClusterNode> nodes,
JobDescriptor<T, R> descriptor,
@Nullable CancellationToken cancellationToken,
@Nullable T arg
) {
Map<ClusterNode, R> map = new HashMap<>();

for (ClusterNode node : nodes) {
map.put(node, execute(JobTarget.node(node), descriptor, cancellationToken, arg));
}

return map;
}
);

/**
* Submits a {@link MapReduceTask} of the given class for an execution.
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,37 @@
/*
* 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;

/**
* Colocated job execution target. Indicates a node that hosts the data for the specified key in the provided table.
*/
public class TableJobTarget implements BroadcastJobTarget {
private final String tableName;

TableJobTarget(String tableName) {
Objects.requireNonNull(tableName);

this.tableName = tableName;
}

public String tableName() {
return tableName;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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<T, R> {
private final Set<ClusterNode> nodes;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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,

Expand Down
Loading