Skip to content

Commit c401bba

Browse files
IGNITE-26909 Thin client optimizations for MultiDC
1 parent 68f6fbf commit c401bba

File tree

5 files changed

+27
-27
lines changed

5 files changed

+27
-27
lines changed

modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientCacheAffinityContext.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -258,7 +258,7 @@ public void writeDataCenterNodesRequest(PayloadOutputChannel ch) {
258258
/**
259259
* @param ch Payload input channel.
260260
*/
261-
public boolean readDataCenterNodesRequest(PayloadInputChannel ch) {
261+
public boolean readDataCenterNodesResponse(PayloadInputChannel ch) {
262262
TopologyNodes top = lastTop.get();
263263

264264
if (top == null)

modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientCacheAffinityMapping.java

Lines changed: 11 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -217,13 +217,13 @@ public static ClientCacheAffinityMapping readResponse(
217217
cacheKeyCfg.put(in.readInt(), readCacheKeyConfiguration(in));
218218

219219
UUID[] primaryPartToNode = readNodePartitions(in);
220-
UUID[] dcBackupPartToNode = primaryPartToNode;
220+
UUID[] dcPartToNode = primaryPartToNode;
221221

222222
if (ch.clientChannel().protocolCtx().isFeatureSupported(DC_AWARE)) {
223-
dcBackupPartToNode = readNodePartitions(in);
223+
dcPartToNode = readNodePartitions(in);
224224

225-
if (dcBackupPartToNode.length == 0)
226-
dcBackupPartToNode = primaryPartToNode;
225+
if (dcPartToNode.length == 0)
226+
dcPartToNode = primaryPartToNode;
227227
}
228228

229229
boolean dfltMapping = true;
@@ -240,7 +240,7 @@ public static ClientCacheAffinityMapping readResponse(
240240
continue;
241241

242242
aff.cacheAff.put(keyCfg.getKey(),
243-
new CacheAffinityInfo(keyCfg.getValue(), primaryPartToNode, dcBackupPartToNode,
243+
new CacheAffinityInfo(keyCfg.getValue(), primaryPartToNode, dcPartToNode,
244244
factory.apply(primaryPartToNode.length))
245245
);
246246
}
@@ -318,8 +318,8 @@ private static class CacheAffinityInfo {
318318
/** Primary partition mapping. */
319319
private final UUID[] primaryPartMapping;
320320

321-
/** Backup partition mapping, located in current DC. */
322-
private final UUID[] dcBackupPartMapping;
321+
/** Partition mapping to nodes located in current DC. */
322+
private final UUID[] dcPartMapping;
323323

324324
/** Mapper a cache key to a partition. */
325325
private final ClientPartitionAwarenessMapper keyMapper;
@@ -328,17 +328,17 @@ private static class CacheAffinityInfo {
328328
* @param keyCfg Cache key configuration or {@code null} if partition awareness is not applicable for this cache.
329329
* @param primaryPartMapping Primary partition to node mapping or {@code null} if partition awareness
330330
* is not applicable for this cache.
331-
* @param dcBackupPartMapping Backup partition to node mapping, located in current DC.
331+
* @param dcPartMapping Partition to node mapping located in current DC.
332332
*/
333333
private CacheAffinityInfo(
334334
Map<Integer, Integer> keyCfg,
335335
UUID[] primaryPartMapping,
336-
UUID[] dcBackupPartMapping,
336+
UUID[] dcPartMapping,
337337
ClientPartitionAwarenessMapper keyMapper
338338
) {
339339
this.keyCfg = keyCfg;
340340
this.primaryPartMapping = primaryPartMapping;
341-
this.dcBackupPartMapping = dcBackupPartMapping;
341+
this.dcPartMapping = dcPartMapping;
342342
this.keyMapper = keyMapper;
343343
}
344344

@@ -362,7 +362,7 @@ private UUID nodeForKey(Object key, boolean primary) {
362362
* @param primary Force primary node.
363363
*/
364364
private UUID nodeForPartition(int part, boolean primary) {
365-
UUID[] partMapping = primary ? primaryPartMapping : dcBackupPartMapping;
365+
UUID[] partMapping = primary ? primaryPartMapping : dcPartMapping;
366366

367367
if (part < 0 || partMapping == null || part >= partMapping.length)
368368
return null;

modules/core/src/main/java/org/apache/ignite/internal/client/thin/ReliableChannel.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -527,7 +527,7 @@ private boolean affinityInfoIsUpToDate(int cacheId) {
527527
&& channel.protocolCtx().isFeatureSupported(ProtocolBitmaskFeature.DC_AWARE)) {
528528
channel.service(ClientOperation.CLUSTER_GET_DC_NODES,
529529
affinityCtx::writeDataCenterNodesRequest,
530-
affinityCtx::readDataCenterNodesRequest);
530+
affinityCtx::readDataCenterNodesResponse);
531531
}
532532

533533
return updated;

modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCachePartitionMapping.java

Lines changed: 8 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -33,20 +33,20 @@ public class ClientCachePartitionMapping {
3333
/** Primary partitions map for caches. */
3434
private final Map<UUID, Set<Integer>> primaryPartitionMap;
3535

36-
/** Backups partitions map, located in current data center for caches. */
37-
@Nullable private final Map<UUID, Set<Integer>> dcBackupPartitionMap;
36+
/** Partitions map to nodes located in current data center for caches. */
37+
@Nullable private final Map<UUID, Set<Integer>> dcPartitionMap;
3838

3939
/**
4040
* @param primaryPartitionMap Primary partition mapping.
41-
* @param dcBackupPartitionMap Backup partition mapping, located in current data center.
41+
* @param dcPartitionMap Partition mapping to nodes located in current data center.
4242
*/
4343
@SuppressWarnings("AssignmentOrReturnOfFieldWithMutableType")
4444
public ClientCachePartitionMapping(
4545
Map<UUID, Set<Integer>> primaryPartitionMap,
46-
@Nullable Map<UUID, Set<Integer>> dcBackupPartitionMap
46+
@Nullable Map<UUID, Set<Integer>> dcPartitionMap
4747
) {
4848
this.primaryPartitionMap = primaryPartitionMap;
49-
this.dcBackupPartitionMap = dcBackupPartitionMap;
49+
this.dcPartitionMap = dcPartitionMap;
5050
}
5151

5252
/**
@@ -57,7 +57,7 @@ public void write(ClientProtocolContext ctx, BinaryRawWriter writer) {
5757
writePartitionMap(writer, primaryPartitionMap);
5858

5959
if (ctx.isFeatureSupported(ClientBitmaskFeature.DC_AWARE))
60-
writePartitionMap(writer, dcBackupPartitionMap);
60+
writePartitionMap(writer, dcPartitionMap);
6161
}
6262

6363
/** */
@@ -93,11 +93,11 @@ private static void writePartitionMap(BinaryRawWriter writer, @Nullable Map<UUID
9393
ClientCachePartitionMapping mapping = (ClientCachePartitionMapping)o;
9494

9595
return Objects.equals(primaryPartitionMap, mapping.primaryPartitionMap)
96-
&& Objects.equals(dcBackupPartitionMap, mapping.dcBackupPartitionMap);
96+
&& Objects.equals(dcPartitionMap, mapping.dcPartitionMap);
9797
}
9898

9999
/** {@inheritDoc} */
100100
@Override public int hashCode() {
101-
return Objects.hash(primaryPartitionMap, dcBackupPartitionMap);
101+
return Objects.hash(primaryPartitionMap, dcPartitionMap);
102102
}
103103
}

modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCachePartitionsRequest.java

Lines changed: 6 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -193,12 +193,12 @@ private ClientCachePartitionAwarenessGroup processCache(
193193
primaryPartitionMap.put(nodeId, parts);
194194
}
195195

196-
Map<UUID, Set<Integer>> dcBackupPartitionMap = null;
196+
Map<UUID, Set<Integer>> dcPartitionMap = null;
197197

198198
if (dcId != null && cacheCtx.config().isReadFromBackup()
199199
&& cacheCtx.config().getWriteSynchronizationMode() != CacheWriteSynchronizationMode.PRIMARY_SYNC) {
200-
// Filter backup partitions, located in current DC.
201-
dcBackupPartitionMap = new HashMap<>();
200+
// Filter partitions, located in current DC.
201+
dcPartitionMap = new HashMap<>();
202202

203203
List<List<ClusterNode>> partAssignments = assignment.assignment();
204204

@@ -208,16 +208,16 @@ private ClientCachePartitionAwarenessGroup processCache(
208208
ClusterNode node = F.find(partAssignment, null, n -> dcId.equals(n.dataCenterId()));
209209

210210
if (node != null)
211-
dcBackupPartitionMap.computeIfAbsent(node.id(), id -> new HashSet<>()).add(p);
211+
dcPartitionMap.computeIfAbsent(node.id(), id -> new HashSet<>()).add(p);
212212
else {
213-
dcBackupPartitionMap = null;
213+
dcPartitionMap = null;
214214

215215
break;
216216
}
217217
}
218218
}
219219

220-
return new ClientCachePartitionMapping(primaryPartitionMap, dcBackupPartitionMap);
220+
return new ClientCachePartitionMapping(primaryPartitionMap, dcPartitionMap);
221221
}
222222
catch (Exception e) {
223223
return null;

0 commit comments

Comments
 (0)