ptupitsyn commented on code in PR #2825: URL: https://github.com/apache/ignite-3/pull/2825#discussion_r1401692258
########## modules/client-handler/src/main/java/org/apache/ignite/client/handler/ClientPrimaryReplicaTracker.java: ########## @@ -0,0 +1,253 @@ +/* + * 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.client.handler; + +import static org.apache.ignite.lang.ErrorGroups.Table.TABLE_NOT_FOUND_ERR; + +import java.util.ArrayList; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.ignite.internal.catalog.CatalogService; +import org.apache.ignite.internal.catalog.descriptors.CatalogTableDescriptor; +import org.apache.ignite.internal.catalog.descriptors.CatalogZoneDescriptor; +import org.apache.ignite.internal.catalog.events.CatalogEvent; +import org.apache.ignite.internal.catalog.events.DropTableEventParameters; +import org.apache.ignite.internal.event.EventListener; +import org.apache.ignite.internal.event.EventParameters; +import org.apache.ignite.internal.hlc.HybridClock; +import org.apache.ignite.internal.hlc.HybridTimestamp; +import org.apache.ignite.internal.placementdriver.PlacementDriver; +import org.apache.ignite.internal.placementdriver.ReplicaMeta; +import org.apache.ignite.internal.placementdriver.event.PrimaryReplicaEvent; +import org.apache.ignite.internal.placementdriver.event.PrimaryReplicaEventParameters; +import org.apache.ignite.internal.replicator.TablePartitionId; +import org.apache.ignite.lang.TableNotFoundException; +import org.jetbrains.annotations.Nullable; + +/** + * Primary partition replica tracker. Shared by all instances of {@link ClientInboundMessageHandler}. + * + * <p>Keeps up-to-date lists of primary replicas by partition for every table, avoiding expensive placement driver calls in most cases. + */ +public class ClientPrimaryReplicaTracker implements EventListener<EventParameters> { + private static final int LRU_CHECK_FREQ_MILLIS = 60 * 60 * 1000; + + private final ConcurrentHashMap<Integer, Holder> primaryReplicas = new ConcurrentHashMap<>(); + + private final AtomicLong updateCount = new AtomicLong(); + + private final PlacementDriver placementDriver; + + private final HybridClock clock; + + private final AtomicLong lruCheckTime = new AtomicLong(0); + + private final CatalogService catalogService; + + /** + * Constructor. + * + * @param placementDriver Placement driver. + * @param catalogService Catalog. + * @param clock Hybrid clock. + */ + public ClientPrimaryReplicaTracker( + PlacementDriver placementDriver, + CatalogService catalogService, + HybridClock clock) { + this.placementDriver = placementDriver; + this.catalogService = catalogService; + this.clock = clock; + } + + /** + * Gets primary replicas by partition for the table. + * + * @param tableId Table ID. + * @return Primary replicas for the table, or null when not yet known. + */ + public CompletableFuture<List<ReplicaHolder>> primaryReplicasAsync(int tableId) { + return primaryReplicas.compute(tableId, (id, hld) -> { + if (hld == null || hld.replicas.isCompletedExceptionally()) { + return new Holder(initReplicasForTableAsync(id)); + } + + hld.lastAccessTime = System.currentTimeMillis(); + return hld; + }).replicas; + } + + long updateCount() { + return updateCount.get(); + } + + @SuppressWarnings({"rawtypes", "unchecked"}) + void start() { + placementDriver.listen(PrimaryReplicaEvent.PRIMARY_REPLICA_ELECTED, (EventListener) this); + catalogService.listen(CatalogEvent.TABLE_DROP, (EventListener) this); + } + + @SuppressWarnings({"rawtypes", "unchecked"}) + void stop() { + catalogService.removeListener(CatalogEvent.TABLE_DROP, (EventListener) this); + placementDriver.removeListener(PrimaryReplicaEvent.PRIMARY_REPLICA_ELECTED, (EventListener) this); + primaryReplicas.clear(); + } + + private CompletableFuture<List<ReplicaHolder>> initReplicasForTableAsync(Integer tableId) { + try { + // Initially, request all primary replicas for the table. + // Then keep them updated via PRIMARY_REPLICA_ELECTED events. + long timestamp = clock.nowLong(); + CatalogTableDescriptor tableDesc = catalogService.table(tableId, timestamp); + + if (tableDesc == null) { + // TODO: Should we wait for table sync? + return CompletableFuture.failedFuture(tableNotFoundException(tableId)); + } + + CatalogZoneDescriptor zoneDesc = catalogService.zone(tableDesc.zoneId(), timestamp); + + if (zoneDesc == null) { + return CompletableFuture.failedFuture(tableNotFoundException(tableId)); + } + + return primaryReplicasAsyncInternal(tableId, zoneDesc.partitions()); + } catch (Throwable t) { + return CompletableFuture.failedFuture(t); + } + } + + private CompletableFuture<List<ReplicaHolder>> primaryReplicasAsyncInternal(int tableId, int partitions) { + CompletableFuture<ReplicaMeta>[] futs = (CompletableFuture<ReplicaMeta>[]) new CompletableFuture[partitions]; + + for (int partition = 0; partition < partitions; partition++) { + futs[partition] = placementDriver.getPrimaryReplica(new TablePartitionId(tableId, partition), clock.now()); + } + + CompletableFuture<Void> all = CompletableFuture.allOf(futs); + + return all.thenApply(v -> { + List<ReplicaHolder> replicaHolders = new ArrayList<>(partitions); + + for (int partition = 0; partition < partitions; partition++) { + ReplicaMeta replicaMeta = futs[partition].join(); + + // Returning null is fine - the client will use default channel for this partition. + replicaHolders.add(replicaMeta == null + ? new ReplicaHolder(null, HybridTimestamp.MIN_VALUE) + : new ReplicaHolder(replicaMeta.getLeaseholder(), replicaMeta.getStartTime())); + } + + return replicaHolders; + }); + } + + @Override + public CompletableFuture<Boolean> notify(EventParameters parameters, @Nullable Throwable exception) { Review Comment: I'd prefer to keep it as is, since type erasure does not allow me to implement the same interface multiple times in the same class with different generic type arguments. The workaround is to put handlers into fields, but this seems like more code for no good reason. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
