Skip to content
Original file line number Diff line number Diff line change
Expand Up @@ -19,16 +19,6 @@
import java.util.List;


public class ChunkProcessor<T> {

protected Router<T> router;

public ChunkProcessor(Router<T> router) {
this.router = router;
}

public void process(ConnectionManager<T> connectionManager, List<T> chunks) {
router.route(connectionManager.connections(), chunks);
}

public interface ChunkProcessor<T> {
void process(ConnectionManager<T> connectionManager, List<T> chunks);
}
Original file line number Diff line number Diff line change
Expand Up @@ -23,11 +23,14 @@
import io.mantisrx.common.metrics.Metrics;
import io.mantisrx.common.metrics.spectator.GaugeCallback;
import io.mantisrx.common.metrics.spectator.MetricGroupId;

import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import rx.functions.Func0;
Expand All @@ -43,10 +46,12 @@ public class ConnectionGroup<T> {
private Counter successfulWrites;
private Counter numSlotSwaps;
private Counter failedWrites;
private final Optional<ProactiveRouter<T>> routerO;

public ConnectionGroup(String groupId) {
public ConnectionGroup(String groupId, Optional<ProactiveRouter<T>> routerO) {
this.groupId = groupId;
this.connections = new HashMap<>();
this.routerO = routerO;

final String grpId = Optional.ofNullable(groupId).orElse("none");
final BasicTag groupIdTag = new BasicTag(MantisMetricStringConstants.GROUP_ID_TAG, grpId);
Expand Down Expand Up @@ -93,6 +98,7 @@ public synchronized void removeConnection(AsyncConnection<T> connection) {
+ " a new connection has already been swapped in the place of the old connection");

}
this.routerO.ifPresent(router -> router.removeConnection(connection));
}

public synchronized void addConnection(AsyncConnection<T> connection) {
Expand All @@ -107,6 +113,7 @@ public synchronized void addConnection(AsyncConnection<T> connection) {
previousConnection.close();
numSlotSwaps.increment();
}
this.routerO.ifPresent(router -> router.addConnection(connection));
}

public synchronized boolean isEmpty() {
Expand All @@ -132,4 +139,11 @@ public String toString() {
return "ConnectionGroup [groupId=" + groupId + ", connections="
+ connections + "]";
}

public void route(List<T> chunks, Router<T> fallbackRouter) {
this.routerO.ifPresentOrElse(
router -> router.route(chunks),
() -> fallbackRouter.route(this.getConnections(), chunks)
);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,9 +20,11 @@
import io.mantisrx.common.metrics.MetricsRegistry;
import io.mantisrx.common.metrics.spectator.GaugeCallback;
import io.mantisrx.common.metrics.spectator.MetricGroupId;

import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedHashMap;
import java.util.Optional;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicBoolean;
Expand All @@ -32,6 +34,7 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import rx.functions.Action0;
import rx.functions.Func1;


public class ConnectionManager<T> {
Expand All @@ -46,9 +49,13 @@ public class ConnectionManager<T> {
private Action0 doOnZeroConnections;
private Lock connectionState = new ReentrantLock();
private AtomicBoolean subscribed = new AtomicBoolean();
private final Func1<String, Optional<ProactiveRouter<T>>> routerFactory;

public ConnectionManager(MetricsRegistry metricsRegistry,
Action0 doOnFirstConnection, Action0 doOnZeroConnections) {
Action0 doOnFirstConnection,
Action0 doOnZeroConnections,
Func1<String, Optional<ProactiveRouter<T>>> routerFactory) {
this.routerFactory = routerFactory;
this.doOnFirstConnection = doOnFirstConnection;
this.doOnZeroConnections = doOnZeroConnections;
this.metricsRegistry = metricsRegistry;
Expand Down Expand Up @@ -119,11 +126,13 @@ protected void add(AsyncConnection<T> connection) {
String groupId = connection.getGroupId();
ConnectionGroup<T> current = managedConnections.get(groupId);
if (current == null) {
ConnectionGroup<T> newGroup = new ConnectionGroup<T>(groupId);
Optional<ProactiveRouter<T>> groupRouter = routerFactory.call(groupId);
ConnectionGroup<T> newGroup = new ConnectionGroup<T>(groupId, groupRouter);
current = managedConnections.putIfAbsent(groupId, newGroup);
if (current == null) {
current = newGroup;
metricsRegistry.registerAndGet(current.getMetrics());
groupRouter.ifPresent(router -> metricsRegistry.registerAndGet(router.getMetrics()));
}
}
current.addConnection(connection);
Expand Down Expand Up @@ -167,19 +176,6 @@ protected void remove(AsyncConnection<T> connection) {
}
}

public Set<AsyncConnection<T>> connections() {
connectionState.lock();
try {
Set<AsyncConnection<T>> connections = new HashSet<>();
for (ConnectionGroup<T> group : managedConnections.values()) {
connections.addAll(group.getConnections());
}
return connections;
} finally {
connectionState.unlock();
}
}

public Map<String, ConnectionGroup<T>> groups() {
connectionState.lock();
try {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,18 +20,18 @@
import java.util.Map;


public class GroupChunkProcessor<T> extends ChunkProcessor<T> {
public class GroupChunkProcessor<T> implements ChunkProcessor<T> {
protected Router<T> fallbackRouter;

public GroupChunkProcessor(Router<T> router) {
super(router);
public GroupChunkProcessor(Router<T> fallbackRouter) {
this.fallbackRouter = fallbackRouter;
}

@Override
public void process(ConnectionManager<T> connectionManager, List<T> chunks) {
Map<String, ConnectionGroup<T>> groups = connectionManager.groups();
for (ConnectionGroup<T> group : groups.values()) {
router.route(group.getConnections(), chunks);
group.route(chunks, fallbackRouter);
}
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,171 @@
/*
* Copyright 2025 Netflix, Inc.
*
* Licensed 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 io.reactivex.mantis.network.push;

import com.netflix.spectator.api.Tag;
import io.mantisrx.common.metrics.Counter;
import io.mantisrx.common.metrics.Metrics;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import rx.functions.Func1;

import java.util.*;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;

public class ProactiveConsistentHashingRouter<K, V> implements ProactiveRouter<KeyValuePair<K, V>> {
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Basically the same as ConsistentHashingRouter except that does not recalculate the whole thing. It just makes the updates to add/remove a connection.

private static final Logger logger = LoggerFactory.getLogger(ProactiveConsistentHashingRouter.class);
private final int connectionRepetitionOnRing;

protected final Func1<KeyValuePair<K, V>, byte[]> encoder;
protected final Counter numEventsRouted;
protected final Counter numEventsProcessed;
protected final Counter numConnectionUpdates;
protected final Metrics metrics;
private final HashFunction hashFunction;
private final NavigableMap<Long, AsyncConnection<KeyValuePair<K, V>>> ring = new TreeMap<>();
private final ReadWriteLock ringLock = new ReentrantReadWriteLock();

public ProactiveConsistentHashingRouter(
String name,
Func1<KeyValuePair<K, V>, byte[]> dataEncoder,
HashFunction hashFunction) {
this(name, dataEncoder, hashFunction, 1000);
}

public ProactiveConsistentHashingRouter(
String name,
Func1<KeyValuePair<K, V>, byte[]> dataEncoder,
HashFunction hashFunction,
int ringRepetitionPerConnection) {
this.connectionRepetitionOnRing = ringRepetitionPerConnection;
this.encoder = dataEncoder;
metrics = new Metrics.Builder()
.id("Router_" + name, Tag.of("router_type", "proactive_consistent_hashing"))
.addCounter("numEventsRouted")
.addCounter("numEventsProcessed")
.addCounter("numConnectionUpdates")
.build();
numEventsRouted = metrics.getCounter("numEventsRouted");
numEventsProcessed = metrics.getCounter("numEventsProcessed");
numConnectionUpdates = metrics.getCounter("numConnectionUpdates");
this.hashFunction = hashFunction;
}

@Override
public void route(List<KeyValuePair<K, V>> chunks) {
if (chunks == null || chunks.isEmpty()) {
return;
}
numEventsProcessed.increment(chunks.size());

// Read lock only for ring access
Map<AsyncConnection<KeyValuePair<K, V>>, List<byte[]>> writes;
ringLock.readLock().lock();
try {
if (ring.isEmpty()) {
return;
}

int numConnections = ring.size() / connectionRepetitionOnRing;
int bufferCapacity = (chunks.size() / numConnections) + 1; // assume even distribution
writes = new HashMap<>(numConnections);

// process chunks (ring access inside lookupConnection)
for (KeyValuePair<K, V> kvp : chunks) {
long hash = kvp.getKeyBytesHashed();
// lookup slot
Map.Entry<Long, AsyncConnection<KeyValuePair<K, V>>> connectionEntry = ring.ceilingEntry(hash);
AsyncConnection<KeyValuePair<K, V>> connection = (connectionEntry == null ? ring.firstEntry() : connectionEntry).getValue();
// add to writes
Func1<KeyValuePair<K, V>, Boolean> predicate = connection.getPredicate();
if (predicate == null || predicate.call(kvp)) {
List<byte[]> buffer = writes.computeIfAbsent(connection, k -> new ArrayList<>(bufferCapacity));
buffer.add(encoder.call(kvp));
}
}
} finally {
ringLock.readLock().unlock();
}

// process writes (outside lock - no ring access)
if (!writes.isEmpty()) {
for (Map.Entry<AsyncConnection<KeyValuePair<K, V>>, List<byte[]>> entry : writes.entrySet()) {
AsyncConnection<KeyValuePair<K, V>> connection = entry.getKey();
List<byte[]> toWrite = entry.getValue();
connection.write(toWrite);
numEventsRouted.increment(toWrite.size());
}
}
}

@Override
public void addConnection(AsyncConnection<KeyValuePair<K, V>> connection) {
String connectionId = connection.getSlotId();
if (connectionId == null) {
throw new IllegalStateException("Connection must specify an id for consistent hashing");
}

List<String> hashCollisions = new ArrayList<>();
ringLock.writeLock().lock();
try {
for (int i = 0; i < connectionRepetitionOnRing; i++) {
// hash node on ring
byte[] connectionBytes = (connectionId + "-" + i).getBytes();
long hash = hashFunction.computeHash(connectionBytes);
if (ring.containsKey(hash)) {
hashCollisions.add(connectionId + "-" + i);
}
ring.put(hash, connection);
}
} finally {
ringLock.writeLock().unlock();
}
numConnectionUpdates.increment();

// Log outside lock
if (!hashCollisions.isEmpty()) {
logger.error("Hash collisions detected when adding connection {}: {}", connectionId, hashCollisions);
}
}

@Override
public void removeConnection(AsyncConnection<KeyValuePair<K, V>> connection) {
String connectionId = connection.getSlotId();
if (connectionId == null) {
throw new IllegalStateException("Connection must specify an id for consistent hashing");
}

ringLock.writeLock().lock();
try {
for (int i = 0; i < connectionRepetitionOnRing; i++) {
// hash node on ring
byte[] connectionBytes = (connectionId + "-" + i).getBytes();
long hash = hashFunction.computeHash(connectionBytes);
ring.remove(hash);
}
} finally {
ringLock.writeLock().unlock();
}
numConnectionUpdates.increment();
}

@Override
public Metrics getMetrics() {
return metrics;
}
}
Loading
Loading