This is an automated email from the ASF dual-hosted git repository.

dataroaring pushed a commit to branch feature/cloud-replica-memory-opt
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 0d10541ed9a4cb873c33e8f46085271a14f361ca
Author: Yongqiang YANG <[email protected]>
AuthorDate: Thu Mar 12 06:18:16 2026 -0700

    [opt](memory) Intern cluster ID strings in CloudReplica
    
    Add a static intern pool for cluster ID strings to eliminate millions
    of duplicate String instances across CloudReplica objects. Each replica
    stores cluster IDs as map keys; without interning, Gson deserialization
    creates a separate String instance per replica (~40-70 bytes each).
    
    - Add ConcurrentHashMap-based intern pool (heap-safe, unlike String.intern)
    - Intern strings at all write points (updateClusterToPrimaryBe/Secondary)
    - Intern keys during gsonPostProcess for deserialized maps
    - For 1M replicas with 3 clusters: saves ~40-70 MB of duplicate Strings
    
    Co-Authored-By: Claude Opus 4.6 <[email protected]>
---
 .../apache/doris/cloud/catalog/CloudReplica.java   | 27 +++++++++++++++++++---
 1 file changed, 24 insertions(+), 3 deletions(-)

diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudReplica.java 
b/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudReplica.java
index b5f119cc803..6948ef2f49c 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudReplica.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudReplica.java
@@ -73,6 +73,18 @@ public class CloudReplica extends Replica implements 
GsonPostProcessable {
 
     private static final Random rand = new Random();
 
+    // Intern pool for cluster ID strings to avoid millions of duplicate 
String instances.
+    // Typically only a handful of distinct cluster IDs exist in the system.
+    private static final ConcurrentHashMap<String, String> CLUSTER_ID_POOL = 
new ConcurrentHashMap<>();
+
+    private static String internClusterId(String clusterId) {
+        if (clusterId == null) {
+            return null;
+        }
+        String existing = CLUSTER_ID_POOL.putIfAbsent(clusterId, clusterId);
+        return existing != null ? existing : clusterId;
+    }
+
     private Map<String, List<Long>> memClusterToBackends = null;
 
     // clusterId, secondaryBe, changeTimestamp
@@ -608,7 +620,7 @@ public class CloudReplica extends Replica implements 
GsonPostProcessable {
     }
 
     public void updateClusterToPrimaryBe(String cluster, long beId) {
-        getOrCreatePrimaryMap().put(cluster, beId);
+        getOrCreatePrimaryMap().put(internClusterId(cluster), beId);
         Map<String, Pair<Long, Long>> secMap = secondaryClusterToBackends;
         if (secMap != null) {
             secMap.remove(cluster);
@@ -626,7 +638,7 @@ public class CloudReplica extends Replica implements 
GsonPostProcessable {
             LOG.debug("add to secondary clusterId {}, beId {}, changeTimestamp 
{}, replica info {}",
                     cluster, beId, changeTimestamp, this);
         }
-        getOrCreateSecondaryMap().put(cluster, Pair.of(beId, changeTimestamp));
+        getOrCreateSecondaryMap().put(internClusterId(cluster), Pair.of(beId, 
changeTimestamp));
     }
 
     public void clearClusterToBe(String cluster) {
@@ -709,10 +721,19 @@ public class CloudReplica extends Replica implements 
GsonPostProcessable {
                 String clusterId = entry.getKey();
                 List<Long> beIds = entry.getValue();
                 if (beIds != null && !beIds.isEmpty()) {
-                    map.put(clusterId, beIds.get(0));
+                    map.put(internClusterId(clusterId), beIds.get(0));
                 }
             }
             this.primaryClusterToBackends = null;
         }
+        // Intern cluster ID keys in deserialized primary map to share String 
instances
+        // across millions of CloudReplica objects
+        if (primaryClusterToBackend != null) {
+            ConcurrentHashMap<String, Long> interned = new 
ConcurrentHashMap<>(primaryClusterToBackend.size());
+            for (Map.Entry<String, Long> entry : 
primaryClusterToBackend.entrySet()) {
+                interned.put(internClusterId(entry.getKey()), 
entry.getValue());
+            }
+            primaryClusterToBackend = interned;
+        }
     }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to