Skip to content

Commit 0def69f

Browse files
authored
Add logic to remove id-conflict KVRanges by considering the lexicographical order of store-id as a tie-breaker (#162)
1 parent 1b1045d commit 0def69f

File tree

2 files changed

+159
-10
lines changed

2 files changed

+159
-10
lines changed

base-kv/base-kv-store-balance-controller/src/main/java/org/apache/bifromq/basekv/balance/impl/RedundantRangeRemovalBalancer.java

Lines changed: 58 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -23,19 +23,25 @@
2323
import static org.apache.bifromq.basekv.utils.DescriptorUtil.getEffectiveRoute;
2424
import static org.apache.bifromq.basekv.utils.DescriptorUtil.organizeByEpoch;
2525

26+
import java.util.Collections;
27+
import java.util.Comparator;
28+
import java.util.HashMap;
29+
import java.util.Map;
30+
import java.util.NavigableMap;
31+
import java.util.Set;
32+
import java.util.SortedSet;
33+
import java.util.TreeSet;
2634
import org.apache.bifromq.basekv.balance.BalanceResult;
2735
import org.apache.bifromq.basekv.balance.NoNeedBalance;
2836
import org.apache.bifromq.basekv.balance.StoreBalancer;
2937
import org.apache.bifromq.basekv.proto.Boundary;
3038
import org.apache.bifromq.basekv.proto.KVRangeDescriptor;
39+
import org.apache.bifromq.basekv.proto.KVRangeId;
3140
import org.apache.bifromq.basekv.proto.KVRangeStoreDescriptor;
3241
import org.apache.bifromq.basekv.raft.proto.RaftNodeStatus;
3342
import org.apache.bifromq.basekv.utils.EffectiveEpoch;
43+
import org.apache.bifromq.basekv.utils.KVRangeIdUtil;
3444
import org.apache.bifromq.basekv.utils.LeaderRange;
35-
import java.util.Collections;
36-
import java.util.Map;
37-
import java.util.NavigableMap;
38-
import java.util.Set;
3945

4046
/**
4147
* The RedundantEpochRemovalBalancer is a specialized StoreBalancer designed to manage and remove redundant replicas
@@ -72,7 +78,7 @@ public BalanceResult balance() {
7278
return NoNeedBalance.INSTANCE;
7379
}
7480
if (latest.size() > 1) {
75-
// deal with higher epoch redundant replicas generated during bootstrap at startup time
81+
// deal with epoch-conflict ranges
7682
Set<KVRangeStoreDescriptor> storeDescriptors = latest.lastEntry().getValue();
7783
for (KVRangeStoreDescriptor storeDescriptor : storeDescriptors) {
7884
if (!storeDescriptor.getId().equals(localStoreId)) {
@@ -82,12 +88,33 @@ public BalanceResult balance() {
8288
if (rangeDescriptor.getRole() != RaftNodeStatus.Leader) {
8389
continue;
8490
}
91+
log.debug("Remove Epoch-Conflict range: {} in store {}",
92+
KVRangeIdUtil.toString(rangeDescriptor.getId()),
93+
storeDescriptor.getId());
8594
return quit(localStoreId, rangeDescriptor);
8695
}
8796
}
97+
return NoNeedBalance.INSTANCE;
8898
}
89-
// deal with redundant replicas generated within the effective epoch but not be included in the effective route
9099
Map.Entry<Long, Set<KVRangeStoreDescriptor>> oldestEntry = latest.firstEntry();
100+
Map<KVRangeId, SortedSet<LeaderRange>> conflictingRanges = findConflictingRanges(oldestEntry.getValue());
101+
if (!conflictingRanges.isEmpty()) {
102+
// deal with id-conflict ranges
103+
for (KVRangeId rangeId : conflictingRanges.keySet()) {
104+
SortedSet<LeaderRange> leaderRanges = conflictingRanges.get(rangeId);
105+
for (LeaderRange leaderRange : leaderRanges) {
106+
if (!leaderRange.ownerStoreDescriptor().getId().equals(localStoreId)) {
107+
return NoNeedBalance.INSTANCE;
108+
}
109+
log.debug("Remove Id-Conflict range: {} in store {}",
110+
KVRangeIdUtil.toString(leaderRange.descriptor().getId()),
111+
leaderRange.ownerStoreDescriptor().getId());
112+
return quit(localStoreId, leaderRange.descriptor());
113+
}
114+
}
115+
return NoNeedBalance.INSTANCE;
116+
}
117+
// deal with boundary-conflict ranges
91118
EffectiveEpoch effectiveEpoch = new EffectiveEpoch(oldestEntry.getKey(), oldestEntry.getValue());
92119
NavigableMap<Boundary, LeaderRange> effectiveLeaders = getEffectiveRoute(effectiveEpoch).leaderRanges();
93120
for (KVRangeStoreDescriptor storeDescriptor : effectiveEpoch.storeDescriptors()) {
@@ -101,10 +128,35 @@ public BalanceResult balance() {
101128
Boundary boundary = rangeDescriptor.getBoundary();
102129
LeaderRange leaderRange = effectiveLeaders.get(boundary);
103130
if (leaderRange == null || !leaderRange.descriptor().getId().equals(rangeDescriptor.getId())) {
131+
log.debug("Remove Boundary-Conflict range: {} in store {}",
132+
KVRangeIdUtil.toString(rangeDescriptor.getId()),
133+
storeDescriptor.getId());
104134
return quit(localStoreId, rangeDescriptor);
105135
}
106136
}
107137
}
108138
return NoNeedBalance.INSTANCE;
109139
}
140+
141+
private Map<KVRangeId, SortedSet<LeaderRange>> findConflictingRanges(Set<KVRangeStoreDescriptor> effectiveEpoch) {
142+
Map<KVRangeId, SortedSet<LeaderRange>> leaderRangesByRangeId = new HashMap<>();
143+
Map<KVRangeId, SortedSet<LeaderRange>> conflictingRanges = new HashMap<>();
144+
for (KVRangeStoreDescriptor storeDescriptor : effectiveEpoch) {
145+
for (KVRangeDescriptor rangeDescriptor : storeDescriptor.getRangesList()) {
146+
if (rangeDescriptor.getRole() != RaftNodeStatus.Leader) {
147+
continue;
148+
}
149+
KVRangeId rangeId = rangeDescriptor.getId();
150+
SortedSet<LeaderRange> leaderRanges = leaderRangesByRangeId.computeIfAbsent(rangeId, k -> new TreeSet<>(
151+
Comparator.comparing((LeaderRange lr) -> lr.ownerStoreDescriptor().getId(), String::compareTo)
152+
.reversed()));
153+
leaderRanges.add(new LeaderRange(rangeDescriptor, storeDescriptor));
154+
if (leaderRanges.size() > 1) {
155+
// More than one leader for the same range, add to conflicting ranges
156+
conflictingRanges.put(rangeId, leaderRanges);
157+
}
158+
}
159+
}
160+
return conflictingRanges;
161+
}
110162
}

base-kv/base-kv-store-balance-controller/src/test/java/org/apache/bifromq/basekv/balance/impl/RedundantRangeRemovalBalancerTest.java

Lines changed: 101 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,10 @@
2222
import static org.testng.Assert.assertEquals;
2323
import static org.testng.Assert.assertSame;
2424

25+
import com.google.protobuf.ByteString;
26+
import java.util.Collections;
27+
import java.util.HashSet;
28+
import java.util.Set;
2529
import org.apache.bifromq.basekv.balance.BalanceNow;
2630
import org.apache.bifromq.basekv.balance.BalanceResult;
2731
import org.apache.bifromq.basekv.balance.BalanceResultType;
@@ -32,10 +36,6 @@
3236
import org.apache.bifromq.basekv.proto.KVRangeStoreDescriptor;
3337
import org.apache.bifromq.basekv.raft.proto.ClusterConfig;
3438
import org.apache.bifromq.basekv.raft.proto.RaftNodeStatus;
35-
import com.google.protobuf.ByteString;
36-
import java.util.Collections;
37-
import java.util.HashSet;
38-
import java.util.Set;
3939
import org.testng.annotations.BeforeMethod;
4040
import org.testng.annotations.Test;
4141

@@ -245,4 +245,101 @@ public void ignoreNonLocalStore() {
245245
BalanceResult result = balancer.balance();
246246
assertSame(result.type(), BalanceResultType.NoNeedBalance);
247247
}
248+
249+
@Test
250+
public void removeIdConflictingRangeWhenLocalStoreIsLoser() {
251+
String peerStoreId = "aStore";
252+
KVRangeId kvRangeId = KVRangeId.newBuilder().setEpoch(1).setId(1).build();
253+
Boundary boundary = Boundary.newBuilder()
254+
.setStartKey(ByteString.copyFromUtf8("a"))
255+
.setEndKey(ByteString.copyFromUtf8("z"))
256+
.build();
257+
258+
// Local store is Leader of the range
259+
KVRangeDescriptor localRange = KVRangeDescriptor.newBuilder()
260+
.setId(kvRangeId)
261+
.setRole(RaftNodeStatus.Leader)
262+
.setVer(1)
263+
.setBoundary(boundary)
264+
.setConfig(ClusterConfig.newBuilder()
265+
.addVoters(localStoreId)
266+
.build())
267+
.build();
268+
269+
KVRangeDescriptor peerRange = KVRangeDescriptor.newBuilder()
270+
.setId(kvRangeId)
271+
.setRole(RaftNodeStatus.Leader)
272+
.setVer(1)
273+
.setBoundary(boundary)
274+
.setConfig(ClusterConfig.newBuilder()
275+
.addVoters(peerStoreId)
276+
.build())
277+
.build();
278+
279+
KVRangeStoreDescriptor localStoreDesc = KVRangeStoreDescriptor.newBuilder()
280+
.setId(localStoreId)
281+
.addRanges(localRange)
282+
.build();
283+
284+
KVRangeStoreDescriptor peerStoreDesc = KVRangeStoreDescriptor.newBuilder()
285+
.setId(peerStoreId)
286+
.addRanges(peerRange)
287+
.build();
288+
289+
balancer.update(Set.of(localStoreDesc, peerStoreDesc));
290+
291+
BalanceResult result = balancer.balance();
292+
assertEquals(result.type(), BalanceResultType.BalanceNow);
293+
294+
ChangeConfigCommand cmd = (ChangeConfigCommand) ((BalanceNow<?>) result).command;
295+
assertEquals(cmd.getToStore(), localStoreId);
296+
assertEquals(cmd.getKvRangeId(), kvRangeId);
297+
assertEquals(cmd.getVoters(), Collections.emptySet());
298+
assertEquals(cmd.getLearners(), Collections.emptySet());
299+
}
300+
301+
@Test
302+
public void ignoreIdConflictingRangeWhenLocalStoreIsWinner() {
303+
String peerStoreId = "zStore"; // larger than "localStore"
304+
KVRangeId kvRangeId = KVRangeId.newBuilder().setEpoch(1).setId(1).build();
305+
Boundary boundary = Boundary.newBuilder()
306+
.setStartKey(ByteString.copyFromUtf8("a"))
307+
.setEndKey(ByteString.copyFromUtf8("z"))
308+
.build();
309+
310+
KVRangeDescriptor localRange = KVRangeDescriptor.newBuilder()
311+
.setId(kvRangeId)
312+
.setRole(RaftNodeStatus.Leader)
313+
.setVer(1)
314+
.setBoundary(boundary)
315+
.setConfig(ClusterConfig.newBuilder()
316+
.addVoters(localStoreId)
317+
.build())
318+
.build();
319+
320+
KVRangeDescriptor peerRange = KVRangeDescriptor.newBuilder()
321+
.setId(kvRangeId)
322+
.setRole(RaftNodeStatus.Leader)
323+
.setVer(1)
324+
.setBoundary(boundary)
325+
.setConfig(ClusterConfig.newBuilder()
326+
.addVoters(peerStoreId)
327+
.build())
328+
.build();
329+
330+
KVRangeStoreDescriptor localStoreDesc = KVRangeStoreDescriptor.newBuilder()
331+
.setId(localStoreId)
332+
.addRanges(localRange)
333+
.build();
334+
335+
KVRangeStoreDescriptor peerStoreDesc = KVRangeStoreDescriptor.newBuilder()
336+
.setId(peerStoreId)
337+
.addRanges(peerRange)
338+
.build();
339+
340+
balancer.update(Set.of(localStoreDesc, peerStoreDesc));
341+
342+
BalanceResult result = balancer.balance();
343+
assertSame(result.type(), BalanceResultType.NoNeedBalance);
344+
}
248345
}

0 commit comments

Comments
 (0)