Skip to content
Merged
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
9 changes: 9 additions & 0 deletions modules/api/src/main/java/org/apache/ignite/table/Table.java
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
package org.apache.ignite.table;

import org.apache.ignite.table.mapper.Mapper;
import org.apache.ignite.table.partition.PartitionDistribution;
import org.apache.ignite.table.partition.PartitionManager;

/**
Expand Down Expand Up @@ -59,8 +60,16 @@ default String name() {
*
* @return Partition manager.
*/
@Deprecated(forRemoval = true, since = "3.2")
PartitionManager partitionManager();

/**
* Gets the partition distribution.
*
* @return Partition distribution.
*/
PartitionDistribution partitionDistribution();

/**
* Gets a record view of the table using the specified record class mapper.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,8 +20,13 @@
import java.io.Serializable;

/**
* Marker interface which represents a partition reference.
* Interface which represents a partition reference.
*/
public interface Partition extends Serializable {

/**
* Retrieves the identifier of the partition. The identifier is only guaranteed to be unique in the context of a table.
*
* @return Identifier of the partition.
*/
long id();
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,137 @@
/*
* 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.table.partition;

import java.util.List;
import java.util.Map;
import java.util.concurrent.CompletableFuture;
import org.apache.ignite.network.ClusterNode;
import org.apache.ignite.table.Tuple;
import org.apache.ignite.table.mapper.Mapper;

/**
* The partition distribution provides the ability to obtain information about table partitions.
* This interface can be used to get all partitions of a table, the location of the primary replica of a partition,
* the partition for a specific table key.
*/
public interface PartitionDistribution {
/**
* Asynchronously gets a list with all partitions.
*
* @return Future with list with all partitions.
*/
CompletableFuture<List<Partition>> partitionsAsync();

/**
* Gets a list with all partitions.
*
* @return List with all partitions.
*/
List<Partition> partitions();

/**
* Asynchronously gets map with all partitions and their locations as of the time of the call.
*
* <p>Note: This assignment may become outdated if a re-assigment happens on the cluster.
*
* @return Future with map from partition to cluster node where primary replica of the partition is located as of the time of the call.
*/
CompletableFuture<Map<Partition, ClusterNode>> primaryReplicasAsync();

/**
* Asynchronously gets all partitions hosted by the specified node as a primary replica.
*
* @return Future with list with all partitions hosted by the specified node as a primary replica.
*/
CompletableFuture<List<Partition>> primaryReplicasAsync(ClusterNode node);

/**
* Gets map with all partitions and their locations as of the time of the call.
*
* <p>Note: This assignment may become outdated if a re-assigment happens on the cluster.
*
* @return Map from partition to cluster node where primary replica of the partition is located as of the time of the call.
*/
Map<Partition, ClusterNode> primaryReplicas();

/**
* Gets all partitions hosted by the specified node as a primary replica as of the time of the call.
*
* <p>Note: This assignment may become outdated if a re-assigment happens on the cluster.
*
* @return List with all partitions hosted by the specified node as a primary replica as of the time of the call.
*/
List<Partition> primaryReplicas(ClusterNode node);

/**
* Asynchronously gets the current location of the primary replica for the provided partition.
*
* <p>Note: This assignment may become outdated if a re-assigment happens on the cluster.
*
* @param partition Partition instance.
* @return Future that represents the pending completion of the operation.
* @see #primaryReplica(Partition)
*/
CompletableFuture<ClusterNode> primaryReplicaAsync(Partition partition);

/**
* Gets the current location of the primary replica for the provided partition.
*
* <p>Note: This assignment may become outdated if a re-assigment happens on the cluster.
*
* @param partition Partition instance.
* @return Cluster node where primary replica of provided partition is located as of the time of the call.
*/
ClusterNode primaryReplica(Partition partition);

/**
* Asynchronously gets partition instance for provided table key.
*
* @param key Table key.
* @param mapper Table key mapper.
* @param <K> Key type.
* @return Future with partition instance which contains provided key.
*/
<K> CompletableFuture<Partition> partitionAsync(K key, Mapper<K> mapper);

/**
* Asynchronously gets partition instance for provided table key.
*
* @param key Table key tuple.
* @return Future with partition instance which contains provided key.
*/
CompletableFuture<Partition> partitionAsync(Tuple key);

/**
* Gets partition instance for provided table key.
*
* @param key Table key.
* @param mapper Table key mapper.
* @param <K> Key type.
* @return Partition instance which contains provided key.
*/
<K> Partition partition(K key, Mapper<K> mapper);

/**
* Gets partition instance for provided table key.
*
* @param key Table key tuple.
* @return Partition instance which contains provided key.
*/
Partition partition(Tuple key);
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,49 +17,12 @@

package org.apache.ignite.table.partition;

import java.util.Map;
import java.util.concurrent.CompletableFuture;
import org.apache.ignite.network.ClusterNode;
import org.apache.ignite.table.Tuple;
import org.apache.ignite.table.mapper.Mapper;

/**
* The partition manager provides the ability to obtain information about table partitions.
* This interface can be used to get all partitions of a table,
* the location of the primary replica of a partition,
* the partition for a specific table key.
* The partition manager provides the ability to obtain information about table partitions. This interface can be used to get all partitions
* of a table, the location of the primary replica of a partition, the partition for a specific table key.
*
* @deprecated Replaced by {@link PartitionDistribution}.
*/
public interface PartitionManager {
/**
* Returns location of primary replica for provided partition.
*
* @param partition Partition instance.
* @return Cluster node where primary replica of provided partition is located.
*/
CompletableFuture<ClusterNode> primaryReplicaAsync(Partition partition);

/**
* Returns map with all partitions and their locations.
*
* @return Map from partition to cluster node where primary replica of the partition is located.
*/
CompletableFuture<Map<Partition, ClusterNode>> primaryReplicasAsync();

/**
* Returns partition instance for provided table key.
*
* @param key Table key.
* @param mapper Table key mapper.
* @param <K> Key type.
* @return Partition instance which contains provided key.
*/
<K> CompletableFuture<Partition> partitionAsync(K key, Mapper<K> mapper);

/**
* Returns partition instance for provided table key.
*
* @param key Table key tuple.
* @return Partition instance which contains provided key.
*/
CompletableFuture<Partition> partitionAsync(Tuple key);
@Deprecated(since = "3.2")
public interface PartitionManager extends PartitionDistribution {
}
Original file line number Diff line number Diff line change
Expand Up @@ -77,7 +77,7 @@ public static CompletableFuture<ResponseWriter> process(
: ReceiverExecutionOptions.DEFAULT;

return readTableAsync(tableId, tables).thenCompose(table -> {
return table.partitionManager()
return table.partitionDistribution()
.primaryReplicaAsync(new HashPartition(partition))
.thenApply(ClusterNodeImpl::fromPublicClusterNode)
.thenCompose(node -> table.internalTable().streamerReceiverRunner()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,6 @@
import org.apache.ignite.client.handler.ResponseWriter;
import org.apache.ignite.client.handler.requests.table.ClientTablePartitionPrimaryReplicasGetRequest;
import org.apache.ignite.internal.client.proto.ClientMessageUnpacker;
import org.apache.ignite.internal.table.partition.HashPartition;
import org.apache.ignite.network.ClusterNode;
import org.apache.ignite.table.IgniteTables;
import org.apache.ignite.table.partition.Partition;
Expand All @@ -49,14 +48,12 @@ public static CompletableFuture<ResponseWriter> process(
) {
int tableId = in.unpackInt();

return readTableAsync(tableId, tables).thenCompose(table -> table.partitionManager()
return readTableAsync(tableId, tables).thenCompose(table -> table.partitionDistribution()
.primaryReplicasAsync()
.thenApply(partitions -> out -> {
out.packInt(partitions.size());
for (Entry<Partition, ClusterNode> e : partitions.entrySet()) {
HashPartition partition = (HashPartition) e.getKey();

out.packInt(partition.partitionId());
out.packLong(e.getKey().id());

packClusterNode(e.getValue(), out);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@
import org.apache.ignite.table.DataStreamerTarget;
import org.apache.ignite.table.IgniteTables;
import org.apache.ignite.table.RecordView;
import org.apache.ignite.table.partition.PartitionManager;
import org.apache.ignite.table.partition.PartitionDistribution;
import org.apache.ignite.tx.Transaction;

/**
Expand Down Expand Up @@ -196,7 +196,7 @@ public enum ClientOperationType {
SQL_EXECUTE_BATCH,

/**
* Get all primary replicas mapping to cluster nodes ({@link PartitionManager#primaryReplicasAsync()}).
* Get all primary replicas mapping to cluster nodes ({@link PartitionDistribution#primaryReplicasAsync()}).
*/
PRIMARY_REPLICAS_GET
}
Original file line number Diff line number Diff line change
Expand Up @@ -66,7 +66,6 @@
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.table.partition.HashPartition;
import org.apache.ignite.internal.util.ExceptionUtils;
import org.apache.ignite.internal.util.ViewUtils;
import org.apache.ignite.lang.CancelHandleHelper;
Expand Down Expand Up @@ -157,7 +156,7 @@ public <T, R> CompletableFuture<BroadcastExecution<R>> submitAsync(
TableJobTarget tableJobTarget = (TableJobTarget) target;
QualifiedName tableName = tableJobTarget.tableName();
return getTable(tableName)
.thenCompose(table -> table.partitionManager().primaryReplicasAsync())
.thenCompose(table -> table.partitionDistribution().primaryReplicasAsync())
.thenCompose(replicas -> {
//noinspection unchecked
CompletableFuture<SubmitResult>[] futures = replicas.keySet().stream()
Expand Down Expand Up @@ -446,21 +445,21 @@ private static <T, R> CompletableFuture<SubmitResult> executePartitioned(
UUID taskId,
@Nullable T arg
) {
int partitionId = ((HashPartition) partition).partitionId();
long partitionId = partition.id();
return t.doSchemaOutOpAsync(
ClientOp.COMPUTE_EXECUTE_PARTITIONED,
(schema, outputChannel, unused) -> {
ClientMessagePacker w = outputChannel.out();

w.packInt(t.tableId());

w.packInt(partitionId);
w.packLong(partitionId);

packJob(outputChannel, descriptor, arg);
packTaskId(outputChannel, taskId);
},
ClientCompute::unpackSubmitResult,
PartitionAwarenessProvider.of(partitionId),
PartitionAwarenessProvider.of(Math.toIntExact(partitionId)),
true,
null
);
Expand Down
Loading