Implement reactor-kafka for updates
This commit is contained in:
parent
799fd4149c
commit
f48a1d321b
4
pom.xml
4
pom.xml
@ -143,6 +143,10 @@
|
|||||||
<groupId>io.projectreactor</groupId>
|
<groupId>io.projectreactor</groupId>
|
||||||
<artifactId>reactor-tools</artifactId>
|
<artifactId>reactor-tools</artifactId>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
<dependency>
|
||||||
|
<groupId>io.projectreactor.kafka</groupId>
|
||||||
|
<artifactId>reactor-kafka</artifactId>
|
||||||
|
</dependency>
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>org.apache.logging.log4j</groupId>
|
<groupId>org.apache.logging.log4j</groupId>
|
||||||
<artifactId>log4j-core</artifactId>
|
<artifactId>log4j-core</artifactId>
|
||||||
|
@ -31,7 +31,6 @@ import java.util.concurrent.CompletionException;
|
|||||||
import java.util.concurrent.ConcurrentHashMap;
|
import java.util.concurrent.ConcurrentHashMap;
|
||||||
import java.util.concurrent.ConcurrentMap;
|
import java.util.concurrent.ConcurrentMap;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
import java.util.stream.Stream;
|
|
||||||
import org.jetbrains.annotations.NotNull;
|
import org.jetbrains.annotations.NotNull;
|
||||||
import org.jetbrains.annotations.Nullable;
|
import org.jetbrains.annotations.Nullable;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
@ -50,6 +49,8 @@ public class AtomixReactiveApi implements ReactiveApi {
|
|||||||
@Nullable
|
@Nullable
|
||||||
private final String nodeId;
|
private final String nodeId;
|
||||||
private final Atomix atomix;
|
private final Atomix atomix;
|
||||||
|
private final KafkaProducer kafkaProducer;
|
||||||
|
private final KafkaConsumer kafkaConsumer;
|
||||||
private final Set<ResultingEventTransformer> resultingEventTransformerSet;
|
private final Set<ResultingEventTransformer> resultingEventTransformerSet;
|
||||||
private final AsyncAtomicIdGenerator nextSessionLiveId;
|
private final AsyncAtomicIdGenerator nextSessionLiveId;
|
||||||
|
|
||||||
@ -67,10 +68,13 @@ public class AtomixReactiveApi implements ReactiveApi {
|
|||||||
|
|
||||||
public AtomixReactiveApi(@Nullable String nodeId,
|
public AtomixReactiveApi(@Nullable String nodeId,
|
||||||
Atomix atomix,
|
Atomix atomix,
|
||||||
|
KafkaParameters kafkaParameters,
|
||||||
@Nullable DiskSessionsManager diskSessions,
|
@Nullable DiskSessionsManager diskSessions,
|
||||||
@NotNull Set<ResultingEventTransformer> resultingEventTransformerSet) {
|
@NotNull Set<ResultingEventTransformer> resultingEventTransformerSet) {
|
||||||
this.nodeId = nodeId;
|
this.nodeId = nodeId;
|
||||||
this.atomix = atomix;
|
this.atomix = atomix;
|
||||||
|
this.kafkaProducer = new KafkaProducer(kafkaParameters);
|
||||||
|
this.kafkaConsumer = new KafkaConsumer(kafkaParameters);
|
||||||
this.resultingEventTransformerSet = resultingEventTransformerSet;
|
this.resultingEventTransformerSet = resultingEventTransformerSet;
|
||||||
|
|
||||||
if (nodeId == null) {
|
if (nodeId == null) {
|
||||||
@ -277,7 +281,7 @@ public class AtomixReactiveApi implements ReactiveApi {
|
|||||||
userId = createBotSessionRequest.userId();
|
userId = createBotSessionRequest.userId();
|
||||||
botToken = createBotSessionRequest.token();
|
botToken = createBotSessionRequest.token();
|
||||||
phoneNumber = null;
|
phoneNumber = null;
|
||||||
reactiveApiPublisher = ReactiveApiPublisher.fromToken(atomix, resultingEventTransformerSet,
|
reactiveApiPublisher = ReactiveApiPublisher.fromToken(atomix, kafkaProducer, resultingEventTransformerSet,
|
||||||
liveId,
|
liveId,
|
||||||
userId,
|
userId,
|
||||||
botToken
|
botToken
|
||||||
@ -287,7 +291,7 @@ public class AtomixReactiveApi implements ReactiveApi {
|
|||||||
userId = createUserSessionRequest.userId();
|
userId = createUserSessionRequest.userId();
|
||||||
botToken = null;
|
botToken = null;
|
||||||
phoneNumber = createUserSessionRequest.phoneNumber();
|
phoneNumber = createUserSessionRequest.phoneNumber();
|
||||||
reactiveApiPublisher = ReactiveApiPublisher.fromPhoneNumber(atomix, resultingEventTransformerSet,
|
reactiveApiPublisher = ReactiveApiPublisher.fromPhoneNumber(atomix, kafkaProducer, resultingEventTransformerSet,
|
||||||
liveId,
|
liveId,
|
||||||
userId,
|
userId,
|
||||||
phoneNumber
|
phoneNumber
|
||||||
@ -298,13 +302,13 @@ public class AtomixReactiveApi implements ReactiveApi {
|
|||||||
botToken = loadSessionFromDiskRequest.token();
|
botToken = loadSessionFromDiskRequest.token();
|
||||||
phoneNumber = loadSessionFromDiskRequest.phoneNumber();
|
phoneNumber = loadSessionFromDiskRequest.phoneNumber();
|
||||||
if (loadSessionFromDiskRequest.phoneNumber() != null) {
|
if (loadSessionFromDiskRequest.phoneNumber() != null) {
|
||||||
reactiveApiPublisher = ReactiveApiPublisher.fromPhoneNumber(atomix, resultingEventTransformerSet,
|
reactiveApiPublisher = ReactiveApiPublisher.fromPhoneNumber(atomix, kafkaProducer, resultingEventTransformerSet,
|
||||||
liveId,
|
liveId,
|
||||||
userId,
|
userId,
|
||||||
phoneNumber
|
phoneNumber
|
||||||
);
|
);
|
||||||
} else {
|
} else {
|
||||||
reactiveApiPublisher = ReactiveApiPublisher.fromToken(atomix, resultingEventTransformerSet,
|
reactiveApiPublisher = ReactiveApiPublisher.fromToken(atomix, kafkaProducer, resultingEventTransformerSet,
|
||||||
liveId,
|
liveId,
|
||||||
userId,
|
userId,
|
||||||
botToken
|
botToken
|
||||||
@ -475,22 +479,24 @@ public class AtomixReactiveApi implements ReactiveApi {
|
|||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ReactiveApiClient dynamicClient(long userId) {
|
public ReactiveApiClient dynamicClient(long userId) {
|
||||||
return new DynamicAtomixReactiveApiClient(this, userId);
|
return new DynamicAtomixReactiveApiClient(this, kafkaConsumer, userId);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ReactiveApiClient liveClient(long liveId, long userId) {
|
public ReactiveApiClient liveClient(long liveId, long userId) {
|
||||||
return new LiveAtomixReactiveApiClient(atomix, liveId, userId);
|
return new LiveAtomixReactiveApiClient(atomix, kafkaConsumer, liveId, userId);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ReactiveApiMultiClient multiClient() {
|
public ReactiveApiMultiClient multiClient() {
|
||||||
return new AtomixReactiveApiMultiClient(this);
|
return new AtomixReactiveApiMultiClient(this, kafkaConsumer);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Mono<Void> close() {
|
public Mono<Void> close() {
|
||||||
return Mono.fromCompletionStage(this.atomix::stop).timeout(Duration.ofSeconds(8), Mono.empty());
|
var atomixStopper = Mono.fromCompletionStage(this.atomix::stop).timeout(Duration.ofSeconds(8), Mono.empty());
|
||||||
|
var kafkaStopper = Mono.fromRunnable(kafkaProducer::close).subscribeOn(Schedulers.boundedElastic());
|
||||||
|
return Mono.when(atomixStopper, kafkaStopper);
|
||||||
}
|
}
|
||||||
|
|
||||||
private record DiskSessionAndId(DiskSession diskSession, long id) {}
|
private record DiskSessionAndId(DiskSession diskSession, long id) {}
|
||||||
|
@ -20,35 +20,21 @@ public class AtomixReactiveApiMultiClient implements ReactiveApiMultiClient, Aut
|
|||||||
|
|
||||||
private final ClusterEventService eventService;
|
private final ClusterEventService eventService;
|
||||||
|
|
||||||
private final Flux<ClientBoundEvent> clientBoundEvents;
|
private final KafkaConsumer kafkaConsumer;
|
||||||
|
|
||||||
private volatile boolean closed = false;
|
private volatile boolean closed = false;
|
||||||
|
|
||||||
AtomixReactiveApiMultiClient(AtomixReactiveApi api) {
|
AtomixReactiveApiMultiClient(AtomixReactiveApi api, KafkaConsumer kafkaConsumer) {
|
||||||
this.eventService = api.getAtomix().getEventService();
|
this.eventService = api.getAtomix().getEventService();
|
||||||
|
this.kafkaConsumer = kafkaConsumer;
|
||||||
clientBoundEvents = Flux
|
|
||||||
.<List<ClientBoundEvent>>push(sink -> {
|
|
||||||
var subscriptionFuture = eventService.subscribe("session-client-bound-events",
|
|
||||||
LiveAtomixReactiveApiClient::deserializeEvents,
|
|
||||||
s -> {
|
|
||||||
sink.next(s);
|
|
||||||
return CompletableFuture.completedFuture(null);
|
|
||||||
},
|
|
||||||
(a) -> null
|
|
||||||
);
|
|
||||||
sink.onDispose(() -> subscriptionFuture.thenAccept(Subscription::close));
|
|
||||||
}, OverflowStrategy.ERROR)
|
|
||||||
.subscribeOn(Schedulers.boundedElastic())
|
|
||||||
.onBackpressureBuffer(0xFFFF, BufferOverflowStrategy.ERROR)
|
|
||||||
.flatMapIterable(list -> list)
|
|
||||||
.takeUntil(s -> closed)
|
|
||||||
.share();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Flux<ClientBoundEvent> clientBoundEvents() {
|
public Flux<ClientBoundEvent> clientBoundEvents(boolean ack) {
|
||||||
return clientBoundEvents;
|
if (closed) {
|
||||||
|
return Flux.empty();
|
||||||
|
}
|
||||||
|
return kafkaConsumer.consumeMessages(kafkaConsumer.newRandomGroupId(), ack).takeUntil(s -> closed);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -0,0 +1,12 @@
|
|||||||
|
package it.tdlight.reactiveapi;
|
||||||
|
|
||||||
|
import it.tdlight.reactiveapi.Event.ClientBoundEvent;
|
||||||
|
import org.apache.kafka.common.serialization.Deserializer;
|
||||||
|
|
||||||
|
public class ClientBoundEventDeserializer implements Deserializer<ClientBoundEvent> {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ClientBoundEvent deserialize(String topic, byte[] data) {
|
||||||
|
return LiveAtomixReactiveApiClient.deserializeEvent(data);
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,12 @@
|
|||||||
|
package it.tdlight.reactiveapi;
|
||||||
|
|
||||||
|
import it.tdlight.reactiveapi.Event.ClientBoundEvent;
|
||||||
|
import org.apache.kafka.common.serialization.Serializer;
|
||||||
|
|
||||||
|
public class ClientBoundEventSerializer implements Serializer<ClientBoundEvent> {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public byte[] serialize(String topic, ClientBoundEvent data) {
|
||||||
|
return ReactiveApiPublisher.serializeEvent(data);
|
||||||
|
}
|
||||||
|
}
|
@ -10,12 +10,15 @@ import java.util.List;
|
|||||||
public class ClusterSettings {
|
public class ClusterSettings {
|
||||||
|
|
||||||
public String id;
|
public String id;
|
||||||
|
public List<String> kafkaBootstrapServers;
|
||||||
public List<NodeSettings> nodes;
|
public List<NodeSettings> nodes;
|
||||||
|
|
||||||
@JsonCreator
|
@JsonCreator
|
||||||
public ClusterSettings(@JsonProperty(required = true, value = "id") String id,
|
public ClusterSettings(@JsonProperty(required = true, value = "id") String id,
|
||||||
|
@JsonProperty(required = true, value = "kafkaBootstrapServers") List<String> kafkaBootstrapServers,
|
||||||
@JsonProperty(required = true, value = "nodes") List<NodeSettings> nodes) {
|
@JsonProperty(required = true, value = "nodes") List<NodeSettings> nodes) {
|
||||||
this.id = id;
|
this.id = id;
|
||||||
|
this.kafkaBootstrapServers = kafkaBootstrapServers;
|
||||||
this.nodes = nodes;
|
this.nodes = nodes;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -33,27 +33,12 @@ public class DynamicAtomixReactiveApiClient implements ReactiveApiClient, AutoCl
|
|||||||
private final Flux<Long> liveIdChange;
|
private final Flux<Long> liveIdChange;
|
||||||
private final Mono<Long> liveIdResolution;
|
private final Mono<Long> liveIdResolution;
|
||||||
|
|
||||||
DynamicAtomixReactiveApiClient(AtomixReactiveApi api, long userId) {
|
DynamicAtomixReactiveApiClient(AtomixReactiveApi api, KafkaConsumer kafkaConsumer, long userId) {
|
||||||
this.api = api;
|
this.api = api;
|
||||||
this.eventService = api.getAtomix().getEventService();
|
this.eventService = api.getAtomix().getEventService();
|
||||||
this.userId = userId;
|
this.userId = userId;
|
||||||
|
|
||||||
clientBoundEvents = Flux
|
clientBoundEvents = kafkaConsumer.consumeMessages(kafkaConsumer.newRandomGroupId(), true, userId)
|
||||||
.<List<ClientBoundEvent>>push(sink -> {
|
|
||||||
var subscriptionFuture = eventService.subscribe("session-client-bound-events",
|
|
||||||
LiveAtomixReactiveApiClient::deserializeEvents,
|
|
||||||
s -> {
|
|
||||||
sink.next(s);
|
|
||||||
return CompletableFuture.completedFuture(null);
|
|
||||||
},
|
|
||||||
(a) -> null
|
|
||||||
);
|
|
||||||
sink.onDispose(() -> subscriptionFuture.thenAccept(Subscription::close));
|
|
||||||
}, OverflowStrategy.ERROR)
|
|
||||||
.subscribeOn(Schedulers.boundedElastic())
|
|
||||||
.onBackpressureBuffer(0xFFFF, BufferOverflowStrategy.ERROR)
|
|
||||||
.flatMapIterable(list -> list)
|
|
||||||
.filter(e -> e.userId() == userId)
|
|
||||||
.doOnNext(e -> liveId.set(e.liveId()))
|
.doOnNext(e -> liveId.set(e.liveId()))
|
||||||
.share();
|
.share();
|
||||||
|
|
||||||
|
@ -27,7 +27,7 @@ public class Entrypoint {
|
|||||||
|
|
||||||
private static final Logger LOG = LoggerFactory.getLogger(Entrypoint.class);
|
private static final Logger LOG = LoggerFactory.getLogger(Entrypoint.class);
|
||||||
|
|
||||||
public static record ValidEntrypointArgs(String clusterPath, String instancePath, String diskSessionsPath) {}
|
public record ValidEntrypointArgs(String clusterPath, String instancePath, String diskSessionsPath) {}
|
||||||
|
|
||||||
public static ValidEntrypointArgs parseArguments(String[] args) {
|
public static ValidEntrypointArgs parseArguments(String[] args) {
|
||||||
// Check arguments validity
|
// Check arguments validity
|
||||||
@ -176,7 +176,9 @@ public class Entrypoint {
|
|||||||
|
|
||||||
atomix.start().join();
|
atomix.start().join();
|
||||||
|
|
||||||
var api = new AtomixReactiveApi(nodeId, atomix, diskSessions, resultingEventTransformerSet);
|
var kafkaParameters = new KafkaParameters(clusterSettings, instanceSettings.id);
|
||||||
|
|
||||||
|
var api = new AtomixReactiveApi(nodeId, atomix, kafkaParameters, diskSessions, resultingEventTransformerSet);
|
||||||
|
|
||||||
LOG.info("Starting ReactiveApi...");
|
LOG.info("Starting ReactiveApi...");
|
||||||
|
|
||||||
|
91
src/main/java/it/tdlight/reactiveapi/KafkaConsumer.java
Normal file
91
src/main/java/it/tdlight/reactiveapi/KafkaConsumer.java
Normal file
@ -0,0 +1,91 @@
|
|||||||
|
package it.tdlight.reactiveapi;
|
||||||
|
|
||||||
|
import it.tdlight.reactiveapi.Event.ClientBoundEvent;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.UUID;
|
||||||
|
import java.util.regex.Pattern;
|
||||||
|
import org.apache.kafka.clients.consumer.ConsumerConfig;
|
||||||
|
import org.apache.kafka.clients.consumer.ConsumerRecord;
|
||||||
|
import org.apache.kafka.common.serialization.IntegerDeserializer;
|
||||||
|
import org.apache.logging.log4j.LogManager;
|
||||||
|
import org.apache.logging.log4j.Logger;
|
||||||
|
import org.jetbrains.annotations.NotNull;
|
||||||
|
import org.jetbrains.annotations.Nullable;
|
||||||
|
import reactor.core.publisher.Flux;
|
||||||
|
import reactor.kafka.receiver.KafkaReceiver;
|
||||||
|
import reactor.kafka.receiver.ReceiverOptions;
|
||||||
|
|
||||||
|
public class KafkaConsumer {
|
||||||
|
|
||||||
|
private static final Logger LOG = LogManager.getLogger(KafkaConsumer.class);
|
||||||
|
|
||||||
|
private final KafkaParameters kafkaParameters;
|
||||||
|
|
||||||
|
public KafkaConsumer(KafkaParameters kafkaParameters) {
|
||||||
|
this.kafkaParameters = kafkaParameters;
|
||||||
|
}
|
||||||
|
|
||||||
|
public KafkaReceiver<Integer, ClientBoundEvent> createReceiver(@NotNull String groupId,
|
||||||
|
@Nullable Long liveId,
|
||||||
|
@Nullable Long userId) {
|
||||||
|
Map<String, Object> props = new HashMap<>();
|
||||||
|
props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaParameters.bootstrapServers());
|
||||||
|
props.put(ConsumerConfig.CLIENT_ID_CONFIG, kafkaParameters.clientId());
|
||||||
|
props.put(ConsumerConfig.GROUP_ID_CONFIG, groupId);
|
||||||
|
props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, IntegerDeserializer.class);
|
||||||
|
props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ClientBoundEventDeserializer.class);
|
||||||
|
props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
|
||||||
|
ReceiverOptions<Integer, ClientBoundEvent> receiverOptions = ReceiverOptions.create(props);
|
||||||
|
ReceiverOptions.create(receiverOptions.consumerProperties());
|
||||||
|
Pattern pattern;
|
||||||
|
if (liveId == null && userId == null) {
|
||||||
|
pattern = Pattern.compile("tdlib\\.event\\.[0-9]+\\.[0-9]+");
|
||||||
|
} else if (liveId == null) {
|
||||||
|
pattern = Pattern.compile("tdlib\\.event\\." + userId + "\\.[0-9]+");
|
||||||
|
} else if (userId == null) {
|
||||||
|
pattern = Pattern.compile("tdlib\\.event\\.[0-9]+\\." + liveId);
|
||||||
|
} else {
|
||||||
|
pattern = Pattern.compile("tdlib\\.event\\." + userId + "\\." + liveId);
|
||||||
|
}
|
||||||
|
ReceiverOptions<Integer, ClientBoundEvent> options = receiverOptions
|
||||||
|
.subscription(pattern)
|
||||||
|
.addAssignListener(partitions -> LOG.debug("onPartitionsAssigned {}", partitions))
|
||||||
|
.addRevokeListener(partitions -> LOG.debug("onPartitionsRevoked {}", partitions));
|
||||||
|
return KafkaReceiver.create(options);
|
||||||
|
}
|
||||||
|
|
||||||
|
public Flux<ClientBoundEvent> consumeMessages(@NotNull String groupId, boolean ack, long userId, long liveId) {
|
||||||
|
if (ack) {
|
||||||
|
return createReceiver(groupId, liveId, userId)
|
||||||
|
.receiveAutoAck()
|
||||||
|
.flatMapSequential(a -> a)
|
||||||
|
.map(ConsumerRecord::value);
|
||||||
|
} else {
|
||||||
|
return createReceiver(groupId, liveId, userId).receive().map(ConsumerRecord::value);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public Flux<ClientBoundEvent> consumeMessages(@NotNull String groupId, boolean ack, long userId) {
|
||||||
|
if (ack) {
|
||||||
|
return createReceiver(groupId, null, userId)
|
||||||
|
.receiveAutoAck()
|
||||||
|
.flatMapSequential(a -> a)
|
||||||
|
.map(ConsumerRecord::value);
|
||||||
|
} else {
|
||||||
|
return createReceiver(groupId, null, userId).receive().map(ConsumerRecord::value);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public Flux<ClientBoundEvent> consumeMessages(@NotNull String groupId, boolean ack) {
|
||||||
|
if (ack) {
|
||||||
|
return createReceiver(groupId, null, null).receiveAutoAck().flatMapSequential(a -> a).map(ConsumerRecord::value);
|
||||||
|
} else {
|
||||||
|
return createReceiver(groupId, null, null).receive().map(ConsumerRecord::value);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public String newRandomGroupId() {
|
||||||
|
return UUID.randomUUID().toString();
|
||||||
|
}
|
||||||
|
}
|
10
src/main/java/it/tdlight/reactiveapi/KafkaParameters.java
Normal file
10
src/main/java/it/tdlight/reactiveapi/KafkaParameters.java
Normal file
@ -0,0 +1,10 @@
|
|||||||
|
package it.tdlight.reactiveapi;
|
||||||
|
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
public record KafkaParameters(String clientId, String bootstrapServers) {
|
||||||
|
|
||||||
|
public KafkaParameters(ClusterSettings clusterSettings, String clientId) {
|
||||||
|
this(clientId, String.join(",", clusterSettings.kafkaBootstrapServers));
|
||||||
|
}
|
||||||
|
}
|
52
src/main/java/it/tdlight/reactiveapi/KafkaProducer.java
Normal file
52
src/main/java/it/tdlight/reactiveapi/KafkaProducer.java
Normal file
@ -0,0 +1,52 @@
|
|||||||
|
package it.tdlight.reactiveapi;
|
||||||
|
|
||||||
|
import it.tdlight.reactiveapi.Event.ClientBoundEvent;
|
||||||
|
import java.time.Duration;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.Map;
|
||||||
|
import org.apache.kafka.clients.producer.ProducerConfig;
|
||||||
|
import org.apache.kafka.clients.producer.ProducerRecord;
|
||||||
|
import org.apache.kafka.common.serialization.IntegerSerializer;
|
||||||
|
import org.apache.logging.log4j.LogManager;
|
||||||
|
import org.apache.logging.log4j.Logger;
|
||||||
|
import reactor.core.publisher.Flux;
|
||||||
|
import reactor.core.publisher.Mono;
|
||||||
|
import reactor.kafka.sender.KafkaSender;
|
||||||
|
import reactor.kafka.sender.SenderOptions;
|
||||||
|
import reactor.kafka.sender.SenderRecord;
|
||||||
|
|
||||||
|
public class KafkaProducer {
|
||||||
|
|
||||||
|
private static final Logger LOG = LogManager.getLogger(KafkaProducer.class);
|
||||||
|
|
||||||
|
private final KafkaSender<Integer, ClientBoundEvent> sender;
|
||||||
|
|
||||||
|
|
||||||
|
public KafkaProducer(KafkaParameters kafkaParameters) {
|
||||||
|
Map<String, Object> props = new HashMap<>();
|
||||||
|
props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaParameters.bootstrapServers());
|
||||||
|
props.put(ProducerConfig.CLIENT_ID_CONFIG, kafkaParameters.clientId());
|
||||||
|
props.put(ProducerConfig.ACKS_CONFIG, "all");
|
||||||
|
props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, IntegerSerializer.class);
|
||||||
|
props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ClientBoundEventSerializer.class);
|
||||||
|
SenderOptions<Integer, ClientBoundEvent> senderOptions = SenderOptions.create(props);
|
||||||
|
|
||||||
|
sender = KafkaSender.create(senderOptions);
|
||||||
|
}
|
||||||
|
|
||||||
|
public Mono<Void> sendMessages(long liveId, long userId, Flux<ClientBoundEvent> eventsFlux) {
|
||||||
|
return eventsFlux
|
||||||
|
.<SenderRecord<Integer, ClientBoundEvent, Integer>>map(event -> SenderRecord.create(new ProducerRecord<>(
|
||||||
|
"tdlib.event.%d.%d".formatted(userId, liveId),
|
||||||
|
event
|
||||||
|
), null))
|
||||||
|
.windowTimeout(1024, Duration.ofMillis(10))
|
||||||
|
.flatMap(sender::send)
|
||||||
|
.doOnError(e -> LOG.error("Send failed", e))
|
||||||
|
.then();
|
||||||
|
}
|
||||||
|
|
||||||
|
public void close() {
|
||||||
|
sender.close();
|
||||||
|
}
|
||||||
|
}
|
@ -1,7 +1,6 @@
|
|||||||
package it.tdlight.reactiveapi;
|
package it.tdlight.reactiveapi;
|
||||||
|
|
||||||
import io.atomix.cluster.messaging.ClusterEventService;
|
import io.atomix.cluster.messaging.ClusterEventService;
|
||||||
import io.atomix.cluster.messaging.Subscription;
|
|
||||||
import io.atomix.core.Atomix;
|
import io.atomix.core.Atomix;
|
||||||
import it.tdlight.jni.TdApi;
|
import it.tdlight.jni.TdApi;
|
||||||
import it.tdlight.reactiveapi.Event.ClientBoundEvent;
|
import it.tdlight.reactiveapi.Event.ClientBoundEvent;
|
||||||
@ -21,11 +20,8 @@ import java.time.Duration;
|
|||||||
import java.time.Instant;
|
import java.time.Instant;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.concurrent.CompletableFuture;
|
|
||||||
import org.apache.commons.lang3.SerializationException;
|
import org.apache.commons.lang3.SerializationException;
|
||||||
import reactor.core.publisher.BufferOverflowStrategy;
|
|
||||||
import reactor.core.publisher.Flux;
|
import reactor.core.publisher.Flux;
|
||||||
import reactor.core.publisher.FluxSink.OverflowStrategy;
|
|
||||||
import reactor.core.publisher.Mono;
|
import reactor.core.publisher.Mono;
|
||||||
import reactor.core.scheduler.Schedulers;
|
import reactor.core.scheduler.Schedulers;
|
||||||
|
|
||||||
@ -37,26 +33,11 @@ public class LiveAtomixReactiveApiClient implements ReactiveApiClient {
|
|||||||
|
|
||||||
private final Flux<ClientBoundEvent> clientBoundEvents;
|
private final Flux<ClientBoundEvent> clientBoundEvents;
|
||||||
|
|
||||||
LiveAtomixReactiveApiClient(Atomix atomix, long liveId, long userId) {
|
LiveAtomixReactiveApiClient(Atomix atomix, KafkaConsumer kafkaConsumer, long liveId, long userId) {
|
||||||
this.eventService = atomix.getEventService();
|
this.eventService = atomix.getEventService();
|
||||||
this.liveId = liveId;
|
this.liveId = liveId;
|
||||||
this.userId = userId;
|
this.userId = userId;
|
||||||
this.clientBoundEvents = Flux
|
this.clientBoundEvents = kafkaConsumer.consumeMessages(kafkaConsumer.newRandomGroupId(), true, userId, liveId).share();
|
||||||
.<List<ClientBoundEvent>>push(sink -> {
|
|
||||||
var subscriptionFuture = eventService.subscribe("session-client-bound-events",
|
|
||||||
LiveAtomixReactiveApiClient::deserializeEvents,
|
|
||||||
s -> {
|
|
||||||
sink.next(s);
|
|
||||||
return CompletableFuture.completedFuture(null);
|
|
||||||
},
|
|
||||||
(a) -> null
|
|
||||||
);
|
|
||||||
sink.onDispose(() -> subscriptionFuture.thenAccept(Subscription::close));
|
|
||||||
}, OverflowStrategy.ERROR)
|
|
||||||
.onBackpressureBuffer(0xFFFF, BufferOverflowStrategy.ERROR)
|
|
||||||
.flatMapIterable(list -> list)
|
|
||||||
.filter(e -> e.userId() == userId && e.liveId() == liveId)
|
|
||||||
.share();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -53,7 +53,7 @@ public class PeriodicRestarter {
|
|||||||
public Mono<Void> start() {
|
public Mono<Void> start() {
|
||||||
return Mono.fromRunnable(() -> {
|
return Mono.fromRunnable(() -> {
|
||||||
LOG.info("Starting periodic restarter...");
|
LOG.info("Starting periodic restarter...");
|
||||||
multiClient.clientBoundEvents().doOnNext(event -> {
|
multiClient.clientBoundEvents(true).doOnNext(event -> {
|
||||||
if (event instanceof OnUpdateData onUpdate) {
|
if (event instanceof OnUpdateData onUpdate) {
|
||||||
if (onUpdate.update() instanceof UpdateAuthorizationState updateAuthorizationState) {
|
if (onUpdate.update() instanceof UpdateAuthorizationState updateAuthorizationState) {
|
||||||
if (updateAuthorizationState.authorizationState instanceof AuthorizationStateReady) {
|
if (updateAuthorizationState.authorizationState instanceof AuthorizationStateReady) {
|
||||||
|
@ -8,7 +8,7 @@ import reactor.core.publisher.Mono;
|
|||||||
|
|
||||||
public interface ReactiveApiMultiClient {
|
public interface ReactiveApiMultiClient {
|
||||||
|
|
||||||
Flux<ClientBoundEvent> clientBoundEvents();
|
Flux<ClientBoundEvent> clientBoundEvents(boolean ack);
|
||||||
|
|
||||||
<T extends TdApi.Object> Mono<T> request(long userId, long liveId, TdApi.Function<T> request, Instant timeout);
|
<T extends TdApi.Object> Mono<T> request(long userId, long liveId, TdApi.Function<T> request, Instant timeout);
|
||||||
|
|
||||||
|
@ -55,7 +55,6 @@ import org.jetbrains.annotations.NotNull;
|
|||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
import reactor.core.Disposable;
|
import reactor.core.Disposable;
|
||||||
import reactor.core.publisher.BufferOverflowStrategy;
|
|
||||||
import reactor.core.publisher.Flux;
|
import reactor.core.publisher.Flux;
|
||||||
import reactor.core.publisher.Mono;
|
import reactor.core.publisher.Mono;
|
||||||
import reactor.core.scheduler.Schedulers;
|
import reactor.core.scheduler.Schedulers;
|
||||||
@ -66,6 +65,7 @@ public abstract class ReactiveApiPublisher {
|
|||||||
private static final Logger LOG = LoggerFactory.getLogger(ReactiveApiPublisher.class);
|
private static final Logger LOG = LoggerFactory.getLogger(ReactiveApiPublisher.class);
|
||||||
private static final Duration SPECIAL_RAW_TIMEOUT_DURATION = Duration.ofSeconds(10);
|
private static final Duration SPECIAL_RAW_TIMEOUT_DURATION = Duration.ofSeconds(10);
|
||||||
|
|
||||||
|
private final KafkaProducer kafkaProducer;
|
||||||
private final ClusterEventService eventService;
|
private final ClusterEventService eventService;
|
||||||
private final Set<ResultingEventTransformer> resultingEventTransformerSet;
|
private final Set<ResultingEventTransformer> resultingEventTransformerSet;
|
||||||
private final ReactiveTelegramClient rawTelegramClient;
|
private final ReactiveTelegramClient rawTelegramClient;
|
||||||
@ -80,9 +80,11 @@ public abstract class ReactiveApiPublisher {
|
|||||||
private final AtomicReference<Path> path = new AtomicReference<>();
|
private final AtomicReference<Path> path = new AtomicReference<>();
|
||||||
|
|
||||||
private ReactiveApiPublisher(Atomix atomix,
|
private ReactiveApiPublisher(Atomix atomix,
|
||||||
|
KafkaProducer kafkaProducer,
|
||||||
Set<ResultingEventTransformer> resultingEventTransformerSet,
|
Set<ResultingEventTransformer> resultingEventTransformerSet,
|
||||||
long liveId,
|
long liveId,
|
||||||
long userId) {
|
long userId) {
|
||||||
|
this.kafkaProducer = kafkaProducer;
|
||||||
this.eventService = atomix.getEventService();
|
this.eventService = atomix.getEventService();
|
||||||
this.resultingEventTransformerSet = resultingEventTransformerSet;
|
this.resultingEventTransformerSet = resultingEventTransformerSet;
|
||||||
this.userId = userId;
|
this.userId = userId;
|
||||||
@ -101,19 +103,27 @@ public abstract class ReactiveApiPublisher {
|
|||||||
}
|
}
|
||||||
|
|
||||||
public static ReactiveApiPublisher fromToken(Atomix atomix,
|
public static ReactiveApiPublisher fromToken(Atomix atomix,
|
||||||
|
KafkaProducer kafkaProducer,
|
||||||
Set<ResultingEventTransformer> resultingEventTransformerSet,
|
Set<ResultingEventTransformer> resultingEventTransformerSet,
|
||||||
Long liveId,
|
Long liveId,
|
||||||
long userId,
|
long userId,
|
||||||
String token) {
|
String token) {
|
||||||
return new ReactiveApiPublisherToken(atomix, resultingEventTransformerSet, liveId, userId, token);
|
return new ReactiveApiPublisherToken(atomix, kafkaProducer, resultingEventTransformerSet, liveId, userId, token);
|
||||||
}
|
}
|
||||||
|
|
||||||
public static ReactiveApiPublisher fromPhoneNumber(Atomix atomix,
|
public static ReactiveApiPublisher fromPhoneNumber(Atomix atomix,
|
||||||
|
KafkaProducer kafkaProducer,
|
||||||
Set<ResultingEventTransformer> resultingEventTransformerSet,
|
Set<ResultingEventTransformer> resultingEventTransformerSet,
|
||||||
Long liveId,
|
Long liveId,
|
||||||
long userId,
|
long userId,
|
||||||
long phoneNumber) {
|
long phoneNumber) {
|
||||||
return new ReactiveApiPublisherPhoneNumber(atomix, resultingEventTransformerSet, liveId, userId, phoneNumber);
|
return new ReactiveApiPublisherPhoneNumber(atomix,
|
||||||
|
kafkaProducer,
|
||||||
|
resultingEventTransformerSet,
|
||||||
|
liveId,
|
||||||
|
userId,
|
||||||
|
phoneNumber
|
||||||
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
public void start(Path path, @Nullable Runnable onClose) {
|
public void start(Path path, @Nullable Runnable onClose) {
|
||||||
@ -170,23 +180,16 @@ public abstract class ReactiveApiPublisher {
|
|||||||
.subscribeOn(Schedulers.parallel())
|
.subscribeOn(Schedulers.parallel())
|
||||||
.subscribe();
|
.subscribe();
|
||||||
|
|
||||||
publishedResultingEvents
|
var messagesToSend = publishedResultingEvents
|
||||||
// Obtain only client-bound events
|
// Obtain only client-bound events
|
||||||
.filter(s -> s instanceof ClientBoundResultingEvent)
|
.filter(s -> s instanceof ClientBoundResultingEvent)
|
||||||
.cast(ClientBoundResultingEvent.class)
|
.cast(ClientBoundResultingEvent.class)
|
||||||
.map(ClientBoundResultingEvent::event)
|
.map(ClientBoundResultingEvent::event)
|
||||||
|
|
||||||
// Buffer requests to avoid halting the event loop
|
// Buffer requests to avoid halting the event loop
|
||||||
.onBackpressureBuffer()
|
.onBackpressureBuffer();
|
||||||
|
|
||||||
.limitRate(1)
|
kafkaProducer.sendMessages(liveId, userId, messagesToSend).subscribeOn(Schedulers.parallel()).subscribe();
|
||||||
.bufferTimeout(512, Duration.ofMillis(100))
|
|
||||||
|
|
||||||
// Send events to the client
|
|
||||||
.subscribeOn(Schedulers.parallel())
|
|
||||||
.publishOn(Schedulers.boundedElastic())
|
|
||||||
.subscribe(clientBoundEvent -> eventService.broadcast("session-client-bound-events",
|
|
||||||
clientBoundEvent, ReactiveApiPublisher::serializeEvents));
|
|
||||||
|
|
||||||
publishedResultingEvents
|
publishedResultingEvents
|
||||||
// Obtain only cluster-bound events
|
// Obtain only cluster-bound events
|
||||||
@ -361,7 +364,7 @@ public abstract class ReactiveApiPublisher {
|
|||||||
return List.of();
|
return List.of();
|
||||||
}
|
}
|
||||||
|
|
||||||
private static byte[] serializeEvents(List<ClientBoundEvent> clientBoundEvents) {
|
public static byte[] serializeEvents(List<ClientBoundEvent> clientBoundEvents) {
|
||||||
try (var byteArrayOutputStream = new ByteArrayOutputStream()) {
|
try (var byteArrayOutputStream = new ByteArrayOutputStream()) {
|
||||||
try (var dataOutputStream = new DataOutputStream(byteArrayOutputStream)) {
|
try (var dataOutputStream = new DataOutputStream(byteArrayOutputStream)) {
|
||||||
dataOutputStream.writeInt(clientBoundEvents.size());
|
dataOutputStream.writeInt(clientBoundEvents.size());
|
||||||
@ -375,7 +378,7 @@ public abstract class ReactiveApiPublisher {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private static byte[] serializeEvent(ClientBoundEvent clientBoundEvent) {
|
public static byte[] serializeEvent(ClientBoundEvent clientBoundEvent) {
|
||||||
try (var byteArrayOutputStream = new ByteArrayOutputStream()) {
|
try (var byteArrayOutputStream = new ByteArrayOutputStream()) {
|
||||||
try (var dataOutputStream = new DataOutputStream(byteArrayOutputStream)) {
|
try (var dataOutputStream = new DataOutputStream(byteArrayOutputStream)) {
|
||||||
writeClientBoundEvent(clientBoundEvent, dataOutputStream);
|
writeClientBoundEvent(clientBoundEvent, dataOutputStream);
|
||||||
@ -496,11 +499,12 @@ public abstract class ReactiveApiPublisher {
|
|||||||
private final String botToken;
|
private final String botToken;
|
||||||
|
|
||||||
public ReactiveApiPublisherToken(Atomix atomix,
|
public ReactiveApiPublisherToken(Atomix atomix,
|
||||||
|
KafkaProducer kafkaProducer,
|
||||||
Set<ResultingEventTransformer> resultingEventTransformerSet,
|
Set<ResultingEventTransformer> resultingEventTransformerSet,
|
||||||
Long liveId,
|
Long liveId,
|
||||||
long userId,
|
long userId,
|
||||||
String botToken) {
|
String botToken) {
|
||||||
super(atomix, resultingEventTransformerSet, liveId, userId);
|
super(atomix, kafkaProducer, resultingEventTransformerSet, liveId, userId);
|
||||||
this.botToken = botToken;
|
this.botToken = botToken;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -529,11 +533,12 @@ public abstract class ReactiveApiPublisher {
|
|||||||
private final long phoneNumber;
|
private final long phoneNumber;
|
||||||
|
|
||||||
public ReactiveApiPublisherPhoneNumber(Atomix atomix,
|
public ReactiveApiPublisherPhoneNumber(Atomix atomix,
|
||||||
|
KafkaProducer kafkaProducer,
|
||||||
Set<ResultingEventTransformer> resultingEventTransformerSet,
|
Set<ResultingEventTransformer> resultingEventTransformerSet,
|
||||||
Long liveId,
|
Long liveId,
|
||||||
long userId,
|
long userId,
|
||||||
long phoneNumber) {
|
long phoneNumber) {
|
||||||
super(atomix, resultingEventTransformerSet, liveId, userId);
|
super(atomix, kafkaProducer, resultingEventTransformerSet, liveId, userId);
|
||||||
this.phoneNumber = phoneNumber;
|
this.phoneNumber = phoneNumber;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -11,12 +11,14 @@
|
|||||||
</Async>
|
</Async>
|
||||||
</Appenders>
|
</Appenders>
|
||||||
<Loggers>
|
<Loggers>
|
||||||
|
<AsyncLogger name="reactor.kafka" level="WARN" additivity="false"/>
|
||||||
|
<AsyncLogger name="org.apache.kafka" level="WARN" additivity="false"/>
|
||||||
<!-- log only INFO, WARN, ERROR and FATAL logging by classes in this package -->
|
<!-- log only INFO, WARN, ERROR and FATAL logging by classes in this package -->
|
||||||
<Logger name="io.atomix" level="INFO" />
|
<Logger name="io.atomix" level="INFO" additivity="false"/>
|
||||||
<!-- log only INFO, WARN, ERROR and FATAL logging by classes in this package -->
|
<!-- log only INFO, WARN, ERROR and FATAL logging by classes in this package -->
|
||||||
<Logger name="io.netty" level="INFO" />
|
<Logger name="io.netty" level="INFO" additivity="false"/>
|
||||||
<!-- log only INFO, WARN, ERROR and FATAL logging by classes in this package -->
|
<!-- log only INFO, WARN, ERROR and FATAL logging by classes in this package -->
|
||||||
<Logger name="io.net5" level="INFO" />
|
<Logger name="io.net5" level="INFO" additivity="false"/>
|
||||||
|
|
||||||
<Root level="INFO">
|
<Root level="INFO">
|
||||||
<filters>
|
<filters>
|
||||||
|
Loading…
x
Reference in New Issue
Block a user