From cbf655c3b556e8593404f292989578d47cb58afe Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C5=81ukasz=20Drumi=C5=84ski?= Date: Fri, 3 Nov 2023 13:29:44 +0100 Subject: [PATCH 01/87] Propagating http headers in Kafka headers (#1612) Propagating http headers in Kafka headers --- .../benchmark/MessageRepositoryBenchmark.java | 6 +- .../environment/HermesServerFactory.java | 6 +- ...rsPropagationAsKafkaHeadersProperties.java | 7 ++ .../kafka/KafkaHeaderNameParameters.java | 2 - ...rsPropagationAsKafkaHeadersProperties.java | 29 ++++++++ .../config/ConsumerReceiverConfiguration.java | 10 ++- .../config/KafkaHeaderNameProperties.java | 10 --- ...KafkaConsumerRecordToMessageConverter.java | 9 ++- .../receiver/kafka/KafkaHeaderExtractor.java | 24 +++++- .../hermes/frontend/buffer/BackupMessage.java | 9 ++- .../frontend/buffer/BackupMessagesLoader.java | 4 +- .../chronicle/ChronicleMapEntryValue.java | 16 +++- .../ChronicleMapMessageRepository.java | 4 +- .../config/FrontendProducerConfiguration.java | 8 +- .../FrontendPublishingConfiguration.java | 5 +- .../config/HTTPHeadersProperties.java | 74 +++++++++++++++++++ .../config/HeaderPropagationProperties.java | 27 ------- .../config/KafkaHeaderNameProperties.java | 11 --- .../producer/kafka/KafkaHeaderFactory.java | 27 +++++-- ...MessageToKafkaProducerRecordConverter.java | 44 ++++------- .../frontend/publishing/avro/AvroMessage.java | 13 +++- .../publishing/message/JsonMessage.java | 13 +++- .../frontend/publishing/message/Message.java | 3 + .../publishing/message/MessageFactory.java | 20 ++--- .../metadata/DefaultHeadersPropagator.java | 64 +++++++++++----- .../metadata/HeadersPropagator.java | 4 +- ...rokerMessageProducerIntegrationTest.groovy | 9 ++- .../message/MessageToJsonConverterTest.groovy | 14 +++- .../DefaultHeadersPropagatorTest.groovy | 57 +++++++++++--- .../preview/MessagePreviewFactoryTest.groovy | 6 +- .../preview/MessagePreviewLogTest.groovy | 19 +++-- .../buffer/BackupMessagesLoaderTest.java | 5 +- .../ChronicleMapMessageRepositoryTest.java | 54 +++++--------- .../kafka/KafkaBrokerMessageProducerTest.java | 10 ++- .../HeadersPropagatorConfiguration.java | 19 ----- .../hermes/integration/JmsConsumingTest.java | 18 +++-- .../integration/MessageBufferLoadingTest.java | 7 +- .../hermes/integration/PublishingTest.java | 10 +-- .../env/HermesIntegrationEnvironment.java | 5 ++ .../helper/ClientBuilderHelper.java | 10 +-- .../integration/metadata/TraceContext.java | 10 +-- .../metadata/TraceHeadersPropagator.java | 22 ------ .../src/test/resources/application.yaml | 7 +- 43 files changed, 448 insertions(+), 283 deletions(-) create mode 100644 hermes-common/src/main/java/pl/allegro/tech/hermes/common/kafka/HTTPHeadersPropagationAsKafkaHeadersProperties.java create mode 100644 hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/ConsumerHTTPHeadersPropagationAsKafkaHeadersProperties.java create mode 100644 hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/HTTPHeadersProperties.java delete mode 100644 hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/HeaderPropagationProperties.java delete mode 100644 integration/src/integration/java/pl/allegro/tech/hermes/frontend/HeadersPropagatorConfiguration.java delete mode 100644 integration/src/integration/java/pl/allegro/tech/hermes/integration/metadata/TraceHeadersPropagator.java diff --git a/hermes-benchmark/src/jmh/java/pl/allegro/tech/hermes/benchmark/MessageRepositoryBenchmark.java b/hermes-benchmark/src/jmh/java/pl/allegro/tech/hermes/benchmark/MessageRepositoryBenchmark.java index eec1a2b04b..a80cd7e429 100644 --- a/hermes-benchmark/src/jmh/java/pl/allegro/tech/hermes/benchmark/MessageRepositoryBenchmark.java +++ b/hermes-benchmark/src/jmh/java/pl/allegro/tech/hermes/benchmark/MessageRepositoryBenchmark.java @@ -30,6 +30,7 @@ import java.util.UUID; import java.util.concurrent.TimeUnit; +import static java.util.Collections.emptyMap; import static pl.allegro.tech.hermes.test.helper.builder.TopicBuilder.topic; @Fork(1) @@ -63,7 +64,7 @@ public void setup() throws IOException { private Message generateMessage() { byte[] messageContent = UUID.randomUUID().toString().getBytes(); String id = MessageIdGenerator.generate(); - return new JsonMessage(id, messageContent, System.currentTimeMillis(), "partition-key"); + return new JsonMessage(id, messageContent, System.currentTimeMillis(), "partition-key", emptyMap()); } private File prepareFile() throws IOException { @@ -109,7 +110,8 @@ public void save(Message message, Topic topic) { topic.getQualifiedName(), message.getPartitionKey(), null, - null); + null, + emptyMap()); map.put(message.getId(), entryValue); } diff --git a/hermes-benchmark/src/jmh/java/pl/allegro/tech/hermes/benchmark/environment/HermesServerFactory.java b/hermes-benchmark/src/jmh/java/pl/allegro/tech/hermes/benchmark/environment/HermesServerFactory.java index 356fe6afb1..c6dc6d21df 100644 --- a/hermes-benchmark/src/jmh/java/pl/allegro/tech/hermes/benchmark/environment/HermesServerFactory.java +++ b/hermes-benchmark/src/jmh/java/pl/allegro/tech/hermes/benchmark/environment/HermesServerFactory.java @@ -9,8 +9,8 @@ import pl.allegro.tech.hermes.common.metric.HermesMetrics; import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.frontend.cache.topic.TopicsCache; +import pl.allegro.tech.hermes.frontend.config.HTTPHeadersProperties; import pl.allegro.tech.hermes.frontend.config.HandlersChainProperties; -import pl.allegro.tech.hermes.frontend.config.HeaderPropagationProperties; import pl.allegro.tech.hermes.frontend.config.HermesServerProperties; import pl.allegro.tech.hermes.frontend.config.SchemaProperties; import pl.allegro.tech.hermes.frontend.config.SslProperties; @@ -81,7 +81,7 @@ static HermesServer provideHermesServer() throws IOException { private static HttpHandler provideHttpHandler(ThroughputLimiter throughputLimiter, TopicsCache topicsCache, BrokerMessageProducer brokerMessageProducer, RawSchemaClient rawSchemaClient, Trackers trackers, AvroMessageContentWrapper avroMessageContentWrapper) { - HeaderPropagationProperties headerPropagationProperties = new HeaderPropagationProperties(); + HTTPHeadersProperties httpHeadersProperties = new HTTPHeadersProperties(); HandlersChainProperties handlersChainProperties = new HandlersChainProperties(); TrackingHeadersExtractor trackingHeadersExtractor = new DefaultTrackingHeaderExtractor(); SchemaProperties schemaProperties = new SchemaProperties(); @@ -97,7 +97,7 @@ private static HttpHandler provideHttpHandler(ThroughputLimiter throughputLimite new DirectSchemaVersionsRepository(rawSchemaClient), new DirectCompiledSchemaRepository<>(rawSchemaClient, SchemaCompilersFactory.avroSchemaCompiler()) ), - new DefaultHeadersPropagator(headerPropagationProperties.isEnabled(), headerPropagationProperties.getAllowFilter()), + new DefaultHeadersPropagator(httpHeadersProperties), new BenchmarkMessageContentWrapper(avroMessageContentWrapper), Clock.systemDefaultZone(), schemaProperties.isIdHeaderEnabled() diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/kafka/HTTPHeadersPropagationAsKafkaHeadersProperties.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/kafka/HTTPHeadersPropagationAsKafkaHeadersProperties.java new file mode 100644 index 0000000000..0c9abaf3b8 --- /dev/null +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/kafka/HTTPHeadersPropagationAsKafkaHeadersProperties.java @@ -0,0 +1,7 @@ +package pl.allegro.tech.hermes.common.kafka; + +public interface HTTPHeadersPropagationAsKafkaHeadersProperties { + boolean isEnabled(); + + String getPrefix(); +} diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/kafka/KafkaHeaderNameParameters.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/kafka/KafkaHeaderNameParameters.java index da9048b89b..63a8274c69 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/kafka/KafkaHeaderNameParameters.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/kafka/KafkaHeaderNameParameters.java @@ -8,6 +8,4 @@ public interface KafkaHeaderNameParameters { String getMessageId(); - String getTimestamp(); - } diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/ConsumerHTTPHeadersPropagationAsKafkaHeadersProperties.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/ConsumerHTTPHeadersPropagationAsKafkaHeadersProperties.java new file mode 100644 index 0000000000..50eb9b40da --- /dev/null +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/ConsumerHTTPHeadersPropagationAsKafkaHeadersProperties.java @@ -0,0 +1,29 @@ +package pl.allegro.tech.hermes.consumers.config; + +import org.springframework.boot.context.properties.ConfigurationProperties; +import pl.allegro.tech.hermes.common.kafka.HTTPHeadersPropagationAsKafkaHeadersProperties; + +@ConfigurationProperties(prefix = "consumer.http.headers.propagation-as-kafka-headers") +public class ConsumerHTTPHeadersPropagationAsKafkaHeadersProperties implements HTTPHeadersPropagationAsKafkaHeadersProperties { + + public boolean enabled = true; + public String prefix = "h-"; + + @Override + public boolean isEnabled() { + return enabled; + } + + @Override + public String getPrefix() { + return prefix; + } + + public void setEnabled(boolean enabled) { + this.enabled = enabled; + } + + public void setPrefix(String prefix) { + this.prefix = prefix; + } +} diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/ConsumerReceiverConfiguration.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/ConsumerReceiverConfiguration.java index 3915bdd64a..0593c487c1 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/ConsumerReceiverConfiguration.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/ConsumerReceiverConfiguration.java @@ -3,6 +3,7 @@ import org.springframework.boot.context.properties.EnableConfigurationProperties; import org.springframework.context.annotation.Bean; import org.springframework.context.annotation.Configuration; +import pl.allegro.tech.hermes.common.kafka.HTTPHeadersPropagationAsKafkaHeadersProperties; import pl.allegro.tech.hermes.common.kafka.KafkaNamesMapper; import pl.allegro.tech.hermes.common.message.wrapper.CompositeMessageContentWrapper; import pl.allegro.tech.hermes.common.metric.MetricsFacade; @@ -27,7 +28,8 @@ KafkaConsumerProperties.class, KafkaClustersProperties.class, CommonConsumerProperties.class, - KafkaHeaderNameProperties.class + KafkaHeaderNameProperties.class, + ConsumerHTTPHeadersPropagationAsKafkaHeadersProperties.class }) public class ConsumerReceiverConfiguration { @@ -78,7 +80,9 @@ public MessageContentReaderFactory messageContentReaderFactory(CompositeMessageC } @Bean - public KafkaHeaderExtractor kafkaHeaderExtractor(KafkaHeaderNameProperties kafkaHeaderNameProperties) { - return new KafkaHeaderExtractor(kafkaHeaderNameProperties); + public KafkaHeaderExtractor kafkaHeaderExtractor( + KafkaHeaderNameProperties kafkaHeaderNameProperties, + HTTPHeadersPropagationAsKafkaHeadersProperties httpHeadersPropagationAsKafkaHeadersProperties) { + return new KafkaHeaderExtractor(kafkaHeaderNameProperties, httpHeadersPropagationAsKafkaHeadersProperties); } } diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/KafkaHeaderNameProperties.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/KafkaHeaderNameProperties.java index 6e1b9c63da..4d90038b9e 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/KafkaHeaderNameProperties.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/KafkaHeaderNameProperties.java @@ -12,8 +12,6 @@ public class KafkaHeaderNameProperties implements KafkaHeaderNameParameters { private String messageId = "id"; - private String timestamp = "ts"; - @Override public String getSchemaVersion() { return schemaVersion; @@ -41,12 +39,4 @@ public void setMessageId(String messageId) { this.messageId = messageId; } - @Override - public String getTimestamp() { - return timestamp; - } - - public void setTimestamp(String timestamp) { - this.timestamp = timestamp; - } } diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/receiver/kafka/KafkaConsumerRecordToMessageConverter.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/receiver/kafka/KafkaConsumerRecordToMessageConverter.java index dee25336ec..537d128de3 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/receiver/kafka/KafkaConsumerRecordToMessageConverter.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/receiver/kafka/KafkaConsumerRecordToMessageConverter.java @@ -37,6 +37,13 @@ public KafkaConsumerRecordToMessageConverter(Topic topic, public Message convertToMessage(ConsumerRecord record, long partitionAssignmentTerm) { KafkaTopic kafkaTopic = topics.get(record.topic()); UnwrappedMessageContent unwrappedContent = messageContentReader.read(record, kafkaTopic.contentType()); + + Map externalMetadata = kafkaHeaderExtractor.extractHTTPHeadersIfEnabled(record.headers()); + // compatibility condition, can be removed when all messages have HTTP headers propagated via Kafka headers. + if (externalMetadata.isEmpty()) { + externalMetadata = unwrappedContent.getMessageMetadata().getExternalMetadata(); + } + return new Message( kafkaHeaderExtractor.extractMessageId(record.headers()), topic.getQualifiedName(), @@ -47,7 +54,7 @@ public Message convertToMessage(ConsumerRecord record, long part clock.millis(), new PartitionOffset(kafkaTopic.name(), record.offset(), record.partition()), partitionAssignmentTerm, - unwrappedContent.getMessageMetadata().getExternalMetadata(), + externalMetadata, subscription.getHeaders(), subscription.getName(), subscription.isSubscriptionIdentityHeadersEnabled() diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/receiver/kafka/KafkaHeaderExtractor.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/receiver/kafka/KafkaHeaderExtractor.java index 9da4fdf43c..f283b52498 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/receiver/kafka/KafkaHeaderExtractor.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/receiver/kafka/KafkaHeaderExtractor.java @@ -3,17 +3,28 @@ import com.google.common.primitives.Ints; import org.apache.kafka.common.header.Header; import org.apache.kafka.common.header.Headers; +import pl.allegro.tech.hermes.common.kafka.HTTPHeadersPropagationAsKafkaHeadersProperties; import pl.allegro.tech.hermes.consumers.config.KafkaHeaderNameProperties; +import java.util.Map; + import static java.nio.charset.StandardCharsets.UTF_8; +import static java.util.Collections.emptyMap; +import static java.util.stream.Collectors.toMap; +import static java.util.stream.StreamSupport.stream; public class KafkaHeaderExtractor { private final KafkaHeaderNameProperties kafkaHeaderNameProperties; + private final boolean isHTTPheadersPropagationAsKafkaHeadersEnabled; + private final String httpHeadersPrefix; - public KafkaHeaderExtractor(KafkaHeaderNameProperties kafkaHeaderNameProperties) { + public KafkaHeaderExtractor(KafkaHeaderNameProperties kafkaHeaderNameProperties, + HTTPHeadersPropagationAsKafkaHeadersProperties httpHeadersPropagationAsKafkaHeadersProperties) { this.kafkaHeaderNameProperties = kafkaHeaderNameProperties; + this.isHTTPheadersPropagationAsKafkaHeadersEnabled = httpHeadersPropagationAsKafkaHeadersProperties.isEnabled(); + this.httpHeadersPrefix = httpHeadersPropagationAsKafkaHeadersProperties.getPrefix(); } public Integer extractSchemaVersion(Headers headers) { @@ -42,4 +53,15 @@ public String extractMessageId(Headers headers) { return new String(header.value(), UTF_8); } + public Map extractHTTPHeadersIfEnabled(Headers headers) { + return isHTTPheadersPropagationAsKafkaHeadersEnabled + ? + stream(headers.spliterator(), false) + .filter(h -> h.key().startsWith(httpHeadersPrefix)) + .collect(toMap( + h -> h.key().substring(httpHeadersPrefix.length()), + h -> new String(h.value(), UTF_8))) + : + emptyMap(); + } } diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/buffer/BackupMessage.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/buffer/BackupMessage.java index 35ba9fa962..b21e6cc12b 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/buffer/BackupMessage.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/buffer/BackupMessage.java @@ -2,6 +2,7 @@ import java.io.Serializable; import java.util.Arrays; +import java.util.Map; import java.util.Objects; public class BackupMessage implements Serializable { @@ -13,9 +14,10 @@ public class BackupMessage implements Serializable { private final String partitionKey; private final Integer schemaVersion; private final Integer schemaId; + private final Map propagatedHTTPHeaders; public BackupMessage(String messageId, byte[] data, long timestamp, String qualifiedTopicName, String partitionKey, - Integer schemaVersion, Integer schemaId) { + Integer schemaVersion, Integer schemaId, Map propagatedHTTPHeaders) { this.messageId = messageId; this.data = data; this.timestamp = timestamp; @@ -23,6 +25,7 @@ public BackupMessage(String messageId, byte[] data, long timestamp, String quali this.partitionKey = partitionKey; this.schemaVersion = schemaVersion; this.schemaId = schemaId; + this.propagatedHTTPHeaders = propagatedHTTPHeaders; } public String getMessageId() { @@ -53,6 +56,10 @@ public Integer getSchemaId() { return schemaId; } + public Map getPropagatedHTTPHeaders() { + return propagatedHTTPHeaders; + } + @Override public boolean equals(Object o) { if (this == o) { diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/buffer/BackupMessagesLoader.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/buffer/BackupMessagesLoader.java index 10ff457ecd..fd5e8bb617 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/buffer/BackupMessagesLoader.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/buffer/BackupMessagesLoader.java @@ -189,12 +189,12 @@ private Message createAvroMessageFromVersion(BackupMessage backupMessage, Option private Message createAvroMessage(BackupMessage backupMessage, CompiledSchema schema) { return new AvroMessage(backupMessage.getMessageId(), backupMessage.getData(), backupMessage.getTimestamp(), schema, - backupMessage.getPartitionKey()); + backupMessage.getPartitionKey(), backupMessage.getPropagatedHTTPHeaders()); } private Message createJsonMessage(BackupMessage backupMessage) { return new JsonMessage(backupMessage.getMessageId(), backupMessage.getData(), backupMessage.getTimestamp(), - backupMessage.getPartitionKey()); + backupMessage.getPartitionKey(), backupMessage.getPropagatedHTTPHeaders()); } private boolean sendMessageIfNeeded(Message message, String topicQualifiedName, Optional cachedTopic, String contextName) { diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/buffer/chronicle/ChronicleMapEntryValue.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/buffer/chronicle/ChronicleMapEntryValue.java index c91c185eaf..80a21001fc 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/buffer/chronicle/ChronicleMapEntryValue.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/buffer/chronicle/ChronicleMapEntryValue.java @@ -2,6 +2,7 @@ import java.io.Serializable; import java.util.Arrays; +import java.util.Map; import java.util.Objects; public class ChronicleMapEntryValue implements Serializable { @@ -13,15 +14,17 @@ public class ChronicleMapEntryValue implements Serializable { private final String partitionKey; private final Integer schemaVersion; private final Integer schemaId; + private final Map propagatedHttpHeaders; public ChronicleMapEntryValue(byte[] data, long timestamp, String qualifiedTopicName, String partitionKey, - Integer schemaVersion, Integer schemaId) { + Integer schemaVersion, Integer schemaId, Map propagatedHttpHeaders) { this.data = data; this.timestamp = timestamp; this.qualifiedTopicName = qualifiedTopicName; this.partitionKey = partitionKey; this.schemaVersion = schemaVersion; this.schemaId = schemaId; + this.propagatedHttpHeaders = propagatedHttpHeaders; } public byte[] getData() { @@ -48,6 +51,10 @@ public Integer getSchemaId() { return schemaId; } + public Map getPropagatedHttpHeaders() { + return propagatedHttpHeaders; + } + @Override public boolean equals(Object o) { if (this == o) { @@ -60,11 +67,14 @@ public boolean equals(Object o) { return Objects.equals(timestamp, that.timestamp) && Arrays.equals(data, that.data) && Objects.equals(qualifiedTopicName, that.qualifiedTopicName) - && Objects.equals(partitionKey, that.partitionKey); + && Objects.equals(partitionKey, that.partitionKey) + && Objects.equals(propagatedHttpHeaders, that.propagatedHttpHeaders); } @Override public int hashCode() { - return Objects.hash(data, timestamp, qualifiedTopicName, partitionKey, schemaVersion, schemaId); + return Objects.hash(data, timestamp, qualifiedTopicName, partitionKey, schemaVersion, schemaId, + propagatedHttpHeaders); } + } diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/buffer/chronicle/ChronicleMapMessageRepository.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/buffer/chronicle/ChronicleMapMessageRepository.java index 086b4ba50f..c4c70dd519 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/buffer/chronicle/ChronicleMapMessageRepository.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/buffer/chronicle/ChronicleMapMessageRepository.java @@ -68,7 +68,7 @@ public void save(Message message, Topic topic) { new ChronicleMapEntryValue( message.getData(), message.getTimestamp(), topic.getQualifiedName(), message.getPartitionKey(), message.getCompiledSchema().map(v -> v.getVersion().value()).orElse(null), - message.getCompiledSchema().map(v -> v.getId().value()).orElse(null))); + message.getCompiledSchema().map(v -> v.getId().value()).orElse(null), message.getHTTPHeaders())); } finally { lock.unlock(); } @@ -92,7 +92,7 @@ public void close() { private BackupMessage toBackupMessage(String id, ChronicleMapEntryValue entryValue) { return new BackupMessage(id, entryValue.getData(), entryValue.getTimestamp(), entryValue.getQualifiedTopicName(), entryValue.getPartitionKey(), entryValue.getSchemaVersion(), - entryValue.getSchemaId()); + entryValue.getSchemaId(), entryValue.getPropagatedHttpHeaders()); } private class LoggingMapSizePreShutdownHook implements Runnable { diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FrontendProducerConfiguration.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FrontendProducerConfiguration.java index e89c46762b..6438546d3d 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FrontendProducerConfiguration.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FrontendProducerConfiguration.java @@ -20,7 +20,8 @@ SchemaProperties.class, KafkaHeaderNameProperties.class, KafkaProducerProperties.class, - KafkaClustersProperties.class + KafkaClustersProperties.class, + HTTPHeadersProperties.class }) public class FrontendProducerConfiguration { @@ -33,8 +34,9 @@ public BrokerMessageProducer kafkaBrokerMessageProducer(Producers producers, } @Bean - public KafkaHeaderFactory kafkaHeaderFactory(KafkaHeaderNameProperties kafkaHeaderNameProperties) { - return new KafkaHeaderFactory(kafkaHeaderNameProperties); + public KafkaHeaderFactory kafkaHeaderFactory(KafkaHeaderNameProperties kafkaHeaderNameProperties, + HTTPHeadersProperties httpHeadersProperties) { + return new KafkaHeaderFactory(kafkaHeaderNameProperties, httpHeadersProperties.getPropagationAsKafkaHeaders()); } @Bean(destroyMethod = "close") diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FrontendPublishingConfiguration.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FrontendPublishingConfiguration.java index ea1b65981b..8931afc318 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FrontendPublishingConfiguration.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FrontendPublishingConfiguration.java @@ -38,7 +38,6 @@ @EnableConfigurationProperties({ ThroughputProperties.class, MessagePreviewProperties.class, - HeaderPropagationProperties.class, HandlersChainProperties.class, SchemaProperties.class }) @@ -95,8 +94,8 @@ public MessageFactory messageFactory(MessageValidators validators, } @Bean - public HeadersPropagator defaultHeadersPropagator(HeaderPropagationProperties headerPropagationProperties) { - return new DefaultHeadersPropagator(headerPropagationProperties.isEnabled(), headerPropagationProperties.getAllowFilter()); + public HeadersPropagator defaultHeadersPropagator(HTTPHeadersProperties httpHeadersProperties) { + return new DefaultHeadersPropagator(httpHeadersProperties); } @Bean diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/HTTPHeadersProperties.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/HTTPHeadersProperties.java new file mode 100644 index 0000000000..af89220b78 --- /dev/null +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/HTTPHeadersProperties.java @@ -0,0 +1,74 @@ +package pl.allegro.tech.hermes.frontend.config; + +import org.springframework.boot.context.properties.ConfigurationProperties; +import pl.allegro.tech.hermes.common.kafka.HTTPHeadersPropagationAsKafkaHeadersProperties; + +import java.util.HashSet; +import java.util.Set; + +@ConfigurationProperties(prefix = "frontend.http.headers") +public class HTTPHeadersProperties { + + public boolean propagationEnabled = false; + public Set allowedSet = new HashSet<>(); + public Set additionalAllowedSetToLog = new HashSet<>(); + public PropagationAsKafkaHeadersProperties propagationAsKafkaHeaders = new PropagationAsKafkaHeadersProperties(); + + public static class PropagationAsKafkaHeadersProperties implements HTTPHeadersPropagationAsKafkaHeadersProperties { + + public boolean enabled = true; + + public String prefix = "h-"; + + @Override + public boolean isEnabled() { + return enabled; + } + + @Override + public String getPrefix() { + return prefix; + } + + public void setEnabled(boolean enabled) { + this.enabled = enabled; + } + + public void setPrefix(String prefix) { + this.prefix = prefix; + } + } + + public boolean isPropagationEnabled() { + return propagationEnabled; + } + + public void setPropagationEnabled(boolean propagationEnabled) { + this.propagationEnabled = propagationEnabled; + } + + public PropagationAsKafkaHeadersProperties getPropagationAsKafkaHeaders() { + return propagationAsKafkaHeaders; + } + + public void setPropagationAsKafkaHeaders( + PropagationAsKafkaHeadersProperties propagationAsKafkaHeaders) { + this.propagationAsKafkaHeaders = propagationAsKafkaHeaders; + } + + public Set getAllowedSet() { + return allowedSet; + } + + public void setAllowedSet(Set allowedSet) { + this.allowedSet = allowedSet; + } + + public Set getAdditionalAllowedSetToLog() { + return additionalAllowedSetToLog; + } + + public void setAdditionalAllowedSetToLog(Set additionalAllowedSetToLog) { + this.additionalAllowedSetToLog = additionalAllowedSetToLog; + } +} diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/HeaderPropagationProperties.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/HeaderPropagationProperties.java deleted file mode 100644 index c047adce2e..0000000000 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/HeaderPropagationProperties.java +++ /dev/null @@ -1,27 +0,0 @@ -package pl.allegro.tech.hermes.frontend.config; - -import org.springframework.boot.context.properties.ConfigurationProperties; - -@ConfigurationProperties(prefix = "frontend.header.propagation") -public class HeaderPropagationProperties { - - private boolean enabled = false; - - private String allowFilter = ""; - - public boolean isEnabled() { - return enabled; - } - - public void setEnabled(boolean enabled) { - this.enabled = enabled; - } - - public String getAllowFilter() { - return allowFilter; - } - - public void setAllowFilter(String allowFilter) { - this.allowFilter = allowFilter; - } -} diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/KafkaHeaderNameProperties.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/KafkaHeaderNameProperties.java index e9983503dc..f75b168d1d 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/KafkaHeaderNameProperties.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/KafkaHeaderNameProperties.java @@ -8,8 +8,6 @@ public class KafkaHeaderNameProperties implements KafkaHeaderNameParameters { private String messageId = "id"; - private String timestamp = "ts"; - private String schemaVersion = "sv"; private String schemaId = "sid"; @@ -23,15 +21,6 @@ public void setMessageId(String messageId) { this.messageId = messageId; } - @Override - public String getTimestamp() { - return timestamp; - } - - public void setTimestamp(String timestamp) { - this.timestamp = timestamp; - } - @Override public String getSchemaVersion() { return schemaVersion; diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaHeaderFactory.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaHeaderFactory.java index 8e0f67e44f..3b21b3549d 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaHeaderFactory.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaHeaderFactory.java @@ -1,27 +1,32 @@ package pl.allegro.tech.hermes.frontend.producer.kafka; import com.google.common.primitives.Ints; -import com.google.common.primitives.Longs; import org.apache.kafka.common.header.Header; import org.apache.kafka.common.header.internals.RecordHeader; +import pl.allegro.tech.hermes.common.kafka.HTTPHeadersPropagationAsKafkaHeadersProperties; import pl.allegro.tech.hermes.common.kafka.KafkaHeaderNameParameters; +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.Map; + public class KafkaHeaderFactory { private final KafkaHeaderNameParameters kafkaHeaderNameParameters; + private final boolean isHTTPHeadersPropagationAsKafkaHeadersEnabled; + private final String httpHeaderPrefix; - public KafkaHeaderFactory(KafkaHeaderNameParameters kafkaHeaderNameParameters) { + public KafkaHeaderFactory(KafkaHeaderNameParameters kafkaHeaderNameParameters, + HTTPHeadersPropagationAsKafkaHeadersProperties httpHeadersPropagationAsKafkaHeadersProperties) { this.kafkaHeaderNameParameters = kafkaHeaderNameParameters; + this.isHTTPHeadersPropagationAsKafkaHeadersEnabled = httpHeadersPropagationAsKafkaHeadersProperties.isEnabled(); + this.httpHeaderPrefix = httpHeadersPropagationAsKafkaHeadersProperties.getPrefix(); } Header messageId(String messageId) { return new RecordHeader(kafkaHeaderNameParameters.getMessageId(), messageId.getBytes()); } - Header timestamp(long timestamp) { - return new RecordHeader(kafkaHeaderNameParameters.getTimestamp(), Longs.toByteArray(timestamp)); - } - Header schemaVersion(int schemaVersion) { return new RecordHeader(kafkaHeaderNameParameters.getSchemaVersion(), Ints.toByteArray(schemaVersion)); } @@ -29,4 +34,14 @@ Header schemaVersion(int schemaVersion) { Header schemaId(int schemaId) { return new RecordHeader(kafkaHeaderNameParameters.getSchemaId(), Ints.toByteArray(schemaId)); } + + void setHTTPHeadersIfEnabled(List
headers, Map httpHeaders) { + if (isHTTPHeadersPropagationAsKafkaHeadersEnabled) { + httpHeaders.forEach((name, value) -> headers.add(createHttpHeader(name, value))); + } + } + + private Header createHttpHeader(String name, String value) { + return new RecordHeader(httpHeaderPrefix + name, value.getBytes(StandardCharsets.UTF_8)); + } } diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/MessageToKafkaProducerRecordConverter.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/MessageToKafkaProducerRecordConverter.java index a2da87693f..51683a03f8 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/MessageToKafkaProducerRecordConverter.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/MessageToKafkaProducerRecordConverter.java @@ -1,17 +1,13 @@ package pl.allegro.tech.hermes.frontend.producer.kafka; +import com.google.common.collect.Lists; import org.apache.avro.Schema; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.header.Header; import pl.allegro.tech.hermes.common.kafka.KafkaTopicName; import pl.allegro.tech.hermes.frontend.publishing.message.Message; -import pl.allegro.tech.hermes.schema.CompiledSchema; -import pl.allegro.tech.hermes.schema.SchemaId; -import pl.allegro.tech.hermes.schema.SchemaVersion; -import java.util.Optional; -import java.util.stream.Collectors; -import java.util.stream.Stream; +import java.util.List; import static java.util.Optional.ofNullable; @@ -27,38 +23,28 @@ public MessageToKafkaProducerRecordConverter(KafkaHeaderFactory kafkaHeaderFacto } public ProducerRecord convertToProducerRecord(Message message, KafkaTopicName kafkaTopicName) { - Optional schemaVersion = message.getCompiledSchema().map(CompiledSchema::getVersion); - Optional schemaId = createSchemaId(message); - Iterable
headers = createRecordHeaders(message.getId(), message.getTimestamp(), schemaId, schemaVersion); + Iterable
headers = createRecordHeaders(message); byte[] partitionKey = ofNullable(message.getPartitionKey()).map(String::getBytes).orElse(null); return new ProducerRecord<>(kafkaTopicName.asString(), null, message.getTimestamp(), partitionKey, message.getData(), headers); } - private Optional createSchemaId(Message message) { - if (schemaIdHeaderEnabled) { - return message.getCompiledSchema().map(CompiledSchema::getId); - } + private Iterable
createRecordHeaders(Message message) { + List
headers = Lists.newArrayList( + kafkaHeaderFactory.messageId(message.getId()) + ); - return Optional.empty(); - } + message.getCompiledSchema().ifPresent(compiledSchemaVersion -> { + headers.add(kafkaHeaderFactory.schemaVersion(compiledSchemaVersion.getVersion().value())); + if (schemaIdHeaderEnabled) { + headers.add(kafkaHeaderFactory.schemaId(compiledSchemaVersion.getId().value())); + } + }); - private Iterable
createRecordHeaders(String id, - long timestamp, - Optional schemaId, - Optional schemaVersion) { - Stream> headers = Stream.of( - Optional.of(kafkaHeaderFactory.messageId(id)), - Optional.of(kafkaHeaderFactory.timestamp(timestamp)), - schemaVersion.map(sv -> kafkaHeaderFactory.schemaVersion(sv.value())), - schemaId.map(sid -> kafkaHeaderFactory.schemaId(sid.value())) - ); + kafkaHeaderFactory.setHTTPHeadersIfEnabled(headers, message.getHTTPHeaders()); - return headers - .filter(Optional::isPresent) - .map(Optional::get) - .collect(Collectors.toList()); + return headers; } } diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/avro/AvroMessage.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/avro/AvroMessage.java index a921259fb3..4a8dbf6f3d 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/avro/AvroMessage.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/avro/AvroMessage.java @@ -5,6 +5,7 @@ import pl.allegro.tech.hermes.frontend.publishing.message.Message; import pl.allegro.tech.hermes.schema.CompiledSchema; +import java.util.Map; import java.util.Optional; public class AvroMessage implements Message { @@ -14,17 +15,20 @@ public class AvroMessage implements Message { private final long timestamp; private final CompiledSchema schema; private final String partitionKey; + private final Map propagatedHTTPHeaders; public AvroMessage(String id, byte[] data, long timestamp, CompiledSchema schema, - String partitionKey) { + String partitionKey, + Map propagatedHTTPHeaders) { this.id = id; this.data = data; this.timestamp = timestamp; this.schema = schema; this.partitionKey = partitionKey; + this.propagatedHTTPHeaders = propagatedHTTPHeaders; } @Override @@ -58,7 +62,12 @@ public Optional> getCompiledSchema() { return Optional.of((CompiledSchema) schema); } + @Override + public Map getHTTPHeaders() { + return propagatedHTTPHeaders; + } + public AvroMessage withDataReplaced(byte[] newData) { - return new AvroMessage(id, newData, timestamp, schema, partitionKey); + return new AvroMessage(id, newData, timestamp, schema, partitionKey, propagatedHTTPHeaders); } } diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/message/JsonMessage.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/message/JsonMessage.java index 4409af800c..414df3f62f 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/message/JsonMessage.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/message/JsonMessage.java @@ -2,18 +2,22 @@ import pl.allegro.tech.hermes.api.ContentType; +import java.util.Map; + public class JsonMessage implements Message { private final String id; private final byte[] data; private final long timestamp; private final String partitionKey; + private final Map propagatedHTTPHeaders; - public JsonMessage(String id, byte[] data, long timestamp, String partitionKey) { + public JsonMessage(String id, byte[] data, long timestamp, String partitionKey, Map propagatedHTTPHeaders) { this.id = id; this.data = data; this.timestamp = timestamp; this.partitionKey = partitionKey; + this.propagatedHTTPHeaders = propagatedHTTPHeaders; } @Override @@ -41,8 +45,13 @@ public String getPartitionKey() { return partitionKey; } + @Override + public Map getHTTPHeaders() { + return propagatedHTTPHeaders; + } + public JsonMessage withDataReplaced(byte[] newData) { - return new JsonMessage(id, newData, timestamp, partitionKey); + return new JsonMessage(id, newData, timestamp, partitionKey, propagatedHTTPHeaders); } } diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/message/Message.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/message/Message.java index 5a13ff72d1..24c80a1497 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/message/Message.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/message/Message.java @@ -3,6 +3,7 @@ import pl.allegro.tech.hermes.api.ContentType; import pl.allegro.tech.hermes.schema.CompiledSchema; +import java.util.Map; import java.util.Optional; public interface Message { @@ -23,4 +24,6 @@ default Optional> getCompiledSchema() { default T getSchema() { return this.getCompiledSchema().get().getSchema(); } + + Map getHTTPHeaders(); } diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/message/MessageFactory.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/message/MessageFactory.java index e0dfd62a68..ff92d5c31c 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/message/MessageFactory.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/message/MessageFactory.java @@ -1,7 +1,6 @@ package pl.allegro.tech.hermes.frontend.publishing.message; import io.undertow.util.HeaderMap; -import io.undertow.util.HeaderValues; import io.undertow.util.Headers; import org.apache.avro.Schema; import org.slf4j.Logger; @@ -27,9 +26,6 @@ import java.util.Optional; import static java.util.Optional.of; -import static java.util.Spliterators.spliteratorUnknownSize; -import static java.util.stream.Collectors.toMap; -import static java.util.stream.StreamSupport.stream; public class MessageFactory { @@ -92,8 +88,9 @@ private Message create(HeaderMap headerMap, Topic topic, String messageId, byte[ } private JsonMessage createJsonMessage(HeaderMap headerMap, String messageId, byte[] messageContent, long timestamp) { - Map extraRequestHeaders = headersPropagator.extract(toHeadersMap(headerMap)); - JsonMessage message = new JsonMessage(messageId, messageContent, timestamp, extractPartitionKey(headerMap)); + Map extraRequestHeaders = headersPropagator.extract(headerMap); + JsonMessage message = new JsonMessage(messageId, messageContent, timestamp, extractPartitionKey(headerMap), + extraRequestHeaders); byte[] wrapped = messageContentWrapper .wrapJson(message.getData(), message.getId(), message.getTimestamp(), extraRequestHeaders); return message.withDataReplaced(wrapped); @@ -113,14 +110,15 @@ private CompiledSchema getCompiledSchema(HeaderMap headerMap, Topic topi private AvroMessage createAvroMessage(HeaderMap headerMap, Topic topic, String messageId, byte[] messageContent, long timestamp) { CompiledSchema schema = getCompiledSchema(headerMap, topic); - Map extraRequestHeaders = headersPropagator.extract(toHeadersMap(headerMap)); + Map extraRequestHeaders = headersPropagator.extract(headerMap); AvroMessage message = new AvroMessage( messageId, enforcer.enforceAvro(headerMap.getFirst(Headers.CONTENT_TYPE_STRING), messageContent, schema.getSchema(), topic), timestamp, schema, - extractPartitionKey(headerMap)); + extractPartitionKey(headerMap), + extraRequestHeaders); validators.check(topic, message); byte[] wrapped = messageContentWrapper.wrapAvro(message.getData(), message.getId(), message.getTimestamp(), @@ -163,10 +161,4 @@ private String extractPartitionKey(HeaderMap headerMap) { return headerMap.getFirst(MessageMetadataHeaders.PARTITION_KEY.getName()); } - private static Map toHeadersMap(HeaderMap headerMap) { - return stream(spliteratorUnknownSize(headerMap.iterator(), 0), false) - .collect(toMap( - h -> h.getHeaderName().toString(), - HeaderValues::getFirst)); - } } diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/metadata/DefaultHeadersPropagator.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/metadata/DefaultHeadersPropagator.java index 880b0b4019..6893e1a46c 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/metadata/DefaultHeadersPropagator.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/metadata/DefaultHeadersPropagator.java @@ -1,44 +1,68 @@ package pl.allegro.tech.hermes.frontend.publishing.metadata; import com.google.common.collect.ImmutableMap; +import io.undertow.util.HeaderMap; +import io.undertow.util.HeaderValues; +import pl.allegro.tech.hermes.frontend.config.HTTPHeadersProperties; +import pl.allegro.tech.hermes.frontend.publishing.handlers.end.TrackingHeadersExtractor; -import java.util.Arrays; +import java.util.HashSet; import java.util.Map; import java.util.Set; -import java.util.stream.Collectors; -import static java.util.Collections.emptySet; -import static java.util.stream.Collectors.toSet; +import static java.util.Spliterators.spliteratorUnknownSize; +import static java.util.stream.Collectors.toMap; +import static java.util.stream.StreamSupport.stream; -public class DefaultHeadersPropagator implements HeadersPropagator { +public class DefaultHeadersPropagator implements HeadersPropagator, TrackingHeadersExtractor { private final boolean propagate; private final Set supportedHeaders; + private final Set trackingHeaders; - public DefaultHeadersPropagator(boolean enabled, String allowFilter) { - if (enabled) { - propagate = true; - supportedHeaders = Arrays.stream(allowFilter.split(",")) - .map(String::trim) - .filter(v -> v.length() > 0) - .collect(toSet()); - } else { - propagate = false; - supportedHeaders = emptySet(); - } + public DefaultHeadersPropagator(HTTPHeadersProperties httpHeadersProperties) { + propagate = httpHeadersProperties.isPropagationEnabled(); + supportedHeaders = httpHeadersProperties.getAllowedSet(); + trackingHeaders = new HashSet<>() { + { + addAll(httpHeadersProperties.getAllowedSet()); + addAll(httpHeadersProperties.getAdditionalAllowedSetToLog()); + } + }; } @Override - public Map extract(Map headers) { + public Map extract(HeaderMap headerMap) { if (propagate) { + Map headers = toHeadersMap(headerMap); if (supportedHeaders.isEmpty()) { return ImmutableMap.copyOf(headers); } - return headers.entrySet().stream() - .filter(e -> this.supportedHeaders.contains(e.getKey())) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + + return extractHeaders(headers, supportedHeaders); } else { return ImmutableMap.of(); } } + + @Override + public Map extractHeadersToLog(HeaderMap headers) { + return extractHeaders(toHeadersMap(headers), trackingHeaders); + } + + private static Map toHeadersMap(HeaderMap headerMap) { + return stream(spliteratorUnknownSize(headerMap.iterator(), 0), false) + .collect(toMap( + h -> h.getHeaderName().toString(), + HeaderValues::getFirst)); + } + + private static Map extractHeaders(Map headers, Set headersToExtract) { + return headers.entrySet().stream() + .filter(headerEntry -> headersToExtract.contains(headerEntry.getKey().toLowerCase())) + .filter(headerEntry -> !headerEntry.getValue().isEmpty()) + .collect(toMap( + Map.Entry::getKey, + Map.Entry::getValue)); + } } diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/metadata/HeadersPropagator.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/metadata/HeadersPropagator.java index cd4406f3a6..c026c69f7f 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/metadata/HeadersPropagator.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/metadata/HeadersPropagator.java @@ -1,9 +1,11 @@ package pl.allegro.tech.hermes.frontend.publishing.metadata; +import io.undertow.util.HeaderMap; + import java.util.Map; public interface HeadersPropagator { - Map extract(Map headers); + Map extract(HeaderMap headers); } diff --git a/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaBrokerMessageProducerIntegrationTest.groovy b/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaBrokerMessageProducerIntegrationTest.groovy index 345beed23a..08246d5de9 100644 --- a/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaBrokerMessageProducerIntegrationTest.groovy +++ b/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaBrokerMessageProducerIntegrationTest.groovy @@ -23,6 +23,7 @@ import pl.allegro.tech.hermes.common.kafka.JsonToAvroMigrationKafkaNamesMapper import pl.allegro.tech.hermes.common.kafka.KafkaNamesMapper import pl.allegro.tech.hermes.common.metric.HermesMetrics import pl.allegro.tech.hermes.common.metric.MetricsFacade +import pl.allegro.tech.hermes.frontend.config.HTTPHeadersProperties import pl.allegro.tech.hermes.frontend.config.SchemaProperties import pl.allegro.tech.hermes.frontend.config.KafkaHeaderNameProperties import pl.allegro.tech.hermes.frontend.config.KafkaProducerProperties @@ -38,6 +39,7 @@ import spock.lang.Specification import java.time.Duration import java.util.stream.Collectors +import static java.util.Collections.emptyMap import static java.util.concurrent.TimeUnit.MILLISECONDS import static org.apache.kafka.clients.consumer.ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG import static org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_ID_CONFIG @@ -103,7 +105,9 @@ class KafkaBrokerMessageProducerIntegrationTest extends Specification { brokerMessageProducer = new KafkaBrokerMessageProducer(producers, new KafkaTopicMetadataFetcher(adminClient, kafkaProducerProperties.getMetadataMaxAge()), new MetricsFacade(new SimpleMeterRegistry(), new HermesMetrics(new MetricRegistry(), new PathsCompiler("localhost"))), - new MessageToKafkaProducerRecordConverter(new KafkaHeaderFactory(kafkaHeaderNameProperties), + new MessageToKafkaProducerRecordConverter(new KafkaHeaderFactory( + kafkaHeaderNameProperties, + new HTTPHeadersProperties.PropagationAsKafkaHeadersProperties()), schemaProperties.isIdHeaderEnabled() ) ) @@ -168,7 +172,8 @@ class KafkaBrokerMessageProducerIntegrationTest extends Specification { private static AvroMessage generateAvroMessage(String partitionKey) { def avroUser = new AvroUser() - return new AvroMessage(UUID.randomUUID().toString(), avroUser.asBytes(), 0L, avroUser.compiledSchema, partitionKey) + return new AvroMessage(UUID.randomUUID().toString(), avroUser.asBytes(), 0L, avroUser.compiledSchema, + partitionKey, emptyMap()) } private static def createTestSubscription(Topic topic, String subscriptionName) { diff --git a/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/publishing/message/MessageToJsonConverterTest.groovy b/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/publishing/message/MessageToJsonConverterTest.groovy index 34bb4dbcab..efd0d0a074 100644 --- a/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/publishing/message/MessageToJsonConverterTest.groovy +++ b/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/publishing/message/MessageToJsonConverterTest.groovy @@ -5,6 +5,8 @@ import pl.allegro.tech.hermes.frontend.publishing.avro.AvroMessage import pl.allegro.tech.hermes.test.helper.avro.AvroUser import spock.lang.Specification +import static java.util.Collections.emptyMap + class MessageToJsonConverterTest extends Specification { def 'should convert avro to JSON'() { @@ -12,10 +14,12 @@ class MessageToJsonConverterTest extends Specification { def avroUser = new AvroUser('name', 16, 'favourite-colour') when: - def converted = new MessageToJsonConverter().convert(new AvroMessage('message-id', avroUser.asBytes(), 0L, avroUser.compiledSchema, "partition-key"), false) + def converted = new MessageToJsonConverter().convert(new AvroMessage( + 'message-id', avroUser.asBytes(), 0L, avroUser.compiledSchema, "partition-key", emptyMap()), false) then: - new String(converted) == JsonOutput.toJson([__metadata: null, name: 'name', age: 16, favoriteColor: 'favourite-colour']) + new String(converted) == JsonOutput.toJson( + [__metadata: null, name: 'name', age: 16, favoriteColor: 'favourite-colour']) } def 'should return bytes when decoding fails'() { @@ -23,7 +27,8 @@ class MessageToJsonConverterTest extends Specification { def avroUser = new AvroUser('name', 16, 'favourite-colour') when: - def converted = new MessageToJsonConverter().convert(new AvroMessage('message-id', 'unable-to-decode'.getBytes(), 0L, avroUser.compiledSchema, null), false) + def converted = new MessageToJsonConverter().convert(new AvroMessage( + 'message-id', 'unable-to-decode'.getBytes(), 0L, avroUser.compiledSchema, null, emptyMap()), false) then: new String(converted) == 'unable-to-decode' @@ -31,7 +36,8 @@ class MessageToJsonConverterTest extends Specification { def 'should return the same when no avro provided'() { when: - def converted = new MessageToJsonConverter().convert(new JsonMessage('message-id', 'given-message'.bytes, 0L, null), false) + def converted = new MessageToJsonConverter().convert(new JsonMessage( + 'message-id', 'given-message'.bytes, 0L, null, emptyMap()), false) then: new String(converted) == 'given-message' diff --git a/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/publishing/metadata/DefaultHeadersPropagatorTest.groovy b/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/publishing/metadata/DefaultHeadersPropagatorTest.groovy index 43e46bf5c5..ce8de9aca6 100644 --- a/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/publishing/metadata/DefaultHeadersPropagatorTest.groovy +++ b/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/publishing/metadata/DefaultHeadersPropagatorTest.groovy @@ -1,15 +1,23 @@ package pl.allegro.tech.hermes.frontend.publishing.metadata +import io.undertow.util.HeaderMap +import io.undertow.util.HttpString +import pl.allegro.tech.hermes.frontend.config.HTTPHeadersProperties import spock.lang.Specification class DefaultHeadersPropagatorTest extends Specification { def 'should propagate all headers when enabled and no filter'() { given: - def propagator = new DefaultHeadersPropagator(true, "") + def properties = new HTTPHeadersProperties() + properties.setPropagationEnabled(true) + def propagator = new DefaultHeadersPropagator(properties) + HeaderMap headerMap = new HeaderMap() + headerMap.add(new HttpString("header1"), "value1") + headerMap.add(new HttpString("header2"), "value2") when: - def extracted = propagator.extract(["header1": "value1", "header2": "value2"]) + def extracted = propagator.extract(headerMap) then: extracted.size() == 2 @@ -17,26 +25,53 @@ class DefaultHeadersPropagatorTest extends Specification { extracted["header2"] == "value2" } - def 'should not propagate headers when disabled'() { + def 'should not propagate headers when propagation is disabled by default'() { given: - def propagator = new DefaultHeadersPropagator(false, "") - + def propagator = new DefaultHeadersPropagator(new HTTPHeadersProperties()) + HeaderMap headerMap = new HeaderMap() + headerMap.add(new HttpString("header1"), "value1") when: - def extracted = propagator.extract(["header1": "value1", "header2": "value2"]) + def extracted = propagator.extract(headerMap) then: extracted.size() == 0 } - def 'should propagate only filtered headers when enabled and filter defined'() { + def 'should propagate only allowed headers when enabled and allowed defined'() { given: - def propagator = new DefaultHeadersPropagator(true, "other-header, header1") - + def properties = new HTTPHeadersProperties() + properties.setPropagationEnabled(true) + properties.setAllowedSet(["header1", "other-header"] as Set) + properties.setAdditionalAllowedSetToLog(["service-name"] as Set) + def propagator = new DefaultHeadersPropagator(properties) + HeaderMap headerMap = new HeaderMap() + headerMap.add(new HttpString("header1"), "value1") + headerMap.add(new HttpString("header2"), "value2") + headerMap.add(new HttpString("service-name"), "value2") when: - def extracted = propagator.extract(["header1": "value1", "header2": "value2"]) + def extracted = propagator.extract(headerMap) then: extracted.size() == 1 extracted["header1"] == "value1" } -} \ No newline at end of file + + def 'should extract "log only" headers'() { + given: + def properties = new HTTPHeadersProperties() + properties.setPropagationEnabled(true) + properties.setAllowedSet(["header1", "other-header"] as Set) + properties.setAdditionalAllowedSetToLog(["service-name"] as Set) + def propagator = new DefaultHeadersPropagator(properties) + HeaderMap headerMap = new HeaderMap() + headerMap.add(new HttpString("header1"), "value1") + headerMap.add(new HttpString("service-name"), "value2") + when: + def extracted = propagator.extractHeadersToLog(headerMap) + + then: + extracted.size() == 2 + extracted["header1"] == "value1" + extracted["service-name"] == "value2" + } +} diff --git a/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/publishing/preview/MessagePreviewFactoryTest.groovy b/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/publishing/preview/MessagePreviewFactoryTest.groovy index d62a1deb63..c60de37224 100644 --- a/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/publishing/preview/MessagePreviewFactoryTest.groovy +++ b/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/publishing/preview/MessagePreviewFactoryTest.groovy @@ -7,6 +7,8 @@ import pl.allegro.tech.hermes.test.helper.avro.AvroUser import spock.lang.Specification import spock.lang.Subject +import static java.util.Collections.emptyMap + class MessagePreviewFactoryTest extends Specification { @Subject @@ -16,7 +18,7 @@ class MessagePreviewFactoryTest extends Specification { given: factory = new MessagePreviewFactory(maxContentSize) when: - MessagePreview preview = factory.create(new JsonMessage('message-id', new byte[messageSize], 0L, "partition-key"), false) + MessagePreview preview = factory.create(new JsonMessage('message-id', new byte[messageSize], 0L, "partition-key", emptyMap()), false) then: preview.truncated == shouldTruncate where: @@ -31,7 +33,7 @@ class MessagePreviewFactoryTest extends Specification { def "should truncate message preview if it is too large after decoding to JSON"() { given: def avroUser = new AvroUser() - def message = new AvroMessage('message-id', avroUser.asBytes(), 0L, avroUser.compiledSchema, null) + def message = new AvroMessage('message-id', avroUser.asBytes(), 0L, avroUser.compiledSchema, null, emptyMap()) factory = new MessagePreviewFactory((avroUser.asJson().length() - 1) / 1024 as int) when: MessagePreview preview = factory.create(message, false) diff --git a/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/publishing/preview/MessagePreviewLogTest.groovy b/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/publishing/preview/MessagePreviewLogTest.groovy index 3a481b5e7c..275d73c50c 100644 --- a/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/publishing/preview/MessagePreviewLogTest.groovy +++ b/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/publishing/preview/MessagePreviewLogTest.groovy @@ -12,6 +12,7 @@ import java.util.concurrent.ExecutorService import java.util.concurrent.Executors import java.util.concurrent.TimeUnit +import static java.util.Collections.emptyMap import static pl.allegro.tech.hermes.api.TopicName.fromQualifiedName class MessagePreviewLogTest extends Specification { @@ -20,8 +21,8 @@ class MessagePreviewLogTest extends Specification { def "should persist JSON messages for topics"() { given: - log.add(TopicBuilder.topic('group.topic-1').build(), new JsonMessage('id', [1] as byte[], 0L, "partition-key")) - log.add(TopicBuilder.topic('group.topic-2').build(), new JsonMessage('id', [2] as byte[], 0L, null)) + log.add(TopicBuilder.topic('group.topic-1').build(), new JsonMessage('id', [1] as byte[], 0L, "partition-key", emptyMap())) + log.add(TopicBuilder.topic('group.topic-2').build(), new JsonMessage('id', [2] as byte[], 0L, null, emptyMap())) when: def messages = log.snapshotAndClean() @@ -33,7 +34,7 @@ class MessagePreviewLogTest extends Specification { def "should persist Avro messages for topics"() { given: def avroUser = new AvroUser() - def message = new AvroMessage('message-id', avroUser.asBytes(), 0L, avroUser.compiledSchema, null) + def message = new AvroMessage('message-id', avroUser.asBytes(), 0L, avroUser.compiledSchema, null, emptyMap()) log.add(TopicBuilder.topic('group.topic-1').build(), message) @@ -51,7 +52,8 @@ class MessagePreviewLogTest extends Specification { def "should persist Avro messages for schema aware topics"() { given: def avroUser = new AvroUser() - def message = new AvroMessage('message-id', SchemaAwareSerDe.serialize(avroUser.compiledSchema.id, avroUser.asBytes()), 0L, avroUser.compiledSchema, null) + def message = new AvroMessage('message-id', SchemaAwareSerDe.serialize(avroUser.compiledSchema.id, + avroUser.asBytes()), 0L, avroUser.compiledSchema, null, emptyMap()) log.add(TopicBuilder.topic('group.topic-1').withSchemaIdAwareSerialization().build(), message) @@ -68,9 +70,9 @@ class MessagePreviewLogTest extends Specification { def "should persist no more than two messages for topic"() { given: - log.add(TopicBuilder.topic('group.topic-1').build(), new JsonMessage('id', [1] as byte[], 0L, null)) - log.add(TopicBuilder.topic('group.topic-1').build(), new JsonMessage('id', [2] as byte[], 0L, null)) - log.add(TopicBuilder.topic('group.topic-1').build(), new JsonMessage('id', [3] as byte[], 0L, null)) + log.add(TopicBuilder.topic('group.topic-1').build(), new JsonMessage('id', [1] as byte[], 0L, null, emptyMap())) + log.add(TopicBuilder.topic('group.topic-1').build(), new JsonMessage('id', [2] as byte[], 0L, null, emptyMap())) + log.add(TopicBuilder.topic('group.topic-1').build(), new JsonMessage('id', [3] as byte[], 0L, null, emptyMap())) when: def messages = log.snapshotAndClean() @@ -88,7 +90,8 @@ class MessagePreviewLogTest extends Specification { threads.times { int executor = it executorService.submit({ - 1000.times { log.add(TopicBuilder.topic("group.topic").build(), new JsonMessage('id', [executor, it] as byte[], 0L, null)) } + 1000.times { log.add(TopicBuilder.topic("group.topic").build(), + new JsonMessage('id', [executor, it] as byte[], 0L, null, emptyMap())) } latch.countDown() }) } diff --git a/hermes-frontend/src/test/java/pl/allegro/tech/hermes/frontend/buffer/BackupMessagesLoaderTest.java b/hermes-frontend/src/test/java/pl/allegro/tech/hermes/frontend/buffer/BackupMessagesLoaderTest.java index a182d2adb8..3617955ec6 100644 --- a/hermes-frontend/src/test/java/pl/allegro/tech/hermes/frontend/buffer/BackupMessagesLoaderTest.java +++ b/hermes-frontend/src/test/java/pl/allegro/tech/hermes/frontend/buffer/BackupMessagesLoaderTest.java @@ -29,6 +29,7 @@ import java.io.File; import java.time.Duration; import java.util.List; +import java.util.Map; import java.util.Optional; import static java.time.LocalDateTime.now; @@ -231,6 +232,7 @@ public void shouldSendMessageWithAllArgumentsFromBackupMessage() { assertThat(sendMessage.getData()).isEqualTo(backupMessages.get(0).getData()); assertThat(sendMessage.getId()).isEqualTo(backupMessages.get(0).getMessageId()); assertThat(sendMessage.getTimestamp()).isEqualTo(backupMessages.get(0).getTimestamp()); + assertThat(sendMessage.getHTTPHeaders().get("propagated-http-header")).isEqualTo("example-value"); } private Message messageOfAge(int ageHours) { @@ -238,7 +240,8 @@ private Message messageOfAge(int ageHours) { MessageIdGenerator.generate(), "{'a':'b'}".getBytes(), now().minusHours(ageHours).toInstant(UTC).toEpochMilli(), - "partition-key" + "partition-key", + Map.of("propagated-http-header", "example-value") ); } } diff --git a/hermes-frontend/src/test/java/pl/allegro/tech/hermes/frontend/buffer/ChronicleMapMessageRepositoryTest.java b/hermes-frontend/src/test/java/pl/allegro/tech/hermes/frontend/buffer/ChronicleMapMessageRepositoryTest.java index 542a1e772c..84c171e95b 100644 --- a/hermes-frontend/src/test/java/pl/allegro/tech/hermes/frontend/buffer/ChronicleMapMessageRepositoryTest.java +++ b/hermes-frontend/src/test/java/pl/allegro/tech/hermes/frontend/buffer/ChronicleMapMessageRepositoryTest.java @@ -14,6 +14,8 @@ import pl.allegro.tech.hermes.test.helper.avro.AvroUserSchemaLoader; import java.io.File; +import java.util.Collections; +import java.util.Map; import java.util.UUID; import static org.assertj.core.api.Assertions.assertThat; @@ -40,8 +42,6 @@ public void shouldSaveFindAndDeleteMessage() { Message message = generateJsonMessage(); String id = message.getId(); - byte[] messageContent = message.getData(); - long timestamp = message.getTimestamp(); Topic topic = topic(qualifiedName).build(); @@ -49,8 +49,7 @@ public void shouldSaveFindAndDeleteMessage() { messageRepository.save(message, topic); //then - assertThat(messageRepository.findAll()).contains( - new BackupMessage(id, messageContent, timestamp, qualifiedName, message.getPartitionKey(), null, null)); + assertThat(messageRepository.findAll()).contains(backupMessage(message, qualifiedName)); //when messageRepository.delete(id); @@ -65,8 +64,6 @@ public void shouldSaveMultipleTimesFindAndDeleteMessage() { String messageContent = "hello world"; Message message1 = generateJsonMessage(messageContent); Message message2 = generateJsonMessage(messageContent); - final String id1 = message1.getId(); - final String id2 = message2.getId(); String qualifiedName = "groupName.topic"; Topic topic = topic(qualifiedName).build(); @@ -78,33 +75,14 @@ public void shouldSaveMultipleTimesFindAndDeleteMessage() { messageRepository.save(message2, topic); //then - assertThat(messageRepository.findAll()).contains( - new BackupMessage( - id1, - messageContent.getBytes(), - message1.getTimestamp(), - qualifiedName, - message1.getPartitionKey(), - null, - null - ) - ); + assertThat(messageRepository.findAll()).contains(backupMessage(message1, qualifiedName)); //when - messageRepository.delete(id1); + messageRepository.delete(message1.getId()); //then assertThat(messageRepository.findAll()).hasSize(1); - assertThat(messageRepository.findAll()).contains( - new BackupMessage( - id2, - messageContent.getBytes(), - message2.getTimestamp(), - qualifiedName, - message2.getPartitionKey(), - null, - null) - ); + assertThat(messageRepository.findAll()).contains(backupMessage(message2, qualifiedName)); } @Test @@ -142,8 +120,7 @@ public void shouldCreateRepositoryThenCloseAndRestore() { messageRepository = new ChronicleMapMessageRepository(file, ENTRIES, AVERAGE_MESSAGE_SIZE); //then - assertThat(messageRepository.findAll()).contains(new BackupMessage( - message.getId(), message.getData(), message.getTimestamp(), qualifiedName, message.getPartitionKey(), null, null)); + assertThat(messageRepository.findAll()).contains(backupMessage(message, qualifiedName)); } @Test @@ -154,10 +131,7 @@ public void shouldSaveFindAndDeleteMessageAvroMessage() { AvroUser avroUser = new AvroUser("Bob", 18, "blue"); String id = MessageIdGenerator.generate(); Message message = new AvroMessage(id, avroUser.asBytes(), System.currentTimeMillis(), - CompiledSchema.of(AvroUserSchemaLoader.load(), 1, 1), "partition-key"); - - byte[] messageContent = message.getData(); - long timestamp = message.getTimestamp(); + CompiledSchema.of(AvroUserSchemaLoader.load(), 1, 1), "partition-key", Collections.emptyMap()); Topic topic = topic(qualifiedName).build(); @@ -165,8 +139,7 @@ public void shouldSaveFindAndDeleteMessageAvroMessage() { messageRepository.save(message, topic); //then - assertThat(messageRepository.findAll()).contains( - new BackupMessage(id, messageContent, timestamp, qualifiedName, message.getPartitionKey(), 1, 1)); + assertThat(messageRepository.findAll()).contains(backupMessage(message, qualifiedName)); //when messageRepository.delete(id); @@ -186,6 +159,13 @@ private Message generateJsonMessage(String content) { private Message generateJsonMessage(String content, long timestamp) { byte[] messageContent = content.getBytes(); String id = MessageIdGenerator.generate(); - return new JsonMessage(id, messageContent, timestamp, "partition-key"); + return new JsonMessage(id, messageContent, timestamp, "partition-key", Map.of("propagated-http-header", "value")); + } + + private BackupMessage backupMessage(Message m, String qualifiedTopicName) { + return new BackupMessage(m.getId(), m.getData(), m.getTimestamp(), qualifiedTopicName, m.getPartitionKey(), + m.getCompiledSchema().map(cs -> cs.getVersion().value()).orElse(null), + m.getCompiledSchema().map(cs -> cs.getId().value()).orElse(null), + m.getHTTPHeaders()); } } diff --git a/hermes-frontend/src/test/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaBrokerMessageProducerTest.java b/hermes-frontend/src/test/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaBrokerMessageProducerTest.java index d0bffcb30c..845a6afc1f 100644 --- a/hermes-frontend/src/test/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaBrokerMessageProducerTest.java +++ b/hermes-frontend/src/test/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaBrokerMessageProducerTest.java @@ -12,10 +12,12 @@ import org.mockito.Mock; import org.mockito.runners.MockitoJUnitRunner; import pl.allegro.tech.hermes.api.Topic; +import pl.allegro.tech.hermes.common.kafka.HTTPHeadersPropagationAsKafkaHeadersProperties; import pl.allegro.tech.hermes.common.kafka.KafkaNamesMapper; import pl.allegro.tech.hermes.common.kafka.NamespaceKafkaNamesMapper; import pl.allegro.tech.hermes.common.metric.HermesMetrics; import pl.allegro.tech.hermes.common.metric.MetricsFacade; +import pl.allegro.tech.hermes.frontend.config.HTTPHeadersProperties; import pl.allegro.tech.hermes.frontend.config.KafkaHeaderNameProperties; import pl.allegro.tech.hermes.frontend.config.KafkaProducerProperties; import pl.allegro.tech.hermes.frontend.config.SchemaProperties; @@ -30,6 +32,7 @@ import static com.google.common.base.Charsets.UTF_8; import static com.jayway.awaitility.Awaitility.await; +import static java.util.Collections.emptyMap; import static org.assertj.core.api.Assertions.assertThat; import static pl.allegro.tech.hermes.test.helper.builder.TopicBuilder.topic; @@ -41,19 +44,22 @@ public class KafkaBrokerMessageProducerTest { private static final String MESSAGE_ID = "id"; private static final Topic TOPIC = topic("group.topic").build(); private static final byte[] CONTENT = "{\"data\":\"json\"}".getBytes(UTF_8); - private static final Message MESSAGE = new JsonMessage(MESSAGE_ID, CONTENT, TIMESTAMP, PARTITION_KEY); + private static final Message MESSAGE = new JsonMessage(MESSAGE_ID, CONTENT, TIMESTAMP, PARTITION_KEY, emptyMap()); private final ByteArraySerializer serializer = new ByteArraySerializer(); private final MockProducer leaderConfirmsProducer = new MockProducer<>(true, serializer, serializer); private final MockProducer everyoneConfirmProducer = new MockProducer<>(true, serializer, serializer); private final KafkaHeaderNameProperties kafkaHeaderNameProperties = new KafkaHeaderNameProperties(); + private final HTTPHeadersPropagationAsKafkaHeadersProperties httpHeadersPropagationAsKafkaHeadersProperties = + new HTTPHeadersProperties.PropagationAsKafkaHeadersProperties(); private final KafkaProducerProperties kafkaProducerProperties = new KafkaProducerProperties(); private final Producers producers = new Producers(leaderConfirmsProducer, everyoneConfirmProducer, kafkaProducerProperties.isReportNodeMetricsEnabled()); private KafkaBrokerMessageProducer producer; private final KafkaNamesMapper kafkaNamesMapper = new NamespaceKafkaNamesMapper("ns", "_"); - private final KafkaHeaderFactory kafkaHeaderFactory = new KafkaHeaderFactory(kafkaHeaderNameProperties); + private final KafkaHeaderFactory kafkaHeaderFactory = new KafkaHeaderFactory(kafkaHeaderNameProperties, + httpHeadersPropagationAsKafkaHeadersProperties); @Mock private HermesMetrics hermesMetrics = new HermesMetrics(new MetricRegistry(), new PathsCompiler("")); diff --git a/integration/src/integration/java/pl/allegro/tech/hermes/frontend/HeadersPropagatorConfiguration.java b/integration/src/integration/java/pl/allegro/tech/hermes/frontend/HeadersPropagatorConfiguration.java deleted file mode 100644 index b59ac343ac..0000000000 --- a/integration/src/integration/java/pl/allegro/tech/hermes/frontend/HeadersPropagatorConfiguration.java +++ /dev/null @@ -1,19 +0,0 @@ -package pl.allegro.tech.hermes.frontend; - -import org.springframework.context.annotation.Bean; -import org.springframework.context.annotation.Configuration; -import org.springframework.context.annotation.Primary; -import org.springframework.context.annotation.Profile; -import pl.allegro.tech.hermes.frontend.publishing.metadata.HeadersPropagator; -import pl.allegro.tech.hermes.integration.metadata.TraceHeadersPropagator; - -@Configuration -public class HeadersPropagatorConfiguration { - - @Bean - @Primary - @Profile("integration") - HeadersPropagator traceHeadersPropagator() { - return new TraceHeadersPropagator(); - } -} diff --git a/integration/src/integration/java/pl/allegro/tech/hermes/integration/JmsConsumingTest.java b/integration/src/integration/java/pl/allegro/tech/hermes/integration/JmsConsumingTest.java index dfc1bd0cfb..bcb92004db 100644 --- a/integration/src/integration/java/pl/allegro/tech/hermes/integration/JmsConsumingTest.java +++ b/integration/src/integration/java/pl/allegro/tech/hermes/integration/JmsConsumingTest.java @@ -61,12 +61,13 @@ public void shouldPublishAndConsumeJmsMessageWithTraceId() { // when Response response = client .request() - .header("Trace-Id", traceId) + .header("trace-id", traceId) .post(Entity.entity(message, MediaType.APPLICATION_JSON)); // then assertThat(response).hasStatus(Response.Status.CREATED); - assertThat(jmsEndpoint.waitAndGetLastMessage()).assertStringProperty("TraceId", traceId); + Message lastMessage = jmsEndpoint.waitAndGetLastMessage(); + assertThat(lastMessage).assertStringProperty("traceid", traceId); } @Test @@ -80,7 +81,7 @@ public void shouldPublishAndConsumeJmsMessageWithTraceHeaders() { Topic topic = operations.buildTopic(randomTopic("publishJmsGroupWithTrace", "topic").build()); operations.createSubscription(topic, "subscription", jmsEndpointAddress(JMS_TOPIC_NAME)); jmsEndpoint.expectMessages(TestMessage.of("hello", "world")); - Invocation.Builder request = createRequestWithTraceHeaders(FRONTEND_URL, topic.getQualifiedName(), trace);; + Invocation.Builder request = createRequestWithTraceHeaders(FRONTEND_URL, topic.getQualifiedName(), trace); // when Response response = request.post(Entity.entity(message, MediaType.APPLICATION_JSON)); @@ -88,11 +89,12 @@ public void shouldPublishAndConsumeJmsMessageWithTraceHeaders() { // then assertThat(response).hasStatus(Response.Status.CREATED); Message lastMessage = jmsEndpoint.waitAndGetLastMessage(); - assertThat(lastMessage).assertStringProperty("TraceId", trace.getTraceId()) - .assertStringProperty("SpanId", trace.getSpanId()) - .assertStringProperty("ParentSpanId", trace.getParentSpanId()) - .assertStringProperty("TraceSampled", trace.getTraceSampled()) - .assertStringProperty("TraceReported", trace.getTraceReported()); + assertThat(lastMessage) + .assertStringProperty("traceid", trace.getTraceId()) + .assertStringProperty("spanid", trace.getSpanId()) + .assertStringProperty("parentspanid", trace.getParentSpanId()) + .assertStringProperty("tracesampled", trace.getTraceSampled()) + .assertStringProperty("tracereported", trace.getTraceReported()); } private String jmsEndpointAddress(String topicName) { diff --git a/integration/src/integration/java/pl/allegro/tech/hermes/integration/MessageBufferLoadingTest.java b/integration/src/integration/java/pl/allegro/tech/hermes/integration/MessageBufferLoadingTest.java index bc36f3d96e..9ffb0764de 100644 --- a/integration/src/integration/java/pl/allegro/tech/hermes/integration/MessageBufferLoadingTest.java +++ b/integration/src/integration/java/pl/allegro/tech/hermes/integration/MessageBufferLoadingTest.java @@ -8,6 +8,7 @@ import pl.allegro.tech.hermes.api.Topic; import pl.allegro.tech.hermes.common.message.wrapper.CompositeMessageContentWrapper; import pl.allegro.tech.hermes.common.message.wrapper.JsonMessageContentWrapper; +import pl.allegro.tech.hermes.frontend.FrontendConfigurationProperties; import pl.allegro.tech.hermes.frontend.buffer.BackupFilesManager; import pl.allegro.tech.hermes.frontend.buffer.MessageRepository; import pl.allegro.tech.hermes.frontend.buffer.chronicle.ChronicleMapMessageRepository; @@ -24,13 +25,13 @@ import java.io.File; import java.time.Clock; -import java.util.Collections; import static com.jayway.awaitility.Awaitility.await; import static jakarta.ws.rs.core.Response.Status.ACCEPTED; import static jakarta.ws.rs.core.Response.Status.CREATED; import static java.nio.charset.Charset.defaultCharset; import static java.time.Instant.now; +import static java.util.Collections.emptyMap; import static java.util.concurrent.TimeUnit.SECONDS; import static org.assertj.core.api.Assertions.assertThat; import static pl.allegro.tech.hermes.frontend.FrontendConfigurationProperties.KAFKA_BROKER_LIST; @@ -132,9 +133,9 @@ private void backupFileWithOneMessage(String tempDirPath, Topic topic) { String messageId = MessageIdGenerator.generate(); long timestamp = now().toEpochMilli(); byte[] content = wrapper.wrapJson("message".getBytes(defaultCharset()), - messageId, timestamp, Collections.emptyMap()); + messageId, timestamp, emptyMap()); - messageRepository.save(new JsonMessage(messageId, content, timestamp, null), topic); + messageRepository.save(new JsonMessage(messageId, content, timestamp, null, emptyMap()), topic); messageRepository.close(); } diff --git a/integration/src/integration/java/pl/allegro/tech/hermes/integration/PublishingTest.java b/integration/src/integration/java/pl/allegro/tech/hermes/integration/PublishingTest.java index 240c04bc04..303bdaeb66 100644 --- a/integration/src/integration/java/pl/allegro/tech/hermes/integration/PublishingTest.java +++ b/integration/src/integration/java/pl/allegro/tech/hermes/integration/PublishingTest.java @@ -277,12 +277,12 @@ public void shouldPublishAndConsumeMessageWithTraceId() { // when Response response = client .request() - .header("Trace-Id", traceId) + .header("trace-id", traceId) .post(Entity.entity(message, MediaType.APPLICATION_JSON)); // then assertThat(response).hasStatus(Response.Status.CREATED); - assertThat(remoteService.waitAndGetLastRequest()).hasHeaderValue("Trace-Id", traceId); + assertThat(remoteService.waitAndGetLastRequest()).hasHeaderValue("trace-id", traceId); } @Test @@ -381,7 +381,7 @@ public void shouldNotOverrideHeadersAddedByMetadataAppendersWithSubscriptionHead Subscription subscription = SubscriptionBuilder.subscription(topic, "subscription") .withEndpoint(remoteService.getUrl()) - .withHeader("Trace-Id", "defaultValue") + .withHeader("trace-id", "defaultValue") .build(); operations.createSubscription(topic, subscription); remoteService.expectMessages(message); @@ -390,12 +390,12 @@ public void shouldNotOverrideHeadersAddedByMetadataAppendersWithSubscriptionHead // when Response response = client .request() - .header("Trace-Id", "valueFromRequest") + .header("trace-id", "valueFromRequest") .post(Entity.entity(message, MediaType.TEXT_PLAIN)); // then assertThat(response).hasStatus(Response.Status.CREATED); - assertThat(remoteService.waitAndGetLastRequest()).hasHeaderValue("Trace-Id", "valueFromRequest"); + assertThat(remoteService.waitAndGetLastRequest()).hasHeaderValue("trace-id", "valueFromRequest"); } @Test diff --git a/integration/src/integration/java/pl/allegro/tech/hermes/integration/env/HermesIntegrationEnvironment.java b/integration/src/integration/java/pl/allegro/tech/hermes/integration/env/HermesIntegrationEnvironment.java index 08b31395a0..33fca84542 100644 --- a/integration/src/integration/java/pl/allegro/tech/hermes/integration/env/HermesIntegrationEnvironment.java +++ b/integration/src/integration/java/pl/allegro/tech/hermes/integration/env/HermesIntegrationEnvironment.java @@ -27,6 +27,7 @@ import java.util.ArrayList; import java.util.Collections; import java.util.LinkedHashMap; +import java.util.Locale; import java.util.Map; import java.util.stream.Stream; @@ -61,6 +62,10 @@ public class HermesIntegrationEnvironment implements EnvironmentAware { "java.security.auth.login.config", HermesIntegrationEnvironment.class.getClassLoader().getResource("kafka_server_jaas.conf").getPath() ); + + // Set English locale as default one for integration tests as on environment with different locale some + // integration tests don't pass. + Locale.setDefault(new Locale("en", "US")); } static { diff --git a/integration/src/integration/java/pl/allegro/tech/hermes/integration/helper/ClientBuilderHelper.java b/integration/src/integration/java/pl/allegro/tech/hermes/integration/helper/ClientBuilderHelper.java index d2b71845cc..dbb670a978 100644 --- a/integration/src/integration/java/pl/allegro/tech/hermes/integration/helper/ClientBuilderHelper.java +++ b/integration/src/integration/java/pl/allegro/tech/hermes/integration/helper/ClientBuilderHelper.java @@ -11,10 +11,10 @@ public static Invocation.Builder createRequestWithTraceHeaders(String uri, Strin WebTarget client = ClientBuilder.newClient().target(uri).path("topics").path(topicName); return client.request() - .header("Trace-Id", traceContext.getTraceId()) - .header("Span-Id", traceContext.getSpanId()) - .header("Parent-Span-Id", traceContext.getParentSpanId()) - .header("Trace-Sampled", traceContext.getTraceSampled()) - .header("Trace-Reported", traceContext.getTraceReported()); + .header("trace-id", traceContext.getTraceId()) + .header("span-id", traceContext.getSpanId()) + .header("parent-span-id", traceContext.getParentSpanId()) + .header("trace-sampled", traceContext.getTraceSampled()) + .header("trace-reported", traceContext.getTraceReported()); } } diff --git a/integration/src/integration/java/pl/allegro/tech/hermes/integration/metadata/TraceContext.java b/integration/src/integration/java/pl/allegro/tech/hermes/integration/metadata/TraceContext.java index 95ee7b3453..c86a4a0c00 100644 --- a/integration/src/integration/java/pl/allegro/tech/hermes/integration/metadata/TraceContext.java +++ b/integration/src/integration/java/pl/allegro/tech/hermes/integration/metadata/TraceContext.java @@ -44,11 +44,11 @@ public String getTraceReported() { public Map asMap() { return ImmutableMap.builder() - .put("Trace-Id", traceId) - .put("Span-Id", spanId) - .put("Parent-Span-Id", parentSpanId) - .put("Trace-Sampled", traceSampled) - .put("Trace-Reported", traceReported) + .put("trace-id", traceId) + .put("span-id", spanId) + .put("parent-span-id", parentSpanId) + .put("trace-sampled", traceSampled) + .put("trace-reported", traceReported) .build(); } diff --git a/integration/src/integration/java/pl/allegro/tech/hermes/integration/metadata/TraceHeadersPropagator.java b/integration/src/integration/java/pl/allegro/tech/hermes/integration/metadata/TraceHeadersPropagator.java deleted file mode 100644 index 53e537db88..0000000000 --- a/integration/src/integration/java/pl/allegro/tech/hermes/integration/metadata/TraceHeadersPropagator.java +++ /dev/null @@ -1,22 +0,0 @@ -package pl.allegro.tech.hermes.integration.metadata; - -import com.google.common.collect.ImmutableSet; -import pl.allegro.tech.hermes.frontend.publishing.metadata.HeadersPropagator; - -import java.util.Map; -import java.util.Set; -import java.util.stream.Collectors; - -public class TraceHeadersPropagator implements HeadersPropagator { - - private static final Set HEADERS = ImmutableSet.of( - "Trace-Id", "Span-Id", "Parent-Span-Id", "Trace-Sampled", "Trace-Reported"); - - @Override - public Map extract(Map headers) { - return headers.entrySet() - .stream() - .filter(entry -> HEADERS.contains(entry.getKey())) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - } -} diff --git a/integration/src/test/resources/application.yaml b/integration/src/test/resources/application.yaml index 1635892834..8e97aea09e 100644 --- a/integration/src/test/resources/application.yaml +++ b/integration/src/test/resources/application.yaml @@ -103,8 +103,13 @@ frontend: metrics: metric-registry: graphiteReporterEnabled: true + http: + headers: + propagationEnabled: true + allowedSet: trace-id, span-id, parent-span-id, trace-sampled, trace-reported + additionalAllowedSetToLog: client-name consumer: metrics: metric-registry: - graphiteReporterEnabled: true \ No newline at end of file + graphiteReporterEnabled: true From 383152226d0eaf379dfee07723a8185123ab8d9f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Piotr=20R=C5=BCysko?= Date: Mon, 6 Nov 2023 11:34:20 +0100 Subject: [PATCH 02/87] Ensure ConsumerNodesRegistry is thread safe (#1776) --- .../consumers/registry/ConsumerNodesRegistry.java | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/registry/ConsumerNodesRegistry.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/registry/ConsumerNodesRegistry.java index 6b61cd5932..af6346de5e 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/registry/ConsumerNodesRegistry.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/registry/ConsumerNodesRegistry.java @@ -1,5 +1,6 @@ package pl.allegro.tech.hermes.consumers.registry; +import org.apache.commons.lang.StringUtils; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.recipes.cache.PathChildrenCache; import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent; @@ -17,6 +18,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; @@ -32,7 +34,7 @@ public class ConsumerNodesRegistry extends PathChildrenCache implements PathChil private final ConsumerNodesRegistryPaths registryPaths; private final String consumerNodeId; private final LeaderLatch leaderLatch; - private final Map consumersLastSeen = new HashMap<>(); + private final Map consumersLastSeen = new ConcurrentHashMap<>(); private final long deathOfConsumerAfterMillis; private final Clock clock; @@ -97,11 +99,18 @@ public List listConsumerNodes() { return new ArrayList<>(consumersLastSeen.keySet()); } - public void refresh() { + public synchronized void refresh() { logger.info("Refreshing current consumers registry"); long currentTime = clock.millis(); - readCurrentNodes().forEach(node -> consumersLastSeen.put(node, currentTime)); + List currentNodes = readCurrentNodes(); + List validNodes = currentNodes.stream() + .filter(StringUtils::isNotBlank) + .toList(); + if (currentNodes.size() != validNodes.size()) { + logger.warn("Found {} invalid consumer nodes.", currentNodes.size() - validNodes.size()); + } + validNodes.forEach(node -> consumersLastSeen.put(node, currentTime)); List deadConsumers = findDeadConsumers(currentTime); if (!deadConsumers.isEmpty()) { From 154f471f815061c2e7c285a18a5036398a341455 Mon Sep 17 00:00:00 2001 From: Mateusz <76775507+szczygiel-m@users.noreply.github.com> Date: Mon, 6 Nov 2023 17:06:50 +0100 Subject: [PATCH 03/87] Fixing bug related to double group in topic creation form (#1773) --- .../src/composables/topic/use-create-topic/useCreateTopic.ts | 2 ++ 1 file changed, 2 insertions(+) diff --git a/hermes-console-vue/src/composables/topic/use-create-topic/useCreateTopic.ts b/hermes-console-vue/src/composables/topic/use-create-topic/useCreateTopic.ts index 5a10e2ff9a..995bf2134d 100644 --- a/hermes-console-vue/src/composables/topic/use-create-topic/useCreateTopic.ts +++ b/hermes-console-vue/src/composables/topic/use-create-topic/useCreateTopic.ts @@ -2,6 +2,7 @@ import { dispatchErrorNotification } from '@/utils/notification-utils'; import { createTopic as doCreateTopic } from '@/api/hermes-client'; import { ref } from 'vue'; import { storeToRefs } from 'pinia'; +import { topicName } from '@/utils/topic-utils/topic-utils'; import { useAppConfigStore } from '@/store/app-config/useAppConfigStore'; import { useFormTopic, @@ -49,6 +50,7 @@ export function useCreateTopic(group: string): UseCreateTopic { notificationsStore, useGlobalI18n().t('notifications.topic.create.failure'), ); + form.value.name = topicName(form.value.name); } finally { creatingTopic.value = false; } From 27551a922721dd5f9e78388b5577f449adc80c9f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Piotr=20R=C5=BCysko?= Date: Tue, 7 Nov 2023 14:39:35 +0100 Subject: [PATCH 04/87] Add possibility of embedding cost report in the new console (#1771) --- hermes-console-vue/json-server/db.json | 8 ++++ .../src/api/app-configuration.ts | 10 +++++ .../components/costs-card/CostsCard.spec.ts | 30 +++++++++++++++ .../src/components/costs-card/CostsCard.vue | 38 +++++++++++++++++++ hermes-console-vue/src/dummy/app-config.ts | 8 ++++ hermes-console-vue/src/i18n/en-US/index.ts | 5 +++ .../src/views/home/HomeView.vue | 14 +++++++ .../subscription/SubscriptionView.spec.ts | 36 +++++++++++++++++- .../views/subscription/SubscriptionView.vue | 26 +++++++++++++ .../src/views/topic/TopicView.spec.ts | 30 +++++++++++++++ .../src/views/topic/TopicView.vue | 35 +++++++++++++---- 11 files changed, 231 insertions(+), 9 deletions(-) create mode 100644 hermes-console-vue/src/components/costs-card/CostsCard.spec.ts create mode 100644 hermes-console-vue/src/components/costs-card/CostsCard.vue diff --git a/hermes-console-vue/json-server/db.json b/hermes-console-vue/json-server/db.json index b1a3f0ecb7..f5174a1005 100644 --- a/hermes-console-vue/json-server/db.json +++ b/hermes-console-vue/json-server/db.json @@ -679,6 +679,14 @@ }, "group": { "nonAdminCreationEnabled": false + }, + "costs": { + "enabled": true, + "globalDetailsUrl": "", + "topicIframeUrl": "", + "topicDetailsUrl": "", + "subscriptionIframeUrl": "", + "subscriptionDetailsUrl": "" } }, "stats": { diff --git a/hermes-console-vue/src/api/app-configuration.ts b/hermes-console-vue/src/api/app-configuration.ts index 72924a90ee..e1eb50b32f 100644 --- a/hermes-console-vue/src/api/app-configuration.ts +++ b/hermes-console-vue/src/api/app-configuration.ts @@ -9,6 +9,7 @@ export interface AppConfiguration { subscription: SubscriptionViewConfiguration; consistency: ConsistencyViewConfiguration; group: GroupViewConfiguration; + costs: CostsConfiguration; } export interface ConsoleConfiguration { @@ -153,3 +154,12 @@ export interface ConsistencyViewConfiguration { export interface GroupViewConfiguration { nonAdminCreationEnabled: boolean; } + +export interface CostsConfiguration { + enabled: boolean; + globalDetailsUrl: string; + topicIframeUrl: string; + topicDetailsUrl: string; + subscriptionIframeUrl: string; + subscriptionDetailsUrl: string; +} diff --git a/hermes-console-vue/src/components/costs-card/CostsCard.spec.ts b/hermes-console-vue/src/components/costs-card/CostsCard.spec.ts new file mode 100644 index 0000000000..15174af94e --- /dev/null +++ b/hermes-console-vue/src/components/costs-card/CostsCard.spec.ts @@ -0,0 +1,30 @@ +import { describe } from 'vitest'; +import { render } from '@/utils/test-utils'; +import CostsCard from '@/components/costs-card/CostsCard.vue'; + +describe('CostsCard', () => { + const props = { + iframeUrl: + 'https://www.openstreetmap.org/export/embed.html?bbox=-0.004017949104309083%2C51.47612752641776%2C0.00030577182769775396%2C51.478569861898606&layer=mapnik', + detailsUrl: 'https://www.openstreetmap.org', + }; + + it('should render title properly', () => { + // when + const { getByText } = render(CostsCard, { props }); + + // then + const row = getByText('costsCard.title'); + expect(row).toBeVisible(); + }); + + it('should render iframe properly', async () => { + // when + const { container } = render(CostsCard, { props }); + const element = container.querySelector('iframe')!!; + + // then + expect(element).toBeVisible(); + expect(element).toHaveAttribute('src', props.iframeUrl); + }); +}); diff --git a/hermes-console-vue/src/components/costs-card/CostsCard.vue b/hermes-console-vue/src/components/costs-card/CostsCard.vue new file mode 100644 index 0000000000..0526ba9d62 --- /dev/null +++ b/hermes-console-vue/src/components/costs-card/CostsCard.vue @@ -0,0 +1,38 @@ + + + diff --git a/hermes-console/src/views/topic/metrics-list/MetricsList.spec.ts b/hermes-console/src/views/topic/metrics-list/MetricsList.spec.ts index 106de46baf..c507ce0b42 100644 --- a/hermes-console/src/views/topic/metrics-list/MetricsList.spec.ts +++ b/hermes-console/src/views/topic/metrics-list/MetricsList.spec.ts @@ -40,8 +40,6 @@ describe('MetricsList', () => { { property: 'topicView.metrics.rate', value: '3.40' }, { property: 'topicView.metrics.deliveryRate', value: '3.50' }, { property: 'topicView.metrics.published', value: 100 }, - { property: 'topicView.metrics.latency', value: '?' }, - { property: 'topicView.metrics.messageSize', value: '?' }, ])('should render all metrics properties %s', ({ property, value }) => { // given vi.mocked(useMetrics).mockReturnValueOnce(useMetricsStub); diff --git a/hermes-console/src/views/topic/metrics-list/MetricsList.vue b/hermes-console/src/views/topic/metrics-list/MetricsList.vue index d1745d9cb5..3fbecf9f4f 100644 --- a/hermes-console/src/views/topic/metrics-list/MetricsList.vue +++ b/hermes-console/src/views/topic/metrics-list/MetricsList.vue @@ -36,11 +36,6 @@ :name="$t('topicView.metrics.published')" :value="formatNumber(props.metrics.published)" /> - - diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSender.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSender.java index 7172e944fd..4f46845e58 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSender.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSender.java @@ -16,6 +16,7 @@ import pl.allegro.tech.hermes.consumers.consumer.sender.MessageSendingResult; import pl.allegro.tech.hermes.consumers.consumer.sender.MessageSendingResultLogInfo; import pl.allegro.tech.hermes.consumers.consumer.sender.timeout.FutureAsyncTimeout; +import pl.allegro.tech.hermes.metrics.HermesCounter; import pl.allegro.tech.hermes.metrics.HermesTimer; import pl.allegro.tech.hermes.metrics.HermesTimerContext; @@ -45,7 +46,9 @@ public class ConsumerMessageSender { private final InflightsPool inflight; private final SubscriptionLoadRecorder loadRecorder; private final HermesTimer consumerLatencyTimer; + private final HermesCounter retries; private final SerialConsumerRateLimiter rateLimiter; + private final HermesTimer rateLimiterAcquireTimer; private final FutureAsyncTimeout async; private final int asyncTimeoutMs; private final LongAdder inflightCount = new LongAdder(); @@ -82,6 +85,8 @@ public ConsumerMessageSender(Subscription subscription, this.inflight = inflight; this.consumerLatencyTimer = metrics.subscriptions().latency(subscription.getQualifiedName()); metrics.subscriptions().registerInflightGauge(subscription.getQualifiedName(), this, sender -> sender.inflightCount.doubleValue()); + this.retries = metrics.subscriptions().retries(subscription.getQualifiedName()); + this.rateLimiterAcquireTimer = metrics.subscriptions().rateLimiterAcquire(subscription.getQualifiedName()); } public void initialize() { @@ -131,6 +136,7 @@ private int calculateMessageDelay(long publishingMessageTimestamp) { */ private void sendMessage(final Message message) { loadRecorder.recordSingleOperation(); + acquireRateLimiterWithTimer(); HermesTimerContext timer = consumerLatencyTimer.time(); CompletableFuture response = messageSender.send(message); @@ -143,6 +149,12 @@ private void sendMessage(final Message message) { }); } + private void acquireRateLimiterWithTimer() { + HermesTimerContext acquireTimer = rateLimiterAcquireTimer.time(); + rateLimiter.acquire(); + acquireTimer.close(); + } + private MessageSender messageSender(Subscription subscription) { Integer requestTimeoutMs = subscription.getSerialSubscriptionPolicy().getRequestTimeout(); ResilientMessageSender resilientMessageSender = new ResilientMessageSender( @@ -200,6 +212,7 @@ private void retrySending(Message message, MessageSendingResult result) { long retryDelay = extractRetryDelay(message, result); if (shouldAttemptResending(message, result, retryDelay)) { + retries.increment(); retrySingleThreadExecutor.schedule(() -> resend(message, result), retryDelay, TimeUnit.MILLISECONDS); } else { handleMessageDiscarding(message, result); diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/ResilientMessageSender.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/ResilientMessageSender.java index 672a141eb7..7f09b6417d 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/ResilientMessageSender.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/ResilientMessageSender.java @@ -46,7 +46,6 @@ public CompletableFuture send( Function exceptionMapper ) { try { - rateLimiter.acquire(); CompletableFuture resultFuture = new CompletableFuture<>(); resultFutureConsumer.accept(resultFuture); CompletableFuture timeoutGuardedResultFuture = async.within( diff --git a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/ResilientMessageSenderTest.groovy b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/ResilientMessageSenderTest.groovy index 07eada26f4..5e9eff443d 100644 --- a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/ResilientMessageSenderTest.groovy +++ b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/ResilientMessageSenderTest.groovy @@ -14,8 +14,8 @@ import java.util.concurrent.Executors import java.util.function.Consumer import java.util.function.Function -import static io.netty.handler.codec.http.HttpResponseStatus.SERVICE_UNAVAILABLE; -import static io.netty.handler.codec.http.HttpResponseStatus.TOO_MANY_REQUESTS; +import static io.netty.handler.codec.http.HttpResponseStatus.SERVICE_UNAVAILABLE +import static io.netty.handler.codec.http.HttpResponseStatus.TOO_MANY_REQUESTS import static pl.allegro.tech.hermes.api.SubscriptionPolicy.Builder.subscriptionPolicy import static pl.allegro.tech.hermes.test.helper.builder.SubscriptionBuilder.subscription @@ -63,7 +63,6 @@ class ResilientMessageSenderTest extends Specification { def "should report successful sending"() { given: SerialConsumerRateLimiter serialConsumerRateLimiter = Mock(SerialConsumerRateLimiter) { - 1 * acquire() 1 * registerSuccessfulSending() } ResilientMessageSender rateLimitingMessageSender = new ResilientMessageSender( @@ -84,7 +83,6 @@ class ResilientMessageSenderTest extends Specification { def "should asynchronously time out send future and report failed sending"() { given: SerialConsumerRateLimiter serialConsumerRateLimiter = Mock(SerialConsumerRateLimiter) { - 1 * acquire() 1 * registerFailedSending() } ResilientMessageSender rateLimitingMessageSender = new ResilientMessageSender( @@ -105,7 +103,6 @@ class ResilientMessageSenderTest extends Specification { def "should treat 4xx response for subscription with no 4xx retry as success"() { given: SerialConsumerRateLimiter serialConsumerRateLimiter = Mock(SerialConsumerRateLimiter) { - 1 * acquire() 1 * registerSuccessfulSending() } @@ -127,7 +124,6 @@ class ResilientMessageSenderTest extends Specification { def "should report failed sending on error response other than 4xx for subscription with no 4xx retry"() { given: SerialConsumerRateLimiter serialConsumerRateLimiter = Mock(SerialConsumerRateLimiter) { - 1 * acquire() 1 * registerFailedSending() } @@ -149,7 +145,6 @@ class ResilientMessageSenderTest extends Specification { def "should report failed sending on 4xx response for subscription with 4xx retry"() { given: SerialConsumerRateLimiter serialConsumerRateLimiter = Mock(SerialConsumerRateLimiter) { - 1 * acquire() 1 * registerFailedSending() } def subscription = subscription(SubscriptionName.fromString("group.topic\$subscription")) @@ -175,7 +170,6 @@ class ResilientMessageSenderTest extends Specification { def "should report successful sending on retry after"() { given: SerialConsumerRateLimiter serialConsumerRateLimiter = Mock(SerialConsumerRateLimiter) { - 1 * acquire() 1 * registerSuccessfulSending() } @@ -197,7 +191,6 @@ class ResilientMessageSenderTest extends Specification { def "should report failed sending on service unavailable without retry after"() { given: SerialConsumerRateLimiter serialConsumerRateLimiter = Mock(SerialConsumerRateLimiter) { - 1 * acquire() 1 * registerFailedSending() } @@ -219,7 +212,6 @@ class ResilientMessageSenderTest extends Specification { def "should not report failed sending on too many requests without retry after"() { given: SerialConsumerRateLimiter serialConsumerRateLimiter = Mock(SerialConsumerRateLimiter) { - 1 * acquire() 1 * registerSuccessfulSending() } @@ -241,7 +233,6 @@ class ResilientMessageSenderTest extends Specification { def "should report failed sending when future completes exceptionally"() { given: SerialConsumerRateLimiter serialConsumerRateLimiter = Mock(SerialConsumerRateLimiter) { - 1 * acquire() 1 * registerFailedSending() } ResilientMessageSender rateLimitingMessageSender = new ResilientMessageSender( @@ -266,7 +257,6 @@ class ResilientMessageSenderTest extends Specification { def "should report failed sending when consumer throws exception"() { given: SerialConsumerRateLimiter serialConsumerRateLimiter = Mock(SerialConsumerRateLimiter) { - 1 * acquire() 1 * registerFailedSending() } ResilientMessageSender rateLimitingMessageSender = new ResilientMessageSender( diff --git a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/sender/SingleRecipientMessageSenderAdapterTest.groovy b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/sender/SingleRecipientMessageSenderAdapterTest.groovy index 9a444d1be5..79eec88352 100644 --- a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/sender/SingleRecipientMessageSenderAdapterTest.groovy +++ b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/sender/SingleRecipientMessageSenderAdapterTest.groovy @@ -49,7 +49,6 @@ class SingleRecipientMessageSenderAdapterTest extends Specification { def "should register successful send in rate limiter"() { given: ConsumerRateLimiter consumerRateLimiter = Mock(ConsumerRateLimiter) { - 1 * acquire() 1 * registerSuccessfulSending() } ResilientMessageSender rateLimitingMessageSender = rateLimitingMessageSender(consumerRateLimiter) @@ -65,7 +64,6 @@ class SingleRecipientMessageSenderAdapterTest extends Specification { def "should register unsuccessful send in rate limiter"() { given: ConsumerRateLimiter consumerRateLimiter = Mock(ConsumerRateLimiter) { - 1 * acquire() 1 * registerFailedSending() } ResilientMessageSender rateLimitingMessageSender = rateLimitingMessageSender(consumerRateLimiter) @@ -77,7 +75,5 @@ class SingleRecipientMessageSenderAdapterTest extends Specification { then: !future.get().succeeded() - } - } diff --git a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/sender/http/JettyBroadCastMessageSenderTest.groovy b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/sender/http/JettyBroadCastMessageSenderTest.groovy index 7445574d47..06d1bf7832 100644 --- a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/sender/http/JettyBroadCastMessageSenderTest.groovy +++ b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/sender/http/JettyBroadCastMessageSenderTest.groovy @@ -71,7 +71,7 @@ class JettyBroadCastMessageSenderTest extends Specification { MessageSender getSender(ConsumerRateLimiter rateLimiter) { def address = new ResolvableEndpointAddress(endpoint, new MultiUrlEndpointAddressResolver(), - EndpointAddressResolverMetadata.empty()); + EndpointAddressResolverMetadata.empty()) def httpRequestFactory = new DefaultHttpRequestFactory(client, 1000, 1000, new DefaultHttpMetadataAppender()) Subscription subscription = subscription(SubscriptionName.fromString("group.topic\$subscription")).build() @@ -87,14 +87,13 @@ class JettyBroadCastMessageSenderTest extends Specification { def "should send message successfully in parallel to all urls"() { given: ConsumerRateLimiter rateLimiter = Mock(ConsumerRateLimiter) { - 4 * acquire() 4 * registerSuccessfulSending() } serviceEndpoints.forEach { endpoint -> endpoint.setDelay(300).expectMessages(TEST_MESSAGE_CONTENT) } when: - def future = getSender(rateLimiter).send(testMessage()); + def future = getSender(rateLimiter).send(testMessage()) then: future.get(10, TimeUnit.SECONDS).succeeded() @@ -107,7 +106,6 @@ class JettyBroadCastMessageSenderTest extends Specification { def "should return not succeeded when sending to one of urls fails"() { given: ConsumerRateLimiter rateLimiter = Mock(ConsumerRateLimiter) { - 4 * acquire() 3 * registerSuccessfulSending() 1 * registerFailedSending() } @@ -133,15 +131,14 @@ class JettyBroadCastMessageSenderTest extends Specification { def "should not send to already sent url on retry"() { given: ConsumerRateLimiter rateLimiter = Mock(ConsumerRateLimiter) { - 3 * acquire() 3 * registerSuccessfulSending() } serviceEndpoints.forEach { endpoint -> endpoint.expectMessages(TEST_MESSAGE_CONTENT) } def alreadySentServiceEndpoint = serviceEndpoints[0] - Message message = testMessage(); - message.incrementRetryCounter([alreadySentServiceEndpoint.url]); + Message message = testMessage() + message.incrementRetryCounter([alreadySentServiceEndpoint.url]) when: def future = getSender(rateLimiter).send(message) diff --git a/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSenderTest.java b/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSenderTest.java index 5443e1421a..a630aafcc8 100644 --- a/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSenderTest.java +++ b/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSenderTest.java @@ -69,9 +69,18 @@ public class ConsumerMessageSenderTest { @Mock private HermesTimer consumerLatencyTimer; + @Mock + private HermesCounter retries; + + @Mock + private HermesTimer rateLimiterAcquireTimer; + @Mock private HermesTimerContext consumerLatencyTimerContext; + @Mock + private HermesTimerContext rateLimiterAcquireTimerContext; + @Mock private HermesCounter failedMeter; @@ -99,9 +108,12 @@ public void setUp() { private void setUpMetrics(Subscription subscription) { when(metricsFacade.subscriptions().latency(subscription.getQualifiedName())).thenReturn(consumerLatencyTimer); + when(metricsFacade.subscriptions().rateLimiterAcquire(subscription.getQualifiedName())).thenReturn(rateLimiterAcquireTimer); when(metricsFacade.subscriptions().otherErrorsCounter(subscription.getQualifiedName())).thenReturn(errors); when(consumerLatencyTimer.time()).thenReturn(consumerLatencyTimerContext); + when(rateLimiterAcquireTimer.time()).thenReturn(rateLimiterAcquireTimerContext); when(metricsFacade.subscriptions().failuresCounter(subscription.getQualifiedName())).thenReturn(failedMeter); + when(metricsFacade.subscriptions().retries(subscription.getQualifiedName())).thenReturn(retries); } @Test @@ -117,8 +129,11 @@ public void shouldHandleSuccessfulSending() { // then verifySemaphoreReleased(); verifyLatencyTimersCountedTimes(subscription, 1, 1); + verifyRateLimiterAcquireTimersCountedTimes(subscription, 1, 1); verifyZeroInteractions(errorHandler); verifyZeroInteractions(failedMeter); + verifyRateLimiterAcquired(); + verifyZeroInteractions(retries); } @Test @@ -134,7 +149,10 @@ public void shouldKeepTryingToSendMessageFailedSending() { // then verifySemaphoreReleased(); verifyLatencyTimersCountedTimes(subscription, 3, 3); + verifyRateLimiterAcquireTimersCountedTimes(subscription, 3, 3); verifyErrorHandlerHandleFailed(message, subscription, 2); + verifyRateLimiterAcquired(3); + verifyRetryCounted(2); } @Test @@ -151,6 +169,9 @@ public void shouldDiscardMessageWhenTTLIsExceeded() { verifySemaphoreReleased(); verifyZeroInteractions(successHandler); verifyLatencyTimersCountedTimes(subscription, 1, 1); + verifyRateLimiterAcquireTimersCountedTimes(subscription, 1, 1); + verifyRateLimiterAcquired(); + verifyZeroInteractions(retries); } @Test @@ -167,11 +188,15 @@ public void shouldNotKeepTryingToSendMessageFailedWithStatusCode4xx() { verifySemaphoreReleased(); verifyZeroInteractions(successHandler); verifyLatencyTimersCountedTimes(subscription, 1, 1); + verifyRateLimiterAcquireTimersCountedTimes(subscription, 1, 1); + verifyRateLimiterAcquired(); + verifyZeroInteractions(retries); } @Test public void shouldKeepTryingToSendMessageFailedWithStatusCode4xxForSubscriptionWith4xxRetry() { // given + final int expectedNumbersOfFailures = 3; ConsumerMessageSender sender = consumerMessageSender(subscriptionWith4xxRetry); Message message = message(); doReturn(failure(403)).doReturn(failure(403)).doReturn(failure(403)).doReturn(success()).when(messageSender).send(message); @@ -183,15 +208,18 @@ public void shouldKeepTryingToSendMessageFailedWithStatusCode4xxForSubscriptionW // then verifySemaphoreReleased(); verify(errorHandler, - timeout(1000).times(3)).handleFailed(eq(message), + timeout(1000).times(expectedNumbersOfFailures)).handleFailed(eq(message), eq(subscriptionWith4xxRetry), any(MessageSendingResult.class) ); + verifyRateLimiterAcquired(expectedNumbersOfFailures + 1); + verifyRetryCounted(expectedNumbersOfFailures); } @Test public void shouldRetryOn401UnauthorizedForOAuthSecuredSubscription() { // given + final int expectedNumbersOfFailures = 2; Subscription subscription = subscriptionWithout4xxRetryAndWithOAuthPolicy(); setUpMetrics(subscription); ConsumerMessageSender sender = consumerMessageSender(subscription); @@ -202,15 +230,18 @@ public void shouldRetryOn401UnauthorizedForOAuthSecuredSubscription() { sender.sendAsync(message); // then - verifyErrorHandlerHandleFailed(message, subscription, 2); + verifyErrorHandlerHandleFailed(message, subscription, expectedNumbersOfFailures); verify(successHandler, timeout(1000)).handleSuccess(eq(message), eq(subscription), any(MessageSendingResult.class)); + verifyRetryCounted(expectedNumbersOfFailures); + verifyRateLimiterAcquired(expectedNumbersOfFailures + 1); } @Test public void shouldBackoffRetriesWhenEndpointFails() throws InterruptedException { // given final int executionTime = 100; - int senderBackoffTime = 50; + final int senderBackoffTime = 50; + final int expectedNumberOfFailures = 1 + executionTime / senderBackoffTime; Subscription subscriptionWithBackoff = subscriptionWithBackoff(senderBackoffTime); setUpMetrics(subscriptionWithBackoff); @@ -223,7 +254,9 @@ public void shouldBackoffRetriesWhenEndpointFails() throws InterruptedException //then Thread.sleep(executionTime); - verifyErrorHandlerHandleFailed(message, subscriptionWithBackoff, 1 + executionTime / senderBackoffTime); + verifyErrorHandlerHandleFailed(message, subscriptionWithBackoff, expectedNumberOfFailures); + verifyRateLimiterAcquired(expectedNumberOfFailures); + verifyRetryCounted(expectedNumberOfFailures); } @Test @@ -241,6 +274,9 @@ public void shouldNotRetryOnRetryAfterAboveTtl() { verifySemaphoreReleased(); verifyZeroInteractions(successHandler); verifyLatencyTimersCountedTimes(subscription, 1, 1); + verifyRateLimiterAcquireTimersCountedTimes(subscription, 1, 1); + verifyRateLimiterAcquired(); + verifyZeroInteractions(retries); } @Test @@ -260,6 +296,8 @@ public void shouldDeliverToModifiedEndpoint() { // then verify(otherMessageSender, timeout(1000)).send(message); + verifyRateLimiterAcquired(); + verifyZeroInteractions(retries); } @Test @@ -279,6 +317,8 @@ public void shouldDeliverToNewSenderAfterModifiedTimeout() { // then verify(otherMessageSender, timeout(1000)).send(message); + verifyRateLimiterAcquired(); + verifyZeroInteractions(retries); } @Test @@ -303,6 +343,8 @@ public void shouldDelaySendingMessageForHalfSecond() { // then long sendingTime = System.currentTimeMillis() - sendingStartTime; assertThat(sendingTime).isGreaterThanOrEqualTo(500); + verifyRateLimiterAcquired(); + verifyZeroInteractions(retries); } @Test @@ -327,11 +369,14 @@ public void shouldCalculateSendingDelayBasingOnPublishingTimestamp() { // then long sendingTime = System.currentTimeMillis() - sendingStartTime; assertThat(sendingTime).isLessThan(300); + verifyRateLimiterAcquired(); + verifyZeroInteractions(retries); } @Test public void shouldIncreaseRetryBackoffExponentially() throws InterruptedException { // given + final int expectedNumberOfFailures = 2; final int backoff = 500; final double multiplier = 2; Subscription subscription = subscriptionWithExponentialRetryBackoff(backoff, multiplier); @@ -346,11 +391,14 @@ public void shouldIncreaseRetryBackoffExponentially() throws InterruptedExceptio // then verifyZeroInteractions(successHandler); + verifyRateLimiterAcquired(expectedNumberOfFailures); + verifyRetryCounted(expectedNumberOfFailures); } @Test public void shouldIgnoreExponentialRetryBackoffWithRetryAfter() { // given + final int expectedNumberOfRetries = 2; final int retrySeconds = 1; final int backoff = 5000; final double multiplier = 3; @@ -366,6 +414,8 @@ public void shouldIgnoreExponentialRetryBackoffWithRetryAfter() { //then verify(successHandler, timeout(retrySeconds * 1000 * 2 + 500)) .handleSuccess(eq(message), eq(subscription), any(MessageSendingResult.class)); + verifyRateLimiterAcquired(expectedNumberOfRetries + 1); + verifyRetryCounted(expectedNumberOfRetries); } @Test @@ -385,6 +435,8 @@ public void shouldIgnoreExponentialRetryBackoffAfterExceededTtl() throws Interru //then verifyZeroInteractions(successHandler); + verifyRateLimiterAcquired(2); + verifyRetryCounted(); } private ConsumerMessageSender consumerMessageSender(Subscription subscription) { @@ -422,6 +474,12 @@ private void verifyLatencyTimersCountedTimes(Subscription subscription, int time verify(consumerLatencyTimerContext, times(closeCount)).close(); } + private void verifyRateLimiterAcquireTimersCountedTimes(Subscription subscription, int timeCount, int closeCount) { + verify(metricsFacade.subscriptions(), times(1)).rateLimiterAcquire(subscription.getQualifiedName()); + verify(rateLimiterAcquireTimer, times(timeCount)).time(); + verify(rateLimiterAcquireTimerContext, times(closeCount)).close(); + } + private Subscription subscriptionWithTtl(int ttl) { return subscriptionBuilderWithTestValues() .withSubscriptionPolicy(subscriptionPolicy().applyDefaults() @@ -506,6 +564,21 @@ private void verifySemaphoreReleased() { assertThat(inflightSemaphore.availablePermits()).isEqualTo(1); } + private void verifyRateLimiterAcquired() { + verifyRateLimiterAcquired(1); + } + + private void verifyRateLimiterAcquired(int times) { + verify(rateLimiter, times(times)).acquire(); + } + + private void verifyRetryCounted() { + verifyRetryCounted(1); + } + + private void verifyRetryCounted(int times) { + verify(retries, times(times)).increment(); + } private Message message() { return messageWithTimestamp(System.currentTimeMillis()); diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ExternalMonitoringConfiguration.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ExternalMonitoringConfiguration.java index e3a828a463..accd325bf2 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ExternalMonitoringConfiguration.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ExternalMonitoringConfiguration.java @@ -15,7 +15,6 @@ import org.springframework.web.client.RestTemplate; import pl.allegro.tech.hermes.management.infrastructure.graphite.CachingGraphiteClient; import pl.allegro.tech.hermes.management.infrastructure.graphite.GraphiteClient; -import pl.allegro.tech.hermes.management.infrastructure.graphite.GraphiteMetricsProvider; import pl.allegro.tech.hermes.management.infrastructure.graphite.RestTemplateGraphiteClient; import pl.allegro.tech.hermes.management.infrastructure.prometheus.CachingPrometheusClient; import pl.allegro.tech.hermes.management.infrastructure.prometheus.PrometheusClient; @@ -29,13 +28,6 @@ @Configuration public class ExternalMonitoringConfiguration { - @Bean - @ConditionalOnProperty(value = "graphite.client.enabled", havingValue = "true") - public GraphiteMetricsProvider graphiteMetricsProvider(GraphiteClient graphiteClient, - GraphiteMonitoringMetricsProperties properties) { - return new GraphiteMetricsProvider(graphiteClient, properties.getPrefix()); - } - @Bean @ConditionalOnProperty(value = "graphite.client.enabled", havingValue = "true") public GraphiteClient graphiteClient(@Qualifier("monitoringRestTemplate") RestTemplate graphiteRestTemplate, diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/graphite/GraphiteMetricsProvider.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/graphite/GraphiteMetricsProvider.java deleted file mode 100644 index 3e355607ec..0000000000 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/graphite/GraphiteMetricsProvider.java +++ /dev/null @@ -1,110 +0,0 @@ -package pl.allegro.tech.hermes.management.infrastructure.graphite; - -import pl.allegro.tech.hermes.api.SubscriptionName; -import pl.allegro.tech.hermes.api.TopicName; -import pl.allegro.tech.hermes.management.infrastructure.metrics.MonitoringMetricsContainer; -import pl.allegro.tech.hermes.management.infrastructure.metrics.MonitoringSubscriptionMetricsProvider; -import pl.allegro.tech.hermes.management.infrastructure.metrics.MonitoringTopicMetricsProvider; - -import static pl.allegro.tech.hermes.common.metric.HermesMetrics.escapeDots; - -public class GraphiteMetricsProvider implements MonitoringSubscriptionMetricsProvider, MonitoringTopicMetricsProvider { - - private static final String SUBSCRIPTION_PATH = "%s.%s.%s"; - - private static final String SUBSCRIPTION_RATE_PATTERN = "sumSeries(%s.consumer.*.meter.%s.m1_rate)"; - private static final String SUBSCRIPTION_THROUGHPUT_PATTERN = "sumSeries(%s.consumer.*.throughput.%s.m1_rate)"; - private static final String SUBSCRIPTION_HTTP_STATUSES_PATTERN = "sumSeries(%s.consumer.*.status.%s.%s.m1_rate)"; - private static final String SUBSCRIPTION_ERROR_TIMEOUT_PATTERN = "sumSeries(%s.consumer.*.status.%s.errors.timeout.m1_rate)"; - private static final String SUBSCRIPTION_ERROR_OTHER_PATTERN = "sumSeries(%s.consumer.*.status.%s.errors.other.m1_rate)"; - private static final String SUBSCRIPTION_BATCH_RATE_PATTERN = "sumSeries(%s.consumer.*.meter.%s.batch.m1_rate)"; - - private static final String TOPIC_RATE_PATTERN = "sumSeries(%s.producer.*.meter.%s.%s.m1_rate)"; - private static final String TOPIC_DELIVERY_RATE_PATTERN = "sumSeries(%s.consumer.*.meter.%s.%s.m1_rate)"; - private static final String TOPIC_THROUGHPUT_PATTERN = "sumSeries(%s.producer.*.throughput.%s.%s.m1_rate)"; - - private final GraphiteClient graphiteClient; - private final String prefix; - - public GraphiteMetricsProvider(GraphiteClient graphiteClient, String prefix) { - this.graphiteClient = graphiteClient; - this.prefix = prefix; - } - - @Override - public MonitoringSubscriptionMetrics subscriptionMetrics(SubscriptionName name) { - String rateMetric = metricPath(name); - String timeouts = metricPathTimeouts(name); - String throughput = metricPathThroughput(name); - String otherErrors = metricPathOtherErrors(name); - String codes2xxPath = metricPathHttpStatuses(name, "2xx"); - String codes4xxPath = metricPathHttpStatuses(name, "4xx"); - String codes5xxPath = metricPathHttpStatuses(name, "5xx"); - String batchPath = metricPathBatchRate(name); - - MonitoringMetricsContainer metricsContainer = graphiteClient.readMetrics(codes2xxPath, codes4xxPath, codes5xxPath, - rateMetric, throughput, timeouts, otherErrors, batchPath); - - return MonitoringSubscriptionMetricsProvider.metricsBuilder() - .withRate(metricsContainer.metricValue(rateMetric)) - .withTimeouts(metricsContainer.metricValue(timeouts)) - .withThroughput(metricsContainer.metricValue(throughput)) - .withOtherErrors(metricsContainer.metricValue(otherErrors)) - .withCodes2xx(metricsContainer.metricValue(codes2xxPath)) - .withCode4xx(metricsContainer.metricValue(codes4xxPath)) - .withCode5xx(metricsContainer.metricValue(codes5xxPath)) - .withMetricPathBatchRate(metricsContainer.metricValue(batchPath)) - .build(); - } - - @Override - public MonitoringTopicMetrics topicMetrics(TopicName topicName) { - String rateMetric = metricPath(TOPIC_RATE_PATTERN, topicName); - String deliveryRateMetric = metricPath(TOPIC_DELIVERY_RATE_PATTERN, topicName); - String throughputMetric = metricPath(TOPIC_THROUGHPUT_PATTERN, topicName); - - MonitoringMetricsContainer metrics = graphiteClient.readMetrics(rateMetric, deliveryRateMetric, throughputMetric); - return MonitoringTopicMetricsProvider.metricsBuilder() - .withRate(metrics.metricValue(rateMetric)) - .withDeliveryRate(metrics.metricValue(deliveryRateMetric)) - .withThroughput(metrics.metricValue(throughputMetric)) - .build(); - } - - private String metricPath(SubscriptionName name) { - return String.format(SUBSCRIPTION_RATE_PATTERN, prefix, subscriptionNameToPath(name) - ); - } - - private String metricPath(String pattern, TopicName topicName) { - return String.format(pattern, prefix, escapeDots(topicName.getGroupName()), - escapeDots(topicName.getName())); - } - - private String metricPathThroughput(SubscriptionName name) { - return String.format(SUBSCRIPTION_THROUGHPUT_PATTERN, prefix, subscriptionNameToPath(name)); - } - - private String metricPathHttpStatuses(SubscriptionName name, String statusCodeClass) { - return String.format(SUBSCRIPTION_HTTP_STATUSES_PATTERN, prefix, subscriptionNameToPath(name), statusCodeClass); - } - - private String metricPathTimeouts(SubscriptionName name) { - return String.format(SUBSCRIPTION_ERROR_TIMEOUT_PATTERN, prefix, subscriptionNameToPath(name) - ); - } - - private String metricPathOtherErrors(SubscriptionName name) { - return String.format(SUBSCRIPTION_ERROR_OTHER_PATTERN, prefix, subscriptionNameToPath(name)); - } - - private String metricPathBatchRate(SubscriptionName name) { - return String.format(SUBSCRIPTION_BATCH_RATE_PATTERN, prefix, subscriptionNameToPath(name)); - } - - private String subscriptionNameToPath(SubscriptionName name) { - return String.format(SUBSCRIPTION_PATH, - escapeDots(name.getTopicName().getGroupName()), name.getTopicName().getName(), escapeDots(name.getName()) - ); - } -} diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridSubscriptionMetricsRepository.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridSubscriptionMetricsRepository.java index ac14b42657..8b82e66be7 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridSubscriptionMetricsRepository.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridSubscriptionMetricsRepository.java @@ -47,6 +47,7 @@ public SubscriptionMetrics loadMetrics(TopicName topicName, String subscriptionN .withCodes2xx(monitoringMetrics.codes2xx()) .withCodes4xx(monitoringMetrics.code4xx()) .withCodes5xx(monitoringMetrics.code5xx()) + .withRetries(monitoringMetrics.retries()) .withTimeouts(monitoringMetrics.timeouts()) .withOtherErrors(monitoringMetrics.otherErrors()) .withThroughput(monitoringMetrics.throughput()) diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/metrics/MonitoringSubscriptionMetricsProvider.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/metrics/MonitoringSubscriptionMetricsProvider.java index d98cccdb19..8745aa1d1c 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/metrics/MonitoringSubscriptionMetricsProvider.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/metrics/MonitoringSubscriptionMetricsProvider.java @@ -13,6 +13,7 @@ record MonitoringSubscriptionMetrics(MetricDecimalValue rate, MetricDecimalValue codes2xx, MetricDecimalValue code4xx, MetricDecimalValue code5xx, + MetricDecimalValue retries, MetricDecimalValue metricPathBatchRate) { } @@ -28,6 +29,7 @@ class MetricsBuilder { private MetricDecimalValue codes2xx; private MetricDecimalValue code4xx; private MetricDecimalValue code5xx; + private MetricDecimalValue retries; private MetricDecimalValue metricPathBatchRate; public MetricsBuilder withRate(MetricDecimalValue rate) { @@ -65,6 +67,11 @@ public MetricsBuilder withCode5xx(MetricDecimalValue code5xx) { return this; } + public MetricsBuilder withRetries(MetricDecimalValue retries) { + this.retries = retries; + return this; + } + public MetricsBuilder withMetricPathBatchRate(MetricDecimalValue metricPathBatchRate) { this.metricPathBatchRate = metricPathBatchRate; return this; @@ -72,7 +79,7 @@ public MetricsBuilder withMetricPathBatchRate(MetricDecimalValue metricPathBatch public MonitoringSubscriptionMetrics build() { return new MonitoringSubscriptionMetrics(rate, timeouts, throughput, otherErrors, codes2xx, - code4xx, code5xx, metricPathBatchRate); + code4xx, code5xx, retries, metricPathBatchRate); } } } \ No newline at end of file diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/VictoriaMetricsMetricsProvider.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/VictoriaMetricsMetricsProvider.java index b2c1e3d008..48bcd67600 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/VictoriaMetricsMetricsProvider.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/VictoriaMetricsMetricsProvider.java @@ -21,6 +21,7 @@ public class VictoriaMetricsMetricsProvider implements MonitoringSubscriptionMet private static final String SUBSCRIPTION_STATUS_CODES_2XX = SUBSCRIPTION_STATUS_CODES + "_2xx"; private static final String SUBSCRIPTION_STATUS_CODES_4XX = SUBSCRIPTION_STATUS_CODES + "_4xx"; private static final String SUBSCRIPTION_STATUS_CODES_5XX = SUBSCRIPTION_STATUS_CODES + "_5xx"; + private static final String SUBSCRIPTION_RETRIES = "subscription_retries_total"; private static final String TOPIC_RATE = "topic_requests_total"; private static final String TOPIC_DELIVERY_RATE = "subscription_delivered_total"; @@ -39,9 +40,8 @@ public VictoriaMetricsMetricsProvider(PrometheusClient prometheusClient, String this.consumersMetricsPrefix = consumersMetricsPrefix.isEmpty() ? "" : consumersMetricsPrefix + "_"; this.frontendMetricsPrefix = frontendMetricsPrefix.isEmpty() ? "" : frontendMetricsPrefix + "_"; this.additionalFilters = additionalFilters; - this.subscriptionMetricsToQuery = Stream.of(SUBSCRIPTION_DELIVERED, SUBSCRIPTION_TIMEOUTS, - SUBSCRIPTION_THROUGHPUT, SUBSCRIPTION_OTHER_ERRORS, SUBSCRIPTION_BATCHES, - SUBSCRIPTION_STATUS_CODES) + this.subscriptionMetricsToQuery = Stream.of(SUBSCRIPTION_DELIVERED, SUBSCRIPTION_TIMEOUTS, SUBSCRIPTION_RETRIES, + SUBSCRIPTION_THROUGHPUT, SUBSCRIPTION_OTHER_ERRORS, SUBSCRIPTION_BATCHES, SUBSCRIPTION_STATUS_CODES) .map(this::consumerMetricName) .collect(Collectors.joining("|")); this.topicMetricsToQuery = String.join("|", List.of( @@ -72,6 +72,7 @@ public MonitoringSubscriptionMetrics subscriptionMetrics(SubscriptionName subscr .withCodes2xx(prometheusMetricsContainer.metricValue(consumerMetricName(SUBSCRIPTION_STATUS_CODES_2XX))) .withCode4xx(prometheusMetricsContainer.metricValue(consumerMetricName(SUBSCRIPTION_STATUS_CODES_4XX))) .withCode5xx(prometheusMetricsContainer.metricValue(consumerMetricName(SUBSCRIPTION_STATUS_CODES_5XX))) + .withRetries(prometheusMetricsContainer.metricValue(consumerMetricName(SUBSCRIPTION_RETRIES))) .build(); } diff --git a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridGraphiteBasedSubscriptionMetricsRepositoryTest.groovy b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridGraphiteBasedSubscriptionMetricsRepositoryTest.groovy deleted file mode 100644 index 0e94f343ac..0000000000 --- a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridGraphiteBasedSubscriptionMetricsRepositoryTest.groovy +++ /dev/null @@ -1,92 +0,0 @@ -package pl.allegro.tech.hermes.management.infrastructure.metrics - -import pl.allegro.tech.hermes.api.MetricLongValue -import pl.allegro.tech.hermes.api.PersistentSubscriptionMetrics -import pl.allegro.tech.hermes.api.SubscriptionMetrics -import pl.allegro.tech.hermes.api.TopicName -import pl.allegro.tech.hermes.infrastructure.zookeeper.ZookeeperPaths -import pl.allegro.tech.hermes.management.domain.subscription.SubscriptionLagSource -import pl.allegro.tech.hermes.management.infrastructure.graphite.GraphiteClient -import pl.allegro.tech.hermes.management.infrastructure.graphite.GraphiteMetricsProvider -import spock.lang.Specification - -import static pl.allegro.tech.hermes.api.MetricDecimalValue.of - -class HybridGraphiteBasedSubscriptionMetricsRepositoryTest extends Specification { - - private GraphiteClient client = Stub(GraphiteClient) - - private SummedSharedCounter summedSharedCounter = Stub(SummedSharedCounter) - - private ZookeeperPaths zookeeperPaths = new ZookeeperPaths("/hermes") - - private SubscriptionLagSource lagSource = new NoOpSubscriptionLagSource() - - private GraphiteMetricsProvider graphiteMetricsProvider = new GraphiteMetricsProvider(client, "stats"); - - private HybridSubscriptionMetricsRepository repository = new HybridSubscriptionMetricsRepository(graphiteMetricsProvider, - summedSharedCounter, zookeeperPaths, lagSource) - - def "should read subscription metrics from multiple places"() { - given: - String rate = 'sumSeries(stats.consumer.*.meter.group.topic.subscription.m1_rate)' - String timeouts = 'sumSeries(stats.consumer.*.status.group.topic.subscription.errors.timeout.m1_rate)' - String otherErrors = 'sumSeries(stats.consumer.*.status.group.topic.subscription.errors.other.m1_rate)' - - client.readMetrics(_ as String, _ as String, _ as String, rate, _ as String, timeouts, otherErrors, _ as String) >> MonitoringMetricsContainer.createEmpty() - .addMetricValue(rate, of('10')) - .addMetricValue(timeouts, of('100')) - .addMetricValue(otherErrors, of('1000')) - summedSharedCounter.getValue('/hermes/groups/group/topics/topic/subscriptions/subscription/metrics/delivered') >> 100 - summedSharedCounter.getValue('/hermes/groups/group/topics/topic/subscriptions/subscription/metrics/discarded') >> 1 - summedSharedCounter.getValue('/hermes/groups/group/topics/topic/subscriptions/subscription/metrics/volume') >> 16 - - when: - SubscriptionMetrics metrics = repository.loadMetrics(new TopicName('group', 'topic'), 'subscription') - - then: - metrics.rate == of('10') - metrics.delivered == 100 - metrics.discarded == 1 - metrics.volume == 16 - metrics.timeouts == of("100") - metrics.otherErrors == of("1000") - metrics.lag == MetricLongValue.of(-1) - } - - def "should read subscription metrics for all http status codes"() { - given: - client.readMetrics(getHttpStatusCodeForFamily(2), getHttpStatusCodeForFamily(4), getHttpStatusCodeForFamily(5), - _ as String, _ as String, _ as String, _ as String, _ as String) >> MonitoringMetricsContainer.createEmpty() - .addMetricValue(getHttpStatusCodeForFamily(2), of('2')) - .addMetricValue(getHttpStatusCodeForFamily(4), of('4')) - .addMetricValue(getHttpStatusCodeForFamily(5), of('5')) - - when: - SubscriptionMetrics metrics = repository.loadMetrics(new TopicName('group', 'topic'), 'subscription') - - then: - metrics.codes2xx == of('2') - metrics.codes4xx == of('4') - metrics.codes5xx == of('5') - } - - def "should read subscription zookeeper metrics"() { - given: - summedSharedCounter.getValue('/hermes/groups/group/topics/topic/subscriptions/subscription/metrics/delivered') >> 1000 - summedSharedCounter.getValue('/hermes/groups/group/topics/topic/subscriptions/subscription/metrics/discarded') >> 10 - summedSharedCounter.getValue('/hermes/groups/group/topics/topic/subscriptions/subscription/metrics/volume') >> 16 - - when: - PersistentSubscriptionMetrics zookeeperMetrics = repository.loadZookeeperMetrics(new TopicName('group', 'topic'), 'subscription') - - then: - zookeeperMetrics.delivered == 1000 - zookeeperMetrics.discarded == 10 - zookeeperMetrics.volume == 16 - } - - private static String getHttpStatusCodeForFamily(int family) { - "sumSeries(stats.consumer.*.status.group.topic.subscription.${family}xx.m1_rate)" - } -} diff --git a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridGraphiteBasedTopicMetricsRepositoryTest.groovy b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridGraphiteBasedTopicMetricsRepositoryTest.groovy deleted file mode 100644 index 4c792c7e36..0000000000 --- a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridGraphiteBasedTopicMetricsRepositoryTest.groovy +++ /dev/null @@ -1,53 +0,0 @@ -package pl.allegro.tech.hermes.management.infrastructure.metrics - -import pl.allegro.tech.hermes.api.TopicMetrics -import pl.allegro.tech.hermes.api.TopicName -import pl.allegro.tech.hermes.domain.subscription.SubscriptionRepository -import pl.allegro.tech.hermes.infrastructure.zookeeper.ZookeeperPaths -import pl.allegro.tech.hermes.management.infrastructure.graphite.GraphiteClient -import pl.allegro.tech.hermes.management.infrastructure.graphite.GraphiteMetricsProvider -import spock.lang.Specification - -import static pl.allegro.tech.hermes.api.MetricDecimalValue.of - -class HybridGraphiteBasedTopicMetricsRepositoryTest extends Specification { - - private GraphiteClient client = Stub(GraphiteClient) - - private SummedSharedCounter summedSharedCounter = Stub(SummedSharedCounter) - - private ZookeeperPaths zookeeperPaths = new ZookeeperPaths("/hermes") - - private SubscriptionRepository subscriptionRepository = Mock(SubscriptionRepository) - - private GraphiteMetricsProvider graphiteMetricsProvider = new GraphiteMetricsProvider(client, "stats") - - private HybridTopicMetricsRepository repository = new HybridTopicMetricsRepository(graphiteMetricsProvider, - summedSharedCounter, zookeeperPaths, subscriptionRepository) - - def "should load metrics from graphite and zookeeper"() { - given: - String rate = 'sumSeries(stats.producer.*.meter.group.topic.m1_rate)' - String deliveryRate = 'sumSeries(stats.consumer.*.meter.group.topic.m1_rate)' - String throughput = 'sumSeries(stats.producer.*.throughput.group.topic.m1_rate)' - TopicName topic = new TopicName('group', 'topic') - - client.readMetrics(rate, deliveryRate, throughput) >> MonitoringMetricsContainer.createEmpty() - .addMetricValue(rate, of('10')) - .addMetricValue(deliveryRate, of('20')) - summedSharedCounter.getValue('/hermes/groups/group/topics/topic/metrics/published') >> 100 - summedSharedCounter.getValue('/hermes/groups/group/topics/topic/metrics/volume') >> 1024 - subscriptionRepository.listSubscriptionNames(topic) >> ["subscription1", "subscription2"] - - when: - TopicMetrics metrics = repository.loadMetrics(topic) - - then: - metrics.rate == of('10') - metrics.deliveryRate == of('20') - metrics.published == 100 - metrics.subscriptions == 2 - metrics.volume == 1024 - } - -} diff --git a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridPrometheusBasedSubscriptionMetricsRepositoryTest.groovy b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridPrometheusBasedSubscriptionMetricsRepositoryTest.groovy index 396d2330cf..6c18b04c48 100644 --- a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridPrometheusBasedSubscriptionMetricsRepositoryTest.groovy +++ b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridPrometheusBasedSubscriptionMetricsRepositoryTest.groovy @@ -23,7 +23,7 @@ class HybridPrometheusBasedSubscriptionMetricsRepositoryTest extends Specificati private SubscriptionLagSource lagSource = new NoOpSubscriptionLagSource() private VictoriaMetricsMetricsProvider prometheusMetricsProvider = new VictoriaMetricsMetricsProvider( - client, "hermes_consumers", "hermes_frontend", "service=~'hermes'"); + client, "hermes_consumers", "hermes_frontend", "service=~'hermes'") private HybridSubscriptionMetricsRepository repository = new HybridSubscriptionMetricsRepository(prometheusMetricsProvider, summedSharedCounter, zookeeperPaths, lagSource) @@ -31,6 +31,7 @@ class HybridPrometheusBasedSubscriptionMetricsRepositoryTest extends Specificati private static final String query = "sum by (__name__, group, topic, subscription, status_code) " + "(irate({__name__=~'hermes_consumers_subscription_delivered_total" + "|hermes_consumers_subscription_timeouts_total" + + "|hermes_consumers_subscription_retries_total" + "|hermes_consumers_subscription_throughput_bytes_total" + "|hermes_consumers_subscription_other_errors_total" + "|hermes_consumers_subscription_batches_total" + @@ -42,6 +43,7 @@ class HybridPrometheusBasedSubscriptionMetricsRepositoryTest extends Specificati client.readMetrics(query) >> MonitoringMetricsContainer.createEmpty() .addMetricValue("hermes_consumers_subscription_delivered_total", of('10')) .addMetricValue("hermes_consumers_subscription_timeouts_total", of('100')) + .addMetricValue("hermes_consumers_subscription_retries_total", of('20')) .addMetricValue("hermes_consumers_subscription_other_errors_total", of('1000')) summedSharedCounter.getValue('/hermes/groups/group/topics/topic/subscriptions/subscription/metrics/delivered') >> 100 summedSharedCounter.getValue('/hermes/groups/group/topics/topic/subscriptions/subscription/metrics/discarded') >> 1 @@ -57,6 +59,7 @@ class HybridPrometheusBasedSubscriptionMetricsRepositoryTest extends Specificati metrics.discarded == 1 metrics.volume == 16 metrics.timeouts == of("100") + metrics.retries == of("20") metrics.otherErrors == of("1000") metrics.lag == MetricLongValue.of(-1) } diff --git a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/prometheus/RestTemplatePrometheusClientTest.groovy b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/prometheus/RestTemplatePrometheusClientTest.groovy index c44a9c8fa7..6f68f65075 100644 --- a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/prometheus/RestTemplatePrometheusClientTest.groovy +++ b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/prometheus/RestTemplatePrometheusClientTest.groovy @@ -21,6 +21,7 @@ class RestTemplatePrometheusClientTest extends Specification { private static final String query = "sum by (__name__,group,topic,subscription,status_code)" + "(irate({__name__=~'hermes_consumers_subscription_delivered_total" + "|hermes_consumers_subscription_timeouts_total" + + "|hermes_consumers_subscription_retries_total" + "|hermes_consumers_subscription_throughput_bytes_total" + "|hermes_consumers_subscription_other_errors_total" + "|hermes_consumers_subscription_batches_total" + @@ -34,13 +35,13 @@ class RestTemplatePrometheusClientTest extends Specification { private RestTemplatePrometheusClient client void setup() { - RestTemplate restTemplate = new RestTemplate(); - client = new RestTemplatePrometheusClient(restTemplate, URI.create("http://localhost:$PROMETHEUS_HTTP_PORT"),); + RestTemplate restTemplate = new RestTemplate() + client = new RestTemplatePrometheusClient(restTemplate, URI.create("http://localhost:$PROMETHEUS_HTTP_PORT")) } def "should get metrics for path"() { given: - mockPrometheus(query, "full_response.json"); + mockPrometheus(query, "full_response.json") when: MonitoringMetricsContainer metrics = client.readMetrics(query) @@ -48,6 +49,7 @@ class RestTemplatePrometheusClientTest extends Specification { then: metrics.metricValue("hermes_consumers_subscription_delivered_total") == of("1.0") metrics.metricValue("hermes_consumers_subscription_timeouts_total") == of("2.0") + metrics.metricValue("hermes_consumers_subscription_retries_total") == of("1.0") metrics.metricValue("hermes_consumers_subscription_throughput_bytes_total") == of("3.0") metrics.metricValue("hermes_consumers_subscription_other_errors_total") == of("4.0") metrics.metricValue("hermes_consumers_subscription_batches_total") == of("5.0") @@ -66,6 +68,7 @@ class RestTemplatePrometheusClientTest extends Specification { then: metrics.metricValue("hermes_consumers_subscription_delivered_total") == of("0.0") metrics.metricValue("hermes_consumers_subscription_timeouts_total") == of("2.0") + metrics.metricValue("hermes_consumers_subscription_retries_total") == of("1.0") metrics.metricValue("hermes_consumers_subscription_throughput_bytes_total") == of("3.0") metrics.metricValue("hermes_consumers_subscription_other_errors_total") == of("4.0") metrics.metricValue("hermes_consumers_subscription_batches_total") == of("5.0") diff --git a/hermes-management/src/test/resources/prometheus-stubs/__files/full_response.json b/hermes-management/src/test/resources/prometheus-stubs/__files/full_response.json index 9a9f457be7..7f813819d0 100644 --- a/hermes-management/src/test/resources/prometheus-stubs/__files/full_response.json +++ b/hermes-management/src/test/resources/prometheus-stubs/__files/full_response.json @@ -29,6 +29,19 @@ ], "group": 1 }, + { + "metric": { + "__name__": "hermes_consumers_subscription_retries_total", + "group": "pl.allegro.tech.hermes", + "subscription": "hermesSubscription", + "topic": "hermesTopic" + }, + "value": [ + 1692281425.609, + "1" + ], + "group": 1 + }, { "metric": { "__name__": "hermes_consumers_subscription_throughput_bytes_total", diff --git a/hermes-management/src/test/resources/prometheus-stubs/__files/partial_response.json b/hermes-management/src/test/resources/prometheus-stubs/__files/partial_response.json index a0f61afb2a..a2da55c2b5 100644 --- a/hermes-management/src/test/resources/prometheus-stubs/__files/partial_response.json +++ b/hermes-management/src/test/resources/prometheus-stubs/__files/partial_response.json @@ -16,6 +16,19 @@ ], "group": 1 }, + { + "metric": { + "__name__": "hermes_consumers_subscription_retries_total", + "group": "pl.allegro.tech.hermes", + "subscription": "hermesSubscription", + "topic": "hermesTopic" + }, + "value": [ + 1692281425.609, + "1" + ], + "group": 1 + }, { "metric": { "__name__": "hermes_consumers_subscription_throughput_bytes_total", diff --git a/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/prometheus/PrometheusExtension.java b/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/prometheus/PrometheusExtension.java index 78a8bb38a3..8daf23be19 100644 --- a/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/prometheus/PrometheusExtension.java +++ b/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/prometheus/PrometheusExtension.java @@ -52,7 +52,7 @@ public void stubSubscriptionMetrics(SubscriptionMetrics metrics) { String query = """ sum by (__name__, group, topic, subscription, status_code) ( irate( - {__name__=~'hermes_consumers_subscription_delivered_total|hermes_consumers_subscription_timeouts_total|hermes_consumers_subscription_throughput_bytes_total|hermes_consumers_subscription_other_errors_total|hermes_consumers_subscription_batches_total|hermes_consumers_subscription_http_status_codes_total', group='%s', topic='%s', subscription='%s', }[1m] + {__name__=~'hermes_consumers_subscription_delivered_total|hermes_consumers_subscription_timeouts_total|hermes_consumers_subscription_retries_total|hermes_consumers_subscription_throughput_bytes_total|hermes_consumers_subscription_other_errors_total|hermes_consumers_subscription_batches_total|hermes_consumers_subscription_http_status_codes_total', group='%s', topic='%s', subscription='%s', }[1m] ) keep_metric_names ) """ From 1460bbb1ba65c78f78ccd9186194000f7dee2ade Mon Sep 17 00:00:00 2001 From: Dawid Urbanowski Date: Mon, 8 Apr 2024 10:41:58 +0200 Subject: [PATCH 38/87] remove crowd from list of supported owner sources (#1840) --- .../hermes/api/CrowdGroupDescription.java | 18 --- .../allegro/tech/hermes/api/CrowdGroups.java | 20 ---- .../pl/allegro/tech/hermes/api/ErrorCode.java | 1 - hermes-console/json-server/db.json | 9 -- .../management/config/CrowdConfiguration.java | 69 ----------- .../management/config/CrowdProperties.java | 110 ------------------ .../config/console/ConsoleProperties.java | 2 +- .../domain/owner/CrowdOwnerSource.java | 58 --------- .../crowd/CachedCrowdClient.java | 38 ------ .../infrastructure/crowd/CrowdClient.java | 23 ---- .../infrastructure/crowd/RestCrowdClient.java | 59 ---------- .../domain/owner/CrowdOwnerSourceSpec.groovy | 41 ------- .../setup/CrowdExtension.java | 87 -------------- .../setup/HermesExtension.java | 5 - .../setup/HermesManagementTestApp.java | 10 -- .../CrowdOwnerSourceIntegrationTest.java | 86 -------------- 16 files changed, 1 insertion(+), 635 deletions(-) delete mode 100644 hermes-api/src/main/java/pl/allegro/tech/hermes/api/CrowdGroupDescription.java delete mode 100644 hermes-api/src/main/java/pl/allegro/tech/hermes/api/CrowdGroups.java delete mode 100644 hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/CrowdConfiguration.java delete mode 100644 hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/CrowdProperties.java delete mode 100644 hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/owner/CrowdOwnerSource.java delete mode 100644 hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/crowd/CachedCrowdClient.java delete mode 100644 hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/crowd/CrowdClient.java delete mode 100644 hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/crowd/RestCrowdClient.java delete mode 100644 hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/domain/owner/CrowdOwnerSourceSpec.groovy delete mode 100644 integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/setup/CrowdExtension.java delete mode 100644 integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/CrowdOwnerSourceIntegrationTest.java diff --git a/hermes-api/src/main/java/pl/allegro/tech/hermes/api/CrowdGroupDescription.java b/hermes-api/src/main/java/pl/allegro/tech/hermes/api/CrowdGroupDescription.java deleted file mode 100644 index 8e0a65db96..0000000000 --- a/hermes-api/src/main/java/pl/allegro/tech/hermes/api/CrowdGroupDescription.java +++ /dev/null @@ -1,18 +0,0 @@ -package pl.allegro.tech.hermes.api; - -import com.fasterxml.jackson.annotation.JsonIgnoreProperties; -import com.fasterxml.jackson.annotation.JsonProperty; - -@JsonIgnoreProperties(ignoreUnknown = true) -public class CrowdGroupDescription { - - private final String name; - - public CrowdGroupDescription(@JsonProperty("name") String name) { - this.name = name; - } - - public String getName() { - return name; - } -} diff --git a/hermes-api/src/main/java/pl/allegro/tech/hermes/api/CrowdGroups.java b/hermes-api/src/main/java/pl/allegro/tech/hermes/api/CrowdGroups.java deleted file mode 100644 index 16bc708a10..0000000000 --- a/hermes-api/src/main/java/pl/allegro/tech/hermes/api/CrowdGroups.java +++ /dev/null @@ -1,20 +0,0 @@ -package pl.allegro.tech.hermes.api; - -import com.fasterxml.jackson.annotation.JsonIgnoreProperties; -import com.fasterxml.jackson.annotation.JsonProperty; - -import java.util.List; - -@JsonIgnoreProperties(ignoreUnknown = true) -public class CrowdGroups { - - private final List crowdGroupDescriptions; - - public CrowdGroups(@JsonProperty("groups") List crowdGroupDescriptions) { - this.crowdGroupDescriptions = crowdGroupDescriptions; - } - - public List getCrowdGroupDescriptions() { - return crowdGroupDescriptions; - } -} diff --git a/hermes-api/src/main/java/pl/allegro/tech/hermes/api/ErrorCode.java b/hermes-api/src/main/java/pl/allegro/tech/hermes/api/ErrorCode.java index 9e48226597..c15dda758b 100644 --- a/hermes-api/src/main/java/pl/allegro/tech/hermes/api/ErrorCode.java +++ b/hermes-api/src/main/java/pl/allegro/tech/hermes/api/ErrorCode.java @@ -48,7 +48,6 @@ public enum ErrorCode { SUBSCRIPTION_ENDPOINT_ADDRESS_CHANGE_EXCEPTION(INTERNAL_SERVER_ERROR), OAUTH_PROVIDER_NOT_EXISTS(NOT_FOUND), OAUTH_PROVIDER_ALREADY_EXISTS(BAD_REQUEST), - CROWD_GROUPS_COULD_NOT_BE_LOADED(INTERNAL_SERVER_ERROR), TOPIC_BLACKLISTED(FORBIDDEN), THROUGHPUT_QUOTA_VIOLATION(429), TOPIC_NOT_UNBLACKLISTED(BAD_REQUEST), diff --git a/hermes-console/json-server/db.json b/hermes-console/json-server/db.json index 607194a4a7..8c2720872d 100644 --- a/hermes-console/json-server/db.json +++ b/hermes-console/json-server/db.json @@ -155,11 +155,6 @@ "autocomplete": true, "deprecated": false }, - { - "name": "Crowd Catalog", - "autocomplete": false, - "deprecated": false - }, { "name": "Deprecated catalog", "autocomplete": true, @@ -593,10 +588,6 @@ }, "owner": { "sources": [ - { - "name": "Crowd Catalog", - "placeholder": "Crowd group (or groups separated by ',')" - }, { "name": "Service Catalog", "placeholder": "Service name from Service Catalog" diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/CrowdConfiguration.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/CrowdConfiguration.java deleted file mode 100644 index 8539c9b2e7..0000000000 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/CrowdConfiguration.java +++ /dev/null @@ -1,69 +0,0 @@ -package pl.allegro.tech.hermes.management.config; - -import org.apache.hc.client5.http.classic.HttpClient; -import org.apache.hc.client5.http.config.ConnectionConfig; -import org.apache.hc.client5.http.config.RequestConfig; -import org.apache.hc.client5.http.impl.classic.HttpClientBuilder; -import org.apache.hc.client5.http.impl.io.PoolingHttpClientConnectionManager; -import org.apache.hc.client5.http.impl.io.PoolingHttpClientConnectionManagerBuilder; -import org.apache.hc.core5.util.Timeout; -import org.springframework.beans.factory.annotation.Qualifier; -import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty; -import org.springframework.boot.context.properties.ConfigurationProperties; -import org.springframework.boot.context.properties.EnableConfigurationProperties; -import org.springframework.context.annotation.Bean; -import org.springframework.context.annotation.Configuration; -import org.springframework.core.annotation.Order; -import org.springframework.http.client.ClientHttpRequestFactory; -import org.springframework.http.client.HttpComponentsClientHttpRequestFactory; -import org.springframework.web.client.RestTemplate; -import pl.allegro.tech.hermes.management.domain.owner.CrowdOwnerSource; -import pl.allegro.tech.hermes.management.domain.owner.PlaintextOwnerSource; -import pl.allegro.tech.hermes.management.infrastructure.crowd.CachedCrowdClient; -import pl.allegro.tech.hermes.management.infrastructure.crowd.RestCrowdClient; - -@Configuration -@EnableConfigurationProperties(CrowdProperties.class) -@ConditionalOnProperty("owner.crowd.enabled") -public class CrowdConfiguration { - - @Bean("managementRequestFactory") - @ConfigurationProperties(prefix = "management.rest-template") - public ClientHttpRequestFactory clientHttpRequestFactory() { - return new HttpComponentsClientHttpRequestFactory(); - } - - @Bean(name = "crowdRestTemplate") - public RestTemplate restTemplate(CrowdProperties properties) { - PoolingHttpClientConnectionManager connectionManager = PoolingHttpClientConnectionManagerBuilder.create() - .setMaxConnTotal(properties.getMaxConnections()) - .setMaxConnPerRoute(properties.getMaxConnectionsPerRoute()) - .setDefaultConnectionConfig( - ConnectionConfig.custom() - .setSocketTimeout(Timeout.ofMilliseconds(properties.getSocketTimeoutMillis())) - .setConnectTimeout(Timeout.ofMilliseconds(properties.getConnectionTimeoutMillis())) - .build()) - .build(); - - RequestConfig requestConfig = RequestConfig.custom() - .setConnectTimeout(Timeout.ofMilliseconds(properties.getConnectionTimeoutMillis())) - .setResponseTimeout(Timeout.ofMilliseconds(properties.getSocketTimeoutMillis())) - .build(); - - HttpClient client = HttpClientBuilder.create() - .setDefaultRequestConfig(requestConfig) - .setConnectionManager(connectionManager) - .build(); - - ClientHttpRequestFactory clientHttpRequestFactory = new HttpComponentsClientHttpRequestFactory(client); - - return new RestTemplate(clientHttpRequestFactory); - } - - @Bean - @Order(PlaintextOwnerSource.ORDER + 1) - public CrowdOwnerSource crowdOwnerSource(CrowdProperties crowdProperties, @Qualifier("crowdRestTemplate") RestTemplate restTemplate) { - return new CrowdOwnerSource(new CachedCrowdClient(new RestCrowdClient(restTemplate, crowdProperties), crowdProperties)); - } - -} diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/CrowdProperties.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/CrowdProperties.java deleted file mode 100644 index 93c423fcc5..0000000000 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/CrowdProperties.java +++ /dev/null @@ -1,110 +0,0 @@ -package pl.allegro.tech.hermes.management.config; - -import org.springframework.boot.context.properties.ConfigurationProperties; - -import java.net.URL; - -@ConfigurationProperties(prefix = "owner.crowd") -public class CrowdProperties { - - private boolean enabled = false; - - private String userName; - - private String password; - - private URL path; - - private Long cacheDurationSeconds = 300L; - - private Long cacheSize = 1000L; - - private int connectionTimeoutMillis = 1000; - - private int socketTimeoutMillis = 2000; - - private int maxConnections = 100; - - private int maxConnectionsPerRoute = 10; - - - public boolean isEnabled() { - return enabled; - } - - public void setEnabled(boolean enabled) { - this.enabled = enabled; - } - - public String getUserName() { - return userName; - } - - public void setUserName(String userName) { - this.userName = userName; - } - - public String getPassword() { - return password; - } - - public void setPassword(String password) { - this.password = password; - } - - public URL getPath() { - return path; - } - - public void setPath(URL path) { - this.path = path; - } - - public Long getCacheDurationSeconds() { - return cacheDurationSeconds; - } - - public void setCacheDurationSeconds(Long cacheDurationSeconds) { - this.cacheDurationSeconds = cacheDurationSeconds; - } - - public Long getCacheSize() { - return cacheSize; - } - - public void setCacheSize(Long cacheSize) { - this.cacheSize = cacheSize; - } - - public int getConnectionTimeoutMillis() { - return connectionTimeoutMillis; - } - - public void setConnectionTimeoutMillis(int connectionTimeoutMillis) { - this.connectionTimeoutMillis = connectionTimeoutMillis; - } - - public int getSocketTimeoutMillis() { - return socketTimeoutMillis; - } - - public void setSocketTimeoutMillis(int socketTimeoutMillis) { - this.socketTimeoutMillis = socketTimeoutMillis; - } - - public int getMaxConnections() { - return maxConnections; - } - - public void setMaxConnections(int maxConnections) { - this.maxConnections = maxConnections; - } - - public int getMaxConnectionsPerRoute() { - return maxConnectionsPerRoute; - } - - public void setMaxConnectionsPerRoute(int maxConnectionsPerRoute) { - this.maxConnectionsPerRoute = maxConnectionsPerRoute; - } -} diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/console/ConsoleProperties.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/console/ConsoleProperties.java index 17f15c1fb5..e6b5435a4c 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/console/ConsoleProperties.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/console/ConsoleProperties.java @@ -607,7 +607,7 @@ public void setLabel(String label) { public static final class Owner { private List sources = Lists.newArrayList( - new OwnerSource("Crowd", "Crowd group (or groups separated by ',')")); + new OwnerSource("Plaintext", "Name of the owner")); public List getSources() { return sources; diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/owner/CrowdOwnerSource.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/owner/CrowdOwnerSource.java deleted file mode 100644 index bfb00aa25e..0000000000 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/owner/CrowdOwnerSource.java +++ /dev/null @@ -1,58 +0,0 @@ -package pl.allegro.tech.hermes.management.domain.owner; - -import com.google.common.base.Strings; -import pl.allegro.tech.hermes.api.Owner; -import pl.allegro.tech.hermes.management.infrastructure.crowd.CrowdClient; - -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Optional; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -public class CrowdOwnerSource implements OwnerSource { - - public static final String NAME = "Crowd"; - - private final CrowdClient crowdClient; - - public CrowdOwnerSource(CrowdClient crowdClient) { - this.crowdClient = crowdClient; - } - - @Override - public String name() { - return NAME; - } - - @Override - public boolean exists(String ownerId) { - return true; - } - - @Override - public Owner get(String id) { - return new Owner(id, id); - } - - @Override - public Optional autocompletion() { - return Optional.of(this::ownersMatching); - } - - private List ownersMatching(String searchString) { - if (Strings.isNullOrEmpty(searchString)) { - return Collections.emptyList(); - } - - List stableBase = Arrays.stream(searchString.split(",")).map(String::trim).collect(Collectors.toList()); - String searchedPrefix = stableBase.remove(stableBase.size() - 1); - return crowdClient.getGroups(searchedPrefix) - .stream() - .map(foundName -> Stream.concat(stableBase.stream(), Stream.of(foundName)).collect(Collectors.joining(", "))) - .map(x -> new Owner(x, x)) - .collect(Collectors.toList()); - } - -} diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/crowd/CachedCrowdClient.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/crowd/CachedCrowdClient.java deleted file mode 100644 index 9a518abd97..0000000000 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/crowd/CachedCrowdClient.java +++ /dev/null @@ -1,38 +0,0 @@ -package pl.allegro.tech.hermes.management.infrastructure.crowd; - -import com.google.common.cache.CacheBuilder; -import com.google.common.cache.CacheLoader; -import com.google.common.cache.LoadingCache; -import com.google.common.util.concurrent.UncheckedExecutionException; -import pl.allegro.tech.hermes.management.config.CrowdProperties; - -import java.util.List; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; - -public class CachedCrowdClient implements CrowdClient { - - private final LoadingCache> cache; - - public CachedCrowdClient(CrowdClient crowdClient, CrowdProperties crowdProperties) { - this.cache = CacheBuilder.newBuilder() - .expireAfterWrite(crowdProperties.getCacheDurationSeconds(), TimeUnit.SECONDS) - .maximumSize(crowdProperties.getCacheSize()) - .build(new CacheLoader>() { - @Override - public List load(String searchString) throws Exception { - return crowdClient.getGroups(searchString); - } - }); - } - - @Override - public List getGroups(String searchString) { - try { - return cache.get(searchString); - } catch (ExecutionException | UncheckedExecutionException e) { - throw new CouldNotLoadCrowdGroupsException(e); - } - } -} - diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/crowd/CrowdClient.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/crowd/CrowdClient.java deleted file mode 100644 index 762bbfcba1..0000000000 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/crowd/CrowdClient.java +++ /dev/null @@ -1,23 +0,0 @@ -package pl.allegro.tech.hermes.management.infrastructure.crowd; - -import pl.allegro.tech.hermes.api.ErrorCode; -import pl.allegro.tech.hermes.common.exception.HermesException; - -import java.util.List; - -public interface CrowdClient { - - List getGroups(String searchString) throws CouldNotLoadCrowdGroupsException; - - class CouldNotLoadCrowdGroupsException extends HermesException { - - CouldNotLoadCrowdGroupsException(Throwable t) { - super(t); - } - - @Override - public ErrorCode getCode() { - return ErrorCode.CROWD_GROUPS_COULD_NOT_BE_LOADED; - } - } -} diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/crowd/RestCrowdClient.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/crowd/RestCrowdClient.java deleted file mode 100644 index 4f84f575cb..0000000000 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/crowd/RestCrowdClient.java +++ /dev/null @@ -1,59 +0,0 @@ -package pl.allegro.tech.hermes.management.infrastructure.crowd; - -import com.google.common.io.BaseEncoding; -import org.springframework.http.HttpEntity; -import org.springframework.http.HttpHeaders; -import org.springframework.http.HttpMethod; -import org.springframework.http.MediaType; -import org.springframework.web.client.RestTemplate; -import pl.allegro.tech.hermes.api.CrowdGroupDescription; -import pl.allegro.tech.hermes.api.CrowdGroups; -import pl.allegro.tech.hermes.management.config.CrowdProperties; - -import java.nio.charset.StandardCharsets; -import java.util.Collections; -import java.util.List; -import java.util.stream.Collectors; - -public class RestCrowdClient implements CrowdClient { - - private static final String CROWD_API_SUFFIX = "/rest/usermanagement/1"; - - private static final String GROUP_SEARCH_SUFFIX = "/search?entity-type=group&restriction=name=\"*{groupName}*\""; - - private final RestTemplate restTemplate; - - private final CrowdProperties crowdProperties; - - private final HttpEntity entity; - - private final String groupSearchUrl; - - public RestCrowdClient(RestTemplate restTemplate, CrowdProperties crowdProperties) { - this.restTemplate = restTemplate; - this.crowdProperties = crowdProperties; - this.groupSearchUrl = this.crowdProperties.getPath() + CROWD_API_SUFFIX + GROUP_SEARCH_SUFFIX; - this.entity = configureEntity(); - } - - private HttpEntity configureEntity() { - HttpHeaders headers = new HttpHeaders(); - headers.setAccept(Collections.singletonList(MediaType.APPLICATION_JSON)); - headers.set(HttpHeaders.AUTHORIZATION, buildBasicAuthorizationValue()); - return new HttpEntity<>(headers); - } - - private String buildBasicAuthorizationValue() { - String encodedValue = String.join(":", crowdProperties.getUserName(), crowdProperties.getPassword()); - return "Basic " + BaseEncoding.base64().encode(encodedValue.getBytes(StandardCharsets.UTF_8)); - } - - @Override - public List getGroups(String searchString) { - CrowdGroups crowdGroups = restTemplate.exchange(this.groupSearchUrl, HttpMethod.GET, this.entity, CrowdGroups.class, searchString) - .getBody(); - return crowdGroups.getCrowdGroupDescriptions().stream() - .map(CrowdGroupDescription::getName) - .collect(Collectors.toList()); - } -} diff --git a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/domain/owner/CrowdOwnerSourceSpec.groovy b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/domain/owner/CrowdOwnerSourceSpec.groovy deleted file mode 100644 index 63c299b107..0000000000 --- a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/domain/owner/CrowdOwnerSourceSpec.groovy +++ /dev/null @@ -1,41 +0,0 @@ -package pl.allegro.tech.hermes.management.domain.owner - -import pl.allegro.tech.hermes.api.Owner -import pl.allegro.tech.hermes.management.infrastructure.crowd.CrowdClient -import spock.lang.Specification - -class CrowdOwnerSourceSpec extends Specification { - - def client = Stub(CrowdClient) { - getGroups("Tea") >> ["Team A", "Team B"] - getGroups("None") >> [] - } - - def source = new CrowdOwnerSource(client) - - def "should search matching owners"() { - expect: - source.ownersMatching("Tea") == [new Owner("Team A", "Team A"), new Owner("Team B", "Team B")] - } - - def "should search matching owners with already matched groups"() { - expect: - source.ownersMatching(searchString) == [ - new Owner("Team A, Team A", "Team A, Team A"), - new Owner("Team A, Team B", "Team A, Team B") - ] - where: - searchString << ["Team A, Tea", "Team A,Tea", " Team A, Tea "] - } - - def "should return empty matches for empty search string"() { - expect: - source.ownersMatching("") == [] - } - - def "should return empty matches for null search string"() { - expect: - source.ownersMatching(null) == [] - } - -} diff --git a/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/setup/CrowdExtension.java b/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/setup/CrowdExtension.java deleted file mode 100644 index 0797a141ca..0000000000 --- a/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/setup/CrowdExtension.java +++ /dev/null @@ -1,87 +0,0 @@ -package pl.allegro.tech.hermes.integrationtests.setup; - -import com.github.tomakehurst.wiremock.WireMockServer; -import jakarta.ws.rs.core.HttpHeaders; -import jakarta.ws.rs.core.MediaType; -import org.junit.jupiter.api.extension.AfterAllCallback; -import org.junit.jupiter.api.extension.AfterEachCallback; -import org.junit.jupiter.api.extension.BeforeAllCallback; -import org.junit.jupiter.api.extension.ExtensionContext; - -import java.time.Duration; - -import static com.github.tomakehurst.wiremock.client.WireMock.aResponse; -import static com.github.tomakehurst.wiremock.client.WireMock.equalTo; -import static com.github.tomakehurst.wiremock.client.WireMock.get; -import static com.github.tomakehurst.wiremock.client.WireMock.getRequestedFor; -import static com.github.tomakehurst.wiremock.client.WireMock.urlMatching; - -public class CrowdExtension implements BeforeAllCallback, AfterEachCallback, AfterAllCallback { - - private static final String BASE_API_PATH = "/crowd/rest/usermanagement/1/search"; - - private final WireMockServer wiremock = new WireMockServer(0); - - @Override - public void beforeAll(ExtensionContext context) { - wiremock.start(); - } - - @Override - public void afterEach(ExtensionContext context) { - wiremock.resetAll(); - } - - @Override - public void afterAll(ExtensionContext context) { - wiremock.stop(); - } - - public String getEndpoint() { - return "http://localhost:" + wiremock.port() + "/crowd"; - } - - public void stubGroups(String ... groups) { - var body = new StringBuilder("{ \"expand\": \"group\", \"groups\": ["); - for (int i = 0; i < groups.length; i++) { - body.append(groupResponse("http://main/crowd/groupname/" + i, groups[i])); - if (i < groups.length - 1) - body.append(","); - } - body.append("]}"); - wiremock.addStubMapping( - get(urlMatching(BASE_API_PATH + ".*")) - .withHeader(HttpHeaders.ACCEPT, equalTo(MediaType.APPLICATION_JSON)) - .willReturn( - aResponse() - .withStatus(200) - .withHeader(HttpHeaders.CONTENT_TYPE, MediaType.APPLICATION_JSON) - .withBody(body.toString()) - ) - .build() - ); - } - - public void assertRequestCount(int count) { - wiremock.verify(count, getRequestedFor(urlMatching(BASE_API_PATH + ".*"))); - } - - private String groupResponse(String path, String name) { - return String.format("{ \"link\": { \"href\": \"%s\", \"rel\": \"self\" }, \"name\": \"%s\"}", path, name); - } - - public void stubDelay(Duration duration) { - wiremock.addStubMapping( - get(urlMatching(BASE_API_PATH + ".*")) - .withHeader(HttpHeaders.ACCEPT, equalTo(MediaType.APPLICATION_JSON)) - .willReturn( - aResponse() - .withStatus(200) - .withHeader(HttpHeaders.CONTENT_TYPE, MediaType.APPLICATION_JSON) - .withBody("{ \"expand\": \"group\", \"groups\": []}") - .withFixedDelay((int) duration.toMillis()) - ) - .build() - ); - } -} diff --git a/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/setup/HermesExtension.java b/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/setup/HermesExtension.java index 65195fa636..7f1ce40cef 100644 --- a/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/setup/HermesExtension.java +++ b/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/setup/HermesExtension.java @@ -158,11 +158,6 @@ public HermesExtension withGooglePubSub(GooglePubSubExtension googlePubSub) { return this; } - public HermesExtension withCrowd(CrowdExtension crowd) { - management.withCrowd(crowd); - return this; - } - public HermesExtension withFrontendProperty(String name, Object value) { frontend.withProperty(name, value); return this; diff --git a/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/setup/HermesManagementTestApp.java b/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/setup/HermesManagementTestApp.java index 175a90ee6a..ffacfc1430 100644 --- a/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/setup/HermesManagementTestApp.java +++ b/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/setup/HermesManagementTestApp.java @@ -42,7 +42,6 @@ public class HermesManagementTestApp implements HermesTestApp { private SpringApplicationBuilder app = null; private List currentArgs = List.of(); private PrometheusExtension prometheus = null; - private CrowdExtension crowd = null; public HermesManagementTestApp(ZookeeperContainer hermesZookeeper, KafkaContainerCluster kafka, @@ -117,14 +116,9 @@ void withPrometheus(PrometheusExtension prometheus) { this.prometheus = prometheus; } - void withCrowd(CrowdExtension crowd) { - this.crowd = crowd; - } - @Override public void restoreDefaultSettings() { prometheus = null; - crowd = null; } @Override @@ -143,10 +137,6 @@ private List createArgs() { args.add("--prometheus.client.externalMonitoringUrl=" + prometheus.getEndpoint()); args.add("--prometheus.client.cacheTtlSeconds=0"); } - if (crowd != null) { - args.add("--owner.crowd.path=" + crowd.getEndpoint()); - args.add("--owner.crowd.enabled=true"); - } args.add("--topic.partitions=2"); args.add("--topic.uncleanLeaderElectionEnabled=false"); int smallestClusterSize = kafkaClusters.values().stream() diff --git a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/CrowdOwnerSourceIntegrationTest.java b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/CrowdOwnerSourceIntegrationTest.java deleted file mode 100644 index 8a45faae41..0000000000 --- a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/CrowdOwnerSourceIntegrationTest.java +++ /dev/null @@ -1,86 +0,0 @@ -package pl.allegro.tech.hermes.integrationtests.management; - -import org.junit.jupiter.api.Order; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.RegisterExtension; -import org.springframework.test.web.reactive.server.WebTestClient; -import pl.allegro.tech.hermes.api.ErrorCode; -import pl.allegro.tech.hermes.api.ErrorDescription; -import pl.allegro.tech.hermes.api.Owner; -import pl.allegro.tech.hermes.integrationtests.setup.CrowdExtension; -import pl.allegro.tech.hermes.integrationtests.setup.HermesExtension; -import pl.allegro.tech.hermes.management.domain.owner.CrowdOwnerSource; - -import java.time.Duration; -import java.util.List; - -import static org.assertj.core.api.Assertions.assertThat; - -public class CrowdOwnerSourceIntegrationTest { - - @Order(0) - @RegisterExtension - public static final CrowdExtension crowd = new CrowdExtension(); - - @Order(1) - @RegisterExtension - public static final HermesExtension hermes = new HermesExtension() - .withCrowd(crowd); - - @Test - public void shouldCrowdServiceBeCalledOnce() { - //given - crowd.stubGroups("Scrum A", "Scrum B"); - - //when - hermes.api().searchOwners(CrowdOwnerSource.NAME, "Scrum"); - - //then - crowd.assertRequestCount(1); - } - - @Test - public void shouldReturnTwoResultsFromCrowd() { - //given - crowd.stubGroups("Scrum A", "Scrum B"); - - //when - WebTestClient.ResponseSpec response = hermes.api().searchOwners(CrowdOwnerSource.NAME, "Scrum"); - - //then - List groups = response - .expectStatus().isOk() - .expectBodyList(Owner.class).returnResult().getResponseBody(); - assertThat(groups).isNotNull(); - assertThat(groups.stream().map(Owner::getId)).containsExactly("Scrum A", "Scrum B"); - } - - @Test - public void shouldReturnNoResultsFromCrowd() { - //given - crowd.stubGroups(); - - //when - WebTestClient.ResponseSpec response = hermes.api().searchOwners(CrowdOwnerSource.NAME, "Non Matching"); - - //then - response.expectStatus().isOk() - .expectBodyList(Owner.class).hasSize(0); - } - - @Test - public void shouldGetAnExceptionOnReadTimeout() { - //given - crowd.stubDelay(Duration.ofMillis(3500)); - - //when - WebTestClient.ResponseSpec response = hermes.api().searchOwners(CrowdOwnerSource.NAME, "Non Matching"); - - //then - ErrorDescription error = response - .expectStatus().is5xxServerError() - .expectBody(ErrorDescription.class).returnResult().getResponseBody(); - assertThat(error).isNotNull(); - assertThat(error.getCode()).isEqualTo(ErrorCode.CROWD_GROUPS_COULD_NOT_BE_LOADED); - } -} From f38952c2a928081ab02fca84a8d42e121f5e484b Mon Sep 17 00:00:00 2001 From: Maciej Moscicki Date: Mon, 8 Apr 2024 13:29:42 +0200 Subject: [PATCH 39/87] add multi dc broker producer (#1823) Co-authored-by: Piotr Rzysko --- config/checkstyle/suppressions.xml | 3 + .../pl/allegro/tech/hermes/api/ErrorCode.java | 5 +- .../environment/HermesServerFactory.java | 9 +- .../hermes/common/metric/GaugeRegistrar.java | 7 + .../tech/hermes/common/metric/Meters.java | 2 + .../hermes/common/metric/ProducerMetrics.java | 75 +++---- .../hermes/common/metric/TopicMetrics.java | 16 +- .../ZookeeperCounterReporterTest.java | 2 +- .../frontend/buffer/BackupMessagesLoader.java | 16 +- .../buffer/SimpleExecutionCallback.java | 27 --- .../FailFastKafkaProducerProperties.java | 208 ++++++++++++++++++ .../config/FrontendConfiguration.java | 10 +- .../config/FrontendProducerConfiguration.java | 65 +++++- .../FrontendPublishingConfiguration.java | 15 +- .../config/HandlersChainProperties.java | 11 + .../config/KafkaProducerProperties.java | 11 + .../config/ReadinessConfiguration.java | 3 +- .../hermes/frontend/metric/CachedTopic.java | 19 +- .../producer/BrokerLatencyReporter.java | 8 +- ...oRemoteDatacenterAwareMessageProducer.java | 37 ++++ .../kafka/KafkaBrokerMessageProducer.java | 104 --------- .../producer/kafka/KafkaMessageSender.java | 103 +++++++-- .../producer/kafka/KafkaMessageSenders.java | 8 +- .../kafka/KafkaMessageSendersFactory.java | 20 +- .../kafka/KafkaProducerParameters.java | 2 + .../kafka/LocalDatacenterMessageProducer.java | 73 ++++++ .../kafka/MultiDatacenterMessageProducer.java | 166 ++++++++++++++ .../kafka/ProducerMetadataLoadingJob.java | 17 +- .../publishing/PublishingCallback.java | 25 ++- .../handlers/HandlersChainFactory.java | 11 +- .../handlers/HandlersChainParameters.java | 17 ++ .../handlers/MessageReadHandler.java | 14 +- .../handlers/PublishingHandler.java | 45 ++-- .../publishing/handlers/TimeoutHandler.java | 29 +++ .../handlers/end/MessageEndProcessor.java | 23 +- .../publishing/message/MessageState.java | 8 +- .../readiness/AdminReadinessService.java | 2 +- ...nterMessageProducerIntegrationTest.groovy} | 25 ++- .../buffer/BackupMessagesLoaderTest.java | 14 +- ...> LocalDatacenterMessageProducerTest.java} | 24 +- .../integration/FrontendTestClient.java | 11 +- .../client/integration/HermesTestClient.java | 4 + .../PrometheusMetricsAssertion.java | 31 ++- .../setup/HermesFrontendTestApp.java | 25 ++- .../BrokerLatencyReportingTest.java | 34 ++- .../RemoteDatacenterProduceFallbackTest.java | 193 ++++++++++++++++ 46 files changed, 1188 insertions(+), 389 deletions(-) delete mode 100644 hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/buffer/SimpleExecutionCallback.java create mode 100644 hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FailFastKafkaProducerProperties.java create mode 100644 hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/FallbackToRemoteDatacenterAwareMessageProducer.java delete mode 100644 hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaBrokerMessageProducer.java create mode 100644 hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducer.java create mode 100644 hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/MultiDatacenterMessageProducer.java rename hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/producer/kafka/{KafkaBrokerMessageProducerIntegrationTest.groovy => LocalDatacenterMessageProducerIntegrationTest.groovy} (92%) rename hermes-frontend/src/test/java/pl/allegro/tech/hermes/frontend/producer/kafka/{KafkaBrokerMessageProducerTest.java => LocalDatacenterMessageProducerTest.java} (89%) create mode 100644 integration-tests/src/slowIntegrationTest/java/pl/allegro/tech/hermes/integrationtests/RemoteDatacenterProduceFallbackTest.java diff --git a/config/checkstyle/suppressions.xml b/config/checkstyle/suppressions.xml index 142e61f8c7..0375f028d1 100644 --- a/config/checkstyle/suppressions.xml +++ b/config/checkstyle/suppressions.xml @@ -36,6 +36,9 @@ + void registerGauge(String name, ToDoubleFunction f) { registerGauge(name, name, stateObj, f); } + + public void registerGauge(String name, + T stateObj, + ToDoubleFunction f, + Iterable tags) { + registerGauge(name, name, stateObj, f, tags); + } } diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/Meters.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/Meters.java index ab18d669c1..9fd5a360f3 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/Meters.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/Meters.java @@ -36,6 +36,8 @@ public class Meters { public static final String DELAYED_PROCESSING = "delayed-processing"; public static final String TOPIC_DELAYED_PROCESSING = DELAYED_PROCESSING + "." + GROUP + "." + TOPIC; + public static final String TOPIC_DUPLICATED_MESSAGE = "duplicated-message" + "." + GROUP + "." + TOPIC; + public static final String OAUTH_SUBSCRIPTION_TOKEN_REQUEST = "oauth.subscription." + GROUP + "." + TOPIC + "." + SUBSCRIPTION + ".token-request." + OAUTH_PROVIDER_NAME; diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/ProducerMetrics.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/ProducerMetrics.java index 14b5edd14f..a89c181da1 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/ProducerMetrics.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/ProducerMetrics.java @@ -7,7 +7,6 @@ import java.util.function.ToDoubleFunction; import static pl.allegro.tech.hermes.common.metric.Gauges.INFLIGHT_REQUESTS; -import static pl.allegro.tech.hermes.common.metric.HermesMetrics.escapeDots; public class ProducerMetrics { private final HermesMetrics hermesMetrics; @@ -20,53 +19,53 @@ public ProducerMetrics(HermesMetrics hermesMetrics, MeterRegistry meterRegistry) this.gaugeRegistrar = new GaugeRegistrar(meterRegistry, hermesMetrics); } - public void registerAckAllTotalBytesGauge(T stateObj, ToDoubleFunction f) { - gaugeRegistrar.registerGauge(ACK_ALL_BUFFER_TOTAL_BYTES, stateObj, f); + public void registerAckAllTotalBytesGauge(T stateObj, ToDoubleFunction f, String sender, String datacenter) { + gaugeRegistrar.registerGauge(ACK_ALL_BUFFER_TOTAL_BYTES, stateObj, f, tags(sender, datacenter)); } - public void registerAckLeaderTotalBytesGauge(T stateObj, ToDoubleFunction f) { - gaugeRegistrar.registerGauge(ACK_LEADER_BUFFER_TOTAL_BYTES, stateObj, f); + public void registerAckLeaderTotalBytesGauge(T stateObj, ToDoubleFunction f, String sender, String datacenter) { + gaugeRegistrar.registerGauge(ACK_LEADER_BUFFER_TOTAL_BYTES, stateObj, f, tags(sender, datacenter)); } - public void registerAckAllAvailableBytesGauge(T stateObj, ToDoubleFunction f) { - gaugeRegistrar.registerGauge(ACK_ALL_BUFFER_AVAILABLE_BYTES, stateObj, f); + public void registerAckAllAvailableBytesGauge(T stateObj, ToDoubleFunction f, String sender, String datacenter) { + gaugeRegistrar.registerGauge(ACK_ALL_BUFFER_AVAILABLE_BYTES, stateObj, f, tags(sender, datacenter)); } - public void registerAckLeaderAvailableBytesGauge(T stateObj, ToDoubleFunction f) { - gaugeRegistrar.registerGauge(ACK_LEADER_BUFFER_AVAILABLE_BYTES, stateObj, f); + public void registerAckLeaderAvailableBytesGauge(T stateObj, ToDoubleFunction f, String sender, String datacenter) { + gaugeRegistrar.registerGauge(ACK_LEADER_BUFFER_AVAILABLE_BYTES, stateObj, f, tags(sender, datacenter)); } - public void registerAckAllCompressionRateGauge(T stateObj, ToDoubleFunction f) { - gaugeRegistrar.registerGauge(ACK_ALL_COMPRESSION_RATE, stateObj, f); + public void registerAckAllCompressionRateGauge(T stateObj, ToDoubleFunction f, String sender, String datacenter) { + gaugeRegistrar.registerGauge(ACK_ALL_COMPRESSION_RATE, stateObj, f, tags(sender, datacenter)); } - public void registerAckLeaderCompressionRateGauge(T stateObj, ToDoubleFunction f) { - gaugeRegistrar.registerGauge(ACK_LEADER_COMPRESSION_RATE, stateObj, f); + public void registerAckLeaderCompressionRateGauge(T stateObj, ToDoubleFunction f, String sender, String datacenter) { + gaugeRegistrar.registerGauge(ACK_LEADER_COMPRESSION_RATE, stateObj, f, tags(sender, datacenter)); } - public void registerAckAllFailedBatchesGauge(T stateObj, ToDoubleFunction f) { - gaugeRegistrar.registerGauge(ACK_ALL_FAILED_BATCHES_TOTAL, stateObj, f); + public void registerAckAllFailedBatchesGauge(T stateObj, ToDoubleFunction f, String sender, String datacenter) { + gaugeRegistrar.registerGauge(ACK_ALL_FAILED_BATCHES_TOTAL, stateObj, f, tags(sender, datacenter)); } - public void registerAckLeaderFailedBatchesGauge(T stateObj, ToDoubleFunction f) { - gaugeRegistrar.registerGauge(ACK_LEADER_FAILED_BATCHES_TOTAL, stateObj, f); + public void registerAckLeaderFailedBatchesGauge(T stateObj, ToDoubleFunction f, String sender, String datacenter) { + gaugeRegistrar.registerGauge(ACK_LEADER_FAILED_BATCHES_TOTAL, stateObj, f, tags(sender, datacenter)); } - public void registerAckAllMetadataAgeGauge(T stateObj, ToDoubleFunction f) { - registerTimeGauge(stateObj, f, ACK_ALL_METADATA_AGE, ACK_ALL_METADATA_AGE, Tags.empty(), TimeUnit.SECONDS); + public void registerAckAllMetadataAgeGauge(T stateObj, ToDoubleFunction f, String sender, String datacenter) { + registerTimeGauge(stateObj, f, ACK_ALL_METADATA_AGE, ACK_ALL_METADATA_AGE, tags(sender, datacenter), TimeUnit.SECONDS); } - public void registerAckLeaderMetadataAgeGauge(T stateObj, ToDoubleFunction f) { - registerTimeGauge(stateObj, f, ACK_LEADER_METADATA_AGE, ACK_LEADER_METADATA_AGE, Tags.empty(), TimeUnit.SECONDS); + public void registerAckLeaderMetadataAgeGauge(T stateObj, ToDoubleFunction f, String sender, String datacenter) { + registerTimeGauge(stateObj, f, ACK_LEADER_METADATA_AGE, ACK_LEADER_METADATA_AGE, tags(sender, datacenter), TimeUnit.SECONDS); } - public void registerAckAllRecordQueueTimeMaxGauge(T stateObj, ToDoubleFunction f) { - registerTimeGauge(stateObj, f, ACK_ALL_RECORD_QUEUE_TIME_MAX, ACK_ALL_RECORD_QUEUE_TIME_MAX, Tags.empty(), TimeUnit.MILLISECONDS); + public void registerAckAllRecordQueueTimeMaxGauge(T stateObj, ToDoubleFunction f, String sender, String datacenter) { + registerTimeGauge(stateObj, f, ACK_ALL_RECORD_QUEUE_TIME_MAX, ACK_ALL_RECORD_QUEUE_TIME_MAX, tags(sender, datacenter), TimeUnit.MILLISECONDS); } - public void registerAckLeaderRecordQueueTimeMaxGauge(T stateObj, ToDoubleFunction f) { + public void registerAckLeaderRecordQueueTimeMaxGauge(T stateObj, ToDoubleFunction f, String sender, String datacenter) { registerTimeGauge(stateObj, f, ACK_LEADER_RECORD_QUEUE_TIME_MAX, - ACK_LEADER_RECORD_QUEUE_TIME_MAX, Tags.empty(), TimeUnit.MILLISECONDS); + ACK_LEADER_RECORD_QUEUE_TIME_MAX, tags(sender, datacenter), TimeUnit.MILLISECONDS); } public double getBufferTotalBytes() { @@ -84,32 +83,10 @@ public void registerProducerInflightRequestGauge(T stateObj, ToDoubleFunctio hermesMetrics.registerProducerInflightRequest(() -> (int) f.applyAsDouble(stateObj)); } - public void registerAckAllMaxLatencyBrokerGauge(T stateObj, ToDoubleFunction f, String brokerNodeId) { - registerLatencyPerBrokerGauge(stateObj, f, "request-latency-max", ACK_ALL, brokerNodeId); + private static Tags tags(String sender, String datacenter) { + return Tags.of("storageDc", datacenter, "sender", sender); } - public void registerAckLeaderMaxLatencyPerBrokerGauge(T stateObj, ToDoubleFunction f, String brokerNodeId) { - registerLatencyPerBrokerGauge(stateObj, f, "request-latency-max", ACK_LEADER, brokerNodeId); - } - - public void registerAckAllAvgLatencyPerBrokerGauge(T stateObj, ToDoubleFunction f, String brokerNodeId) { - registerLatencyPerBrokerGauge(stateObj, f, "request-latency-avg", ACK_ALL, brokerNodeId); - } - - public void registerAckLeaderAvgLatencyPerBrokerGauge(T stateObj, ToDoubleFunction f, String brokerNodeId) { - registerLatencyPerBrokerGauge(stateObj, f, "request-latency-avg", ACK_LEADER, brokerNodeId); - } - - private void registerLatencyPerBrokerGauge(T stateObj, - ToDoubleFunction f, - String metricName, - String producerName, - String brokerNodeId) { - String baseMetricName = KAFKA_PRODUCER + producerName + metricName; - String graphiteMetricName = baseMetricName + "." + escapeDots(brokerNodeId); - - registerTimeGauge(stateObj, f, graphiteMetricName, baseMetricName, Tags.of("broker", brokerNodeId), TimeUnit.MILLISECONDS); - } private void registerTimeGauge(T stateObj, ToDoubleFunction f, diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/TopicMetrics.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/TopicMetrics.java index 1d840e3b5d..bab7b92e01 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/TopicMetrics.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/TopicMetrics.java @@ -76,9 +76,9 @@ public MeterBackedHermesCounter topicGlobalThroughputBytes() { ); } - public HermesCounter topicPublished(TopicName topicName) { + public HermesCounter topicPublished(TopicName topicName, String datacenter) { return HermesCounters.from( - micrometerCounter(TopicMetricsNames.TOPIC_PUBLISHED, topicName), + micrometerCounter(TopicMetricsNames.TOPIC_PUBLISHED, topicName, Tag.of("storageDc", datacenter)), hermesMetrics.counter(Counters.PUBLISHED, topicName) ); } @@ -126,6 +126,13 @@ public HermesCounter topicHttpStatusCodeCounter(TopicName topicName, int statusC ); } + public HermesCounter topicDuplicatedMessageCounter(TopicName topicName) { + return HermesCounters.from( + micrometerCounter(TopicMetricsNames.TOPIC_DUPLICATED_MESSAGE, topicName), + hermesMetrics.meter(Meters.TOPIC_DUPLICATED_MESSAGE, topicName) + ); + } + public HermesHistogram topicGlobalMessageContentSizeHistogram() { return DefaultHermesHistogram.of( DistributionSummary.builder(TopicMetricsNames.TOPIC_GLOBAL_MESSAGE_SIZE_BYTES) @@ -147,8 +154,8 @@ private Timer micrometerTimer(String metricName, TopicName topicName) { return meterRegistry.timer(metricName, topicTags(topicName)); } - private Counter micrometerCounter(String metricName, TopicName topicName) { - return meterRegistry.counter(metricName, topicTags(topicName)); + private Counter micrometerCounter(String metricName, TopicName topicName, Tag ... tags) { + return meterRegistry.counter(metricName, topicTags(topicName).and(tags)); } private Tags topicTags(TopicName topicName) { @@ -176,5 +183,6 @@ public static class TopicMetricsNames { public static final String TOPIC_HTTP_STATUS_CODES = "topic-http-status-codes"; public static final String TOPIC_GLOBAL_MESSAGE_SIZE_BYTES = "topic-global-message-size-bytes"; public static final String TOPIC_MESSAGE_SIZE_BYTES = "topic-message-size-bytes"; + public static final String TOPIC_DUPLICATED_MESSAGE = "topic-duplicated-message"; } } diff --git a/hermes-common/src/test/java/pl/allegro/tech/hermes/common/metric/counter/zookeeper/ZookeeperCounterReporterTest.java b/hermes-common/src/test/java/pl/allegro/tech/hermes/common/metric/counter/zookeeper/ZookeeperCounterReporterTest.java index 3788cfb46e..5ded453054 100644 --- a/hermes-common/src/test/java/pl/allegro/tech/hermes/common/metric/counter/zookeeper/ZookeeperCounterReporterTest.java +++ b/hermes-common/src/test/java/pl/allegro/tech/hermes/common/metric/counter/zookeeper/ZookeeperCounterReporterTest.java @@ -51,7 +51,7 @@ public void before() { @Test public void shouldReportPublishedMessages() { // given - metricsFacade.topics().topicPublished(topic).increment(COUNT); + metricsFacade.topics().topicPublished(topic, "dc1").increment(COUNT); // when zookeeperCounterReporter.report(); diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/buffer/BackupMessagesLoader.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/buffer/BackupMessagesLoader.java index b8aa287fb9..1ddff796f8 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/buffer/BackupMessagesLoader.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/buffer/BackupMessagesLoader.java @@ -16,7 +16,6 @@ import pl.allegro.tech.hermes.frontend.publishing.avro.AvroMessage; import pl.allegro.tech.hermes.frontend.publishing.message.JsonMessage; import pl.allegro.tech.hermes.frontend.publishing.message.Message; -import pl.allegro.tech.hermes.metrics.HermesTimerContext; import pl.allegro.tech.hermes.schema.CompiledSchema; import pl.allegro.tech.hermes.schema.SchemaExistenceEnsurer; import pl.allegro.tech.hermes.schema.SchemaId; @@ -56,7 +55,6 @@ public class BackupMessagesLoader { private final int maxResendRetries; private final Duration resendSleep; private final Duration readTopicInfoSleep; - private final String datacenter; private final Set topicsAvailabilityCache = new HashSet<>(); private final AtomicReference>> toResend = new AtomicReference<>(); @@ -68,9 +66,7 @@ public BackupMessagesLoader(BrokerTopicAvailabilityChecker brokerTopicAvailabili SchemaRepository schemaRepository, SchemaExistenceEnsurer schemaExistenceEnsurer, Trackers trackers, - BackupMessagesLoaderParameters backupMessagesLoaderParameters, - String datacenter - ) { + BackupMessagesLoaderParameters backupMessagesLoaderParameters) { this.brokerTopicAvailabilityChecker = brokerTopicAvailabilityChecker; this.brokerMessageProducer = brokerMessageProducer; this.brokerListeners = brokerListeners; @@ -82,7 +78,6 @@ public BackupMessagesLoader(BrokerTopicAvailabilityChecker brokerTopicAvailabili this.resendSleep = backupMessagesLoaderParameters.getLoadingPauseBetweenResend(); this.readTopicInfoSleep = backupMessagesLoaderParameters.getLoadingWaitForBrokerTopicInfo(); this.maxResendRetries = backupMessagesLoaderParameters.getMaxResendRetries(); - this.datacenter = datacenter; } public void loadMessages(List messages) { @@ -254,11 +249,9 @@ private boolean isNotStale(Message message) { } private void sendMessage(Message message, CachedTopic cachedTopic) { - HermesTimerContext brokerTimer = cachedTopic.startBrokerLatencyTimer(); brokerMessageProducer.send(message, cachedTopic, new PublishingCallback() { @Override public void onUnpublished(Message message, Topic topic, Exception exception) { - brokerTimer.close(); brokerListeners.onError(message, topic, exception); trackers.get(topic).logError(message.getId(), topic.getName(), exception.getMessage(), "", Collections.emptyMap()); toResend.get().add(ImmutablePair.of(message, cachedTopic)); @@ -266,9 +259,12 @@ public void onUnpublished(Message message, Topic topic, Exception exception) { @Override public void onPublished(Message message, Topic topic) { - brokerTimer.close(); - cachedTopic.incrementPublished(); brokerListeners.onAcknowledge(message, topic); + } + + @Override + public void onEachPublished(Message message, Topic topic, String datacenter) { + cachedTopic.incrementPublished(datacenter); trackers.get(topic).logPublished(message.getId(), topic.getName(), "", datacenter, Collections.emptyMap()); } }); diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/buffer/SimpleExecutionCallback.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/buffer/SimpleExecutionCallback.java deleted file mode 100644 index 38905ef34a..0000000000 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/buffer/SimpleExecutionCallback.java +++ /dev/null @@ -1,27 +0,0 @@ -package pl.allegro.tech.hermes.frontend.buffer; - -import com.google.common.collect.Lists; -import pl.allegro.tech.hermes.api.Topic; -import pl.allegro.tech.hermes.frontend.publishing.PublishingCallback; -import pl.allegro.tech.hermes.frontend.publishing.message.Message; - -import java.util.List; - -public class SimpleExecutionCallback implements PublishingCallback { - - private final List callbacks; - - public SimpleExecutionCallback(PublishingCallback... callbacks) { - this.callbacks = Lists.newArrayList(callbacks); - } - - @Override - public void onUnpublished(Message message, Topic topic, Exception exception) { - callbacks.forEach(c -> c.onUnpublished(message, topic, exception)); - } - - @Override - public void onPublished(Message message, Topic topic) { - callbacks.forEach(c -> c.onPublished(message, topic)); - } -} diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FailFastKafkaProducerProperties.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FailFastKafkaProducerProperties.java new file mode 100644 index 0000000000..e611bad3ca --- /dev/null +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FailFastKafkaProducerProperties.java @@ -0,0 +1,208 @@ +package pl.allegro.tech.hermes.frontend.config; + +import org.springframework.boot.context.properties.ConfigurationProperties; +import pl.allegro.tech.hermes.frontend.producer.kafka.KafkaProducerParameters; + +import java.time.Duration; + +@ConfigurationProperties(prefix = "frontend.kafka.fail-fast-producer") +public class FailFastKafkaProducerProperties implements KafkaProducerParameters { + + private Duration speculativeSendDelay = Duration.ofMillis(250); + + private FallbackSchedulerProperties fallbackScheduler = new FallbackSchedulerProperties(); + + private Duration maxBlock = Duration.ofMillis(500); + + private Duration metadataMaxAge = Duration.ofMinutes(5); + + private String compressionCodec = "none"; + + private int retries = Integer.MAX_VALUE; + + private Duration retryBackoff = Duration.ofMillis(50); + + private Duration requestTimeout = Duration.ofMillis(500); + + private Duration deliveryTimeout = Duration.ofMillis(500); + + private int batchSize = 16 * 1024; + + private int tcpSendBuffer = 128 * 1024; + + private int maxRequestSize = 1024 * 1024; + + private Duration linger = Duration.ofMillis(0); + + private Duration metricsSampleWindow = Duration.ofSeconds(30); + + private int maxInflightRequestsPerConnection = 5; + + private boolean reportNodeMetricsEnabled = false; + + @Override + public Duration getMaxBlock() { + return maxBlock; + } + + public void setMaxBlock(Duration maxBlock) { + this.maxBlock = maxBlock; + } + + @Override + public Duration getMetadataMaxAge() { + return metadataMaxAge; + } + + public void setMetadataMaxAge(Duration metadataMaxAge) { + this.metadataMaxAge = metadataMaxAge; + } + + @Override + public String getCompressionCodec() { + return compressionCodec; + } + + public void setCompressionCodec(String compressionCodec) { + this.compressionCodec = compressionCodec; + } + + @Override + public int getRetries() { + return retries; + } + + public void setRetries(int retries) { + this.retries = retries; + } + + @Override + public Duration getRetryBackoff() { + return retryBackoff; + } + + public void setRetryBackoff(Duration retryBackoff) { + this.retryBackoff = retryBackoff; + } + + @Override + public Duration getRequestTimeout() { + return requestTimeout; + } + + public void setRequestTimeout(Duration requestTimeout) { + this.requestTimeout = requestTimeout; + } + + @Override + public int getBatchSize() { + return batchSize; + } + + public void setBatchSize(int batchSize) { + this.batchSize = batchSize; + } + + @Override + public int getTcpSendBuffer() { + return tcpSendBuffer; + } + + public void setTcpSendBuffer(int tcpSendBuffer) { + this.tcpSendBuffer = tcpSendBuffer; + } + + @Override + public int getMaxRequestSize() { + return maxRequestSize; + } + + public void setMaxRequestSize(int maxRequestSize) { + this.maxRequestSize = maxRequestSize; + } + + @Override + public Duration getLinger() { + return linger; + } + + public void setLinger(Duration linger) { + this.linger = linger; + } + + @Override + public Duration getMetricsSampleWindow() { + return metricsSampleWindow; + } + + public void setMetricsSampleWindow(Duration metricsSampleWindow) { + this.metricsSampleWindow = metricsSampleWindow; + } + + @Override + public int getMaxInflightRequestsPerConnection() { + return maxInflightRequestsPerConnection; + } + + public void setMaxInflightRequestsPerConnection(int maxInflightRequestsPerConnection) { + this.maxInflightRequestsPerConnection = maxInflightRequestsPerConnection; + } + + @Override + public boolean isReportNodeMetricsEnabled() { + return reportNodeMetricsEnabled; + } + + public void setReportNodeMetricsEnabled(boolean reportNodeMetricsEnabled) { + this.reportNodeMetricsEnabled = reportNodeMetricsEnabled; + } + + public Duration getSpeculativeSendDelay() { + return speculativeSendDelay; + } + + public void setSpeculativeSendDelay(Duration speculativeSendDelay) { + this.speculativeSendDelay = speculativeSendDelay; + } + + @Override + public Duration getDeliveryTimeout() { + return deliveryTimeout; + } + + public void setDeliveryTimeout(Duration deliveryTimeout) { + this.deliveryTimeout = deliveryTimeout; + } + + public FallbackSchedulerProperties getFallbackScheduler() { + return fallbackScheduler; + } + + public void setFallbackScheduler(FallbackSchedulerProperties fallbackScheduler) { + this.fallbackScheduler = fallbackScheduler; + } + + public static class FallbackSchedulerProperties { + + private int threadPoolSize = 16; + + private boolean threadPoolMonitoringEnabled = false; + + + public int getThreadPoolSize() { + return threadPoolSize; + } + + public void setThreadPoolSize(int threadPoolSize) { + this.threadPoolSize = threadPoolSize; + } + + public boolean isThreadPoolMonitoringEnabled() { + return threadPoolMonitoringEnabled; + } + + public void setThreadPoolMonitoringEnabled(boolean threadPoolMonitoringEnabled) { + this.threadPoolMonitoringEnabled = threadPoolMonitoringEnabled; + } + } +} diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FrontendConfiguration.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FrontendConfiguration.java index deffcd8af6..c219608443 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FrontendConfiguration.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FrontendConfiguration.java @@ -1,5 +1,6 @@ package pl.allegro.tech.hermes.frontend.config; +import jakarta.inject.Named; import org.apache.curator.framework.CuratorFramework; import org.springframework.boot.context.properties.EnableConfigurationProperties; import org.springframework.context.annotation.Bean; @@ -18,7 +19,6 @@ import pl.allegro.tech.hermes.frontend.producer.BrokerMessageProducer; import pl.allegro.tech.hermes.frontend.validator.MessageValidators; import pl.allegro.tech.hermes.frontend.validator.TopicMessageValidator; -import pl.allegro.tech.hermes.infrastructure.dc.DatacenterNameProvider; import pl.allegro.tech.hermes.infrastructure.zookeeper.ZookeeperPaths; import pl.allegro.tech.hermes.schema.SchemaExistenceEnsurer; import pl.allegro.tech.hermes.schema.SchemaRepository; @@ -49,13 +49,12 @@ public TopicsCache notificationBasedTopicsCache(InternalNotificationsBus interna } @Bean - public BackupMessagesLoader backupMessagesLoader(BrokerMessageProducer brokerMessageProducer, + public BackupMessagesLoader backupMessagesLoader(@Named("localDatacenterBrokerProducer") BrokerMessageProducer brokerMessageProducer, BrokerListeners brokerListeners, TopicsCache topicsCache, SchemaRepository schemaRepository, Trackers trackers, - LocalMessageStorageProperties localMessageStorageProperties, - DatacenterNameProvider datacenterNameProvider) { + LocalMessageStorageProperties localMessageStorageProperties) { return new BackupMessagesLoader( brokerMessageProducer, brokerMessageProducer, @@ -64,8 +63,7 @@ public BackupMessagesLoader backupMessagesLoader(BrokerMessageProducer brokerMes schemaRepository, new SchemaExistenceEnsurer(schemaRepository), trackers, - localMessageStorageProperties, - datacenterNameProvider.getDatacenterName() + localMessageStorageProperties ); } diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FrontendProducerConfiguration.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FrontendProducerConfiguration.java index ca9a27e5c8..46ae60ba74 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FrontendProducerConfiguration.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FrontendProducerConfiguration.java @@ -1,24 +1,31 @@ package pl.allegro.tech.hermes.frontend.config; +import jakarta.inject.Named; import org.apache.kafka.clients.admin.AdminClient; import org.springframework.boot.context.properties.EnableConfigurationProperties; import org.springframework.context.annotation.Bean; import org.springframework.context.annotation.Configuration; import pl.allegro.tech.hermes.common.kafka.KafkaParameters; import pl.allegro.tech.hermes.common.metric.MetricsFacade; +import pl.allegro.tech.hermes.common.metric.executor.InstrumentedExecutorServiceFactory; import pl.allegro.tech.hermes.frontend.cache.topic.TopicsCache; +import pl.allegro.tech.hermes.frontend.config.FailFastKafkaProducerProperties.FallbackSchedulerProperties; import pl.allegro.tech.hermes.frontend.producer.BrokerLatencyReporter; import pl.allegro.tech.hermes.frontend.producer.BrokerMessageProducer; -import pl.allegro.tech.hermes.frontend.producer.kafka.KafkaBrokerMessageProducer; +import pl.allegro.tech.hermes.frontend.producer.kafka.FallbackToRemoteDatacenterAwareMessageProducer; import pl.allegro.tech.hermes.frontend.producer.kafka.KafkaHeaderFactory; import pl.allegro.tech.hermes.frontend.producer.kafka.KafkaMessageSenders; import pl.allegro.tech.hermes.frontend.producer.kafka.KafkaMessageSendersFactory; +import pl.allegro.tech.hermes.frontend.producer.kafka.LocalDatacenterMessageProducer; import pl.allegro.tech.hermes.frontend.producer.kafka.MessageToKafkaProducerRecordConverter; +import pl.allegro.tech.hermes.frontend.producer.kafka.MultiDatacenterMessageProducer; import pl.allegro.tech.hermes.frontend.producer.kafka.ProducerMetadataLoadingJob; +import pl.allegro.tech.hermes.frontend.readiness.AdminReadinessService; import pl.allegro.tech.hermes.infrastructure.dc.DatacenterNameProvider; import java.util.List; import java.util.Properties; +import java.util.concurrent.ScheduledExecutorService; import static org.apache.kafka.clients.CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG; import static org.apache.kafka.clients.CommonClientConfigs.DEFAULT_SECURITY_PROTOCOL; @@ -33,16 +40,46 @@ SchemaProperties.class, KafkaHeaderNameProperties.class, KafkaProducerProperties.class, + FailFastKafkaProducerProperties.class, KafkaClustersProperties.class, HTTPHeadersProperties.class }) public class FrontendProducerConfiguration { @Bean - public BrokerMessageProducer kafkaBrokerMessageProducer(KafkaMessageSenders kafkaMessageSenders, - MetricsFacade metricsFacade, - MessageToKafkaProducerRecordConverter messageConverter) { - return new KafkaBrokerMessageProducer(kafkaMessageSenders, metricsFacade, messageConverter); + public BrokerMessageProducer kafkaBrokerMessageProducer(@Named("localDatacenterBrokerProducer") BrokerMessageProducer localDatacenterBrokerProducer, + @Named("multiDatacenterBrokerProducer") BrokerMessageProducer multiDatacenterBrokerProducer) { + return new FallbackToRemoteDatacenterAwareMessageProducer( + localDatacenterBrokerProducer, + multiDatacenterBrokerProducer + ); + } + + @Bean + public BrokerMessageProducer localDatacenterBrokerProducer(@Named("kafkaMessageSenders") KafkaMessageSenders kafkaMessageSenders, + MessageToKafkaProducerRecordConverter messageConverter) { + return new LocalDatacenterMessageProducer(kafkaMessageSenders, messageConverter); + } + + @Bean + public BrokerMessageProducer multiDatacenterBrokerProducer(@Named("failFastKafkaMessageSenders") KafkaMessageSenders kafkaMessageSenders, + MessageToKafkaProducerRecordConverter messageConverter, + FailFastKafkaProducerProperties kafkaProducerProperties, + AdminReadinessService adminReadinessService, + InstrumentedExecutorServiceFactory executorServiceFactory) { + FallbackSchedulerProperties fallbackSchedulerProperties = kafkaProducerProperties.getFallbackScheduler(); + ScheduledExecutorService fallbackScheduler = executorServiceFactory.getScheduledExecutorService( + "fallback-to-remote", + fallbackSchedulerProperties.getThreadPoolSize(), + fallbackSchedulerProperties.isThreadPoolMonitoringEnabled() + ); + return new MultiDatacenterMessageProducer( + kafkaMessageSenders, + adminReadinessService, + messageConverter, + kafkaProducerProperties.getSpeculativeSendDelay(), + fallbackScheduler + ); } @Bean @@ -54,7 +91,13 @@ public KafkaHeaderFactory kafkaHeaderFactory(KafkaHeaderNameProperties kafkaHead @Bean(destroyMethod = "close") public KafkaMessageSenders kafkaMessageSenders(KafkaProducerProperties kafkaProducerProperties, KafkaMessageSendersFactory kafkaMessageSendersFactory) { - return kafkaMessageSendersFactory.provide(kafkaProducerProperties); + return kafkaMessageSendersFactory.provide(kafkaProducerProperties, "default"); + } + + @Bean(destroyMethod = "close") + public KafkaMessageSenders failFastKafkaMessageSenders(FailFastKafkaProducerProperties kafkaProducerProperties, + KafkaMessageSendersFactory kafkaMessageSendersFactory) { + return kafkaMessageSendersFactory.provide(kafkaProducerProperties, "failFast"); } @Bean(destroyMethod = "close") @@ -63,12 +106,16 @@ public KafkaMessageSendersFactory kafkaMessageSendersFactory(KafkaClustersProper TopicLoadingProperties topicLoadingProperties, TopicsCache topicsCache, LocalMessageStorageProperties localMessageStorageProperties, - DatacenterNameProvider datacenterNameProvider) { + DatacenterNameProvider datacenterNameProvider, + BrokerLatencyReporter brokerLatencyReporter, + MetricsFacade metricsFacade) { KafkaProperties kafkaProperties = kafkaClustersProperties.toKafkaProperties(datacenterNameProvider); List remoteKafkaProperties = kafkaClustersProperties.toRemoteKafkaProperties(datacenterNameProvider); return new KafkaMessageSendersFactory( kafkaProperties, remoteKafkaProperties, + brokerLatencyReporter, + metricsFacade, createAdminClient(kafkaProperties), topicsCache, topicLoadingProperties.getMetadata().getRetryCount(), @@ -93,10 +140,10 @@ private static AdminClient createAdminClient(KafkaProperties kafkaProperties) { } @Bean(initMethod = "start", destroyMethod = "stop") - public ProducerMetadataLoadingJob producerMetadataLoadingJob(KafkaMessageSenders kafkaMessageSenders, + public ProducerMetadataLoadingJob producerMetadataLoadingJob(List kafkaMessageSendersList, TopicLoadingProperties topicLoadingProperties) { return new ProducerMetadataLoadingJob( - kafkaMessageSenders, + kafkaMessageSendersList, topicLoadingProperties.getMetadataRefreshJob().isEnabled(), topicLoadingProperties.getMetadataRefreshJob().getInterval() ); diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FrontendPublishingConfiguration.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FrontendPublishingConfiguration.java index 5e664a9ca4..69f57beaae 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FrontendPublishingConfiguration.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FrontendPublishingConfiguration.java @@ -2,6 +2,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import io.undertow.server.HttpHandler; +import jakarta.inject.Named; import org.springframework.boot.context.properties.EnableConfigurationProperties; import org.springframework.context.annotation.Bean; import org.springframework.context.annotation.Configuration; @@ -10,7 +11,6 @@ import pl.allegro.tech.hermes.domain.topic.preview.MessagePreviewRepository; import pl.allegro.tech.hermes.frontend.cache.topic.TopicsCache; import pl.allegro.tech.hermes.frontend.listeners.BrokerListeners; -import pl.allegro.tech.hermes.frontend.producer.BrokerLatencyReporter; import pl.allegro.tech.hermes.frontend.producer.BrokerMessageProducer; import pl.allegro.tech.hermes.frontend.publishing.handlers.HandlersChainFactory; import pl.allegro.tech.hermes.frontend.publishing.handlers.ThroughputLimiter; @@ -29,7 +29,6 @@ import pl.allegro.tech.hermes.frontend.publishing.preview.MessagePreviewLog; import pl.allegro.tech.hermes.frontend.server.auth.AuthenticationConfiguration; import pl.allegro.tech.hermes.frontend.validator.MessageValidators; -import pl.allegro.tech.hermes.infrastructure.dc.DatacenterNameProvider; import pl.allegro.tech.hermes.schema.SchemaRepository; import pl.allegro.tech.hermes.tracker.frontend.Trackers; @@ -48,13 +47,12 @@ public class FrontendPublishingConfiguration { @Bean public HttpHandler httpHandler(TopicsCache topicsCache, MessageErrorProcessor messageErrorProcessor, MessageEndProcessor messageEndProcessor, MessageFactory messageFactory, - BrokerMessageProducer brokerMessageProducer, MessagePreviewLog messagePreviewLog, + @Named("kafkaBrokerMessageProducer") BrokerMessageProducer brokerMessageProducer, MessagePreviewLog messagePreviewLog, ThroughputLimiter throughputLimiter, Optional authConfig, - MessagePreviewProperties messagePreviewProperties, HandlersChainProperties handlersChainProperties, - BrokerLatencyReporter brokerLatencyReporter) { + MessagePreviewProperties messagePreviewProperties, HandlersChainProperties handlersChainProperties) { return new HandlersChainFactory(topicsCache, messageErrorProcessor, messageEndProcessor, messageFactory, brokerMessageProducer, messagePreviewLog, throughputLimiter, authConfig, messagePreviewProperties.isEnabled(), - handlersChainProperties, brokerLatencyReporter).provide(); + handlersChainProperties).provide(); } @Bean @@ -64,9 +62,8 @@ public ThroughputLimiter throughputLimiter(ThroughputProperties throughputProper @Bean public MessageEndProcessor messageEndProcessor(Trackers trackers, BrokerListeners brokerListeners, - TrackingHeadersExtractor trackingHeadersExtractor, - DatacenterNameProvider datacenterNameProvider) { - return new MessageEndProcessor(trackers, brokerListeners, trackingHeadersExtractor, datacenterNameProvider.getDatacenterName()); + TrackingHeadersExtractor trackingHeadersExtractor) { + return new MessageEndProcessor(trackers, brokerListeners, trackingHeadersExtractor); } @Bean diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/HandlersChainProperties.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/HandlersChainProperties.java index 1e0aa17905..451227270e 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/HandlersChainProperties.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/HandlersChainProperties.java @@ -12,6 +12,8 @@ public class HandlersChainProperties implements HandlersChainParameters { private Duration longIdleTimeout = Duration.ofMillis(400); + private Duration maxPublishRequestDuration = Duration.ofMillis(500); + private boolean forceTopicMaxMessageSize = false; @Override @@ -61,6 +63,15 @@ public void setForceTopicMaxMessageSize(boolean forceTopicMaxMessageSize) { this.forceTopicMaxMessageSize = forceTopicMaxMessageSize; } + @Override + public Duration getMaxPublishRequestDuration() { + return maxPublishRequestDuration; + } + + public void setMaxPublishRequestDuration(Duration maxPublishRequestDuration) { + this.maxPublishRequestDuration = maxPublishRequestDuration; + } + private AuthenticationProperties authentication = new AuthenticationProperties(); private KeepAliveHeaderProperties keepAliveHeader = new KeepAliveHeaderProperties(); diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/KafkaProducerProperties.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/KafkaProducerProperties.java index 17562ec453..8bdb084e47 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/KafkaProducerProperties.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/KafkaProducerProperties.java @@ -20,6 +20,8 @@ public class KafkaProducerProperties implements KafkaProducerParameters { private Duration requestTimeout = Duration.ofMinutes(30); + private Duration deliveryTimeout = Duration.ofMinutes(30); + private int batchSize = 16 * 1024; private int tcpSendBuffer = 128 * 1024; @@ -150,4 +152,13 @@ public boolean isReportNodeMetricsEnabled() { public void setReportNodeMetricsEnabled(boolean reportNodeMetricsEnabled) { this.reportNodeMetricsEnabled = reportNodeMetricsEnabled; } + + @Override + public Duration getDeliveryTimeout() { + return deliveryTimeout; + } + + public void setDeliveryTimeout(Duration deliveryTimeout) { + this.deliveryTimeout = deliveryTimeout; + } } diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/ReadinessConfiguration.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/ReadinessConfiguration.java index 8febee759e..de4e563123 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/ReadinessConfiguration.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/ReadinessConfiguration.java @@ -1,6 +1,7 @@ package pl.allegro.tech.hermes.frontend.config; import com.fasterxml.jackson.databind.ObjectMapper; +import jakarta.inject.Named; import org.apache.curator.framework.CuratorFramework; import org.springframework.boot.context.properties.EnableConfigurationProperties; import org.springframework.context.annotation.Bean; @@ -18,7 +19,7 @@ public class ReadinessConfiguration { @Bean public DefaultReadinessChecker readinessChecker(ReadinessCheckProperties readinessCheckProperties, - BrokerTopicAvailabilityChecker brokerTopicAvailabilityChecker, + @Named("localDatacenterBrokerProducer") BrokerTopicAvailabilityChecker brokerTopicAvailabilityChecker, AdminReadinessService adminReadinessService) { return new DefaultReadinessChecker( brokerTopicAvailabilityChecker, diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/metric/CachedTopic.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/metric/CachedTopic.java index cb2a96f6ec..033351b232 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/metric/CachedTopic.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/metric/CachedTopic.java @@ -39,7 +39,9 @@ public class CachedTopic { private final MeterBackedHermesCounter topicThroughputMeter; private final MeterBackedHermesCounter globalThroughputMeter; - private final HermesCounter published; + private final HermesCounter topicDuplicatedMessageCounter; + + private final Map published = new ConcurrentHashMap<>(); private final Map httpStatusCodesMeters = new ConcurrentHashMap<>(); @@ -64,8 +66,6 @@ public CachedTopic(Topic topic, MetricsFacade metricsFacade, globalMessageContentSize = metricsFacade.topics().topicGlobalMessageContentSizeHistogram(); topicMessageContentSize = metricsFacade.topics().topicMessageContentSizeHistogram(topic.getName()); - published = metricsFacade.topics().topicPublished(topic.getName()); - globalThroughputMeter = metricsFacade.topics().topicGlobalThroughputBytes(); topicThroughputMeter = metricsFacade.topics().topicThroughputBytes(topic.getName()); @@ -78,6 +78,8 @@ public CachedTopic(Topic topic, MetricsFacade metricsFacade, topicProducerLatencyTimer = metricsFacade.topics().ackLeaderTopicLatency(topic.getName()); topicBrokerLatencyTimer = metricsFacade.topics().ackLeaderBrokerLatency(); } + + topicDuplicatedMessageCounter = metricsFacade.topics().topicDuplicatedMessageCounter(topic.getName()); } public Topic getTopic() { @@ -122,8 +124,11 @@ public HermesTimerContext startBrokerLatencyTimer() { return topicBrokerLatencyTimer.time(); } - public void incrementPublished() { - published.increment(1L); + public void incrementPublished(String datacenter) { + published.computeIfAbsent( + datacenter, + dc -> metricsFacade.topics().topicPublished(topic.getName(), datacenter) + ).increment(); } public void reportMessageContentSize(int size) { @@ -141,4 +146,8 @@ public void markDelayedProcessing() { public HermesRateMeter getThroughput() { return topicThroughputMeter; } + + public void markMessageDuplicated() { + topicDuplicatedMessageCounter.increment(); + } } diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/BrokerLatencyReporter.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/BrokerLatencyReporter.java index 797d5e6a8f..53be7793d5 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/BrokerLatencyReporter.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/BrokerLatencyReporter.java @@ -1,6 +1,5 @@ package pl.allegro.tech.hermes.frontend.producer; -import jakarta.annotation.Nullable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import pl.allegro.tech.hermes.api.Topic; @@ -10,7 +9,6 @@ import pl.allegro.tech.hermes.metrics.HermesTimerContext; import java.time.Duration; -import java.util.Optional; import java.util.concurrent.ExecutorService; import java.util.concurrent.RejectedExecutionException; import java.util.function.Supplier; @@ -37,7 +35,7 @@ public BrokerLatencyReporter(boolean perBrokerLatencyEnabled, public void report(HermesTimerContext timerContext, Message message, Topic.Ack ack, - @Nullable Supplier produceMetadata) { + Supplier produceMetadata) { Duration duration = timerContext.closeAndGet(); if (perBrokerLatencyEnabled) { try { @@ -53,8 +51,8 @@ public void report(HermesTimerContext timerContext, private void doReport(Duration duration, String messageId, Topic.Ack ack, - @Nullable Supplier produceMetadata) { - String broker = Optional.ofNullable(produceMetadata).flatMap(metadata -> metadata.get().getBroker()).orElse("unknown"); + Supplier produceMetadata) { + String broker = produceMetadata.get().getBroker().orElse("unknown"); if (duration.compareTo(slowResponseThreshold) > 0) { logger.debug("Slow produce request, broker response time: {} ms, ackLevel: {}, messageId: {}, broker: {}", diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/FallbackToRemoteDatacenterAwareMessageProducer.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/FallbackToRemoteDatacenterAwareMessageProducer.java new file mode 100644 index 0000000000..f6cddafab9 --- /dev/null +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/FallbackToRemoteDatacenterAwareMessageProducer.java @@ -0,0 +1,37 @@ +package pl.allegro.tech.hermes.frontend.producer.kafka; + +import pl.allegro.tech.hermes.frontend.metric.CachedTopic; +import pl.allegro.tech.hermes.frontend.producer.BrokerMessageProducer; +import pl.allegro.tech.hermes.frontend.publishing.PublishingCallback; +import pl.allegro.tech.hermes.frontend.publishing.message.Message; + +public class FallbackToRemoteDatacenterAwareMessageProducer implements BrokerMessageProducer { + + private final BrokerMessageProducer localDatacenterMessageProducer; + private final BrokerMessageProducer multiDatacenterMessageProducer; + + public FallbackToRemoteDatacenterAwareMessageProducer(BrokerMessageProducer localDatacenterMessageProducer, + BrokerMessageProducer multiDatacenterMessageProducer) { + this.localDatacenterMessageProducer = localDatacenterMessageProducer; + this.multiDatacenterMessageProducer = multiDatacenterMessageProducer; + } + + @Override + public void send(Message message, CachedTopic topic, PublishingCallback callback) { + if (topic.getTopic().isFallbackToRemoteDatacenterEnabled()) { + this.multiDatacenterMessageProducer.send(message, topic, callback); + } else { + this.localDatacenterMessageProducer.send(message, topic, callback); + } + } + + @Override + public boolean areAllTopicsAvailable() { + return localDatacenterMessageProducer.areAllTopicsAvailable(); + } + + @Override + public boolean isTopicAvailable(CachedTopic topic) { + return localDatacenterMessageProducer.isTopicAvailable(topic); + } +} diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaBrokerMessageProducer.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaBrokerMessageProducer.java deleted file mode 100644 index 09cf06185d..0000000000 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaBrokerMessageProducer.java +++ /dev/null @@ -1,104 +0,0 @@ -package pl.allegro.tech.hermes.frontend.producer.kafka; - -import jakarta.inject.Singleton; -import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.clients.producer.RecordMetadata; -import org.apache.kafka.common.PartitionInfo; -import org.apache.kafka.common.errors.InterruptException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import pl.allegro.tech.hermes.common.metric.MetricsFacade; -import pl.allegro.tech.hermes.frontend.metric.CachedTopic; -import pl.allegro.tech.hermes.frontend.producer.BrokerMessageProducer; -import pl.allegro.tech.hermes.frontend.publishing.PublishingCallback; -import pl.allegro.tech.hermes.frontend.publishing.message.Message; -import pl.allegro.tech.hermes.frontend.publishing.metadata.ProduceMetadata; - -import java.util.List; -import java.util.Optional; -import java.util.function.Supplier; - -@Singleton -public class KafkaBrokerMessageProducer implements BrokerMessageProducer { - - private static final Logger logger = LoggerFactory.getLogger(KafkaBrokerMessageProducer.class); - private final KafkaMessageSenders kafkaMessageSenders; - private final MessageToKafkaProducerRecordConverter messageConverter; - - public KafkaBrokerMessageProducer(KafkaMessageSenders kafkaMessageSenders, - MetricsFacade metricsFacade, - MessageToKafkaProducerRecordConverter messageConverter) { - this.kafkaMessageSenders = kafkaMessageSenders; - this.messageConverter = messageConverter; - kafkaMessageSenders.registerLocalSenderMetrics(metricsFacade); - } - - @Override - public void send(Message message, CachedTopic cachedTopic, final PublishingCallback callback) { - ProducerRecord producerRecord = - messageConverter.convertToProducerRecord(message, cachedTopic.getKafkaTopics().getPrimary().name()); - - try { - var producer = kafkaMessageSenders.get(cachedTopic.getTopic()); - producer.send(producerRecord, new SendCallback(message, cachedTopic, callback)); - } catch (Exception e) { - // message didn't get to internal producer buffer and it will not be send to a broker - callback.onUnpublished(message, cachedTopic.getTopic(), e); - } - } - - private Supplier produceMetadataSupplier(CachedTopic topic, RecordMetadata recordMetadata) { - return () -> { - String kafkaTopicName = topic.getKafkaTopics().getPrimary().name().asString(); - try { - List topicPartitions = kafkaMessageSenders.get(topic.getTopic()).loadPartitionMetadataFor(kafkaTopicName); - - Optional partitionInfo = topicPartitions.stream() - .filter(p -> p.partition() == recordMetadata.partition()) - .findFirst(); - - return partitionInfo.map(partition -> partition.leader().host()) - .map(ProduceMetadata::new) - .orElse(ProduceMetadata.empty()); - } catch (InterruptException e) { - Thread.currentThread().interrupt(); - } catch (Exception e) { - logger.warn("Could not read information about partitions for topic {}. {}", kafkaTopicName, e.getMessage()); - } - return ProduceMetadata.empty(); - }; - } - - @Override - public boolean areAllTopicsAvailable() { - return kafkaMessageSenders.areAllTopicsAvailable(); - } - - @Override - public boolean isTopicAvailable(CachedTopic cachedTopic) { - return kafkaMessageSenders.isTopicAvailable(cachedTopic); - } - - private class SendCallback implements org.apache.kafka.clients.producer.Callback { - - private final Message message; - private final CachedTopic topic; - private final PublishingCallback callback; - - public SendCallback(Message message, CachedTopic topic, PublishingCallback callback) { - this.message = message; - this.topic = topic; - this.callback = callback; - } - - @Override - public void onCompletion(RecordMetadata recordMetadata, Exception e) { - Supplier produceMetadata = produceMetadataSupplier(topic, recordMetadata); - if (e == null) { - callback.onPublished(message, topic.getTopic(), produceMetadata); - } else { - callback.onUnpublished(message, topic.getTopic(), produceMetadata, e); - } - } - } -} diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSender.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSender.java index 2dc76d23d6..c88bd5d94d 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSender.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSender.java @@ -3,26 +3,45 @@ import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.common.Metric; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.errors.InterruptException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import pl.allegro.tech.hermes.api.Topic; import pl.allegro.tech.hermes.common.metric.MetricsFacade; +import pl.allegro.tech.hermes.frontend.metric.CachedTopic; +import pl.allegro.tech.hermes.frontend.producer.BrokerLatencyReporter; +import pl.allegro.tech.hermes.frontend.publishing.message.Message; +import pl.allegro.tech.hermes.frontend.publishing.metadata.ProduceMetadata; +import pl.allegro.tech.hermes.metrics.HermesTimerContext; import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.function.Predicate; +import java.util.function.Supplier; import java.util.function.ToDoubleFunction; public class KafkaMessageSender { + private static final Logger logger = LoggerFactory.getLogger(KafkaMessageSender.class); + private final Producer producer; + private final BrokerLatencyReporter brokerLatencyReporter; + private final MetricsFacade metricsFacade; private final String datacenter; - KafkaMessageSender(Producer kafkaProducer, String datacenter) { + KafkaMessageSender(Producer kafkaProducer, + BrokerLatencyReporter brokerLatencyReporter, + MetricsFacade metricsFacade, + String datacenter) { this.producer = kafkaProducer; + this.brokerLatencyReporter = brokerLatencyReporter; + this.metricsFacade = metricsFacade; this.datacenter = datacenter; } @@ -30,8 +49,13 @@ public String getDatacenter() { return datacenter; } - public void send(ProducerRecord producerRecord, Callback callback) { - producer.send(producerRecord, callback); + public void send(ProducerRecord producerRecord, + CachedTopic cachedTopic, + Message message, + Callback callback) { + HermesTimerContext timer = cachedTopic.startBrokerLatencyTimer(); + Callback meteredCallback = new MeteredCallback(timer, message, cachedTopic, callback); + producer.send(producerRecord, meteredCallback); } List loadPartitionMetadataFor(String topic) { @@ -42,7 +66,51 @@ public void close() { producer.close(); } - public void registerGauges(MetricsFacade metricsFacade, Topic.Ack ack) { + private Supplier produceMetadataSupplier(RecordMetadata recordMetadata) { + return () -> { + String kafkaTopicName = recordMetadata.topic(); + try { + List topicPartitions = producer.partitionsFor(kafkaTopicName); + + Optional partitionInfo = topicPartitions.stream() + .filter(p -> p.partition() == recordMetadata.partition()) + .findFirst(); + + return partitionInfo.map(partition -> partition.leader().host()) + .map(ProduceMetadata::new) + .orElse(ProduceMetadata.empty()); + } catch (InterruptException e) { + Thread.currentThread().interrupt(); + } catch (Exception e) { + logger.warn("Could not read information about partitions for topic {}. {}", kafkaTopicName, e.getMessage()); + } + return ProduceMetadata.empty(); + }; + } + + private class MeteredCallback implements Callback { + + private final HermesTimerContext hermesTimerContext; + private final Message message; + private final CachedTopic cachedTopic; + private final Callback callback; + + public MeteredCallback(HermesTimerContext hermesTimerContext, Message message, CachedTopic cachedTopic, Callback callback) { + this.hermesTimerContext = hermesTimerContext; + this.message = message; + this.cachedTopic = cachedTopic; + this.callback = callback; + } + + @Override + public void onCompletion(RecordMetadata metadata, Exception exception) { + callback.onCompletion(metadata, exception); + Supplier produceMetadataSupplier = produceMetadataSupplier(metadata); + brokerLatencyReporter.report(hermesTimerContext, message, cachedTopic.getTopic().getAck(), produceMetadataSupplier); + } + } + + public void registerGauges(Topic.Ack ack, String sender) { MetricName bufferTotalBytes = producerMetric("buffer-total-bytes", "producer-metrics", "buffer total bytes"); MetricName bufferAvailableBytes = producerMetric("buffer-available-bytes", "producer-metrics", "buffer available bytes"); MetricName compressionRate = producerMetric("compression-rate-avg", "producer-metrics", "average compression rate"); @@ -50,21 +118,20 @@ public void registerGauges(MetricsFacade metricsFacade, Topic.Ack ack) { MetricName metadataAge = producerMetric("metadata-age", "producer-metrics", "age [s] of metadata"); MetricName queueTimeMax = producerMetric("record-queue-time-max", "producer-metrics", "maximum time [ms] that batch spent in the send buffer"); - // TODO: add 'datacenter' label if (ack == Topic.Ack.ALL) { - metricsFacade.producer().registerAckAllTotalBytesGauge(producer, producerGauge(bufferTotalBytes)); - metricsFacade.producer().registerAckAllAvailableBytesGauge(producer, producerGauge(bufferAvailableBytes)); - metricsFacade.producer().registerAckAllCompressionRateGauge(producer, producerGauge(compressionRate)); - metricsFacade.producer().registerAckAllFailedBatchesGauge(producer, producerGauge(failedBatches)); - metricsFacade.producer().registerAckAllMetadataAgeGauge(producer, producerGauge(metadataAge)); - metricsFacade.producer().registerAckAllRecordQueueTimeMaxGauge(producer, producerGauge(queueTimeMax)); + metricsFacade.producer().registerAckAllTotalBytesGauge(producer, producerGauge(bufferTotalBytes), sender, datacenter); + metricsFacade.producer().registerAckAllAvailableBytesGauge(producer, producerGauge(bufferAvailableBytes), sender, datacenter); + metricsFacade.producer().registerAckAllCompressionRateGauge(producer, producerGauge(compressionRate), sender, datacenter); + metricsFacade.producer().registerAckAllFailedBatchesGauge(producer, producerGauge(failedBatches), sender, datacenter); + metricsFacade.producer().registerAckAllMetadataAgeGauge(producer, producerGauge(metadataAge), sender, datacenter); + metricsFacade.producer().registerAckAllRecordQueueTimeMaxGauge(producer, producerGauge(queueTimeMax), sender, datacenter); } else if (ack == Topic.Ack.LEADER) { - metricsFacade.producer().registerAckLeaderTotalBytesGauge(producer, producerGauge(bufferTotalBytes)); - metricsFacade.producer().registerAckLeaderAvailableBytesGauge(producer, producerGauge(bufferAvailableBytes)); - metricsFacade.producer().registerAckLeaderCompressionRateGauge(producer, producerGauge(compressionRate)); - metricsFacade.producer().registerAckLeaderFailedBatchesGauge(producer, producerGauge(failedBatches)); - metricsFacade.producer().registerAckLeaderMetadataAgeGauge(producer, producerGauge(metadataAge)); - metricsFacade.producer().registerAckLeaderRecordQueueTimeMaxGauge(producer, producerGauge(queueTimeMax)); + metricsFacade.producer().registerAckLeaderTotalBytesGauge(producer, producerGauge(bufferTotalBytes), sender, datacenter); + metricsFacade.producer().registerAckLeaderAvailableBytesGauge(producer, producerGauge(bufferAvailableBytes), sender, datacenter); + metricsFacade.producer().registerAckLeaderCompressionRateGauge(producer, producerGauge(compressionRate), sender, datacenter); + metricsFacade.producer().registerAckLeaderFailedBatchesGauge(producer, producerGauge(failedBatches), sender, datacenter); + metricsFacade.producer().registerAckLeaderMetadataAgeGauge(producer, producerGauge(metadataAge), sender, datacenter); + metricsFacade.producer().registerAckLeaderRecordQueueTimeMaxGauge(producer, producerGauge(queueTimeMax), sender, datacenter); } } @@ -76,7 +143,6 @@ private double findProducerMetric(Producer producer, return value < 0 ? 0.0 : value; } - private ToDoubleFunction> producerGauge(MetricName producerMetricName) { Predicate> predicate = entry -> entry.getKey().group().equals(producerMetricName.group()) && entry.getKey().name().equals(producerMetricName.name()); @@ -86,5 +152,4 @@ private ToDoubleFunction> producerGauge(MetricName producerMetric private static MetricName producerMetric(String name, String group, String description) { return new MetricName(name, group, description, Collections.emptyMap()); } - } diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSenders.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSenders.java index 221afe5737..4b32b24e0e 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSenders.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSenders.java @@ -95,9 +95,11 @@ private boolean anyUnderReplicatedPartition(List partitionInfos, return partitionInfos.stream().anyMatch(p -> p.inSyncReplicas().length < minInSyncReplicas); } - public void registerLocalSenderMetrics(MetricsFacade metricsFacade) { - ackLeader.registerGauges(metricsFacade, Topic.Ack.LEADER); - ackAll.registerGauges(metricsFacade, Topic.Ack.ALL); + public void registerSenderMetrics(String name) { + ackLeader.registerGauges(Topic.Ack.LEADER, name); + ackAll.registerGauges(Topic.Ack.ALL, name); + remoteAckLeader.forEach(sender -> sender.registerGauges(Topic.Ack.LEADER, name)); + remoteAckAll.forEach(sender -> sender.registerGauges(Topic.Ack.ALL, name)); } static class Tuple { diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSendersFactory.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSendersFactory.java index df1ba5d9a9..d4d8583439 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSendersFactory.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSendersFactory.java @@ -2,7 +2,9 @@ import org.apache.kafka.clients.admin.AdminClient; import pl.allegro.tech.hermes.common.kafka.KafkaParameters; +import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.frontend.cache.topic.TopicsCache; +import pl.allegro.tech.hermes.frontend.producer.BrokerLatencyReporter; import java.time.Duration; import java.util.HashMap; @@ -15,6 +17,7 @@ import static org.apache.kafka.clients.producer.ProducerConfig.BOOTSTRAP_SERVERS_CONFIG; import static org.apache.kafka.clients.producer.ProducerConfig.BUFFER_MEMORY_CONFIG; import static org.apache.kafka.clients.producer.ProducerConfig.COMPRESSION_TYPE_CONFIG; +import static org.apache.kafka.clients.producer.ProducerConfig.DELIVERY_TIMEOUT_MS_CONFIG; import static org.apache.kafka.clients.producer.ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG; import static org.apache.kafka.clients.producer.ProducerConfig.LINGER_MS_CONFIG; import static org.apache.kafka.clients.producer.ProducerConfig.MAX_BLOCK_MS_CONFIG; @@ -39,10 +42,14 @@ public class KafkaMessageSendersFactory { private final MinInSyncReplicasLoader localMinInSyncReplicasLoader; private final KafkaParameters kafkaParameters; private final List remoteKafkaParameters; + private final BrokerLatencyReporter brokerLatencyReporter; + private final MetricsFacade metricsFacade; private final long bufferedSizeBytes; public KafkaMessageSendersFactory(KafkaParameters kafkaParameters, List remoteKafkaParameters, + BrokerLatencyReporter brokerLatencyReporter, + MetricsFacade metricsFacade, AdminClient localAdminClient, TopicsCache topicsCache, int retryCount, @@ -55,10 +62,12 @@ public KafkaMessageSendersFactory(KafkaParameters kafkaParameters, this.bufferedSizeBytes = bufferedSizeBytes; this.kafkaParameters = kafkaParameters; this.remoteKafkaParameters = remoteKafkaParameters; + this.metricsFacade = metricsFacade; + this.brokerLatencyReporter = brokerLatencyReporter; } - public KafkaMessageSenders provide(KafkaProducerParameters kafkaProducerParameters) { - KafkaMessageSenders.Tuple localProducers = new KafkaMessageSenders.Tuple( + public KafkaMessageSenders provide(KafkaProducerParameters kafkaProducerParameters, String senderName) { + KafkaMessageSenders.Tuple localProducers = new KafkaMessageSenders.Tuple( sender(kafkaParameters, kafkaProducerParameters, ACK_LEADER), sender(kafkaParameters, kafkaProducerParameters, ACK_ALL) ); @@ -67,12 +76,14 @@ public KafkaMessageSenders provide(KafkaProducerParameters kafkaProducerParamete kafkaProperties -> new KafkaMessageSenders.Tuple( sender(kafkaProperties, kafkaProducerParameters, ACK_LEADER), sender(kafkaProperties, kafkaProducerParameters, ACK_ALL))).toList(); - return new KafkaMessageSenders( + KafkaMessageSenders senders = new KafkaMessageSenders( topicMetadataLoadingExecutor, localMinInSyncReplicasLoader, localProducers, remoteProducers ); + senders.registerSenderMetrics(senderName); + return senders; } private KafkaMessageSender sender(KafkaParameters kafkaParameters, @@ -84,6 +95,7 @@ private KafkaMessageSender sender(KafkaParameters kafkaParameter props.put(COMPRESSION_TYPE_CONFIG, kafkaProducerParameters.getCompressionCodec()); props.put(BUFFER_MEMORY_CONFIG, bufferedSizeBytes); props.put(REQUEST_TIMEOUT_MS_CONFIG, (int) kafkaProducerParameters.getRequestTimeout().toMillis()); + props.put(DELIVERY_TIMEOUT_MS_CONFIG, (int) kafkaProducerParameters.getDeliveryTimeout().toMillis()); props.put(BATCH_SIZE_CONFIG, kafkaProducerParameters.getBatchSize()); props.put(SEND_BUFFER_CONFIG, kafkaProducerParameters.getTcpSendBuffer()); props.put(RETRIES_CONFIG, kafkaProducerParameters.getRetries()); @@ -104,6 +116,8 @@ private KafkaMessageSender sender(KafkaParameters kafkaParameter } return new KafkaMessageSender<>( new org.apache.kafka.clients.producer.KafkaProducer<>(props), + brokerLatencyReporter, + metricsFacade, kafkaParameters.getDatacenter() ); } diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaProducerParameters.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaProducerParameters.java index f7e05d4283..79454b522f 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaProducerParameters.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaProducerParameters.java @@ -22,6 +22,8 @@ public interface KafkaProducerParameters { int getMaxRequestSize(); + Duration getDeliveryTimeout(); + Duration getLinger(); Duration getMetricsSampleWindow(); diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducer.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducer.java new file mode 100644 index 0000000000..94b11793fe --- /dev/null +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducer.java @@ -0,0 +1,73 @@ +package pl.allegro.tech.hermes.frontend.producer.kafka; + +import jakarta.inject.Singleton; +import org.apache.kafka.clients.producer.Callback; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import pl.allegro.tech.hermes.frontend.metric.CachedTopic; +import pl.allegro.tech.hermes.frontend.producer.BrokerMessageProducer; +import pl.allegro.tech.hermes.frontend.publishing.PublishingCallback; +import pl.allegro.tech.hermes.frontend.publishing.message.Message; + +@Singleton +public class LocalDatacenterMessageProducer implements BrokerMessageProducer { + + private final KafkaMessageSenders kafkaMessageSenders; + private final MessageToKafkaProducerRecordConverter messageConverter; + + public LocalDatacenterMessageProducer(KafkaMessageSenders kafkaMessageSenders, + MessageToKafkaProducerRecordConverter messageConverter) { + this.kafkaMessageSenders = kafkaMessageSenders; + this.messageConverter = messageConverter; + } + + @Override + public void send(Message message, CachedTopic cachedTopic, final PublishingCallback callback) { + ProducerRecord producerRecord = + messageConverter.convertToProducerRecord(message, cachedTopic.getKafkaTopics().getPrimary().name()); + + try { + var producer = kafkaMessageSenders.get(cachedTopic.getTopic()); + Callback wrappedCallback = new SendCallback(message, cachedTopic, callback, producer.getDatacenter()); + producer.send(producerRecord, cachedTopic, message, wrappedCallback); + } catch (Exception e) { + // message didn't get to internal producer buffer and it will not be send to a broker + callback.onUnpublished(message, cachedTopic.getTopic(), e); + } + } + + @Override + public boolean areAllTopicsAvailable() { + return kafkaMessageSenders.areAllTopicsAvailable(); + } + + @Override + public boolean isTopicAvailable(CachedTopic cachedTopic) { + return kafkaMessageSenders.isTopicAvailable(cachedTopic); + } + + private static class SendCallback implements org.apache.kafka.clients.producer.Callback { + + private final Message message; + private final CachedTopic topic; + private final PublishingCallback callback; + private final String datacenter; + + public SendCallback(Message message, CachedTopic topic, PublishingCallback callback, String datacenter) { + this.message = message; + this.topic = topic; + this.callback = callback; + this.datacenter = datacenter; + } + + @Override + public void onCompletion(RecordMetadata recordMetadata, Exception e) { + if (e == null) { + callback.onEachPublished(message, topic.getTopic(), datacenter); + callback.onPublished(message, topic.getTopic()); + } else { + callback.onUnpublished(message, topic.getTopic(), e); + } + } + } +} diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/MultiDatacenterMessageProducer.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/MultiDatacenterMessageProducer.java new file mode 100644 index 0000000000..e659aa5161 --- /dev/null +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/MultiDatacenterMessageProducer.java @@ -0,0 +1,166 @@ +package pl.allegro.tech.hermes.frontend.producer.kafka; + +import org.apache.kafka.clients.producer.Callback; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import pl.allegro.tech.hermes.frontend.metric.CachedTopic; +import pl.allegro.tech.hermes.frontend.producer.BrokerMessageProducer; +import pl.allegro.tech.hermes.frontend.publishing.PublishingCallback; +import pl.allegro.tech.hermes.frontend.publishing.message.Message; +import pl.allegro.tech.hermes.frontend.readiness.AdminReadinessService; + +import java.time.Duration; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.apache.commons.lang3.exception.ExceptionUtils.getRootCauseMessage; + +public class MultiDatacenterMessageProducer implements BrokerMessageProducer { + + private final KafkaMessageSenders kafkaMessageSenders; + private final MessageToKafkaProducerRecordConverter messageConverter; + private final Duration speculativeSendDelay; + private final AdminReadinessService adminReadinessService; + private final ScheduledExecutorService fallbackScheduler; + + public MultiDatacenterMessageProducer(KafkaMessageSenders kafkaMessageSenders, + AdminReadinessService adminReadinessService, + MessageToKafkaProducerRecordConverter messageConverter, + Duration speculativeSendDelay, + ScheduledExecutorService fallbackScheduler) { + this.messageConverter = messageConverter; + this.kafkaMessageSenders = kafkaMessageSenders; + this.speculativeSendDelay = speculativeSendDelay; + this.adminReadinessService = adminReadinessService; + this.fallbackScheduler = fallbackScheduler; + } + + @Override + public void send(Message message, CachedTopic cachedTopic, PublishingCallback callback) { + var producerRecord = messageConverter.convertToProducerRecord(message, cachedTopic.getKafkaTopics().getPrimary().name()); + + Optional> remoteSender = getRemoteSender(cachedTopic); + + final SendCallback sendCallback = remoteSender.isPresent() + ? SendCallback.withFallback(callback) + : SendCallback.withoutFallback(callback); + + fallbackScheduler.schedule(() -> { + if (!sendCallback.sent.get() && remoteSender.isPresent()) { + send(remoteSender.get(), + producerRecord, + sendCallback, + cachedTopic, + message); + } + }, speculativeSendDelay.toMillis(), TimeUnit.MILLISECONDS); + + send(kafkaMessageSenders.get(cachedTopic.getTopic()), producerRecord, sendCallback, cachedTopic, message); + } + + private void send(KafkaMessageSender sender, + ProducerRecord producerRecord, + SendCallback callback, + CachedTopic cachedTopic, + Message message) { + String datacenter = sender.getDatacenter(); + try { + sender.send(producerRecord, cachedTopic, message, new DCAwareCallback( + message, + cachedTopic, + datacenter, + callback)); + } catch (Exception e) { + // message didn't get to internal producer buffer and it will not be sent to a broker + callback.onUnpublished(message, cachedTopic, datacenter, e); + } + } + + private Optional> getRemoteSender(CachedTopic cachedTopic) { + return kafkaMessageSenders.getRemote(cachedTopic.getTopic()) + .stream() + .filter(producer -> adminReadinessService.isDatacenterReady(producer.getDatacenter())) + .findFirst(); + } + + private record DCAwareCallback(Message message, CachedTopic cachedTopic, String datacenter, + SendCallback callback) implements Callback { + + @Override + public void onCompletion(RecordMetadata metadata, Exception exception) { + if (exception == null) { + callback.onPublished(message, cachedTopic, datacenter); + } else { + callback.onUnpublished(message, cachedTopic, datacenter, exception); + } + } + } + + private static class SendCallback { + + private final PublishingCallback callback; + private final AtomicBoolean sent = new AtomicBoolean(false); + private final AtomicInteger tries; + private final ConcurrentHashMap errors; + + private SendCallback(PublishingCallback callback, int tries) { + this.callback = callback; + this.tries = new AtomicInteger(tries); + this.errors = new ConcurrentHashMap<>(tries); + } + + static SendCallback withFallback(PublishingCallback callback) { + return new SendCallback(callback, 2); + } + + static SendCallback withoutFallback(PublishingCallback callback) { + return new SendCallback(callback, 1); + } + + private void onUnpublished(Message message, CachedTopic cachedTopic, String datacenter, Exception exception) { + errors.put(datacenter, exception); + if (tries.decrementAndGet() == 0) { + callback.onUnpublished(message, cachedTopic.getTopic(), new MultiDCPublishException(errors)); + } + } + + private void onPublished(Message message, CachedTopic cachedTopic, String datacenter) { + callback.onEachPublished(message, cachedTopic.getTopic(), datacenter); + if (sent.compareAndSet(false, true)) { + callback.onPublished(message, cachedTopic.getTopic()); + } else { + cachedTopic.markMessageDuplicated(); + } + } + } + + public static class MultiDCPublishException extends RuntimeException { + + public MultiDCPublishException(Map exceptionsPerDC) { + super(errorMessage(exceptionsPerDC)); + } + + private static String errorMessage(Map exceptionsPerDC) { + StringBuilder builder = new StringBuilder(); + exceptionsPerDC.forEach( + (dc, exception) -> builder.append(String.format("[%s]: %s, ", dc, getRootCauseMessage(exception))) + ); + return builder.toString(); + } + } + + @Override + public boolean areAllTopicsAvailable() { + return kafkaMessageSenders.areAllTopicsAvailable(); + } + + @Override + public boolean isTopicAvailable(CachedTopic topic) { + return kafkaMessageSenders.isTopicAvailable(topic); + } +} diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/ProducerMetadataLoadingJob.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/ProducerMetadataLoadingJob.java index 30eff9e89c..8e868afa98 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/ProducerMetadataLoadingJob.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/ProducerMetadataLoadingJob.java @@ -3,6 +3,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; import java.time.Duration; +import java.util.List; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledFuture; @@ -20,17 +21,17 @@ */ public class ProducerMetadataLoadingJob implements Runnable { - private final KafkaMessageSenders kafkaMessageSenders; + private final List kafkaMessageSendersList; private final ScheduledExecutorService executorService; private final boolean enabled; private final Duration interval; private ScheduledFuture job; - public ProducerMetadataLoadingJob(KafkaMessageSenders kafkaMessageSenders, + public ProducerMetadataLoadingJob(List kafkaMessageSendersList, boolean enabled, Duration interval) { - this.kafkaMessageSenders = kafkaMessageSenders; + this.kafkaMessageSendersList = kafkaMessageSendersList; this.enabled = enabled; this.interval = interval; ThreadFactory threadFactory = new ThreadFactoryBuilder() @@ -40,12 +41,12 @@ public ProducerMetadataLoadingJob(KafkaMessageSenders kafkaMessageSenders, @Override public void run() { - kafkaMessageSenders.refreshTopicMetadata(); + refreshTopicMetadata(); } public void start() { if (enabled) { - kafkaMessageSenders.refreshTopicMetadata(); + refreshTopicMetadata(); job = executorService.scheduleAtFixedRate(this, interval.toSeconds(), interval.toSeconds(), TimeUnit.SECONDS); } } @@ -57,4 +58,10 @@ public void stop() throws InterruptedException { executorService.awaitTermination(1, TimeUnit.MINUTES); } } + + private void refreshTopicMetadata() { + for (KafkaMessageSenders kafkaMessageSenders : kafkaMessageSendersList) { + kafkaMessageSenders.refreshTopicMetadata(); + } + } } diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/PublishingCallback.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/PublishingCallback.java index 53a83ccf62..d175b1895e 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/PublishingCallback.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/PublishingCallback.java @@ -2,21 +2,26 @@ import pl.allegro.tech.hermes.api.Topic; import pl.allegro.tech.hermes.frontend.publishing.message.Message; -import pl.allegro.tech.hermes.frontend.publishing.metadata.ProduceMetadata; - -import java.util.function.Supplier; public interface PublishingCallback { + /** + * Invoked when publishing to the broker fails and the message won't be delivered. + */ void onUnpublished(Message message, Topic topic, Exception exception); - default void onUnpublished(Message message, Topic topic, Supplier produceMetadata, Exception exception) { - onUnpublished(message, topic, exception); - } - + /** + * Invoked the first time the message is successfully published to the broker. + */ void onPublished(Message message, Topic topic); - default void onPublished(Message message, Topic topic, Supplier produceMetadata) { - onPublished(message, topic); - } + /** + * Invoked every time the message is successfully published to the broker. + * Could be invoked one or many times depending on the underlying implementation. + * + * @param message the delivered message + * @param topic the topic that the message was delivered to + * @param datacenter the datacenter that the messages was delivered to + */ + void onEachPublished(Message message, Topic topic, String datacenter); } diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/handlers/HandlersChainFactory.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/handlers/HandlersChainFactory.java index 953b9dd0d9..f16394a340 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/handlers/HandlersChainFactory.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/handlers/HandlersChainFactory.java @@ -8,7 +8,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import pl.allegro.tech.hermes.frontend.cache.topic.TopicsCache; -import pl.allegro.tech.hermes.frontend.producer.BrokerLatencyReporter; import pl.allegro.tech.hermes.frontend.producer.BrokerMessageProducer; import pl.allegro.tech.hermes.frontend.publishing.handlers.end.MessageEndProcessor; import pl.allegro.tech.hermes.frontend.publishing.handlers.end.MessageErrorProcessor; @@ -33,14 +32,12 @@ public class HandlersChainFactory { private final ThroughputLimiter throughputLimiter; private final Optional authenticationConfiguration; private final HandlersChainParameters handlersChainParameters; - private final BrokerLatencyReporter brokerLatencyReporter; public HandlersChainFactory(TopicsCache topicsCache, MessageErrorProcessor messageErrorProcessor, MessageEndProcessor messageEndProcessor, MessageFactory messageFactory, BrokerMessageProducer brokerMessageProducer, MessagePreviewLog messagePreviewLog, ThroughputLimiter throughputLimiter, Optional authenticationConfiguration, - boolean messagePreviewEnabled, HandlersChainParameters handlersChainParameters, - BrokerLatencyReporter brokerLatencyReporter) { + boolean messagePreviewEnabled, HandlersChainParameters handlersChainParameters) { this.topicsCache = topicsCache; this.messageErrorProcessor = messageErrorProcessor; this.messageEndProcessor = messageEndProcessor; @@ -51,12 +48,11 @@ public HandlersChainFactory(TopicsCache topicsCache, MessageErrorProcessor messa this.throughputLimiter = throughputLimiter; this.authenticationConfiguration = authenticationConfiguration; this.handlersChainParameters = handlersChainParameters; - this.brokerLatencyReporter = brokerLatencyReporter; } public HttpHandler provide() { HttpHandler publishing = new PublishingHandler(brokerMessageProducer, messageErrorProcessor, - messageEndProcessor, brokerLatencyReporter); + messageEndProcessor); HttpHandler messageCreateHandler = new MessageCreateHandler(publishing, messageFactory, messageErrorProcessor); HttpHandler timeoutHandler = new TimeoutHandler(messageEndProcessor, messageErrorProcessor); HttpHandler handlerAfterRead = previewEnabled ? new PreviewHandler(messageCreateHandler, previewLog) : messageCreateHandler; @@ -67,7 +63,8 @@ public HttpHandler provide() { throughputLimiter, handlersChainParameters.isForceTopicMaxMessageSize(), handlersChainParameters.getIdleTimeout(), - handlersChainParameters.getLongIdleTimeout()); + handlersChainParameters.getLongIdleTimeout(), + handlersChainParameters.getMaxPublishRequestDuration()); TopicHandler topicHandler = new TopicHandler(readHandler, topicsCache, messageErrorProcessor); boolean keepAliveHeaderEnabled = handlersChainParameters.isKeepAliveHeaderEnabled(); HttpHandler rootPublishingHandler = keepAliveHeaderEnabled ? withKeepAliveHeaderHandler(topicHandler) : topicHandler; diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/handlers/HandlersChainParameters.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/handlers/HandlersChainParameters.java index 6f1204b7e1..98a0a37522 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/handlers/HandlersChainParameters.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/handlers/HandlersChainParameters.java @@ -4,10 +4,27 @@ public interface HandlersChainParameters { + /** + * Defines the time allowed for processing a publish request to a topic with ack=leader. + * After this time elapses a TimeoutHandler will be run which, with PersistentBufferExtension enabled, + * would result in message being sent to local buffer. + */ Duration getIdleTimeout(); + /** + * Defines the time allowed for processing a publish request to a topic with ack=all. + * After this time elapses a TimeoutHandler will be run which, with PersistentBufferExtension enabled, + * would result in message being sent to local buffer. + */ Duration getLongIdleTimeout(); + /** + * Defines the time allowed for processing a publish request to a topic with + * fallbackToRemoteDatacenterEnabled=true. After this time elapses a TimeoutHandler will be run which + * would result in error being returned to a client. + */ + Duration getMaxPublishRequestDuration(); + boolean isForceTopicMaxMessageSize(); boolean isKeepAliveHeaderEnabled(); diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/handlers/MessageReadHandler.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/handlers/MessageReadHandler.java index 384800de70..050771f72a 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/handlers/MessageReadHandler.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/handlers/MessageReadHandler.java @@ -6,6 +6,7 @@ import io.undertow.server.HttpServerExchange; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import pl.allegro.tech.hermes.api.Topic; import pl.allegro.tech.hermes.frontend.publishing.handlers.end.MessageErrorProcessor; import pl.allegro.tech.hermes.frontend.publishing.message.MessageState; @@ -29,11 +30,12 @@ class MessageReadHandler implements HttpHandler { private final ContentLengthChecker contentLengthChecker; private final Duration defaultAsyncTimeout; private final Duration longAsyncTimeout; + private final Duration maxPublishRequestDuration; private final ThroughputLimiter throughputLimiter; MessageReadHandler(HttpHandler next, HttpHandler timeoutHandler, MessageErrorProcessor messageErrorProcessor, ThroughputLimiter throughputLimiter, - boolean forceMaxMessageSizePerTopic, Duration idleTime, Duration longIdleTime) { + boolean forceMaxMessageSizePerTopic, Duration idleTime, Duration longIdleTime, Duration maxPublishRequestDuration) { this.next = next; this.timeoutHandler = timeoutHandler; this.messageErrorProcessor = messageErrorProcessor; @@ -41,13 +43,14 @@ class MessageReadHandler implements HttpHandler { this.defaultAsyncTimeout = idleTime; this.longAsyncTimeout = longIdleTime; this.throughputLimiter = throughputLimiter; + this.maxPublishRequestDuration = maxPublishRequestDuration; } @Override public void handleRequest(HttpServerExchange exchange) { AttachmentContent attachment = exchange.getAttachment(AttachmentContent.KEY); - Duration timeout = attachment.getTopic().isReplicationConfirmRequired() ? longAsyncTimeout : defaultAsyncTimeout; + Duration timeout = calculateTimeout(attachment.getTopic()); attachment.setTimeoutHolder(new TimeoutHolder( (int) timeout.toMillis(), @@ -66,6 +69,13 @@ public void handleRequest(HttpServerExchange exchange) { } } + private Duration calculateTimeout(Topic topic) { + if (topic.isFallbackToRemoteDatacenterEnabled()) { + return maxPublishRequestDuration; + } + return topic.isReplicationConfirmRequired() ? longAsyncTimeout : defaultAsyncTimeout; + } + private void runTimeoutHandler(HttpServerExchange exchange, AttachmentContent attachment) { try { timeoutHandler.handleRequest(exchange); diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/handlers/PublishingHandler.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/handlers/PublishingHandler.java index 4ca09c80a7..9f795fb3b5 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/handlers/PublishingHandler.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/handlers/PublishingHandler.java @@ -3,17 +3,12 @@ import io.undertow.server.HttpHandler; import io.undertow.server.HttpServerExchange; import pl.allegro.tech.hermes.api.Topic; -import pl.allegro.tech.hermes.frontend.producer.BrokerLatencyReporter; import pl.allegro.tech.hermes.frontend.producer.BrokerMessageProducer; import pl.allegro.tech.hermes.frontend.publishing.PublishingCallback; import pl.allegro.tech.hermes.frontend.publishing.handlers.end.MessageEndProcessor; import pl.allegro.tech.hermes.frontend.publishing.handlers.end.MessageErrorProcessor; import pl.allegro.tech.hermes.frontend.publishing.message.Message; import pl.allegro.tech.hermes.frontend.publishing.message.MessageState; -import pl.allegro.tech.hermes.frontend.publishing.metadata.ProduceMetadata; -import pl.allegro.tech.hermes.metrics.HermesTimerContext; - -import java.util.function.Supplier; import static org.apache.commons.lang3.exception.ExceptionUtils.getRootCauseMessage; import static pl.allegro.tech.hermes.api.ErrorCode.INTERNAL_ERROR; @@ -24,14 +19,12 @@ class PublishingHandler implements HttpHandler { private final BrokerMessageProducer brokerMessageProducer; private final MessageErrorProcessor messageErrorProcessor; private final MessageEndProcessor messageEndProcessor; - private final BrokerLatencyReporter brokerBrokerLatencyReporter; PublishingHandler(BrokerMessageProducer brokerMessageProducer, MessageErrorProcessor messageErrorProcessor, - MessageEndProcessor messageEndProcessor, BrokerLatencyReporter brokerLatencyReporter) { + MessageEndProcessor messageEndProcessor) { this.brokerMessageProducer = brokerMessageProducer; this.messageErrorProcessor = messageErrorProcessor; this.messageEndProcessor = messageEndProcessor; - this.brokerBrokerLatencyReporter = brokerLatencyReporter; } @Override @@ -52,50 +45,46 @@ private void handle(HttpServerExchange exchange) { MessageState messageState = attachment.getMessageState(); messageState.setSendingToKafkaProducerQueue(); - HermesTimerContext brokerLatencyTimers = attachment.getCachedTopic().startBrokerLatencyTimer(); brokerMessageProducer.send(attachment.getMessage(), attachment.getCachedTopic(), new PublishingCallback() { - // called from kafka producer thread @Override - public void onPublished(Message message, Topic topic, Supplier produceMetadata) { + public void onPublished(Message message, Topic topic) { exchange.getConnection().getWorker().execute(() -> { - brokerBrokerLatencyReporter.report(brokerLatencyTimers, message, topic.getAck(), produceMetadata); if (messageState.setSentToKafka()) { attachment.removeTimeout(); messageEndProcessor.sent(exchange, attachment); } else if (messageState.setDelayedSentToKafka()) { - messageEndProcessor.delayedSent(exchange, attachment.getCachedTopic(), message); + messageEndProcessor.delayedSent(attachment.getCachedTopic(), message); } }); } @Override - public void onPublished(Message message, Topic topic) { - onPublished(message, topic, null); - } - - - // in most cases this method should be called from worker thread, - // therefore there is no need to switch it to another worker thread - @Override - public void onUnpublished(Message message, Topic topic, Supplier produceMetadata, Exception exception) { - messageState.setErrorInSendingToKafka(); - brokerBrokerLatencyReporter.report(brokerLatencyTimers, message, topic.getAck(), produceMetadata); - attachment.removeTimeout(); - handleNotPublishedMessage(exchange, topic, attachment.getMessageId(), exception); + public void onEachPublished(Message message, Topic topic, String datacenter) { + exchange.getConnection().getWorker().execute(() -> { + attachment.getCachedTopic().incrementPublished(datacenter); + messageEndProcessor.eachSent(exchange, attachment, datacenter); + }); } @Override public void onUnpublished(Message message, Topic topic, Exception exception) { - onUnpublished(message, topic, null, exception); + exchange.getConnection().getWorker().execute(() -> { + messageState.setErrorInSendingToKafka(); + attachment.removeTimeout(); + handleNotPublishedMessage(exchange, topic, attachment.getMessageId(), exception); + }); } }); - if (messageState.setSendingToKafka() && messageState.setDelayedProcessing()) { + if (messageState.setSendingToKafka() + && !attachment.getCachedTopic().getTopic().isFallbackToRemoteDatacenterEnabled() + && messageState.setDelayedProcessing()) { messageEndProcessor.bufferedButDelayedProcessing(exchange, attachment); } } + private void handleNotPublishedMessage(HttpServerExchange exchange, Topic topic, String messageId, Exception exception) { messageErrorProcessor.sendAndLog( exchange, diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/handlers/TimeoutHandler.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/handlers/TimeoutHandler.java index 206c79cb37..1dd2774e88 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/handlers/TimeoutHandler.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/handlers/TimeoutHandler.java @@ -2,11 +2,14 @@ import io.undertow.server.HttpHandler; import io.undertow.server.HttpServerExchange; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import pl.allegro.tech.hermes.frontend.publishing.handlers.end.MessageEndProcessor; import pl.allegro.tech.hermes.frontend.publishing.handlers.end.MessageErrorProcessor; import pl.allegro.tech.hermes.frontend.publishing.message.MessageState; import static pl.allegro.tech.hermes.api.ErrorCode.INTERNAL_ERROR; +import static pl.allegro.tech.hermes.api.ErrorCode.SENDING_TO_KAFKA_TIMEOUT; import static pl.allegro.tech.hermes.api.ErrorCode.TIMEOUT; import static pl.allegro.tech.hermes.api.ErrorDescription.error; @@ -14,6 +17,7 @@ class TimeoutHandler implements HttpHandler { private final MessageErrorProcessor messageErrorProcessor; private final MessageEndProcessor messageEndProcessor; + private static final Logger logger = LoggerFactory.getLogger(TimeoutHandler.class); TimeoutHandler(MessageEndProcessor messageEndProcessor, MessageErrorProcessor messageErrorProcessor) { this.messageErrorProcessor = messageErrorProcessor; @@ -24,10 +28,13 @@ class TimeoutHandler implements HttpHandler { public void handleRequest(HttpServerExchange exchange) throws Exception { AttachmentContent attachment = exchange.getAttachment(AttachmentContent.KEY); MessageState state = attachment.getMessageState(); + boolean buffersDisabled = attachment.getCachedTopic().getTopic().isFallbackToRemoteDatacenterEnabled(); state.setTimeoutHasPassed(); if (state.setReadingTimeout()) { readingTimeout(exchange, attachment); + } else if (buffersDisabled && state.setTimeoutSendingToKafka()) { + sendingToKafkaTimeout(exchange, attachment); } else if (state.setDelayedSending()) { delayedSending(exchange, attachment); } else { @@ -66,4 +73,26 @@ private void readingTimeout(HttpServerExchange exchange, AttachmentContent attac } }); } + + private void sendingToKafkaTimeout(HttpServerExchange exchange, AttachmentContent attachment) { + exchange.getConnection().getWorker().execute(() -> { + TimeoutHolder timeoutHolder = attachment.getTimeoutHolder(); + + if (timeoutHolder != null) { + timeoutHolder.timeout(); + messageErrorProcessor.sendAndLog( + exchange, + attachment.getTopic(), + attachment.getMessageId(), + error("Timeout while sending to kafka message after " + timeoutHolder.getTimeout() + " milliseconds", SENDING_TO_KAFKA_TIMEOUT)); + } else { + messageErrorProcessor.sendAndLog( + exchange, + attachment.getTopic(), + attachment.getMessageId(), + error("Probably context switching problem as timeout task was started before it was attached to an exchange", + INTERNAL_ERROR)); + } + }); + } } diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/handlers/end/MessageEndProcessor.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/handlers/end/MessageEndProcessor.java index 505a587f71..dee57afd57 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/handlers/end/MessageEndProcessor.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/handlers/end/MessageEndProcessor.java @@ -23,34 +23,25 @@ public class MessageEndProcessor { private final Trackers trackers; private final BrokerListeners brokerListeners; private final TrackingHeadersExtractor trackingHeadersExtractor; - private final String datacenter; - public MessageEndProcessor(Trackers trackers, BrokerListeners brokerListeners, TrackingHeadersExtractor trackingHeadersExtractor, String datacenter) { + public MessageEndProcessor(Trackers trackers, BrokerListeners brokerListeners, TrackingHeadersExtractor trackingHeadersExtractor) { this.trackers = trackers; this.brokerListeners = brokerListeners; this.trackingHeadersExtractor = trackingHeadersExtractor; - this.datacenter = datacenter; } - public void sent(HttpServerExchange exchange, AttachmentContent attachment) { + public void eachSent(HttpServerExchange exchange, AttachmentContent attachment, String datacenter) { trackers.get(attachment.getTopic()).logPublished(attachment.getMessageId(), - attachment.getTopic().getName(), - readHostAndPort(exchange), - datacenter, + attachment.getTopic().getName(), readHostAndPort(exchange), datacenter, trackingHeadersExtractor.extractHeadersToLog(exchange.getRequestHeaders())); + } + + public void sent(HttpServerExchange exchange, AttachmentContent attachment) { sendResponse(exchange, attachment, StatusCodes.CREATED); - attachment.getCachedTopic().incrementPublished(); } - public void delayedSent(HttpServerExchange exchange, CachedTopic cachedTopic, Message message) { - trackers.get(cachedTopic.getTopic()).logPublished( - message.getId(), - cachedTopic.getTopic().getName(), - readHostAndPort(exchange), - datacenter, - trackingHeadersExtractor.extractHeadersToLog(exchange.getRequestHeaders())); + public void delayedSent(CachedTopic cachedTopic, Message message) { brokerListeners.onAcknowledge(message, cachedTopic.getTopic()); - cachedTopic.incrementPublished(); } public void bufferedButDelayedProcessing(HttpServerExchange exchange, AttachmentContent attachment) { diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/message/MessageState.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/message/MessageState.java index a18baa68d2..159e57faac 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/message/MessageState.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/message/MessageState.java @@ -15,6 +15,7 @@ import static pl.allegro.tech.hermes.frontend.publishing.message.MessageState.State.SENDING_TO_KAFKA; import static pl.allegro.tech.hermes.frontend.publishing.message.MessageState.State.SENDING_TO_KAFKA_PRODUCER_QUEUE; import static pl.allegro.tech.hermes.frontend.publishing.message.MessageState.State.SENT_TO_KAFKA; +import static pl.allegro.tech.hermes.frontend.publishing.message.MessageState.State.TIMEOUT_SENDING_TO_KAFKA; public class MessageState { @@ -31,7 +32,8 @@ enum State { SENT_TO_KAFKA, DELAYED_SENDING, DELAYED_PROCESSING, - DELAYED_SENT_TO_KAFKA + DELAYED_SENT_TO_KAFKA, + TIMEOUT_SENDING_TO_KAFKA, } private volatile boolean timeoutHasPassed = false; @@ -69,6 +71,10 @@ public boolean setDelayedSending() { return state.compareAndSet(SENDING_TO_KAFKA, DELAYED_SENDING); } + public boolean setTimeoutSendingToKafka() { + return state.compareAndSet(SENDING_TO_KAFKA_PRODUCER_QUEUE, TIMEOUT_SENDING_TO_KAFKA) || state.compareAndSet(SENDING_TO_KAFKA, TIMEOUT_SENDING_TO_KAFKA); + } + public boolean setReadingTimeout() { return state.compareAndSet(READING, READING_TIMEOUT); } diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/readiness/AdminReadinessService.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/readiness/AdminReadinessService.java index 942a8cce4a..15285ab214 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/readiness/AdminReadinessService.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/readiness/AdminReadinessService.java @@ -81,6 +81,6 @@ public boolean isLocalDatacenterReady() { } public boolean isDatacenterReady(String datacenter) { - return readinessPerDatacenter.getOrDefault(datacenter, true); + return readinessPerDatacenter != null && readinessPerDatacenter.getOrDefault(datacenter, true); } } diff --git a/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaBrokerMessageProducerIntegrationTest.groovy b/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducerIntegrationTest.groovy similarity index 92% rename from hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaBrokerMessageProducerIntegrationTest.groovy rename to hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducerIntegrationTest.groovy index f2486d36c1..f7b9d91c1b 100644 --- a/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaBrokerMessageProducerIntegrationTest.groovy +++ b/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducerIntegrationTest.groovy @@ -27,6 +27,7 @@ import pl.allegro.tech.hermes.frontend.config.HTTPHeadersProperties import pl.allegro.tech.hermes.frontend.config.KafkaHeaderNameProperties import pl.allegro.tech.hermes.frontend.config.SchemaProperties import pl.allegro.tech.hermes.frontend.metric.CachedTopic +import pl.allegro.tech.hermes.frontend.producer.BrokerLatencyReporter import pl.allegro.tech.hermes.frontend.publishing.avro.AvroMessage import pl.allegro.tech.hermes.frontend.server.CachedTopicsTestHelper import pl.allegro.tech.hermes.metrics.PathsCompiler @@ -38,6 +39,7 @@ import spock.lang.Specification import java.time.Duration import java.util.stream.Collectors +import static java.util.Collections.emptyList import static java.util.Collections.emptyMap import static java.util.concurrent.TimeUnit.MILLISECONDS import static org.apache.kafka.clients.consumer.ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG @@ -48,10 +50,13 @@ import static org.apache.kafka.clients.producer.ProducerConfig.KEY_SERIALIZER_CL import static org.apache.kafka.clients.producer.ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG @Testcontainers -class KafkaBrokerMessageProducerIntegrationTest extends Specification { +class LocalDatacenterMessageProducerIntegrationTest extends Specification { static Integer NUMBER_OF_PARTITION = 3 + @Shared + BrokerLatencyReporter brokerLatencyReporter = new BrokerLatencyReporter(false, null, null, null) + @Shared KafkaContainer kafkaContainer = new KafkaContainer() @@ -62,7 +67,7 @@ class KafkaBrokerMessageProducerIntegrationTest extends Specification { KafkaProducer everyoneConfirms @Shared - KafkaBrokerMessageProducer brokerMessageProducer + LocalDatacenterMessageProducer brokerMessageProducer @Shared KafkaNamesMapper kafkaNamesMapper = new JsonToAvroMigrationKafkaNamesMapper("", "_") @@ -85,6 +90,12 @@ class KafkaBrokerMessageProducerIntegrationTest extends Specification { @Shared String datacenter = "dc"; + @Shared + MetricsFacade metricsFacade = new MetricsFacade( + new SimpleMeterRegistry(), + new HermesMetrics(new MetricRegistry(), new PathsCompiler("")) + ) + def setupSpec() { kafkaContainer.start() kafkaContainer.waitingFor(Wait.forHealthcheck()) @@ -106,13 +117,13 @@ class KafkaBrokerMessageProducerIntegrationTest extends Specification { topicMetadataLoadingExecutor, minInSyncReplicasLoader, new KafkaMessageSenders.Tuple( - new KafkaMessageSender(leaderConfirms, datacenter), - new KafkaMessageSender(everyoneConfirms, datacenter) + new KafkaMessageSender(leaderConfirms, brokerLatencyReporter, metricsFacade, datacenter), + new KafkaMessageSender(everyoneConfirms, brokerLatencyReporter, metricsFacade, datacenter) ), - Collections.emptyList() + emptyList() ) - brokerMessageProducer = new KafkaBrokerMessageProducer(producers, - new MetricsFacade(new SimpleMeterRegistry(), new HermesMetrics(new MetricRegistry(), new PathsCompiler("localhost"))), + brokerMessageProducer = new LocalDatacenterMessageProducer( + producers, new MessageToKafkaProducerRecordConverter(new KafkaHeaderFactory( kafkaHeaderNameProperties, new HTTPHeadersProperties.PropagationAsKafkaHeadersProperties()), diff --git a/hermes-frontend/src/test/java/pl/allegro/tech/hermes/frontend/buffer/BackupMessagesLoaderTest.java b/hermes-frontend/src/test/java/pl/allegro/tech/hermes/frontend/buffer/BackupMessagesLoaderTest.java index 02b7a007ea..d4a8f5de74 100644 --- a/hermes-frontend/src/test/java/pl/allegro/tech/hermes/frontend/buffer/BackupMessagesLoaderTest.java +++ b/hermes-frontend/src/test/java/pl/allegro/tech/hermes/frontend/buffer/BackupMessagesLoaderTest.java @@ -66,8 +66,6 @@ public class BackupMessagesLoaderTest { private final Topic topic = TopicBuilder.topic("pl.allegro.tech.hermes.test").build(); - private final String datacenter = "dc1"; - @Before public void setUp() { tempDir = Files.createTempDir(); @@ -107,8 +105,7 @@ public void shouldNotSendOldMessages() { schemaRepository, schemaExistenceEnsurer, trackers, - localMessageStorageProperties, - datacenter + localMessageStorageProperties ); messageRepository.save(messageOfAge(1), topic); @@ -152,8 +149,7 @@ public void shouldSendAndResendMessages() { schemaRepository, schemaExistenceEnsurer, trackers, - localMessageStorageProperties, - datacenter + localMessageStorageProperties ); messageRepository.save(messageOfAge(1), topic); @@ -183,8 +179,7 @@ public void shouldSendOnlyWhenBrokerTopicIsAvailable() { schemaRepository, schemaExistenceEnsurer, trackers, - localMessageStorageProperties, - datacenter + localMessageStorageProperties ); MessageRepository messageRepository = new ChronicleMapMessageRepository( new File(tempDir.getAbsoluteFile(), "messages.dat"), @@ -223,8 +218,7 @@ public void shouldSendMessageWithAllArgumentsFromBackupMessage() { schemaRepository, schemaExistenceEnsurer, trackers, - localMessageStorageProperties, - datacenter + localMessageStorageProperties ); messageRepository.save(messageOfAge(1), topic); diff --git a/hermes-frontend/src/test/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaBrokerMessageProducerTest.java b/hermes-frontend/src/test/java/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducerTest.java similarity index 89% rename from hermes-frontend/src/test/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaBrokerMessageProducerTest.java rename to hermes-frontend/src/test/java/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducerTest.java index ce4a76a640..9d0abf3232 100644 --- a/hermes-frontend/src/test/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaBrokerMessageProducerTest.java +++ b/hermes-frontend/src/test/java/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducerTest.java @@ -23,6 +23,7 @@ import pl.allegro.tech.hermes.frontend.config.KafkaHeaderNameProperties; import pl.allegro.tech.hermes.frontend.config.SchemaProperties; import pl.allegro.tech.hermes.frontend.metric.CachedTopic; +import pl.allegro.tech.hermes.frontend.producer.BrokerLatencyReporter; import pl.allegro.tech.hermes.frontend.publishing.PublishingCallback; import pl.allegro.tech.hermes.frontend.publishing.message.JsonMessage; import pl.allegro.tech.hermes.frontend.publishing.message.Message; @@ -30,6 +31,7 @@ import java.time.Duration; import java.util.List; +import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicBoolean; import static com.google.common.base.Charsets.UTF_8; @@ -40,7 +42,7 @@ import static pl.allegro.tech.hermes.test.helper.builder.TopicBuilder.topic; @RunWith(MockitoJUnitRunner.class) -public class KafkaBrokerMessageProducerTest { +public class LocalDatacenterMessageProducerTest { private static final Long TIMESTAMP = 1L; private static final String PARTITION_KEY = "partition-key"; @@ -55,10 +57,12 @@ public class KafkaBrokerMessageProducerTest { private HermesMetrics hermesMetrics = new HermesMetrics(new MetricRegistry(), new PathsCompiler("")); private final MetricsFacade metricsFacade = new MetricsFacade(new SimpleMeterRegistry(), hermesMetrics); + private final BrokerLatencyReporter brokerLatencyReporter = new BrokerLatencyReporter(false, metricsFacade, Duration.ZERO, Executors.newSingleThreadExecutor()); + private final MockProducer leaderConfirmsProducer = new MockProducer<>(true, serializer, serializer); private final MockProducer everyoneConfirmProducer = new MockProducer<>(true, serializer, serializer); - private final KafkaMessageSender leaderConfirmsProduceWrapper = new KafkaMessageSender<>(leaderConfirmsProducer, datacenter); - private final KafkaMessageSender everyoneConfirmsProduceWrapper = new KafkaMessageSender<>(everyoneConfirmProducer, datacenter); + private final KafkaMessageSender leaderConfirmsProduceWrapper = new KafkaMessageSender<>(leaderConfirmsProducer, brokerLatencyReporter, metricsFacade, datacenter); + private final KafkaMessageSender everyoneConfirmsProduceWrapper = new KafkaMessageSender<>(everyoneConfirmProducer, brokerLatencyReporter, metricsFacade, datacenter); private final KafkaHeaderNameProperties kafkaHeaderNameProperties = new KafkaHeaderNameProperties(); private final HTTPHeadersPropagationAsKafkaHeadersProperties httpHeadersPropagationAsKafkaHeadersProperties = @@ -78,7 +82,7 @@ public class KafkaBrokerMessageProducerTest { emptyList() ); - private KafkaBrokerMessageProducer producer; + private LocalDatacenterMessageProducer producer; private final KafkaNamesMapper kafkaNamesMapper = new NamespaceKafkaNamesMapper("ns", "_"); private final KafkaHeaderFactory kafkaHeaderFactory = new KafkaHeaderFactory(kafkaHeaderNameProperties, httpHeadersPropagationAsKafkaHeadersProperties); @@ -92,7 +96,7 @@ public void before() { cachedTopic = new CachedTopic(TOPIC, metricsFacade, kafkaNamesMapper.toKafkaTopics(TOPIC)); MessageToKafkaProducerRecordConverter messageConverter = new MessageToKafkaProducerRecordConverter(kafkaHeaderFactory, schemaProperties.isIdHeaderEnabled()); - producer = new KafkaBrokerMessageProducer(kafkaMessageSenders, metricsFacade, messageConverter); + producer = new LocalDatacenterMessageProducer(kafkaMessageSenders, messageConverter); } @After @@ -141,6 +145,11 @@ public void onUnpublished(Message message, Topic topic, Exception exception) { public void onPublished(Message message, Topic topic) { callbackCalled.set(true); } + + @Override + public void onEachPublished(Message message, Topic topic, String datacenter) { + callbackCalled.set(true); + } }); //then @@ -169,6 +178,11 @@ public void onUnpublished(Message message, Topic topic, Exception exception) { public void onPublished(Message message, Topic topic) { } + + @Override + public void onEachPublished(Message message, Topic topic, String datacenter) { + + } } } diff --git a/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/client/integration/FrontendTestClient.java b/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/client/integration/FrontendTestClient.java index 1cc6f1b4c2..86964105e9 100644 --- a/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/client/integration/FrontendTestClient.java +++ b/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/client/integration/FrontendTestClient.java @@ -28,6 +28,7 @@ public class FrontendTestClient { private static final String STATUS_HEALTH_PATH = "/status/health"; private static final String STATUS_READY_PATH = "/status/ready"; private static final String STATUS_PING_PATH = "/status/ping"; + private static final String METRICS_PATH = "/status/prometheus"; private final WebTestClient webTestClient; private final FrontendSlowClient slowTestClient; @@ -169,7 +170,7 @@ WebTestClient.ResponseSpec publishWithHeaders(String topicQualifiedName, String } String publishSlowly(int clientTimeout, int pauseTimeBetweenChunks, int delayBeforeSendingFirstData, - String topicName, boolean chunkedEncoding) throws IOException, InterruptedException{ + String topicName, boolean chunkedEncoding) throws IOException, InterruptedException { return slowTestClient.slowEvent(clientTimeout, pauseTimeBetweenChunks, delayBeforeSendingFirstData, topicName, chunkedEncoding); } @@ -196,4 +197,12 @@ public WebTestClient.ResponseSpec getStatusPing() { .build()) .exchange(); } + + public WebTestClient.ResponseSpec getMetrics() { + return webTestClient.get().uri(UriBuilder + .fromUri(frontendContainerUrl) + .path(METRICS_PATH) + .build()) + .exchange(); + } } diff --git a/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/client/integration/HermesTestClient.java b/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/client/integration/HermesTestClient.java index fd2654e8d0..43c183f27a 100644 --- a/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/client/integration/HermesTestClient.java +++ b/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/client/integration/HermesTestClient.java @@ -265,6 +265,10 @@ public WebTestClient.ResponseSpec getConsumersMetrics() { return consumerTestClient.getMetrics(); } + public WebTestClient.ResponseSpec getFrontendMetrics() { + return frontendTestClient.getMetrics(); + } + public WebTestClient.ResponseSpec verifyFilters(String qualifiedTopicName, MessageFiltersVerificationInput input) { return managementTestClient.verifyFilters(qualifiedTopicName, input); diff --git a/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/assertions/PrometheusMetricsAssertion.java b/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/assertions/PrometheusMetricsAssertion.java index ef2273feac..8bfc48bd49 100644 --- a/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/assertions/PrometheusMetricsAssertion.java +++ b/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/assertions/PrometheusMetricsAssertion.java @@ -33,8 +33,16 @@ public static class PrometheusMetricWithNameAssertion { } public PrometheusMetricAssertion withLabels(String label0, String value0, - String label1, String value1, - String label2, String value2) { + String label1, String value1) { + return withLabels( + new String[]{label0, label1}, + new String[]{value0, value1} + ); + } + + public PrometheusMetricAssertion withLabels(String label0, String value0, + String label1, String value1, + String label2, String value2) { return withLabels( new String[]{label0, label1, label2}, new String[]{value0, value1, value2} @@ -42,9 +50,9 @@ public PrometheusMetricAssertion withLabels(String label0, String value0, } public PrometheusMetricAssertion withLabels(String label0, String value0, - String label1, String value1, - String label2, String value2, - String label3, String value3) { + String label1, String value1, + String label2, String value2, + String label3, String value3) { return withLabels( new String[]{label0, label1, label2, label3}, new String[]{value0, value1, value2, value3} @@ -84,11 +92,20 @@ public static class PrometheusMetricAssertion { } public void withValue(double expectedValue) { + double actualValue = extractValue(); + assertThat(actualValue).isEqualTo(expectedValue); + } + + public void withValueGreaterThan(double expectedValue) { + double actualValue = extractValue(); + assertThat(actualValue).isGreaterThan(expectedValue); + } + + private double extractValue() { Matcher matcher = METRIC_LINE_PATTERN.matcher(actualLine); if (matcher.matches()) { String valueStr = matcher.group(2); - double actualValue = Double.parseDouble(valueStr); - assertThat(actualValue).isEqualTo(expectedValue); + return Double.parseDouble(valueStr); } else { throw new IllegalStateException("Unexpected line: " + actualLine); } diff --git a/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/setup/HermesFrontendTestApp.java b/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/setup/HermesFrontendTestApp.java index 9093a641d2..5a4aa9d67c 100644 --- a/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/setup/HermesFrontendTestApp.java +++ b/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/setup/HermesFrontendTestApp.java @@ -30,7 +30,6 @@ import static pl.allegro.tech.hermes.frontend.FrontendConfigurationProperties.FRONTEND_READINESS_CHECK_KAFKA_CHECK_ENABLED; import static pl.allegro.tech.hermes.frontend.FrontendConfigurationProperties.FRONTEND_THROUGHPUT_FIXED_MAX; import static pl.allegro.tech.hermes.frontend.FrontendConfigurationProperties.FRONTEND_THROUGHPUT_TYPE; -import static pl.allegro.tech.hermes.frontend.FrontendConfigurationProperties.KAFKA_BROKER_LIST; import static pl.allegro.tech.hermes.frontend.FrontendConfigurationProperties.KAFKA_NAMESPACE; import static pl.allegro.tech.hermes.frontend.FrontendConfigurationProperties.KAFKA_PRODUCER_METADATA_MAX_AGE; import static pl.allegro.tech.hermes.frontend.FrontendConfigurationProperties.METRICS_MICROMETER_REPORT_PERIOD; @@ -42,7 +41,7 @@ public class HermesFrontendTestApp implements HermesTestApp { private final ZookeeperContainer hermesZookeeper; - private final KafkaContainerCluster kafka; + private final Map kafkaClusters; private final ConfluentSchemaRegistryContainer schemaRegistry; private SpringApplicationBuilder app; @@ -58,8 +57,20 @@ public HermesFrontendTestApp(ZookeeperContainer hermesZookeeper, KafkaContainerCluster kafka, ConfluentSchemaRegistryContainer schemaRegistry) { this.hermesZookeeper = hermesZookeeper; - this.kafka = kafka; this.schemaRegistry = schemaRegistry; + this.kafkaClusters = Map.of("dc", kafka); + + } + public HermesFrontendTestApp(ZookeeperContainer hermesZookeeper, + Map kafkaClusters, + ConfluentSchemaRegistryContainer schemaRegistry) { + this.hermesZookeeper = hermesZookeeper; + this.schemaRegistry = schemaRegistry; + this.kafkaClusters = kafkaClusters; + } + + private String kafkaClusterProperty(int index, String name) { + return String.format("frontend.kafka.clusters[%d].%s", index, name); } public HermesFrontendTestApp withProperty(String name, Object value) { @@ -78,7 +89,13 @@ private List createArgs() { args.put(FRONTEND_PORT, 0); args.put(KAFKA_NAMESPACE, "itTest"); - args.put(KAFKA_BROKER_LIST, kafka.getBootstrapServersForExternalClients()); + + var i = 0; + for (var entry : kafkaClusters.entrySet()) { + args.put(kafkaClusterProperty(i, "datacenter"), entry.getKey()); + args.put(kafkaClusterProperty(i, "brokerList"), entry.getValue().getBootstrapServersForExternalClients()); + i++; + } args.put(ZOOKEEPER_CONNECTION_STRING, hermesZookeeper.getConnectionString()); diff --git a/integration-tests/src/slowIntegrationTest/java/pl/allegro/tech/hermes/integrationtests/BrokerLatencyReportingTest.java b/integration-tests/src/slowIntegrationTest/java/pl/allegro/tech/hermes/integrationtests/BrokerLatencyReportingTest.java index 582c4e9471..fed2d0e62f 100644 --- a/integration-tests/src/slowIntegrationTest/java/pl/allegro/tech/hermes/integrationtests/BrokerLatencyReportingTest.java +++ b/integration-tests/src/slowIntegrationTest/java/pl/allegro/tech/hermes/integrationtests/BrokerLatencyReportingTest.java @@ -1,8 +1,6 @@ package pl.allegro.tech.hermes.integrationtests; import com.jayway.awaitility.Duration; -import jakarta.ws.rs.client.Client; -import jakarta.ws.rs.client.ClientBuilder; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Order; @@ -16,17 +14,12 @@ import pl.allegro.tech.hermes.test.helper.client.integration.FrontendTestClient; import pl.allegro.tech.hermes.test.helper.message.TestMessage; -import java.util.Arrays; -import java.util.Optional; - import static com.jayway.awaitility.Awaitility.waitAtMost; -import static org.assertj.core.api.AssertionsForClassTypes.assertThat; +import static pl.allegro.tech.hermes.integrationtests.assertions.HermesAssertions.assertThatMetrics; import static pl.allegro.tech.hermes.test.helper.builder.TopicBuilder.topicWithRandomName; public class BrokerLatencyReportingTest { - private final Client client = ClientBuilder.newClient(); - @Order(0) @RegisterExtension public static InfrastructureExtension infra = new InfrastructureExtension(); @@ -39,7 +32,6 @@ public class BrokerLatencyReportingTest { private static FrontendTestClient frontendTestClient; - @BeforeAll public static void setup() { frontend = new HermesFrontendTestApp(infra.hermesZookeeper(), infra.kafka(), infra.schemaRegistry()); @@ -65,18 +57,18 @@ public void shouldReportBrokerLatencyMetrics() { // then waitAtMost(Duration.FIVE_SECONDS).until(() -> { - Double metricValue = getMetricValue("hermes_frontend_broker_latency_seconds_count{ack=\"LEADER\",broker=\"localhost\",") - .orElse(0.0); - assertThat(metricValue).isGreaterThan(0.0d); + frontendTestClient.getMetrics() + .expectStatus() + .isOk() + .expectBody(String.class) + .value((body) -> assertThatMetrics(body) + .contains("hermes_frontend_broker_latency_seconds_count") + .withLabels( + "ack", "LEADER", + "broker", "localhost" + ) + .withValueGreaterThan(0.0d) + ); }); } - - private Optional getMetricValue(String metricPrefix) { - String metricsResponse = client.target("http://127.0.0.1:" + frontend.getPort() + "/status/prometheus").request().get(String.class); - return Arrays.stream(metricsResponse.split("\n")) - .filter(metricName -> metricName.startsWith(metricPrefix)) - .findFirst() - .map(line -> line.split(" ")[1]) // metrics have format " " - .map(Double::valueOf); - } } \ No newline at end of file diff --git a/integration-tests/src/slowIntegrationTest/java/pl/allegro/tech/hermes/integrationtests/RemoteDatacenterProduceFallbackTest.java b/integration-tests/src/slowIntegrationTest/java/pl/allegro/tech/hermes/integrationtests/RemoteDatacenterProduceFallbackTest.java new file mode 100644 index 0000000000..eb299a22ca --- /dev/null +++ b/integration-tests/src/slowIntegrationTest/java/pl/allegro/tech/hermes/integrationtests/RemoteDatacenterProduceFallbackTest.java @@ -0,0 +1,193 @@ +package pl.allegro.tech.hermes.integrationtests; + +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.testcontainers.lifecycle.Startable; +import pl.allegro.tech.hermes.api.Topic; +import pl.allegro.tech.hermes.integrationtests.setup.HermesConsumersTestApp; +import pl.allegro.tech.hermes.integrationtests.setup.HermesFrontendTestApp; +import pl.allegro.tech.hermes.integrationtests.setup.HermesManagementTestApp; +import pl.allegro.tech.hermes.integrationtests.subscriber.TestSubscriber; +import pl.allegro.tech.hermes.integrationtests.subscriber.TestSubscribersExtension; +import pl.allegro.tech.hermes.test.helper.client.integration.HermesInitHelper; +import pl.allegro.tech.hermes.test.helper.client.integration.HermesTestClient; +import pl.allegro.tech.hermes.test.helper.containers.ConfluentSchemaRegistryContainer; +import pl.allegro.tech.hermes.test.helper.containers.KafkaContainerCluster; +import pl.allegro.tech.hermes.test.helper.containers.ZookeeperContainer; +import pl.allegro.tech.hermes.test.helper.message.TestMessage; + +import java.util.Map; +import java.util.stream.Stream; + +import static pl.allegro.tech.hermes.infrastructure.dc.DefaultDatacenterNameProvider.DEFAULT_DC_NAME; +import static pl.allegro.tech.hermes.integrationtests.assertions.HermesAssertions.assertThatMetrics; +import static pl.allegro.tech.hermes.test.helper.builder.SubscriptionBuilder.subscription; +import static pl.allegro.tech.hermes.test.helper.builder.TopicBuilder.topicWithRandomName; + +public class RemoteDatacenterProduceFallbackTest { + + @RegisterExtension + public static final TestSubscribersExtension subscribers = new TestSubscribersExtension(); + + private static final ConfluentSchemaRegistryContainer schemaRegistry = new ConfluentSchemaRegistryContainer(); + private static final HermesDatacenter dc1 = new HermesDatacenter(); + private static final HermesDatacenter dc2 = new HermesDatacenter(); + + private static HermesManagementTestApp management; + private static HermesInitHelper initHelper; + private static HermesFrontendTestApp frontendDC1; + private static HermesConsumersTestApp consumerDC1; + private static HermesConsumersTestApp consumerDC2; + + private static HermesTestClient DC1; + + @BeforeAll + public static void setup() { + Stream.of(dc1, dc2) + .parallel() + .forEach(HermesDatacenter::startKafkaAndZookeeper); + schemaRegistry.start(); + management = new HermesManagementTestApp( + Map.of(DEFAULT_DC_NAME, dc1.hermesZookeeper, "dc2", dc2.hermesZookeeper), + Map.of(DEFAULT_DC_NAME, dc1.kafka, "dc2", dc2.kafka), + schemaRegistry + ); + management.start(); + frontendDC1 = new HermesFrontendTestApp(dc1.hermesZookeeper, + Map.of("dc", dc1.kafka, "dc2", dc2.kafka), + schemaRegistry + ); + frontendDC1.start(); + + consumerDC1 = new HermesConsumersTestApp(dc1.hermesZookeeper, dc1.kafka, schemaRegistry); + consumerDC1.start(); + + consumerDC2 = new HermesConsumersTestApp(dc2.hermesZookeeper, dc2.kafka, schemaRegistry); + consumerDC2.start(); + + DC1 = new HermesTestClient(management.getPort(), frontendDC1.getPort(), consumerDC1.getPort()); + initHelper = new HermesInitHelper(management.getPort()); + } + + @AfterAll + public static void clean() { + management.stop(); + consumerDC2.stop(); + frontendDC1.stop(); + consumerDC1.stop(); + schemaRegistry.stop(); + Stream.of(dc1, dc2) + .parallel() + .forEach(HermesDatacenter::stop); + } + + @AfterEach + public void afterEach() { + Stream.of(dc1, dc2).forEach(dc -> dc.kafka.restoreConnectionsBetweenBrokersAndClients()); + DC1.setReadiness(DEFAULT_DC_NAME, true); + DC1.setReadiness("dc2", true); + } + + @Test + public void shouldPublishAndConsumeViaRemoteDCWhenLocalKafkaIsUnavailable() { + // given + TestSubscriber subscriber = subscribers.createSubscriber(); + Topic topic = initHelper.createTopic(topicWithRandomName().withFallbackToRemoteDatacenterEnabled().build()); + initHelper.createSubscription( + subscription(topic.getQualifiedName(), "subscription", subscriber.getEndpoint()).build() + ); + + // when dc1 is not available + dc1.kafka.cutOffConnectionsBetweenBrokersAndClients(); + + // and message is published to dc1 + TestMessage message = TestMessage.of("key1", "value1"); + DC1.publishUntilSuccess(topic.getQualifiedName(), message.body()); + + // then message is received in dc2 + subscriber.waitUntilReceived(message.body()); + + // and metrics that message was published to remote dc is incremented + DC1.getFrontendMetrics() + .expectStatus() + .isOk() + .expectBody(String.class) + .value((body) -> assertThatMetrics(body) + .contains("hermes_frontend_topic_published_total") + .withLabels( + "group", topic.getName().getGroupName(), + "topic", topic.getName().getName(), + "storageDc", "dc2" + ) + .withValue(1.0) + ); + } + + @Test + public void shouldReturn500whenBothDCsAreUnavailable() { + // given + TestSubscriber subscriber = subscribers.createSubscriber(); + Topic topic = initHelper.createTopic(topicWithRandomName().withFallbackToRemoteDatacenterEnabled().build()); + initHelper.createSubscription( + subscription(topic.getQualifiedName(), "subscription", subscriber.getEndpoint()).build() + ); + + // when both dcs are not available + dc1.kafka.cutOffConnectionsBetweenBrokersAndClients(); + dc2.kafka.cutOffConnectionsBetweenBrokersAndClients(); + + // and message is published + TestMessage message = TestMessage.of("key1", "value1"); + DC1.publishUntilStatus(topic.getQualifiedName(), message.body(), 503); + + // then no messages are received + subscriber.noMessagesReceived(); + } + + @Test + public void shouldNotFallBackToNotReadyDatacenter() { + // given + TestSubscriber subscriber = subscribers.createSubscriber(); + Topic topic = initHelper.createTopic(topicWithRandomName().withFallbackToRemoteDatacenterEnabled().build()); + initHelper.createSubscription( + subscription(topic.getQualifiedName(), "subscription", subscriber.getEndpoint()).build() + ); + + // when local datacenter is not available and remote is not ready + dc1.kafka.cutOffConnectionsBetweenBrokersAndClients(); + DC1.setReadiness("dc2", false); + + // and message is published + TestMessage message = TestMessage.of("key1", "value1"); + DC1.publishUntilStatus(topic.getQualifiedName(), message.body(), 503); + + // then no messages are received + subscriber.noMessagesReceived(); + } + + private static class HermesDatacenter { + + private final ZookeeperContainer hermesZookeeper = new ZookeeperContainer("HermesZookeeper"); + private final KafkaContainerCluster kafka = new KafkaContainerCluster(1); + + public HermesDatacenter() { + schemaRegistry.withKafkaCluster(kafka); + } + + void startKafkaAndZookeeper() { + Stream.of(hermesZookeeper, kafka) + .parallel() + .forEach(Startable::start); + } + + + void stop() { + Stream.of(hermesZookeeper, kafka) + .parallel() + .forEach(Startable::stop); + } + } +} From 300dd1fc4ce4c6143e6b48c8d8a4b4180bfc9221 Mon Sep 17 00:00:00 2001 From: Maciej Moscicki Date: Fri, 12 Apr 2024 15:41:47 +0200 Subject: [PATCH 40/87] add visibility into empty broadcast messages (#1843) --- .../sender/MultiMessageSendingResult.java | 10 ++++++--- .../http/JettyBroadCastMessageSender.java | 22 ++++++++++++++----- 2 files changed, 24 insertions(+), 8 deletions(-) diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/sender/MultiMessageSendingResult.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/sender/MultiMessageSendingResult.java index fed25180c4..6f7f18caec 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/sender/MultiMessageSendingResult.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/sender/MultiMessageSendingResult.java @@ -94,9 +94,13 @@ public List getChildren() { @Override public String getRootCause() { - return children.stream() - .map(child -> child.getRequestUri().map(Object::toString).orElse("") + ":" + child.getRootCause()) - .collect(joining(";")); + if (children.isEmpty()) { + return "Empty children message results"; + } else { + return children.stream() + .map(child -> child.getRequestUri().map(Object::toString).orElse("") + ":" + child.getRootCause()) + .collect(joining(";")); + } } } diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/sender/http/JettyBroadCastMessageSender.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/sender/http/JettyBroadCastMessageSender.java index 3b1607d3be..73cabc57e8 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/sender/http/JettyBroadCastMessageSender.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/sender/http/JettyBroadCastMessageSender.java @@ -2,6 +2,8 @@ import com.google.common.collect.ImmutableList; import org.eclipse.jetty.client.Request; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import pl.allegro.tech.hermes.consumers.consumer.Message; import pl.allegro.tech.hermes.consumers.consumer.ResilientMessageSender; import pl.allegro.tech.hermes.consumers.consumer.sender.MessageSender; @@ -14,6 +16,7 @@ import pl.allegro.tech.hermes.consumers.consumer.sender.resolver.EndpointAddressResolutionException; import pl.allegro.tech.hermes.consumers.consumer.sender.resolver.ResolvableEndpointAddress; +import java.net.URI; import java.util.Collections; import java.util.List; import java.util.concurrent.CompletableFuture; @@ -22,6 +25,8 @@ public class JettyBroadCastMessageSender implements MessageSender { + private static final Logger logger = LoggerFactory.getLogger(JettyBroadCastMessageSender.class); + private final HttpRequestFactory requestFactory; private final ResolvableEndpointAddress endpoint; private final HttpHeadersProvider requestHeadersProvider; @@ -70,11 +75,18 @@ private List> collectResults( HttpRequestHeaders headers = requestHeadersProvider.getHeaders(message, requestData); - return endpoint.resolveAllFor(message).stream() - .filter(uri -> message.hasNotBeenSentTo(uri.toString())) - .map(uri -> requestFactory.buildRequest(message, uri, headers)) - .map(this::processResponse) - .collect(Collectors.toList()); + List resolvedUris = endpoint.resolveAllFor(message).stream() + .filter(uri -> message.hasNotBeenSentTo(uri.toString())).toList(); + + if (resolvedUris.isEmpty()) { + logger.debug("Empty resolved URIs for message: {}", message.getId()); + return Collections.emptyList(); + } else { + return resolvedUris.stream() + .map(uri -> requestFactory.buildRequest(message, uri, headers)) + .map(this::processResponse) + .collect(Collectors.toList()); + } } private CompletableFuture> mergeResults(List> results) { From 9da6649ab409806b44ce9259ef4c3760e4d80b5a Mon Sep 17 00:00:00 2001 From: michal494 <84710247+michal494@users.noreply.github.com> Date: Wed, 17 Apr 2024 11:06:20 +0200 Subject: [PATCH 41/87] #1748 Use metrics provided by Micrometer for monitoring executors (#1824) * #1748 WIP * #1748 Add monitoring for Zookeeper cache executor service * #1748 Remove unnecessary method * #1748 Remove unused methods * #1748 Remove unused constants * #1748 Test for monitoring of the executors with Micrometer --------- Co-authored-by: Michal Ciszewski --- ...elAwareZookeeperNotifyingCacheFactory.java | 23 +++++- .../hermes/common/metric/ExecutorMetrics.java | 49 +++---------- .../tech/hermes/common/metric/Gauges.java | 9 --- .../hermes/common/metric/HermesMetrics.java | 28 -------- .../hermes/common/metric/MetricsFacade.java | 2 +- .../InstrumentedExecutorServiceFactory.java | 38 ++++------ .../metric/executor/ThreadPoolMetrics.java | 43 ------------ .../ModelAwareZookeeperNotifyingCache.java | 14 ++-- ...edExecutorServiceFactoryMetricsTest.groovy | 70 +++++++++++++++++++ .../consumers/config/CommonConfiguration.java | 14 ++-- .../HttpClientConnectionMonitoringTest.groovy | 4 +- .../sender/http/JettyMessageSenderTest.java | 3 +- .../ConsumerTestRuntimeEnvironment.java | 4 +- .../frontend/config/CommonConfiguration.java | 13 ++-- 14 files changed, 128 insertions(+), 186 deletions(-) delete mode 100644 hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/executor/ThreadPoolMetrics.java create mode 100644 hermes-common/src/test/groovy/pl/allegro/tech/hermes/common/metric/executor/InstrumentedExecutorServiceFactoryMetricsTest.groovy diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/di/factories/ModelAwareZookeeperNotifyingCacheFactory.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/di/factories/ModelAwareZookeeperNotifyingCacheFactory.java index cbd1f1998e..21326ac177 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/di/factories/ModelAwareZookeeperNotifyingCacheFactory.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/di/factories/ModelAwareZookeeperNotifyingCacheFactory.java @@ -1,23 +1,35 @@ package pl.allegro.tech.hermes.common.di.factories; +import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.curator.framework.CuratorFramework; +import pl.allegro.tech.hermes.common.cache.queue.LinkedHashSetBlockingQueue; +import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.infrastructure.zookeeper.cache.ModelAwareZookeeperNotifyingCache; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + public class ModelAwareZookeeperNotifyingCacheFactory { private final CuratorFramework curator; + private final MetricsFacade metricsFacade; + private final ZookeeperParameters zookeeperParameters; - public ModelAwareZookeeperNotifyingCacheFactory(CuratorFramework curator, ZookeeperParameters zookeeperParameters) { + public ModelAwareZookeeperNotifyingCacheFactory(CuratorFramework curator, MetricsFacade metricaFacade, ZookeeperParameters zookeeperParameters) { this.curator = curator; + this.metricsFacade = metricaFacade; this.zookeeperParameters = zookeeperParameters; } public ModelAwareZookeeperNotifyingCache provide() { String rootPath = zookeeperParameters.getRoot(); + ExecutorService executor = createExecutor(rootPath, zookeeperParameters.getProcessingThreadPoolSize()); ModelAwareZookeeperNotifyingCache cache = new ModelAwareZookeeperNotifyingCache( - curator, rootPath, zookeeperParameters.getProcessingThreadPoolSize() + curator, executor, rootPath ); try { cache.start(); @@ -26,4 +38,11 @@ public ModelAwareZookeeperNotifyingCache provide() { } return cache; } + + private ExecutorService createExecutor(String rootPath, int processingThreadPoolSize) { + ThreadFactory threadFactory = new ThreadFactoryBuilder().setNameFormat(rootPath + "-zk-cache-%d").build(); + ExecutorService executor = new ThreadPoolExecutor(1, processingThreadPoolSize, + Integer.MAX_VALUE, TimeUnit.SECONDS, new LinkedHashSetBlockingQueue<>(), threadFactory); + return metricsFacade.executor().monitor(executor, rootPath + "zk-cache"); + } } diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/ExecutorMetrics.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/ExecutorMetrics.java index e51acfe8fc..8843d79829 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/ExecutorMetrics.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/ExecutorMetrics.java @@ -1,56 +1,23 @@ package pl.allegro.tech.hermes.common.metric; import io.micrometer.core.instrument.MeterRegistry; -import io.micrometer.core.instrument.Tags; +import io.micrometer.core.instrument.binder.jvm.ExecutorServiceMetrics; -import java.util.function.ToDoubleFunction; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.ScheduledExecutorService; public class ExecutorMetrics { - private final HermesMetrics hermesMetrics; private final MeterRegistry meterRegistry; - public ExecutorMetrics(HermesMetrics hermesMetrics, MeterRegistry meterRegistry) { - this.hermesMetrics = hermesMetrics; + public ExecutorMetrics(MeterRegistry meterRegistry) { this.meterRegistry = meterRegistry; } - public void registerThreadPoolCapacity(String executorName, T stateObj, ToDoubleFunction f) { - hermesMetrics.registerThreadPoolCapacity(executorName, () -> (int) f.applyAsDouble(stateObj)); - registerMicrometerGauge("executors.capacity", executorName, stateObj, f); + public ExecutorService monitor(ExecutorService executorService, String executorName) { + return ExecutorServiceMetrics.monitor(meterRegistry, executorService, executorName); } - public void registerThreadPoolActiveThreads(String executorName, T stateObj, ToDoubleFunction f) { - hermesMetrics.registerThreadPoolActiveThreads(executorName, () -> (int) f.applyAsDouble(stateObj)); - registerMicrometerGauge("executors.active-threads", executorName, stateObj, f); - } - - public void registerThreadPoolUtilization(String executorName, T stateObj, ToDoubleFunction f) { - hermesMetrics.registerThreadPoolUtilization(executorName, () -> f.applyAsDouble(stateObj)); - registerMicrometerGauge("executors.utilization", executorName, stateObj, f); - } - - public void registerThreadPoolTaskQueueCapacity(String executorName, T stateObj, ToDoubleFunction f) { - hermesMetrics.registerThreadPoolTaskQueueCapacity(executorName, () -> (int) f.applyAsDouble(stateObj)); - registerMicrometerGauge("executors.task-queue-capacity", executorName, stateObj, f); - } - - public void registerThreadPoolTaskQueued(String executorName, T stateObj, ToDoubleFunction f) { - hermesMetrics.registerThreadPoolTaskQueued(executorName, () -> (int) f.applyAsDouble(stateObj)); - registerMicrometerGauge("executors.task-queue-size", executorName, stateObj, f); - } - - public void registerThreadPoolTaskQueueUtilization(String executorName, T stateObj, ToDoubleFunction f) { - hermesMetrics.registerThreadPoolTaskQueueUtilization(executorName, () -> f.applyAsDouble(stateObj)); - registerMicrometerGauge("executors.task-queue-utilization", executorName, stateObj, f); - } - - public void incrementRequestRejectedCounter(String executorName) { - hermesMetrics.incrementThreadPoolTaskRejectedCount(executorName); - meterRegistry.counter("executors.task-rejected", Tags.of("executor_name", executorName)).increment(); - } - - - private void registerMicrometerGauge(String name, String executorName, T stateObj, ToDoubleFunction f) { - meterRegistry.gauge(name, Tags.of("executor_name", executorName), stateObj, f); + public ScheduledExecutorService monitor(ScheduledExecutorService scheduledExecutorService, String executorName) { + return ExecutorServiceMetrics.monitor(meterRegistry, scheduledExecutorService, executorName); } } diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/Gauges.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/Gauges.java index e7f2f1fa8e..c75cbfcdea 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/Gauges.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/Gauges.java @@ -1,6 +1,5 @@ package pl.allegro.tech.hermes.common.metric; -import static pl.allegro.tech.hermes.metrics.PathsCompiler.EXECUTOR_NAME; import static pl.allegro.tech.hermes.metrics.PathsCompiler.GROUP; import static pl.allegro.tech.hermes.metrics.PathsCompiler.SUBSCRIPTION; import static pl.allegro.tech.hermes.metrics.PathsCompiler.TOPIC; @@ -34,13 +33,5 @@ public class Gauges { public static final String CONSUMER_SENDER_HTTP_2_SERIAL_CLIENT_CONNECTIONS = "http-clients.serial.http2.connections"; public static final String CONSUMER_SENDER_HTTP_2_SERIAL_CLIENT_PENDING_CONNECTIONS = "http-clients.serial.http2.pending-connections"; - public static final String EXECUTORS = "executors."; - public static final String EXECUTOR_ACTIVE_THREADS = EXECUTORS + EXECUTOR_NAME + ".active-threads"; - public static final String EXECUTOR_CAPACITY = EXECUTORS + EXECUTOR_NAME + ".capacity"; - public static final String UTILIZATION = EXECUTORS + EXECUTOR_NAME + ".utilization"; - public static final String TASK_QUEUE_CAPACITY = EXECUTORS + EXECUTOR_NAME + ".task-queue-capacity"; - public static final String TASK_QUEUED = EXECUTORS + EXECUTOR_NAME + ".task-queue-size"; - public static final String TASKS_QUEUE_UTILIZATION = EXECUTORS + EXECUTOR_NAME + ".task-queue-utilization"; - public static final String TASKS_REJECTED_COUNT = EXECUTORS + EXECUTOR_NAME + "task-rejected"; public static final String INFLIGHT = "inflight." + GROUP + "." + TOPIC + "." + SUBSCRIPTION + ".count"; } diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/HermesMetrics.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/HermesMetrics.java index cb341c0253..23d477ac6e 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/HermesMetrics.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/HermesMetrics.java @@ -112,34 +112,6 @@ public void registerConsumerSenderHttp2RequestQueueSize(Gauge gauge) { metricRegistry.register(metricRegistryName(Gauges.CONSUMER_SENDER_HTTP_2_SERIAL_CLIENT_REQUEST_QUEUE_SIZE), gauge); } - public void registerThreadPoolActiveThreads(String executorName, Gauge gauge) { - registerExecutorGauge(Gauges.EXECUTOR_ACTIVE_THREADS, executorName, gauge); - } - - public void registerThreadPoolCapacity(String executorName, Gauge gauge) { - registerExecutorGauge(Gauges.EXECUTOR_CAPACITY, executorName, gauge); - } - - public void registerThreadPoolUtilization(String executorName, Gauge gauge) { - registerExecutorGauge(Gauges.UTILIZATION, executorName, gauge); - } - - public void registerThreadPoolTaskQueueCapacity(String executorName, Gauge gauge) { - registerExecutorGauge(Gauges.TASK_QUEUE_CAPACITY, executorName, gauge); - } - - public void registerThreadPoolTaskQueued(String executorName, Gauge gauge) { - registerExecutorGauge(Gauges.TASK_QUEUED, executorName, gauge); - } - - public void registerThreadPoolTaskQueueUtilization(String executorName, Gauge gauge) { - registerExecutorGauge(Gauges.TASKS_QUEUE_UTILIZATION, executorName, gauge); - } - - public void incrementThreadPoolTaskRejectedCount(String executorName) { - executorCounter(Gauges.TASKS_REJECTED_COUNT, executorName).inc(); - } - public void registerInflightGauge(SubscriptionName subscription, Gauge gauge) { registerGauge(metricRegistryName(Gauges.INFLIGHT, subscription.getTopicName(), subscription.getName()), gauge); } diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/MetricsFacade.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/MetricsFacade.java index 016cf8436b..ab57f56456 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/MetricsFacade.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/MetricsFacade.java @@ -56,7 +56,7 @@ public MetricsFacade(MeterRegistry meterRegistry, HermesMetrics hermesMetrics) { this.trackerElasticSearchMetrics = new TrackerElasticSearchMetrics(hermesMetrics, meterRegistry); this.persistentBufferMetrics = new PersistentBufferMetrics(hermesMetrics, meterRegistry); this.producerMetrics = new ProducerMetrics(hermesMetrics, meterRegistry); - this.executorMetrics = new ExecutorMetrics(hermesMetrics, meterRegistry); + this.executorMetrics = new ExecutorMetrics(meterRegistry); this.schemaClientMetrics = new SchemaClientMetrics(hermesMetrics, meterRegistry); this.undeliveredMessagesMetrics = new UndeliveredMessagesMetrics(hermesMetrics, meterRegistry); this.deserializationMetrics = new DeserializationMetrics(hermesMetrics, meterRegistry); diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/executor/InstrumentedExecutorServiceFactory.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/executor/InstrumentedExecutorServiceFactory.java index f137e1cf7f..621876228e 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/executor/InstrumentedExecutorServiceFactory.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/executor/InstrumentedExecutorServiceFactory.java @@ -1,6 +1,7 @@ package pl.allegro.tech.hermes.common.metric.executor; import com.google.common.util.concurrent.ThreadFactoryBuilder; +import pl.allegro.tech.hermes.common.metric.MetricsFacade; import java.util.concurrent.ExecutorService; import java.util.concurrent.LinkedBlockingQueue; @@ -11,14 +12,13 @@ import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; - public class InstrumentedExecutorServiceFactory { - private final ThreadPoolMetrics threadPoolMetrics; + private final MetricsFacade metricsFacade; private final RejectedExecutionHandler rejectedExecutionHandler = new ThreadPoolExecutor.AbortPolicy(); - public InstrumentedExecutorServiceFactory(ThreadPoolMetrics threadPoolMetrics) { - this.threadPoolMetrics = threadPoolMetrics; + public InstrumentedExecutorServiceFactory(MetricsFacade metricsFacade) { + this.metricsFacade = metricsFacade; } public ExecutorService getExecutorService(String name, int size, boolean monitoringEnabled) { @@ -30,11 +30,7 @@ public ExecutorService getExecutorService(String name, int size, boolean monitor ThreadPoolExecutor executor = newFixedThreadPool(name, size, threadFactory, queueCapacity); executor.prestartAllCoreThreads(); - if (monitoringEnabled) { - monitor(name, executor); - } - - return executor; + return monitoringEnabled ? monitor(name, executor) : executor; } public ScheduledExecutorService getScheduledExecutorService( @@ -42,18 +38,16 @@ public ScheduledExecutorService getScheduledExecutorService( ) { ThreadFactory threadFactory = new ThreadFactoryBuilder().setNameFormat(name + "-scheduled-executor-%d").build(); - ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(size, threadFactory); + return monitoringEnabled ? monitor(name, executor) : executor; + } - if (monitoringEnabled) { - monitor(name, executor); - } - - return executor; + private ExecutorService monitor(String threadPoolName, ExecutorService executor) { + return metricsFacade.executor().monitor(executor, threadPoolName); } - private void monitor(String threadPoolName, ThreadPoolExecutor executor) { - threadPoolMetrics.createGauges(threadPoolName, executor); + private ScheduledExecutorService monitor(String threadPoolName, ScheduledExecutorService executor) { + return metricsFacade.executor().monitor(executor, threadPoolName); } /** @@ -68,16 +62,8 @@ private ThreadPoolExecutor newFixedThreadPool(String executorName, int size, Thr TimeUnit.MILLISECONDS, new LinkedBlockingQueue<>(queueCapacity), threadFactory, - getMeteredRejectedExecutionHandler(executorName) + rejectedExecutionHandler ); return executor; } - - RejectedExecutionHandler getMeteredRejectedExecutionHandler(String executorName) { - return (r, executor) -> { - threadPoolMetrics.markRequestRejected(executorName); - rejectedExecutionHandler.rejectedExecution(r, executor); - }; - } - } diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/executor/ThreadPoolMetrics.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/executor/ThreadPoolMetrics.java deleted file mode 100644 index 200d0d8fcb..0000000000 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/executor/ThreadPoolMetrics.java +++ /dev/null @@ -1,43 +0,0 @@ -package pl.allegro.tech.hermes.common.metric.executor; - -import pl.allegro.tech.hermes.common.metric.MetricsFacade; - -import java.util.concurrent.ThreadPoolExecutor; - -public class ThreadPoolMetrics { - - private final MetricsFacade metricsFacade; - - public ThreadPoolMetrics(MetricsFacade metricsFacade) { - this.metricsFacade = metricsFacade; - } - - public void createGauges( - String executorName, - ThreadPoolExecutor executor) { - - metricsFacade.executor().registerThreadPoolCapacity(executorName, executor, ThreadPoolExecutor::getPoolSize); - metricsFacade.executor().registerThreadPoolActiveThreads(executorName, executor, ThreadPoolExecutor::getActiveCount); - metricsFacade.executor().registerThreadPoolUtilization(executorName, executor, - e -> (double) e.getActiveCount() / (double) e.getPoolSize() - ); - metricsFacade.executor().registerThreadPoolTaskQueueCapacity(executorName, executor, - e -> { - int qCapacity = e.getQueue().size() + e.getQueue().remainingCapacity(); - // overflow in case of unbounded queue, set queueCapacity to Integer.MAX_VALUE - return qCapacity < 0 ? Integer.MAX_VALUE : qCapacity; - }); - metricsFacade.executor().registerThreadPoolTaskQueued(executorName, executor, e -> e.getQueue().size()); - metricsFacade.executor().registerThreadPoolTaskQueueUtilization(executorName, executor, - e -> { - int calculatedCapacity = e.getQueue().size() + e.getQueue().remainingCapacity(); - int queueCapacity = calculatedCapacity < 0 ? Integer.MAX_VALUE : calculatedCapacity; - return (double) e.getQueue().size() / (double) queueCapacity; - }); - } - - public void markRequestRejected(String executorName) { - metricsFacade.executor().incrementRequestRejectedCounter(executorName); - } - -} diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/infrastructure/zookeeper/cache/ModelAwareZookeeperNotifyingCache.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/infrastructure/zookeeper/cache/ModelAwareZookeeperNotifyingCache.java index 6a544acd01..c42921f558 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/infrastructure/zookeeper/cache/ModelAwareZookeeperNotifyingCache.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/infrastructure/zookeeper/cache/ModelAwareZookeeperNotifyingCache.java @@ -1,18 +1,14 @@ package pl.allegro.tech.hermes.infrastructure.zookeeper.cache; -import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import pl.allegro.tech.hermes.common.cache.queue.LinkedHashSetBlockingQueue; import pl.allegro.tech.hermes.infrastructure.zookeeper.ZookeeperPaths; import java.util.Arrays; import java.util.List; -import java.util.concurrent.ThreadFactory; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; +import java.util.concurrent.ExecutorService; import java.util.function.Consumer; public class ModelAwareZookeeperNotifyingCache { @@ -26,15 +22,13 @@ public class ModelAwareZookeeperNotifyingCache { private static final int SUBSCRIPTION_LEVEL = 2; private final HierarchicalCache cache; - private final ThreadPoolExecutor executor; + private final ExecutorService executor; - public ModelAwareZookeeperNotifyingCache(CuratorFramework curator, String rootPath, int processingThreadPoolSize) { + public ModelAwareZookeeperNotifyingCache(CuratorFramework curator, ExecutorService executor, String rootPath) { List levelPrefixes = Arrays.asList( ZookeeperPaths.GROUPS_PATH, ZookeeperPaths.TOPICS_PATH, ZookeeperPaths.SUBSCRIPTIONS_PATH ); - ThreadFactory threadFactory = new ThreadFactoryBuilder().setNameFormat(rootPath + "-zk-cache-%d").build(); - executor = new ThreadPoolExecutor(1, processingThreadPoolSize, - Integer.MAX_VALUE, TimeUnit.SECONDS, new LinkedHashSetBlockingQueue<>(), threadFactory); + this.executor = executor; this.cache = new HierarchicalCache( curator, executor, diff --git a/hermes-common/src/test/groovy/pl/allegro/tech/hermes/common/metric/executor/InstrumentedExecutorServiceFactoryMetricsTest.groovy b/hermes-common/src/test/groovy/pl/allegro/tech/hermes/common/metric/executor/InstrumentedExecutorServiceFactoryMetricsTest.groovy new file mode 100644 index 0000000000..91d5ad69ac --- /dev/null +++ b/hermes-common/src/test/groovy/pl/allegro/tech/hermes/common/metric/executor/InstrumentedExecutorServiceFactoryMetricsTest.groovy @@ -0,0 +1,70 @@ +package pl.allegro.tech.hermes.common.metric.executor + +import com.codahale.metrics.MetricRegistry +import io.micrometer.core.instrument.MeterRegistry +import io.micrometer.core.instrument.search.Search +import io.micrometer.core.instrument.simple.SimpleMeterRegistry +import pl.allegro.tech.hermes.common.metric.HermesMetrics +import pl.allegro.tech.hermes.common.metric.MetricsFacade +import pl.allegro.tech.hermes.metrics.PathsCompiler +import spock.lang.Specification +import spock.lang.Subject + +import java.util.concurrent.ExecutorService +import java.util.concurrent.Future +import java.util.concurrent.ScheduledExecutorService +import java.util.concurrent.ScheduledFuture + +import static java.util.concurrent.TimeUnit.SECONDS + +class InstrumentedExecutorServiceFactoryMetricsTest extends Specification { + + private final MeterRegistry meterRegistry = new SimpleMeterRegistry() + + @Subject + private final InstrumentedExecutorServiceFactory factory = + new InstrumentedExecutorServiceFactory( + new MetricsFacade( + meterRegistry, + new HermesMetrics(new MetricRegistry(), new PathsCompiler("host")) + ) + ) + + def "should record metrics for executor service (monitoring enabled: #monitoringEnabled)"() { + given: + ExecutorService executor = factory.getExecutorService("test-executor", 10, monitoringEnabled) + + when: + Future task = executor.submit { println("task executed") } + task.get() + + then: + metric("executor.completed", "test-executor").functionCounter()?.count() == expectedCompletedCount + + where: + monitoringEnabled || expectedCompletedCount + true || 1.0d + false || null + } + + def "should record metrics for scheduled executor service (monitoring enabled: #monitoringEnabled)"() { + given: + ScheduledExecutorService executor = factory.getScheduledExecutorService("test-scheduled-executor", 10, monitoringEnabled) + + when: + ScheduledFuture task = executor.schedule({ println("scheduled task executed") }, 1, SECONDS) + task.get() + + then: + metric("executor.completed", "test-scheduled-executor").functionCounter()?.count() == expectedCompletedCount + + where: + monitoringEnabled || expectedCompletedCount + true || 1.0d + false || null + } + + private Search metric(String name, String executorName) { + return meterRegistry.find(name).tag("name", executorName) + } +} diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/CommonConfiguration.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/CommonConfiguration.java index 260e224196..218ccec73f 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/CommonConfiguration.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/CommonConfiguration.java @@ -40,7 +40,6 @@ import pl.allegro.tech.hermes.common.metric.counter.CounterStorage; import pl.allegro.tech.hermes.common.metric.counter.zookeeper.ZookeeperCounterStorage; import pl.allegro.tech.hermes.common.metric.executor.InstrumentedExecutorServiceFactory; -import pl.allegro.tech.hermes.common.metric.executor.ThreadPoolMetrics; import pl.allegro.tech.hermes.common.util.InetAddressInstanceIdResolver; import pl.allegro.tech.hermes.common.util.InstanceIdResolver; import pl.allegro.tech.hermes.domain.filtering.MessageFilters; @@ -151,10 +150,11 @@ public InternalNotificationsBus zookeeperInternalNotificationBus(ObjectMapper ob @Bean(destroyMethod = "stop") public ModelAwareZookeeperNotifyingCache modelAwareZookeeperNotifyingCache(CuratorFramework curator, + MetricsFacade metricsFacade, ZookeeperClustersProperties zookeeperClustersProperties, DatacenterNameProvider datacenterNameProvider) { ZookeeperProperties zookeeperProperties = zookeeperClustersProperties.toZookeeperProperties(datacenterNameProvider); - return new ModelAwareZookeeperNotifyingCacheFactory(curator, zookeeperProperties).provide(); + return new ModelAwareZookeeperNotifyingCacheFactory(curator, metricsFacade, zookeeperProperties).provide(); } @Bean @@ -165,15 +165,9 @@ public UndeliveredMessageLog undeliveredMessageLog(CuratorFramework zookeeper, return new ZookeeperUndeliveredMessageLog(zookeeper, paths, mapper, metricsFacade); } - - @Bean - public ThreadPoolMetrics threadPoolMetrics(MetricsFacade metricsFacade) { - return new ThreadPoolMetrics(metricsFacade); - } - @Bean - public InstrumentedExecutorServiceFactory instrumentedExecutorServiceFactory(ThreadPoolMetrics threadPoolMetrics) { - return new InstrumentedExecutorServiceFactory(threadPoolMetrics); + public InstrumentedExecutorServiceFactory instrumentedExecutorServiceFactory(MetricsFacade metricsFacade) { + return new InstrumentedExecutorServiceFactory(metricsFacade); } @Bean diff --git a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/sender/http/HttpClientConnectionMonitoringTest.groovy b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/sender/http/HttpClientConnectionMonitoringTest.groovy index e6ad62185c..fed0b23ef0 100644 --- a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/sender/http/HttpClientConnectionMonitoringTest.groovy +++ b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/sender/http/HttpClientConnectionMonitoringTest.groovy @@ -9,7 +9,6 @@ import org.eclipse.jetty.client.HttpClient import pl.allegro.tech.hermes.common.metric.HermesMetrics import pl.allegro.tech.hermes.common.metric.MetricsFacade import pl.allegro.tech.hermes.common.metric.executor.InstrumentedExecutorServiceFactory -import pl.allegro.tech.hermes.common.metric.executor.ThreadPoolMetrics import pl.allegro.tech.hermes.consumers.config.ConsumerSenderConfiguration import pl.allegro.tech.hermes.consumers.config.Http1ClientProperties import pl.allegro.tech.hermes.consumers.config.SslContextProperties @@ -32,7 +31,6 @@ class HttpClientConnectionMonitoringTest extends Specification { HermesMetrics hermesMetrics = new HermesMetrics(metricRegistry, new PathsCompiler("localhost")) MeterRegistry meterRegistry = new SimpleMeterRegistry() MetricsFacade metrics = new MetricsFacade(meterRegistry, hermesMetrics) - ThreadPoolMetrics threadPoolMetrics = new ThreadPoolMetrics(metrics) def setupSpec() { port = Ports.nextAvailable() @@ -45,7 +43,7 @@ class HttpClientConnectionMonitoringTest extends Specification { SslContextFactoryProvider sslContextFactoryProvider = new SslContextFactoryProvider(null, new SslContextProperties()) ConsumerSenderConfiguration consumerConfiguration = new ConsumerSenderConfiguration(); client = consumerConfiguration.http1SerialClient(new HttpClientsFactory( - new InstrumentedExecutorServiceFactory(threadPoolMetrics), + new InstrumentedExecutorServiceFactory(metrics), sslContextFactoryProvider), new Http1ClientProperties() ) batchClient = Mock(HttpClient) diff --git a/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/sender/http/JettyMessageSenderTest.java b/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/sender/http/JettyMessageSenderTest.java index c473a4ffb0..7123eca3a5 100644 --- a/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/sender/http/JettyMessageSenderTest.java +++ b/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/sender/http/JettyMessageSenderTest.java @@ -9,7 +9,6 @@ import pl.allegro.tech.hermes.api.EndpointAddress; import pl.allegro.tech.hermes.api.EndpointAddressResolverMetadata; import pl.allegro.tech.hermes.common.metric.executor.InstrumentedExecutorServiceFactory; -import pl.allegro.tech.hermes.common.metric.executor.ThreadPoolMetrics; import pl.allegro.tech.hermes.consumers.config.ConsumerSenderConfiguration; import pl.allegro.tech.hermes.consumers.config.Http1ClientProperties; import pl.allegro.tech.hermes.consumers.config.SslContextProperties; @@ -65,7 +64,7 @@ public static void setupEnvironment() throws Exception { client = consumerConfiguration.http1SerialClient( new HttpClientsFactory( new InstrumentedExecutorServiceFactory( - new ThreadPoolMetrics(TestMetricsFacadeFactory.create()) + TestMetricsFacadeFactory.create() ), sslContextFactoryProvider), new Http1ClientProperties() diff --git a/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/supervisor/workload/ConsumerTestRuntimeEnvironment.java b/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/supervisor/workload/ConsumerTestRuntimeEnvironment.java index b6965869f5..05057dbc78 100644 --- a/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/supervisor/workload/ConsumerTestRuntimeEnvironment.java +++ b/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/supervisor/workload/ConsumerTestRuntimeEnvironment.java @@ -138,7 +138,7 @@ private WorkloadSupervisor createConsumer(String consumerId, workloadProperties.setMonitorScanInterval(Duration.ofSeconds(1)); ModelAwareZookeeperNotifyingCache modelAwareCache = new ModelAwareZookeeperNotifyingCacheFactory( - curator, zookeeperProperties + curator, metricsSupplier.get(), zookeeperProperties ).provide(); InternalNotificationsBus notificationsBus = @@ -239,7 +239,7 @@ ConsumersRuntimeMonitor monitor(String consumerId, Duration monitorScanInterval) { CuratorFramework curator = consumerZookeeperConnections.get(consumerId); ModelAwareZookeeperNotifyingCache modelAwareCache = - new ModelAwareZookeeperNotifyingCacheFactory(curator, zookeeperProperties).provide(); + new ModelAwareZookeeperNotifyingCacheFactory(curator, metricsSupplier.get(), zookeeperProperties).provide(); InternalNotificationsBus notificationsBus = new ZookeeperInternalNotificationBus(objectMapper, modelAwareCache); SubscriptionsCache subscriptionsCache = new NotificationsBasedSubscriptionCache( diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/CommonConfiguration.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/CommonConfiguration.java index 6ee77fb667..92b0b446bd 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/CommonConfiguration.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/CommonConfiguration.java @@ -38,7 +38,6 @@ import pl.allegro.tech.hermes.common.metric.counter.CounterStorage; import pl.allegro.tech.hermes.common.metric.counter.zookeeper.ZookeeperCounterStorage; import pl.allegro.tech.hermes.common.metric.executor.InstrumentedExecutorServiceFactory; -import pl.allegro.tech.hermes.common.metric.executor.ThreadPoolMetrics; import pl.allegro.tech.hermes.common.util.InetAddressInstanceIdResolver; import pl.allegro.tech.hermes.common.util.InstanceIdResolver; import pl.allegro.tech.hermes.domain.filtering.MessageFilter; @@ -160,10 +159,11 @@ public InternalNotificationsBus zookeeperInternalNotificationBus(ObjectMapper ob @Bean(initMethod = "start", destroyMethod = "stop") public ModelAwareZookeeperNotifyingCache modelAwareZookeeperNotifyingCache(CuratorFramework curator, + MetricsFacade metricsFacade, ZookeeperClustersProperties zookeeperClustersProperties, DatacenterNameProvider datacenterNameProvider) { ZookeeperProperties zookeeperProperties = zookeeperClustersProperties.toZookeeperProperties(datacenterNameProvider); - return new ModelAwareZookeeperNotifyingCacheFactory(curator, zookeeperProperties).provide(); + return new ModelAwareZookeeperNotifyingCacheFactory(curator, metricsFacade, zookeeperProperties).provide(); } @Bean @@ -175,13 +175,8 @@ public UndeliveredMessageLog undeliveredMessageLog(CuratorFramework zookeeper, } @Bean - public ThreadPoolMetrics threadPoolMetrics(MetricsFacade metricsFacade) { - return new ThreadPoolMetrics(metricsFacade); - } - - @Bean - public InstrumentedExecutorServiceFactory instrumentedExecutorServiceFactory(ThreadPoolMetrics threadPoolMetrics) { - return new InstrumentedExecutorServiceFactory(threadPoolMetrics); + public InstrumentedExecutorServiceFactory instrumentedExecutorServiceFactory(MetricsFacade metricsFacade) { + return new InstrumentedExecutorServiceFactory(metricsFacade); } @Bean From 857d5885435e005b92a28a5b68bb1af04acb22e3 Mon Sep 17 00:00:00 2001 From: Mateusz <76775507+szczygiel-m@users.noreply.github.com> Date: Wed, 17 Apr 2024 11:37:13 +0200 Subject: [PATCH 42/87] Configured logback (#1827) * Configured logback * Configured logback * Configured logback * Configured logback * Configured logback --- .../src/integrationTest/resources/logback.xml | 16 ++++++++++++++++ .../slowIntegrationTest/resources/logback.xml | 16 ++++++++++++++++ 2 files changed, 32 insertions(+) diff --git a/integration-tests/src/integrationTest/resources/logback.xml b/integration-tests/src/integrationTest/resources/logback.xml index 6b79faabfb..25978ddfbd 100644 --- a/integration-tests/src/integrationTest/resources/logback.xml +++ b/integration-tests/src/integrationTest/resources/logback.xml @@ -8,6 +8,22 @@ + + + + + + + + + + + + + + + + diff --git a/integration-tests/src/slowIntegrationTest/resources/logback.xml b/integration-tests/src/slowIntegrationTest/resources/logback.xml index 6b79faabfb..25978ddfbd 100644 --- a/integration-tests/src/slowIntegrationTest/resources/logback.xml +++ b/integration-tests/src/slowIntegrationTest/resources/logback.xml @@ -8,6 +8,22 @@ + + + + + + + + + + + + + + + + From b2b5071bee7e62cb51099de7d0762b4ea47454e6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Piotr=20R=C5=BCysko?= Date: Wed, 17 Apr 2024 12:52:09 +0200 Subject: [PATCH 43/87] Skip building console while running tests (#1845) --- build.gradle | 2 ++ hermes-management/build.gradle | 10 ++++++++++ 2 files changed, 12 insertions(+) diff --git a/build.gradle b/build.gradle index 6de28b2eae..55d670209e 100644 --- a/build.gradle +++ b/build.gradle @@ -123,6 +123,8 @@ configure(subprojects - project(':integration-tests')) { withSourcesJar() } + javadoc.options.addStringOption('Xdoclint:none', '-quiet') + publishing { publications { mavenJava(MavenPublication) { diff --git a/hermes-management/build.gradle b/hermes-management/build.gradle index 93215853a8..db5903e8de 100644 --- a/hermes-management/build.gradle +++ b/hermes-management/build.gradle @@ -62,6 +62,16 @@ yarnSetup.dependsOn(nodeSetup) yarn.dependsOn(npmSetup) task buildHermesConsole(type: YarnTask, dependsOn: yarn) { + def tasksThatDontRequireConsole = [ + 'integrationTest', + 'slowIntegrationTest', + 'check' + ] + + onlyIf { + tasksThatDontRequireConsole.intersect(gradle.startParameter.taskNames).isEmpty() + } + args = ['build-only'] } From 0d9323d33e6bc655b861dd8d7186ff3f707f1343 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Piotr=20R=C5=BCysko?= Date: Thu, 25 Apr 2024 07:31:43 +0200 Subject: [PATCH 44/87] fix QueryEndpointTest (#1849) --- .../test/helper/builder/GroupBuilder.java | 10 +- .../test/helper/builder/TopicBuilder.java | 13 +- .../management/QueryEndpointTest.java | 312 +++++++++++++++--- 3 files changed, 284 insertions(+), 51 deletions(-) diff --git a/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/builder/GroupBuilder.java b/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/builder/GroupBuilder.java index 7f7fc24a99..f93033081d 100644 --- a/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/builder/GroupBuilder.java +++ b/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/builder/GroupBuilder.java @@ -23,6 +23,14 @@ public Group build() { } public static GroupBuilder groupWithRandomName() { - return group(GroupBuilder.class.getSimpleName() + "Group" + sequence.incrementAndGet()); + return groupWithRandomNameEndedWith(""); + } + + public static GroupBuilder groupWithRandomNameEndedWith(String suffix) { + return group(GroupBuilder.class.getSimpleName() + "Group" + sequence.incrementAndGet() + suffix); + } + + public static GroupBuilder groupWithRandomNameContaining(String string) { + return group(GroupBuilder.class.getSimpleName() + "Group" + string + sequence.incrementAndGet()); } } diff --git a/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/builder/TopicBuilder.java b/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/builder/TopicBuilder.java index b4cc5d9e3a..81eccae2d2 100644 --- a/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/builder/TopicBuilder.java +++ b/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/builder/TopicBuilder.java @@ -63,9 +63,20 @@ private TopicBuilder(TopicName topicName) { } public static TopicBuilder topicWithRandomName() { + return topicWithRandomNameEndedWith(""); + } + + public static TopicBuilder topicWithRandomNameContaining(String string) { + return topic( + TopicBuilder.class.getSimpleName() + "Group" + sequence.incrementAndGet(), + TopicBuilder.class.getSimpleName() + "Topic" + string + sequence.incrementAndGet() + ); + } + + public static TopicBuilder topicWithRandomNameEndedWith(String suffix) { return topic( TopicBuilder.class.getSimpleName() + "Group" + sequence.incrementAndGet(), - TopicBuilder.class.getSimpleName() + "Topic" + sequence.incrementAndGet() + TopicBuilder.class.getSimpleName() + "Topic" + sequence.incrementAndGet() + suffix ); } diff --git a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/QueryEndpointTest.java b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/QueryEndpointTest.java index a64751e5df..fb4bd6045a 100644 --- a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/QueryEndpointTest.java +++ b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/QueryEndpointTest.java @@ -2,7 +2,6 @@ import com.jayway.awaitility.Duration; import org.assertj.core.api.Assertions; -import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Order; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.RegisterExtension; @@ -16,12 +15,14 @@ import pl.allegro.tech.hermes.api.SubscriptionNameWithMetrics; import pl.allegro.tech.hermes.api.Topic; import pl.allegro.tech.hermes.api.TopicNameWithMetrics; +import pl.allegro.tech.hermes.api.TopicWithSchema; import pl.allegro.tech.hermes.api.TrackingMode; import pl.allegro.tech.hermes.integrationtests.prometheus.PrometheusExtension; import pl.allegro.tech.hermes.integrationtests.setup.HermesExtension; import pl.allegro.tech.hermes.test.helper.avro.AvroUserSchemaLoader; import pl.allegro.tech.hermes.test.helper.builder.SubscriptionBuilder; +import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -39,10 +40,14 @@ import static pl.allegro.tech.hermes.api.TopicWithSchema.topicWithSchema; import static pl.allegro.tech.hermes.integrationtests.prometheus.SubscriptionMetrics.subscriptionMetrics; import static pl.allegro.tech.hermes.test.helper.builder.GroupBuilder.groupWithRandomName; +import static pl.allegro.tech.hermes.test.helper.builder.GroupBuilder.groupWithRandomNameContaining; +import static pl.allegro.tech.hermes.test.helper.builder.GroupBuilder.groupWithRandomNameEndedWith; import static pl.allegro.tech.hermes.test.helper.builder.SubscriptionBuilder.subscription; import static pl.allegro.tech.hermes.test.helper.builder.SubscriptionBuilder.subscriptionWithRandomName; import static pl.allegro.tech.hermes.test.helper.builder.TopicBuilder.topic; import static pl.allegro.tech.hermes.test.helper.builder.TopicBuilder.topicWithRandomName; +import static pl.allegro.tech.hermes.test.helper.builder.TopicBuilder.topicWithRandomNameContaining; +import static pl.allegro.tech.hermes.test.helper.builder.TopicBuilder.topicWithRandomNameEndedWith; import static pl.allegro.tech.hermes.test.helper.endpoint.TimeoutAdjuster.adjust; public class QueryEndpointTest { @@ -58,80 +63,281 @@ public class QueryEndpointTest { public static final HermesExtension hermes = new HermesExtension() .withPrometheus(prometheus); - @BeforeEach - void cleanup() { - hermes.clearManagementData(); + @Test + public void shouldReturnAllGroupsWhenQueryIsEmpty() { + // given + createGroupWithRandomName(); + createGroupWithRandomName(); + + // when + List found = hermes.api().queryGroups("{\"query\": {}}") + .expectStatus().isOk() + .expectBodyList(Group.class).returnResult().getResponseBody(); + + // then + assertThat(found) + .extracting(Group::getGroupName) + .containsExactlyInAnyOrderElementsOf(hermes.api().getGroups()); } - public static Stream groupData() { - return Stream.of( - arguments("{\"query\": {}}", asList(1, 2, 3, 4)), - arguments("{\"query\": {\"groupName\": \"testGroup1\"}}", asList(1)), - arguments("{\"query\": {\"groupName\": {\"like\": \".*Group2\"}}}", asList(3)), - arguments("{\"query\": {\"groupName\": {\"like\": \".*Group.*\"}}}", asList(1, 3, 4))); + @Test + public void shouldReturnGroupsWithExactName() { + // given + Group group = createGroupWithRandomName(); + createGroupWithRandomName(); + + // when + List found = hermes.api().queryGroups("{\"query\": {\"groupName\": \"" + group.getGroupName() + "\"}}") + .expectStatus().isOk() + .expectBodyList(Group.class).returnResult().getResponseBody(); + + // then + assertThat(found) + .containsExactly(group); } - @ParameterizedTest - @MethodSource("groupData") - public void shouldQueryGroup(String query, List positions) { + @Test + public void shouldReturnGroupsWithNameSuffix() { // given - List groups = List.of( - new Group("testGroup1"), - new Group("testNot1"), - new Group("testGroup2"), - new Group("testGroup3") - ); + String suffix = "GroupSuffix"; + hermes.initHelper().createGroup(groupWithRandomNameEndedWith(suffix).build()); + createGroupWithRandomName(); - groups.forEach(g -> hermes.initHelper().createGroup(g)); + // when + List found = hermes.api().queryGroups("{\"query\": {\"groupName\": {\"like\": \".*" + suffix + "\"}}}") + .expectStatus().isOk() + .expectBodyList(Group.class).returnResult().getResponseBody(); + + // then + List groupsWithSuffix = hermes.api().getGroups().stream() + .filter(name -> name.endsWith(suffix)) + .toList(); + assertThat(found) + .extracting(Group::getGroupName) + .containsExactlyInAnyOrderElementsOf(groupsWithSuffix); + } + + @Test + public void shouldReturnGroupsWithNameContainingString() { + // given + String string = "SomeString"; + hermes.initHelper().createGroup(groupWithRandomNameContaining(string).build()); + createGroupWithRandomName(); // when - List found = hermes.api().queryGroups(query) + List found = hermes.api().queryGroups("{\"query\": {\"groupName\": {\"like\": \".*" + string + ".*\"}}}") .expectStatus().isOk() .expectBodyList(Group.class).returnResult().getResponseBody(); // then - assertListMatches(groups, found, positions); + List groupsContainingString = hermes.api().getGroups().stream() + .filter(name -> name.contains(string)) + .toList(); + assertThat(found) + .extracting(Group::getGroupName) + .containsExactlyInAnyOrderElementsOf(groupsContainingString); } - public static Stream topicData() { - return Stream.of( - arguments("{\"query\": {}}", asList(1, 2, 3, 4)), - arguments("{\"query\": {\"name\": \"testGroup1.testTopic1\"}}", asList(1)), - arguments("{\"query\": {\"name\": {\"like\": \".*testTopic1\"}}}", asList(1)), - arguments("{\"query\": {\"name\": {\"like\": \".*testTopic.*\"}}}", asList(1, 2, 3)), - arguments("{\"query\": {\"trackingEnabled\": \"true\", \"contentType\": \"AVRO\"}}", asList(3)), - arguments("{\"query\": {\"and\": [{\"trackingEnabled\": \"true\"}, {\"contentType\": \"AVRO\"}]}}", asList(3)), - arguments("{\"query\": {\"or\": [{\"trackingEnabled\": \"true\"}, {\"contentType\": \"AVRO\"}]}}", asList(1, 3, 4)), - arguments("{\"query\": {\"owner.id\": \"Team Alpha\"}}", asList(4)), - arguments("{\"query\": {\"owner.id\": {\"like\": \".*Alph.*\"}}}", asList(4))); + @Test + public void shouldReturnAllTopicsWhenQueryIsEmpty() { + // given + createTopicWithRandomName(); + createTopicWithRandomName(); + + // when + List found = hermes.api().queryTopics("{\"query\": {}}") + .expectStatus().isOk() + .expectBodyList(Topic.class).returnResult().getResponseBody(); + + // then + assertThat(found) + .containsExactlyInAnyOrderElementsOf(listAllTopics()); } - @ParameterizedTest - @MethodSource("topicData") - public void shouldQueryTopic(String query, List positions) { + @Test + public void shouldReturnTopicsWithExactName() { + // given + Topic topic = createTopicWithRandomName(); + createTopicWithRandomName(); + + // when + List found = hermes.api().queryTopics("{\"query\": {\"name\": \"" + topic.getQualifiedName() + "\"}}") + .expectStatus().isOk() + .expectBodyList(Topic.class).returnResult().getResponseBody(); + + // then + assertThat(found) + .containsExactly(topic); + } + + @Test + public void shouldReturnTopicsWithNameSuffix() { // given - Topic topic1 = topic("testGroup1", "testTopic1").withContentType(AVRO).withTrackingEnabled(false).build(); - Topic topic2 = topic("testGroup1", "testTopic2").withContentType(JSON).withTrackingEnabled(false).build(); - Topic topic3 = topic("testGroup1", "testTopic3").withContentType(AVRO).withTrackingEnabled(true).build(); - Topic topic4 = topic("testGroup2", "testOtherTopic").withContentType(JSON).withTrackingEnabled(true) - .withOwner(new OwnerId("Plaintext", "Team Alpha")).build(); + String suffix = "TopicSuffix"; + hermes.initHelper().createTopic(topicWithRandomNameEndedWith(suffix).build()); + createTopicWithRandomName(); + + // when + List found = hermes.api().queryTopics("{\"query\": {\"name\": {\"like\": \".*" + suffix + "\"}}}") + .expectStatus().isOk() + .expectBodyList(Topic.class).returnResult().getResponseBody(); + + // then + List topicsWithSuffix = listAllTopics().stream() + .filter(topic -> topic.getQualifiedName().endsWith(suffix)) + .toList(); + assertThat(found) + .containsExactlyInAnyOrderElementsOf(topicsWithSuffix); + } + @Test + public void shouldReturnTopicsWithNameContainingString() { + // given + String string = "SomeString"; + hermes.initHelper().createTopic(topicWithRandomNameContaining(string).build()); + createTopicWithRandomName(); + + // when + List found = hermes.api().queryTopics("{\"query\": {\"name\": {\"like\": \".*" + string + ".*\"}}}") + .expectStatus().isOk() + .expectBodyList(Topic.class).returnResult().getResponseBody(); + + // then + List topicsContainingString = listAllTopics().stream() + .filter(topic -> topic.getQualifiedName().contains(string)) + .toList(); + assertThat(found) + .containsExactlyInAnyOrderElementsOf(topicsContainingString); + } + + @Test + public void shouldReturnTopicsWithAllMatchingProperties() { + // given + Topic topic1 = topicWithRandomName() + .withContentType(AVRO) + .withTrackingEnabled(false) + .build(); + Topic topic2 = topicWithRandomName() + .withContentType(JSON) + .withTrackingEnabled(false) + .build(); + Topic topic3 = topicWithRandomName() + .withContentType(AVRO) + .withTrackingEnabled(true) + .build(); hermes.initHelper().createTopicWithSchema(topicWithSchema(topic1, SCHEMA)); hermes.initHelper().createTopicWithSchema(topicWithSchema(topic3, SCHEMA)); hermes.initHelper().createTopic(topic2); - hermes.initHelper().createTopic(topic4); - List topics = asList(topic1, topic2, topic3, topic4); + // when + List found = hermes.api().queryTopics("{\"query\": {\"and\": [{\"trackingEnabled\": \"true\"}, {\"contentType\": \"AVRO\"}]}}") + .expectStatus().isOk() + .expectBodyList(Topic.class).returnResult().getResponseBody(); + + // then + List topicsWithAvroAndTracking = listAllTopics().stream() + .filter(topic -> topic.getContentType() == AVRO && topic.isTrackingEnabled()) + .toList(); + assertThat(found) + .containsExactlyInAnyOrderElementsOf(topicsWithAvroAndTracking); + } + + @Test + public void shouldReturnTopicsWithAtLeastOneMatchingProperty() { + // given + Topic topic1 = topicWithRandomName() + .withContentType(AVRO) + .withTrackingEnabled(false) + .build(); + Topic topic2 = topicWithRandomName() + .withContentType(JSON) + .withTrackingEnabled(false) + .build(); + Topic topic3 = topicWithRandomName() + .withContentType(AVRO) + .withTrackingEnabled(true) + .build(); + hermes.initHelper().createTopicWithSchema(topicWithSchema(topic1, SCHEMA)); + hermes.initHelper().createTopicWithSchema(topicWithSchema(topic3, SCHEMA)); + hermes.initHelper().createTopic(topic2); // when - List found = hermes.api().queryTopics(query) + List found = hermes.api().queryTopics("{\"query\": {\"or\": [{\"trackingEnabled\": \"true\"}, {\"contentType\": \"AVRO\"}]}}") .expectStatus().isOk() .expectBodyList(Topic.class).returnResult().getResponseBody(); // then - assertListMatches(topics, found, positions); + List topicsWithAvroOrTracking = listAllTopics().stream() + .filter(topic -> topic.getContentType() == AVRO || topic.isTrackingEnabled()) + .toList(); + assertThat(found) + .containsExactlyInAnyOrderElementsOf(topicsWithAvroOrTracking); } + @Test + public void shouldReturnTopicsWithExactOwnerId() { + // given + Topic topic = topicWithRandomName() + .withContentType(JSON) + .withTrackingEnabled(true) + .withOwner(new OwnerId("Plaintext", "Team Alpha")) + .build(); + hermes.initHelper().createTopic(topic); + createTopicWithRandomName(); + + // when + List found = hermes.api().queryTopics("{\"query\": {\"owner.id\": \"Team Alpha\"}}") + .expectStatus().isOk() + .expectBodyList(Topic.class).returnResult().getResponseBody(); + + // then + List topicsOwnerId = listAllTopics().stream() + .filter(t -> t.getOwner().getId().equals("Team Alpha")) + .toList(); + assertThat(found) + .containsExactlyInAnyOrderElementsOf(topicsOwnerId); + } + + @Test + public void shouldReturnTopicsWithOwnerIdContainingString() { + // given + Topic topic = topicWithRandomName() + .withContentType(JSON) + .withTrackingEnabled(true) + .withOwner(new OwnerId("Plaintext", "Team Alpha")) + .build(); + hermes.initHelper().createTopic(topic); + createTopicWithRandomName(); + + // when + List found = hermes.api().queryTopics("{\"query\": {\"owner.id\": {\"like\": \".*Alph.*\"}}}") + .expectStatus().isOk() + .expectBodyList(Topic.class).returnResult().getResponseBody(); + + // then + List topicsOwnerId = listAllTopics().stream() + .filter(t -> t.getOwner().getId().contains("Alph")) + .toList(); + assertThat(found) + .containsExactlyInAnyOrderElementsOf(topicsOwnerId); + } + + private List listAllTopics() { + List groups = hermes.api().getGroups(); + List topics = new ArrayList<>(); + for (String groupName : groups) { + String[] topicNames = hermes.api().listTopics(groupName) + .expectStatus() + .isOk() + .expectBody(String[].class).returnResult().getResponseBody(); + for (String topicName : topicNames) { + TopicWithSchema topic = hermes.api().getTopicResponse(topicName) + .expectBody(TopicWithSchema.class).returnResult().getResponseBody(); + topics.add(topic.getTopic()); + } + } + return topics; + } public static Stream subscriptionData() { return Stream.of( @@ -233,8 +439,8 @@ public void shouldQueryTopicsMetrics(String topicName1, String topicName2, Strin hermes.api().publishUntilSuccess(group.getGroupName() + "." + topicName2, "testMessage5"); List qualifiedNames = topicNames.stream() - .map(topicName -> group.getGroupName() + "." + topicName) - .collect(toList()); + .map(topicName -> group.getGroupName() + "." + topicName) + .collect(toList()); waitAtMost(adjust(Duration.ONE_MINUTE)).until(() -> { // when @@ -340,7 +546,7 @@ public void shouldHandleUnavailableSubscriptionsMetrics() { }); } - private static void assertThatRateIsUnavailable(List allSubscriptions, Subscription ... subscriptions) { + private static void assertThatRateIsUnavailable(List allSubscriptions, Subscription... subscriptions) { subscriptionsMatchesToNamesAndTheirTopicsNames(allSubscriptions, subscriptions); for (SubscriptionNameWithMetrics metrics : allSubscriptions) { assertThat(metrics.getRate().asString()).isEqualTo("unavailable"); @@ -348,7 +554,7 @@ private static void assertThatRateIsUnavailable(List found, - Subscription ... expectedSubscriptions) { + Subscription... expectedSubscriptions) { assertThat(found).isNotNull(); Map foundSubscriptionsAndTheirTopicNames = found.stream() .collect(Collectors.toMap(SubscriptionNameWithMetrics::getName, SubscriptionNameWithMetrics::getTopicName)); @@ -379,4 +585,12 @@ private void assertTopicMetricsMatchesToNames(List found, Assertions.assertThat(foundQualifiedNames).containsAll(expectedQualifiedNames); } + + private Topic createTopicWithRandomName() { + return hermes.initHelper().createTopic(topicWithRandomName().build()); + } + + private Group createGroupWithRandomName() { + return hermes.initHelper().createGroup(groupWithRandomName().build()); + } } From 491dee3fbdad493bc9f8b1d45dd9d159b550973d Mon Sep 17 00:00:00 2001 From: Maciej Moscicki Date: Thu, 25 Apr 2024 08:05:57 +0200 Subject: [PATCH 45/87] fix local setup for mgmt (#1851) --- hermes-management/build.gradle | 3 +++ .../src/main/resources/application-local.yaml | 19 +++++++++++++++++++ 2 files changed, 22 insertions(+) diff --git a/hermes-management/build.gradle b/hermes-management/build.gradle index db5903e8de..f1105eadcb 100644 --- a/hermes-management/build.gradle +++ b/hermes-management/build.gradle @@ -92,3 +92,6 @@ tasks.register('prepareIndexTemplate') { } jar.dependsOn(attachHermesConsole, 'prepareIndexTemplate') + +run.dependsOn(attachHermesConsole, 'prepareIndexTemplate') + diff --git a/hermes-management/src/main/resources/application-local.yaml b/hermes-management/src/main/resources/application-local.yaml index 67e4a58cc2..845e296a4a 100644 --- a/hermes-management/src/main/resources/application-local.yaml +++ b/hermes-management/src/main/resources/application-local.yaml @@ -33,3 +33,22 @@ console: label: SERIAL - value: BATCH label: BATCH +prometheus: + client: + enabled: true + +management: + server: + servlet: + context-path: / + +spring: + main: + allow-bean-definition-overriding: true + jersey: + type: filter + filter: + order: 2147483647 + mvc: + servlet: + path: / \ No newline at end of file From 6e68738c38e51c1c2be92820648f2236994e86f4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Piotr=20R=C5=BCysko?= Date: Thu, 25 Apr 2024 12:31:29 +0200 Subject: [PATCH 46/87] add publishing chaos experiments (#1842) --- .../hermes/api/PublishingChaosPolicy.java | 43 ++++++ .../pl/allegro/tech/hermes/api/Topic.java | 20 ++- .../tech/hermes/api/TopicWithSchema.java | 7 +- .../FailFastKafkaProducerProperties.java | 32 ++++ .../config/FrontendProducerConfiguration.java | 10 +- .../producer/kafka/ChaosException.java | 8 + .../producer/kafka/KafkaMessageSenders.java | 10 +- .../kafka/MultiDatacenterMessageProducer.java | 130 ++++++++++++++-- .../topic/validator/TopicValidator.java | 32 ++++ .../test/helper/builder/TopicBuilder.java | 10 +- .../management/TopicManagementTest.java | 143 ++++++++++++++++++ .../RemoteDatacenterProduceFallbackTest.java | 89 +++++++++++ 12 files changed, 512 insertions(+), 22 deletions(-) create mode 100644 hermes-api/src/main/java/pl/allegro/tech/hermes/api/PublishingChaosPolicy.java create mode 100644 hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/ChaosException.java diff --git a/hermes-api/src/main/java/pl/allegro/tech/hermes/api/PublishingChaosPolicy.java b/hermes-api/src/main/java/pl/allegro/tech/hermes/api/PublishingChaosPolicy.java new file mode 100644 index 0000000000..ebe9620fc8 --- /dev/null +++ b/hermes-api/src/main/java/pl/allegro/tech/hermes/api/PublishingChaosPolicy.java @@ -0,0 +1,43 @@ +package pl.allegro.tech.hermes.api; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Collections; +import java.util.Map; + +public record PublishingChaosPolicy(ChaosMode mode, ChaosPolicy globalPolicy, Map datacenterPolicies) { + + @JsonCreator + public PublishingChaosPolicy(@JsonProperty("mode") ChaosMode mode, + @JsonProperty("globalPolicy") ChaosPolicy globalPolicy, + @JsonProperty("datacenterPolicies") Map datacenterPolicies) { + this.mode = mode; + this.globalPolicy = globalPolicy; + this.datacenterPolicies = datacenterPolicies == null ? Map.of() : datacenterPolicies; + } + + public static PublishingChaosPolicy disabled() { + return new PublishingChaosPolicy(ChaosMode.DISABLED, null, Collections.emptyMap()); + } + + public record ChaosPolicy(int probability, int delayFrom, int delayTo, boolean completeWithError) { + + @JsonCreator + public ChaosPolicy(@JsonProperty("probability") int probability, + @JsonProperty("delayFrom") int delayFrom, + @JsonProperty("delayTo") int delayTo, + @JsonProperty("completeWithError") boolean completeWithError) { + this.probability = probability; + this.delayFrom = delayFrom; + this.delayTo = delayTo; + this.completeWithError = completeWithError; + } + } + + public enum ChaosMode { + DISABLED, + GLOBAL, + DATACENTER + } +} diff --git a/hermes-api/src/main/java/pl/allegro/tech/hermes/api/Topic.java b/hermes-api/src/main/java/pl/allegro/tech/hermes/api/Topic.java index 2b1353ccfa..3cc67c855f 100644 --- a/hermes-api/src/main/java/pl/allegro/tech/hermes/api/Topic.java +++ b/hermes-api/src/main/java/pl/allegro/tech/hermes/api/Topic.java @@ -38,6 +38,7 @@ public class Topic { @NotNull private Ack ack; private boolean fallbackToRemoteDatacenterEnabled; + private PublishingChaosPolicy chaos; @NotNull private ContentType contentType; @Min(MIN_MESSAGE_SIZE) @@ -57,7 +58,7 @@ public class Topic { private Instant modifiedAt; public Topic(TopicName name, String description, OwnerId owner, RetentionTime retentionTime, - boolean migratedFromJsonType, Ack ack, boolean fallbackToRemoteDatacenterEnabled, + boolean migratedFromJsonType, Ack ack, boolean fallbackToRemoteDatacenterEnabled, PublishingChaosPolicy chaos, boolean trackingEnabled, ContentType contentType, boolean jsonToAvroDryRunEnabled, @JacksonInject(value = DEFAULT_SCHEMA_ID_SERIALIZATION_ENABLED_KEY, useInput = OptBoolean.TRUE) Boolean schemaIdAwareSerializationEnabled, @@ -69,6 +70,7 @@ public Topic(TopicName name, String description, OwnerId owner, RetentionTime re this.retentionTime = retentionTime; this.ack = (ack == null ? Ack.LEADER : ack); this.fallbackToRemoteDatacenterEnabled = fallbackToRemoteDatacenterEnabled; + this.chaos = chaos; this.trackingEnabled = trackingEnabled; this.migratedFromJsonType = migratedFromJsonType; this.contentType = contentType; @@ -92,6 +94,7 @@ public Topic( @JsonProperty("jsonToAvroDryRun") boolean jsonToAvroDryRunEnabled, @JsonProperty("ack") Ack ack, @JsonProperty("fallbackToRemoteDatacenterEnabled") boolean fallbackToRemoteDatacenterEnabled, + @JsonProperty("chaos") PublishingChaosPolicy chaos, @JsonProperty("trackingEnabled") boolean trackingEnabled, @JsonProperty("migratedFromJsonType") boolean migratedFromJsonType, @JsonProperty("schemaIdAwareSerializationEnabled") @@ -107,10 +110,10 @@ public Topic( @JsonProperty("modifiedAt") Instant modifiedAt ) { this(TopicName.fromQualifiedName(qualifiedName), description, owner, retentionTime, migratedFromJsonType, ack, - fallbackToRemoteDatacenterEnabled, trackingEnabled, contentType, jsonToAvroDryRunEnabled, - schemaIdAwareSerializationEnabled, maxMessageSize == null ? DEFAULT_MAX_MESSAGE_SIZE : maxMessageSize, - publishingAuth == null ? PublishingAuth.disabled() : publishingAuth, - subscribingRestricted, + fallbackToRemoteDatacenterEnabled, chaos == null ? PublishingChaosPolicy.disabled() : chaos, + trackingEnabled, contentType, jsonToAvroDryRunEnabled, schemaIdAwareSerializationEnabled, + maxMessageSize == null ? DEFAULT_MAX_MESSAGE_SIZE : maxMessageSize, + publishingAuth == null ? PublishingAuth.disabled() : publishingAuth, subscribingRestricted, offlineStorage == null ? TopicDataOfflineStorage.defaultOfflineStorage() : offlineStorage, labels == null ? Collections.emptySet() : labels, createdAt, modifiedAt @@ -124,7 +127,7 @@ public RetentionTime getRetentionTime() { @Override public int hashCode() { return Objects.hash(name, description, owner, retentionTime, migratedFromJsonType, trackingEnabled, ack, - fallbackToRemoteDatacenterEnabled, contentType, jsonToAvroDryRunEnabled, schemaIdAwareSerializationEnabled, + fallbackToRemoteDatacenterEnabled, chaos, contentType, jsonToAvroDryRunEnabled, schemaIdAwareSerializationEnabled, maxMessageSize, publishingAuth, subscribingRestricted, offlineStorage, labels); } @@ -148,6 +151,7 @@ public boolean equals(Object obj) { && Objects.equals(this.schemaIdAwareSerializationEnabled, other.schemaIdAwareSerializationEnabled) && Objects.equals(this.ack, other.ack) && Objects.equals(this.fallbackToRemoteDatacenterEnabled, other.fallbackToRemoteDatacenterEnabled) + && Objects.equals(this.chaos, other.chaos) && Objects.equals(this.contentType, other.contentType) && Objects.equals(this.maxMessageSize, other.maxMessageSize) && Objects.equals(this.subscribingRestricted, other.subscribingRestricted) @@ -187,6 +191,10 @@ public boolean isFallbackToRemoteDatacenterEnabled() { return fallbackToRemoteDatacenterEnabled; } + public PublishingChaosPolicy getChaos() { + return chaos; + } + public ContentType getContentType() { return contentType; } diff --git a/hermes-api/src/main/java/pl/allegro/tech/hermes/api/TopicWithSchema.java b/hermes-api/src/main/java/pl/allegro/tech/hermes/api/TopicWithSchema.java index 8dafae5495..a8f6496198 100644 --- a/hermes-api/src/main/java/pl/allegro/tech/hermes/api/TopicWithSchema.java +++ b/hermes-api/src/main/java/pl/allegro/tech/hermes/api/TopicWithSchema.java @@ -19,7 +19,7 @@ public class TopicWithSchema extends Topic { public TopicWithSchema(Topic topic, String schema) { this(schema, topic.getQualifiedName(), topic.getDescription(), topic.getOwner(), topic.getRetentionTime(), topic.isJsonToAvroDryRunEnabled(), topic.getAck(), topic.isFallbackToRemoteDatacenterEnabled(), - topic.isTrackingEnabled(), topic.wasMigratedFromJsonType(), topic.isSchemaIdAwareSerializationEnabled(), + topic.getChaos(), topic.isTrackingEnabled(), topic.wasMigratedFromJsonType(), topic.isSchemaIdAwareSerializationEnabled(), topic.getContentType(), topic.getMaxMessageSize(), topic.getPublishingAuth(), topic.isSubscribingRestricted(), topic.getOfflineStorage(), topic.getLabels(), topic.getCreatedAt(), topic.getModifiedAt()); } @@ -33,6 +33,7 @@ public TopicWithSchema(@JsonProperty("schema") String schema, @JsonProperty("jsonToAvroDryRun") boolean jsonToAvroDryRunEnabled, @JsonProperty("ack") Ack ack, @JsonProperty("fallbackToRemoteDatacenterEnabled") boolean fallbackToRemoteDatacenterEnabled, + @JsonProperty("chaos") PublishingChaosPolicy chaos, @JsonProperty("trackingEnabled") boolean trackingEnabled, @JsonProperty("migratedFromJsonType") boolean migratedFromJsonType, @JsonProperty("schemaIdAwareSerializationEnabled") @@ -47,7 +48,7 @@ public TopicWithSchema(@JsonProperty("schema") String schema, @JsonProperty("createdAt") Instant createdAt, @JsonProperty("modifiedAt") Instant modifiedAt) { super(qualifiedName, description, owner, retentionTime, jsonToAvroDryRunEnabled, ack, - fallbackToRemoteDatacenterEnabled, trackingEnabled, migratedFromJsonType, schemaIdAwareSerializationEnabled, + fallbackToRemoteDatacenterEnabled, chaos, trackingEnabled, migratedFromJsonType, schemaIdAwareSerializationEnabled, contentType, maxMessageSize, publishingAuth, subscribingRestricted, offlineStorage, labels, createdAt, modifiedAt); this.topic = convertToTopic(); @@ -64,7 +65,7 @@ public static TopicWithSchema topicWithSchema(Topic topic) { private Topic convertToTopic() { return new Topic(this.getQualifiedName(), this.getDescription(), this.getOwner(), this.getRetentionTime(), - this.isJsonToAvroDryRunEnabled(), this.getAck(), this.isFallbackToRemoteDatacenterEnabled(), + this.isJsonToAvroDryRunEnabled(), this.getAck(), this.isFallbackToRemoteDatacenterEnabled(), this.getChaos(), this.isTrackingEnabled(), this.wasMigratedFromJsonType(), this.isSchemaIdAwareSerializationEnabled(), this.getContentType(), this.getMaxMessageSize(), this.getPublishingAuth(), this.isSubscribingRestricted(), this.getOfflineStorage(), this.getLabels(), this.getCreatedAt(), this.getModifiedAt()); diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FailFastKafkaProducerProperties.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FailFastKafkaProducerProperties.java index e611bad3ca..5dde59c5bc 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FailFastKafkaProducerProperties.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FailFastKafkaProducerProperties.java @@ -12,6 +12,8 @@ public class FailFastKafkaProducerProperties implements KafkaProducerParameters private FallbackSchedulerProperties fallbackScheduler = new FallbackSchedulerProperties(); + private ChaosSchedulerProperties chaosScheduler = new ChaosSchedulerProperties(); + private Duration maxBlock = Duration.ofMillis(500); private Duration metadataMaxAge = Duration.ofMinutes(5); @@ -182,12 +184,42 @@ public void setFallbackScheduler(FallbackSchedulerProperties fallbackScheduler) this.fallbackScheduler = fallbackScheduler; } + public ChaosSchedulerProperties getChaosScheduler() { + return chaosScheduler; + } + + public void setChaosScheduler(ChaosSchedulerProperties chaosScheduler) { + this.chaosScheduler = chaosScheduler; + } + public static class FallbackSchedulerProperties { private int threadPoolSize = 16; private boolean threadPoolMonitoringEnabled = false; + public int getThreadPoolSize() { + return threadPoolSize; + } + + public void setThreadPoolSize(int threadPoolSize) { + this.threadPoolSize = threadPoolSize; + } + + public boolean isThreadPoolMonitoringEnabled() { + return threadPoolMonitoringEnabled; + } + + public void setThreadPoolMonitoringEnabled(boolean threadPoolMonitoringEnabled) { + this.threadPoolMonitoringEnabled = threadPoolMonitoringEnabled; + } + } + + public static class ChaosSchedulerProperties { + + private int threadPoolSize = 16; + + private boolean threadPoolMonitoringEnabled = false; public int getThreadPoolSize() { return threadPoolSize; diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FrontendProducerConfiguration.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FrontendProducerConfiguration.java index 46ae60ba74..44924c138f 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FrontendProducerConfiguration.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FrontendProducerConfiguration.java @@ -9,6 +9,7 @@ import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.common.metric.executor.InstrumentedExecutorServiceFactory; import pl.allegro.tech.hermes.frontend.cache.topic.TopicsCache; +import pl.allegro.tech.hermes.frontend.config.FailFastKafkaProducerProperties.ChaosSchedulerProperties; import pl.allegro.tech.hermes.frontend.config.FailFastKafkaProducerProperties.FallbackSchedulerProperties; import pl.allegro.tech.hermes.frontend.producer.BrokerLatencyReporter; import pl.allegro.tech.hermes.frontend.producer.BrokerMessageProducer; @@ -73,12 +74,19 @@ public BrokerMessageProducer multiDatacenterBrokerProducer(@Named("failFastKafka fallbackSchedulerProperties.getThreadPoolSize(), fallbackSchedulerProperties.isThreadPoolMonitoringEnabled() ); + ChaosSchedulerProperties chaosSchedulerProperties = kafkaProducerProperties.getChaosScheduler(); + ScheduledExecutorService chaosScheduler = executorServiceFactory.getScheduledExecutorService( + "chaos", + chaosSchedulerProperties.getThreadPoolSize(), + chaosSchedulerProperties.isThreadPoolMonitoringEnabled() + ); return new MultiDatacenterMessageProducer( kafkaMessageSenders, adminReadinessService, messageConverter, kafkaProducerProperties.getSpeculativeSendDelay(), - fallbackScheduler + fallbackScheduler, + chaosScheduler ); } diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/ChaosException.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/ChaosException.java new file mode 100644 index 0000000000..c2b1a27e7e --- /dev/null +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/ChaosException.java @@ -0,0 +1,8 @@ +package pl.allegro.tech.hermes.frontend.producer.kafka; + +public class ChaosException extends RuntimeException { + + public ChaosException(String datacenter, long delayMs, String messageId) { + super("Scheduled failure occurred for datacenter: " + datacenter + ", messageId: " + messageId + " after " + delayMs + "ms"); + } +} diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSenders.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSenders.java index 4b32b24e0e..109c381122 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSenders.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSenders.java @@ -4,7 +4,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import pl.allegro.tech.hermes.api.Topic; -import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.frontend.metric.CachedTopic; import java.util.List; @@ -26,6 +25,7 @@ public class KafkaMessageSenders { private final TopicMetadataLoadingExecutor topicMetadataLoadingExecutor; private final List localDatacenterTopicMetadataLoaders; private final List kafkaProducerMetadataRefreshers; + private final List datacenters; KafkaMessageSenders(TopicMetadataLoadingExecutor topicMetadataLoadingExecutor, MinInSyncReplicasLoader localMinInSyncReplicasLoader, @@ -43,6 +43,10 @@ public class KafkaMessageSenders { this.kafkaProducerMetadataRefreshers = Stream.concat(Stream.of(localSenders), remoteSenders.stream()) .map(KafkaProducerMetadataRefresher::new) .collect(Collectors.toList()); + this.datacenters = Stream.concat(Stream.of(localSenders), remoteSenders.stream()) + .map(tuple -> tuple.ackAll) + .map(KafkaMessageSender::getDatacenter) + .toList(); } KafkaMessageSender get(Topic topic) { @@ -53,6 +57,10 @@ List> getRemote(Topic topic) { return topic.isReplicationConfirmRequired() ? remoteAckLeader : remoteAckAll; } + List getDatacenters() { + return datacenters; + } + void refreshTopicMetadata() { topicMetadataLoadingExecutor.execute(kafkaProducerMetadataRefreshers); } diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/MultiDatacenterMessageProducer.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/MultiDatacenterMessageProducer.java index e659aa5161..b60cccd65d 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/MultiDatacenterMessageProducer.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/MultiDatacenterMessageProducer.java @@ -3,6 +3,11 @@ import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.clients.producer.RecordMetadata; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import pl.allegro.tech.hermes.api.PublishingChaosPolicy; +import pl.allegro.tech.hermes.api.PublishingChaosPolicy.ChaosPolicy; +import pl.allegro.tech.hermes.api.Topic; import pl.allegro.tech.hermes.frontend.metric.CachedTopic; import pl.allegro.tech.hermes.frontend.producer.BrokerMessageProducer; import pl.allegro.tech.hermes.frontend.publishing.PublishingCallback; @@ -10,10 +15,13 @@ import pl.allegro.tech.hermes.frontend.readiness.AdminReadinessService; import java.time.Duration; +import java.util.HashMap; import java.util.Map; import java.util.Optional; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -22,45 +30,63 @@ public class MultiDatacenterMessageProducer implements BrokerMessageProducer { + private static final Logger logger = LoggerFactory.getLogger(MultiDatacenterMessageProducer.class); + private final KafkaMessageSenders kafkaMessageSenders; private final MessageToKafkaProducerRecordConverter messageConverter; private final Duration speculativeSendDelay; private final AdminReadinessService adminReadinessService; private final ScheduledExecutorService fallbackScheduler; + private final ScheduledExecutorService chaosScheduler; public MultiDatacenterMessageProducer(KafkaMessageSenders kafkaMessageSenders, AdminReadinessService adminReadinessService, MessageToKafkaProducerRecordConverter messageConverter, Duration speculativeSendDelay, - ScheduledExecutorService fallbackScheduler) { + ScheduledExecutorService fallbackScheduler, + ScheduledExecutorService chaosScheduler) { this.messageConverter = messageConverter; this.kafkaMessageSenders = kafkaMessageSenders; this.speculativeSendDelay = speculativeSendDelay; this.adminReadinessService = adminReadinessService; this.fallbackScheduler = fallbackScheduler; + this.chaosScheduler = chaosScheduler; } @Override public void send(Message message, CachedTopic cachedTopic, PublishingCallback callback) { var producerRecord = messageConverter.convertToProducerRecord(message, cachedTopic.getKafkaTopics().getPrimary().name()); + KafkaMessageSender localSender = kafkaMessageSenders.get(cachedTopic.getTopic()); Optional> remoteSender = getRemoteSender(cachedTopic); final SendCallback sendCallback = remoteSender.isPresent() ? SendCallback.withFallback(callback) : SendCallback.withoutFallback(callback); + Map experiments = createChaosExperimentsPerDatacenter(cachedTopic.getTopic()); + fallbackScheduler.schedule(() -> { if (!sendCallback.sent.get() && remoteSender.isPresent()) { - send(remoteSender.get(), + sendOrScheduleChaosExperiment( + remoteSender.get(), producerRecord, sendCallback, cachedTopic, - message); + message, + experiments.getOrDefault(remoteSender.get().getDatacenter(), ChaosExperiment.DISABLED) + ); } }, speculativeSendDelay.toMillis(), TimeUnit.MILLISECONDS); - send(kafkaMessageSenders.get(cachedTopic.getTopic()), producerRecord, sendCallback, cachedTopic, message); + sendOrScheduleChaosExperiment( + localSender, + producerRecord, + sendCallback, + cachedTopic, + message, + experiments.getOrDefault(localSender.getDatacenter(), ChaosExperiment.DISABLED) + ); } private void send(KafkaMessageSender sender, @@ -81,6 +107,90 @@ private void send(KafkaMessageSender sender, } } + private void sendOrScheduleChaosExperiment(KafkaMessageSender sender, + ProducerRecord producerRecord, + SendCallback callback, + CachedTopic cachedTopic, + Message message, + ChaosExperiment experiment) { + if (experiment.enabled()) { + scheduleChaosExperiment(experiment, sender, producerRecord, callback, cachedTopic, message); + } else { + send(sender, producerRecord, callback, cachedTopic, message); + } + } + + private void scheduleChaosExperiment(ChaosExperiment experiment, + KafkaMessageSender sender, + ProducerRecord producerRecord, + SendCallback callback, + CachedTopic cachedTopic, + Message message) { + try { + chaosScheduler.schedule(() -> { + if (experiment.completeWithError()) { + var datacenter = sender.getDatacenter(); + var exception = new ChaosException(datacenter, experiment.delayInMillis(), message.getId()); + callback.onUnpublished(message, cachedTopic, datacenter, exception); + } else { + send(sender, producerRecord, callback, cachedTopic, message); + } + }, experiment.delayInMillis(), TimeUnit.MILLISECONDS); + } catch (RejectedExecutionException e) { + logger.warn("Failed while scheduling chaos experiment. Sending message to Kafka.", e); + send(sender, producerRecord, callback, cachedTopic, message); + } + } + + private Map createChaosExperimentsPerDatacenter(Topic topic) { + PublishingChaosPolicy chaos = topic.getChaos(); + return switch (chaos.mode()) { + case DISABLED -> Map.of(); + case GLOBAL -> { + Map experiments = new HashMap<>(); + ChaosPolicy policy = chaos.globalPolicy(); + boolean enabled = computeIfShouldBeEnabled(policy); + for (String datacenter : kafkaMessageSenders.getDatacenters()) { + experiments.put(datacenter, createChaosExperimentForDatacenter(policy, enabled)); + } + yield experiments; + } + case DATACENTER -> { + Map experiments = new HashMap<>(); + Map policies = chaos.datacenterPolicies(); + for (String datacenter : kafkaMessageSenders.getDatacenters()) { + ChaosPolicy policy = policies.get(datacenter); + boolean enabled = computeIfShouldBeEnabled(policy); + experiments.put(datacenter, createChaosExperimentForDatacenter(policy, enabled)); + } + yield experiments; + } + }; + } + + private boolean computeIfShouldBeEnabled(ChaosPolicy policy) { + if (policy == null) { + return false; + } + return ThreadLocalRandom.current().nextInt(100) < policy.probability(); + } + + private ChaosExperiment createChaosExperimentForDatacenter(ChaosPolicy policy, boolean enabled) { + if (!enabled) { + return ChaosExperiment.DISABLED; + } + long delayMillisFrom = policy.delayFrom(); + long delayMillisTo = policy.delayTo(); + long delayMillis = ThreadLocalRandom.current().nextLong(delayMillisTo - delayMillisFrom) + delayMillisFrom; + return new ChaosExperiment(true, policy.completeWithError(), delayMillis); + } + + private record ChaosExperiment(boolean enabled, boolean completeWithError, long delayInMillis) { + + private static final ChaosExperiment DISABLED = new ChaosExperiment(false, false, 0); + + } + private Optional> getRemoteSender(CachedTopic cachedTopic) { return kafkaMessageSenders.getRemote(cachedTopic.getTopic()) .stream() @@ -92,14 +202,14 @@ private record DCAwareCallback(Message message, CachedTopic cachedTopic, String SendCallback callback) implements Callback { @Override - public void onCompletion(RecordMetadata metadata, Exception exception) { - if (exception == null) { - callback.onPublished(message, cachedTopic, datacenter); - } else { - callback.onUnpublished(message, cachedTopic, datacenter, exception); - } + public void onCompletion(RecordMetadata metadata, Exception exception) { + if (exception == null) { + callback.onPublished(message, cachedTopic, datacenter); + } else { + callback.onUnpublished(message, cachedTopic, datacenter, exception); } } + } private static class SendCallback { diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/validator/TopicValidator.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/validator/TopicValidator.java index 39bed768b0..b1a425a9cc 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/validator/TopicValidator.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/validator/TopicValidator.java @@ -3,6 +3,9 @@ import org.springframework.beans.factory.annotation.Autowired; import org.springframework.stereotype.Component; import pl.allegro.tech.hermes.api.ContentType; +import pl.allegro.tech.hermes.api.PublishingChaosPolicy; +import pl.allegro.tech.hermes.api.PublishingChaosPolicy.ChaosMode; +import pl.allegro.tech.hermes.api.PublishingChaosPolicy.ChaosPolicy; import pl.allegro.tech.hermes.api.Topic; import pl.allegro.tech.hermes.management.api.validator.ApiPreconditions; import pl.allegro.tech.hermes.management.domain.auth.RequestUser; @@ -44,6 +47,11 @@ public void ensureCreatedTopicIsValid(Topic created, RequestUser createdBy, Crea throw new TopicValidationException("User is not allowed to enable fallback to remote datacenter"); } + if (created.getChaos().mode() != ChaosMode.DISABLED && !createdBy.isAdmin()) { + throw new TopicValidationException("User is not allowed to set chaos policy for this topic"); + } + validateChaosPolicy(created.getChaos()); + if (created.wasMigratedFromJsonType()) { throw new TopicValidationException("Newly created topic cannot have migratedFromJsonType flag set to true"); } @@ -62,6 +70,11 @@ public void ensureUpdatedTopicIsValid(Topic updated, Topic previous, RequestUser throw new TopicValidationException("User is not allowed to enable fallback to remote datacenter"); } + if (!previous.getChaos().equals(updated.getChaos()) && !modifiedBy.isAdmin()) { + throw new TopicValidationException("User is not allowed to update chaos policy for this topic"); + } + validateChaosPolicy(updated.getChaos()); + if (migrationFromJsonTypeFlagChangedToTrue(updated, previous)) { if (updated.getContentType() != ContentType.AVRO) { throw new TopicValidationException("Change content type to AVRO together with setting migratedFromJsonType flag"); @@ -103,4 +116,23 @@ private void checkContentType(Topic checked) { private void checkTopicLabels(Topic checked) { topicLabelsValidator.check(checked.getLabels()); } + + private void validateChaosPolicy(PublishingChaosPolicy chaosPolicy) { + for (ChaosPolicy policy : chaosPolicy.datacenterPolicies().values()) { + validate(policy); + } + validate(chaosPolicy.globalPolicy()); + } + + private void validate(ChaosPolicy chaosPolicy) { + if (chaosPolicy == null) { + return; + } + if (chaosPolicy.delayFrom() < 0 || chaosPolicy.delayTo() < 0 || chaosPolicy.delayFrom() > chaosPolicy.delayTo()) { + throw new TopicValidationException("Invalid chaos policy: 'delayFrom' and 'delayTo' must be >= 0, and 'delayFrom' <= 'delayTo'."); + } + if (chaosPolicy.probability() < 0 || chaosPolicy.probability() > 100) { + throw new TopicValidationException("Invalid chaos policy: 'probability' must be within the range 0 to 100"); + } + } } diff --git a/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/builder/TopicBuilder.java b/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/builder/TopicBuilder.java index 81eccae2d2..cc15900358 100644 --- a/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/builder/TopicBuilder.java +++ b/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/builder/TopicBuilder.java @@ -4,6 +4,7 @@ import pl.allegro.tech.hermes.api.OfflineRetentionTime; import pl.allegro.tech.hermes.api.OwnerId; import pl.allegro.tech.hermes.api.PublishingAuth; +import pl.allegro.tech.hermes.api.PublishingChaosPolicy; import pl.allegro.tech.hermes.api.RetentionTime; import pl.allegro.tech.hermes.api.Topic; import pl.allegro.tech.hermes.api.TopicDataOfflineStorage; @@ -34,6 +35,8 @@ public class TopicBuilder { private boolean fallbackToRemoteDatacenterEnabled = false; + private PublishingChaosPolicy chaos = PublishingChaosPolicy.disabled(); + private ContentType contentType = ContentType.JSON; private RetentionTime retentionTime = RetentionTime.of(1, TimeUnit.DAYS); @@ -99,7 +102,7 @@ public static TopicBuilder topic(String qualifiedName) { public Topic build() { return new Topic( name, description, owner, retentionTime, migratedFromJsonType, ack, fallbackToRemoteDatacenterEnabled, - trackingEnabled, contentType, jsonToAvroDryRunEnabled, schemaIdAwareSerialization, maxMessageSize, + chaos, trackingEnabled, contentType, jsonToAvroDryRunEnabled, schemaIdAwareSerialization, maxMessageSize, new PublishingAuth(publishers, authEnabled, unauthenticatedAccessEnabled), subscribingRestricted, offlineStorage, labels, null, null ); @@ -194,4 +197,9 @@ public TopicBuilder withLabels(Set labels) { this.labels = labels; return this; } + + public TopicBuilder withPublishingChaosPolicy(PublishingChaosPolicy chaos) { + this.chaos = chaos; + return this; + } } diff --git a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/TopicManagementTest.java b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/TopicManagementTest.java index 0d6574d2be..f23c76d5df 100644 --- a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/TopicManagementTest.java +++ b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/TopicManagementTest.java @@ -12,6 +12,8 @@ import pl.allegro.tech.hermes.api.ErrorDescription; import pl.allegro.tech.hermes.api.Group; import pl.allegro.tech.hermes.api.PatchData; +import pl.allegro.tech.hermes.api.PublishingChaosPolicy; +import pl.allegro.tech.hermes.api.PublishingChaosPolicy.ChaosPolicy; import pl.allegro.tech.hermes.api.Subscription; import pl.allegro.tech.hermes.api.Topic; import pl.allegro.tech.hermes.api.TopicLabel; @@ -22,6 +24,7 @@ import java.util.Arrays; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Set; import java.util.stream.Stream; @@ -31,6 +34,7 @@ import static pl.allegro.tech.hermes.api.ContentType.AVRO; import static pl.allegro.tech.hermes.api.ContentType.JSON; import static pl.allegro.tech.hermes.api.PatchData.patchData; +import static pl.allegro.tech.hermes.api.PublishingChaosPolicy.ChaosMode.DATACENTER; import static pl.allegro.tech.hermes.api.TopicWithSchema.topicWithSchema; import static pl.allegro.tech.hermes.integrationtests.setup.HermesExtension.auditEvents; import static pl.allegro.tech.hermes.integrationtests.setup.HermesExtension.brokerOperations; @@ -685,6 +689,145 @@ public void shouldAllowNonAdminUserToModifyTopicWithFallbackToRemoteDatacenterEn response.expectStatus().isOk(); } + @Test + public void shouldNotAllowNonAdminUserCreateTopicWithChaosEnabled() { + // given + TestSecurityProvider.setUserIsAdmin(false); + TopicWithSchema topic = topicWithSchema( + topicWithRandomName() + .withPublishingChaosPolicy(new PublishingChaosPolicy(DATACENTER, null, Map.of())) + .build() + ); + hermes.initHelper().createGroup(Group.from(topic.getName().getGroupName())); + + // when + WebTestClient.ResponseSpec response = hermes.api().createTopic(topic); + + //then + response.expectStatus().isBadRequest(); + assertThat(response.expectBody(String.class).returnResult().getResponseBody()) + .contains("User is not allowed to set chaos policy for this topic"); + } + + @Test + public void shouldAllowAdminUserCreateTopicWithChaosEnabled() { + // given + TestSecurityProvider.setUserIsAdmin(true); + TopicWithSchema topic = topicWithSchema( + topicWithRandomName() + .withPublishingChaosPolicy(new PublishingChaosPolicy(DATACENTER, null, Map.of())) + .build() + ); + hermes.initHelper().createGroup(Group.from(topic.getName().getGroupName())); + + // when + WebTestClient.ResponseSpec response = hermes.api().createTopic(topic); + + //then + response.expectStatus().isCreated(); + } + + @Test + public void shouldNotCreateTopicWithInvalidChaosPolicy() { + // given + TestSecurityProvider.setUserIsAdmin(true); + TopicWithSchema topic = topicWithSchema( + topicWithRandomName() + .withPublishingChaosPolicy( + new PublishingChaosPolicy(DATACENTER, null, Map.of("dc1", new ChaosPolicy(100, 100, 99, false))) + ) + .build() + ); + hermes.initHelper().createGroup(Group.from(topic.getName().getGroupName())); + + // when + WebTestClient.ResponseSpec response = hermes.api().createTopic(topic); + + //then + response.expectStatus().isBadRequest(); + assertThat(response.expectBody(String.class).returnResult().getResponseBody()) + .contains("Invalid chaos policy: 'delayFrom' and 'delayTo' must be >= 0, and 'delayFrom' <= 'delayTo'."); + } + + @Test + public void shouldNotAllowNonAdminUserToEnableChaos() { + // given + Topic topic = hermes.initHelper().createTopic(topicWithRandomName().build()); + TestSecurityProvider.setUserIsAdmin(false); + PatchData patchData = PatchData.from(ImmutableMap.of("chaos", ImmutableMap.of("mode", DATACENTER))); + + // when + WebTestClient.ResponseSpec response = hermes.api().updateTopic(topic.getQualifiedName(), patchData); + + //then + response.expectStatus().isBadRequest(); + assertThat(response.expectBody(String.class).returnResult().getResponseBody()) + .contains("User is not allowed to update chaos policy for this topic"); + } + + @Test + public void shouldAllowAdminUserToEnableChaos() { + // given + Topic topic = hermes.initHelper().createTopic(topicWithRandomName().build()); + TestSecurityProvider.setUserIsAdmin(true); + PatchData patchData = PatchData.from(ImmutableMap.of("chaos", ImmutableMap.of("mode", DATACENTER))); + + // when + WebTestClient.ResponseSpec response = hermes.api().updateTopic(topic.getQualifiedName(), patchData); + + //then + response.expectStatus().isOk(); + } + + @Test + public void shouldAllowNonAdminUserToModifyTopicWithChaosEnabled() { + // given + TestSecurityProvider.setUserIsAdmin(true); + Topic topic = hermes.initHelper().createTopic( + topicWithRandomName() + .withPublishingChaosPolicy(new PublishingChaosPolicy(DATACENTER, null, Map.of())) + .build() + ); + TestSecurityProvider.setUserIsAdmin(false); + PatchData patchData = PatchData.from(ImmutableMap.of("description", "new description")); + + // when + WebTestClient.ResponseSpec response = hermes.api().updateTopic(topic.getQualifiedName(), patchData); + + //then + response.expectStatus().isOk(); + } + + @Test + public void shouldNotUpdateTopicWithInvalidChaosPolicy() { + // given + TestSecurityProvider.setUserIsAdmin(true); + Topic topic = hermes.initHelper().createTopic( + topicWithRandomName() + .withPublishingChaosPolicy( + new PublishingChaosPolicy(DATACENTER, null, Map.of("dc1", new ChaosPolicy(100, 100, 100, false))) + ) + .build() + ); + PatchData patchData = PatchData.from( + ImmutableMap.of( + "chaos", + ImmutableMap.of( + "datacenterPolicies", + ImmutableMap.of("dc1", ImmutableMap.of("delayTo", 99)) + ) + ) + ); + + // when + WebTestClient.ResponseSpec response = hermes.api().updateTopic(topic.getQualifiedName(), patchData); + + //then + response.expectStatus().isBadRequest(); + assertThat(response.expectBody(String.class).returnResult().getResponseBody()) + .contains("Invalid chaos policy: 'delayFrom' and 'delayTo' must be >= 0, and 'delayFrom' <= 'delayTo'."); + } + private static List getGroupTopicsList(String groupName) { return Arrays.stream(Objects.requireNonNull(hermes.api().listTopics(groupName) .expectStatus() diff --git a/integration-tests/src/slowIntegrationTest/java/pl/allegro/tech/hermes/integrationtests/RemoteDatacenterProduceFallbackTest.java b/integration-tests/src/slowIntegrationTest/java/pl/allegro/tech/hermes/integrationtests/RemoteDatacenterProduceFallbackTest.java index eb299a22ca..3cf7b5eca3 100644 --- a/integration-tests/src/slowIntegrationTest/java/pl/allegro/tech/hermes/integrationtests/RemoteDatacenterProduceFallbackTest.java +++ b/integration-tests/src/slowIntegrationTest/java/pl/allegro/tech/hermes/integrationtests/RemoteDatacenterProduceFallbackTest.java @@ -5,7 +5,11 @@ import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.RegisterExtension; +import org.springframework.test.web.reactive.server.WebTestClient; import org.testcontainers.lifecycle.Startable; +import pl.allegro.tech.hermes.api.PublishingChaosPolicy; +import pl.allegro.tech.hermes.api.PublishingChaosPolicy.ChaosMode; +import pl.allegro.tech.hermes.api.PublishingChaosPolicy.ChaosPolicy; import pl.allegro.tech.hermes.api.Topic; import pl.allegro.tech.hermes.integrationtests.setup.HermesConsumersTestApp; import pl.allegro.tech.hermes.integrationtests.setup.HermesFrontendTestApp; @@ -168,6 +172,91 @@ public void shouldNotFallBackToNotReadyDatacenter() { subscriber.noMessagesReceived(); } + @Test + public void shouldPublishAndConsumeViaRemoteDCWhenChaosExperimentIsEnabledForLocalKafka() { + // given + TestSubscriber subscriber = subscribers.createSubscriber(); + + Topic topic = initHelper.createTopic( + topicWithRandomName() + .withFallbackToRemoteDatacenterEnabled() + .withPublishingChaosPolicy(completeWithErrorForDatacenter(DEFAULT_DC_NAME)) + .build() + ); + initHelper.createSubscription( + subscription(topic.getQualifiedName(), "subscription", subscriber.getEndpoint()).build() + ); + + // and message is published to dc1 + TestMessage message = TestMessage.of("key1", "value1"); + DC1.publishUntilSuccess(topic.getQualifiedName(), message.body()); + + // then message is received in dc2 + subscriber.waitUntilReceived(message.body()); + + // and metrics that message was published to remote dc is incremented + DC1.getFrontendMetrics() + .expectStatus() + .isOk() + .expectBody(String.class) + .value((body) -> assertThatMetrics(body) + .contains("hermes_frontend_topic_published_total") + .withLabels( + "group", topic.getName().getGroupName(), + "topic", topic.getName().getName(), + "storageDc", "dc2" + ) + .withValue(1.0) + ); + } + + @Test + public void shouldReturnErrorWhenChaosExperimentIsEnabledForAllDatacenters() { + // given + TestSubscriber subscriber = subscribers.createSubscriber(); + + Topic topic = initHelper.createTopic( + topicWithRandomName() + .withFallbackToRemoteDatacenterEnabled() + .withPublishingChaosPolicy(completeWithErrorForAllDatacenters()) + .build() + ); + initHelper.createSubscription( + subscription(topic.getQualifiedName(), "subscription", subscriber.getEndpoint()).build() + ); + TestMessage message = TestMessage.of("key1", "value1"); + + // when + DC1.publishUntilStatus(topic.getQualifiedName(), message.body(), 500); + + // then + subscriber.noMessagesReceived(); + } + + private static PublishingChaosPolicy completeWithErrorForAllDatacenters() { + int delayFrom = 100; + int delayTo = 200; + int probability = 100; + boolean completeWithError = true; + return new PublishingChaosPolicy( + ChaosMode.GLOBAL, + new ChaosPolicy(probability, delayFrom, delayTo, completeWithError), + null + ); + } + + private static PublishingChaosPolicy completeWithErrorForDatacenter(String datacenter) { + int delayFrom = 100; + int delayTo = 200; + int probability = 100; + boolean completeWithError = true; + return new PublishingChaosPolicy( + ChaosMode.DATACENTER, + null, + Map.of(datacenter, new ChaosPolicy(probability, delayFrom, delayTo, completeWithError)) + ); + } + private static class HermesDatacenter { private final ZookeeperContainer hermesZookeeper = new ZookeeperContainer("HermesZookeeper"); From eacfaf526573c7b1c880422ab6eaab5940f082e1 Mon Sep 17 00:00:00 2001 From: Mateusz <76775507+szczygiel-m@users.noreply.github.com> Date: Thu, 25 Apr 2024 12:49:42 +0200 Subject: [PATCH 47/87] Updated console dependencies (#1850) * Updated console dependencies * Minor UI fixes --- .github/workflows/ci-console.yml | 2 +- hermes-console/README.md | 2 +- hermes-console/package.json | 74 +- .../ConfirmationDialog.spec.ts | 10 +- .../useCreateSubscription.ts | 1 + .../use-form-subscription/form-mapper.ts | 4 +- .../use-form-subscription/types.ts | 2 + .../useFormSubscription.ts | 6 + .../useImportSubscription.ts | 3 +- .../topic/use-import-topic/useImportTopic.ts | 2 +- hermes-console/src/dummy/subscription-form.ts | 5 + hermes-console/src/i18n/en-US/index.ts | 2 +- .../src/i18n/en-US/subscription-form.ts | 6 + hermes-console/src/mocks/handlers.ts | 491 +-- hermes-console/src/store/auth/useAuthStore.ts | 4 +- .../admin/constraints/ConstraintsView.vue | 5 +- .../views/group-topics/GroupTopicsView.vue | 2 +- .../views/groups/group-form/GroupForm.spec.ts | 4 +- .../subscription-form/SubscriptionForm.vue | 16 +- .../{vite.config.ts => vite.config.mts} | 0 hermes-console/yarn.lock | 3333 ++++++++--------- 21 files changed, 1922 insertions(+), 2052 deletions(-) rename hermes-console/{vite.config.ts => vite.config.mts} (100%) diff --git a/.github/workflows/ci-console.yml b/.github/workflows/ci-console.yml index c104c881e9..3e53793f4c 100644 --- a/.github/workflows/ci-console.yml +++ b/.github/workflows/ci-console.yml @@ -16,7 +16,7 @@ jobs: - name: Setup node uses: actions/setup-node@v4 with: - node-version: 18 + node-version: 20 - name: Run linter run: yarn && yarn lint - name: Run frontend tests diff --git a/hermes-console/README.md b/hermes-console/README.md index 71dff2200f..c3a8fd8b63 100644 --- a/hermes-console/README.md +++ b/hermes-console/README.md @@ -4,7 +4,7 @@ Hermes console written in Vue 3. ## Requirements -* node >=18.0.0 +* node >=20.0.0 * yarn ## Project Setup diff --git a/hermes-console/package.json b/hermes-console/package.json index 6a0195b9c6..321306bf5e 100644 --- a/hermes-console/package.json +++ b/hermes-console/package.json @@ -3,13 +3,13 @@ "description": "Console for Hermes Management", "license": "Apache-2.0", "engines": { - "node": ">=18.0.0" + "node": ">=20.0.0" }, "scripts": { "dev": "vite", "build": "run-p type-check build-only", "preview": "vite preview", - "test:unit": "vitest --environment jsdom", + "test:unit": "vitest --silent --environment jsdom", "build-only": "vite build", "type-check": "vue-tsc --noEmit -p tsconfig.vitest.json --composite false", "lint": "eslint . --ext .vue,.js,.jsx,.cjs,.mjs,.ts,.tsx,.cts,.mts --ignore-path .gitignore", @@ -17,49 +17,49 @@ "dev-server": "node json-server/server.ts" }, "dependencies": { - "ace-builds": "1.28.0", - "axios": "1.4.0", + "ace-builds": "1.33.0", + "axios": "1.6.8", "base64-arraybuffer": "1.0.2", - "jwt-decode": "3.1.2", - "pinia": "2.1.4", - "pinia-plugin-persistedstate": "3.2.0", - "query-string": "8.1.0", - "uuid": "9.0.0", - "vue": "3.3.4", - "vue-i18n": "9.2.2", - "vue-router": "4.2.2", - "vue3-ace-editor": "2.2.3", - "vuetify": "3.3.2" + "jwt-decode": "4.0.0", + "pinia": "2.1.7", + "pinia-plugin-persistedstate": "3.2.1", + "query-string": "9.0.0", + "uuid": "9.0.1", + "vue": "3.4.23", + "vue-i18n": "9.13.0", + "vue-router": "4.3.2", + "vue3-ace-editor": "2.2.4", + "vuetify": "3.5.16" }, "devDependencies": { - "@mdi/font": "7.1.96", + "@mdi/font": "7.4.47", "@pinia/testing": "0.1.3", - "@rushstack/eslint-patch": "1.2.0", - "@testing-library/jest-dom": "5.16.5", - "@testing-library/user-event": "14.4.3", - "@testing-library/vue": "7.0.0", - "@types/jsdom": "21.1.0", - "@types/node": "18.13.0", - "@types/uuid": "9.0.2", - "@vitejs/plugin-vue": "4.0.0", - "@vue/eslint-config-prettier": "7.0.0", - "@vue/eslint-config-typescript": "11.0.2", - "@vue/test-utils": "2.3.2", + "@rushstack/eslint-patch": "1.10.2", + "@testing-library/jest-dom": "6.4.2", + "@testing-library/user-event": "14.5.2", + "@testing-library/vue": "8.0.3", + "@types/jsdom": "21.1.6", + "@types/node": "20.12.7", + "@types/uuid": "9.0.8", + "@vitejs/plugin-vue": "5.0.4", + "@vue/eslint-config-prettier": "9.0.0", + "@vue/eslint-config-typescript": "13.0.0", + "@vue/test-utils": "2.4.5", "@vue/tsconfig": "0.1.3", - "eslint": "8.34.0", + "eslint": "8.57.0", "eslint-plugin-sort-imports-es6-autofix": "0.6.0", - "eslint-plugin-vue": "9.9.0", - "jsdom": "22.1.0", - "json-server": "0.17.1", - "msw": "1.2.2", + "eslint-plugin-vue": "9.25.0", + "jsdom": "24.0.0", + "json-server": "0.17.4", + "msw": "2.2.14", "npm-run-all": "4.1.5", - "prettier": "2.8.4", - "sass": "1.58.1", + "prettier": "3.2.5", + "sass": "1.75.0", "typescript": "4.9.5", - "vite": "4.1.1", + "vite": "5.2.9", "vite-plugin-rewrite-all": "1.0.1", - "vite-plugin-vuetify": "1.0.2", - "vitest": "0.31.4", - "vue-tsc": "1.0.24" + "vite-plugin-vuetify": "2.0.3", + "vitest": "1.5.0", + "vue-tsc": "2.0.13" } } diff --git a/hermes-console/src/components/confirmation-dialog/ConfirmationDialog.spec.ts b/hermes-console/src/components/confirmation-dialog/ConfirmationDialog.spec.ts index 5ac517f014..b32945bdf5 100644 --- a/hermes-console/src/components/confirmation-dialog/ConfirmationDialog.spec.ts +++ b/hermes-console/src/components/confirmation-dialog/ConfirmationDialog.spec.ts @@ -126,20 +126,20 @@ describe('ConfirmationDialog', () => { ).toBeEnabled(); //when - await userEvent.type(getAllByRole('textbox')[1], 'not matching text'); + await userEvent.type(getAllByRole('textbox')[0], 'not matching text'); //then - expect(getAllByRole('textbox')[1]).toHaveValue('not matching text'); + expect(getAllByRole('textbox')[0]).toHaveValue('not matching text'); expect( getByText('confirmationDialog.confirm').closest('button'), ).toBeDisabled(); //when - await userEvent.clear(getAllByRole('textbox')[1]); - await userEvent.type(getAllByRole('textbox')[1], 'prod'); + await userEvent.clear(getAllByRole('textbox')[0]); + await userEvent.type(getAllByRole('textbox')[0], 'prod'); //then - expect(getAllByRole('textbox')[1]).toHaveValue('prod'); + expect(getAllByRole('textbox')[0]).toHaveValue('prod'); expect( getByText('confirmationDialog.confirm').closest('button'), ).toBeEnabled(); diff --git a/hermes-console/src/composables/subscription/use-create-subscription/useCreateSubscription.ts b/hermes-console/src/composables/subscription/use-create-subscription/useCreateSubscription.ts index 2a616c680c..813f71ade4 100644 --- a/hermes-console/src/composables/subscription/use-create-subscription/useCreateSubscription.ts +++ b/hermes-console/src/composables/subscription/use-create-subscription/useCreateSubscription.ts @@ -128,6 +128,7 @@ function initializeForm( .messageTtl || 3600, retryBackoff: 1000, retryBackoffMultiplier: 1.0, + backoffMaxIntervalInSec: 600, sendingDelay: 0, requestTimeout: loadedConfig.value.subscription.defaults.subscriptionPolicy diff --git a/hermes-console/src/composables/subscription/use-form-subscription/form-mapper.ts b/hermes-console/src/composables/subscription/use-form-subscription/form-mapper.ts index f14b32f87c..e6d6effaec 100644 --- a/hermes-console/src/composables/subscription/use-form-subscription/form-mapper.ts +++ b/hermes-console/src/composables/subscription/use-form-subscription/form-mapper.ts @@ -68,7 +68,9 @@ function mapSerialSubscriptionPolicy( form: SubscriptionForm, ): SerialSubscriptionPolicyJson { return { - backoffMaxIntervalInSec: 600, + backoffMaxIntervalInSec: parseFloat( + String(form.subscriptionPolicy.backoffMaxIntervalInSec), + ), backoffMultiplier: parseFloat( String(form.subscriptionPolicy.retryBackoffMultiplier), ), diff --git a/hermes-console/src/composables/subscription/use-form-subscription/types.ts b/hermes-console/src/composables/subscription/use-form-subscription/types.ts index 3d21268760..c678a440b8 100644 --- a/hermes-console/src/composables/subscription/use-form-subscription/types.ts +++ b/hermes-console/src/composables/subscription/use-form-subscription/types.ts @@ -41,6 +41,7 @@ export interface FormSubscriptionPolicy { retryBackoff: number; sendingDelay: number; retryBackoffMultiplier: number; + backoffMaxIntervalInSec: number; requestTimeout: number; batchSize: number | null; batchTime: number | null; @@ -70,6 +71,7 @@ export interface FormValidators { inflightMessageTTL: FieldValidator[]; retryBackoff: FieldValidator[]; retryBackoffMultiplier: FieldValidator[]; + backoffMaxIntervalInSec: FieldValidator[]; messageDeliveryTrackingMode: FieldValidator[]; monitoringSeverity: FieldValidator[]; } diff --git a/hermes-console/src/composables/subscription/use-form-subscription/useFormSubscription.ts b/hermes-console/src/composables/subscription/use-form-subscription/useFormSubscription.ts index 865ab8ccdf..6f4c679ad1 100644 --- a/hermes-console/src/composables/subscription/use-form-subscription/useFormSubscription.ts +++ b/hermes-console/src/composables/subscription/use-form-subscription/useFormSubscription.ts @@ -70,6 +70,7 @@ function formValidators(form: Ref): FormValidators { inflightMessageTTL: [required(), min(0), max(7200)], retryBackoff: [required(), min(0), max(1000000)], retryBackoffMultiplier: [required(), min(1), max(10)], + backoffMaxIntervalInSec: [required(), min(1), max(600)], messageDeliveryTrackingMode: [required()], monitoringSeverity: [required()], }; @@ -179,6 +180,7 @@ function createEmptyForm(): Ref { inflightMessageTTL: 3600, retryBackoff: 1000, retryBackoffMultiplier: 1.0, + backoffMaxIntervalInSec: 600, sendingDelay: 0, requestTimeout: 1000, batchSize: null, @@ -225,6 +227,10 @@ export function initializeFullyFilledForm( subscription.deliveryType === DeliveryType.SERIAL ? subscription.subscriptionPolicy.backoffMultiplier : 1.0, + backoffMaxIntervalInSec: + subscription.deliveryType === DeliveryType.SERIAL + ? subscription.subscriptionPolicy.backoffMaxIntervalInSec + : 600, sendingDelay: subscription.deliveryType === DeliveryType.SERIAL ? subscription.subscriptionPolicy.sendingDelay diff --git a/hermes-console/src/composables/subscription/use-import-subscription/useImportSubscription.ts b/hermes-console/src/composables/subscription/use-import-subscription/useImportSubscription.ts index bb8904fe7a..3acffbb1ca 100644 --- a/hermes-console/src/composables/subscription/use-import-subscription/useImportSubscription.ts +++ b/hermes-console/src/composables/subscription/use-import-subscription/useImportSubscription.ts @@ -22,8 +22,7 @@ export function useImportSubscription(): UseImportSubscription { ) { if (importedFile.value) { const reader = new FileReader(); - - reader.readAsText(importedFile.value[0]); + reader.readAsText(importedFile.value); reader.onload = function () { const subscription = JSON.parse(reader.result); diff --git a/hermes-console/src/composables/topic/use-import-topic/useImportTopic.ts b/hermes-console/src/composables/topic/use-import-topic/useImportTopic.ts index 9f756c72fa..3d33c58722 100644 --- a/hermes-console/src/composables/topic/use-import-topic/useImportTopic.ts +++ b/hermes-console/src/composables/topic/use-import-topic/useImportTopic.ts @@ -23,7 +23,7 @@ export function useImportTopic(): UseImportTopic { if (importedFile.value) { const reader = new FileReader(); - reader.readAsText(importedFile.value[0]); + reader.readAsText(importedFile.value); reader.onload = function () { const topic = JSON.parse(reader.result); diff --git a/hermes-console/src/dummy/subscription-form.ts b/hermes-console/src/dummy/subscription-form.ts index ee096c6158..c78e67b274 100644 --- a/hermes-console/src/dummy/subscription-form.ts +++ b/hermes-console/src/dummy/subscription-form.ts @@ -20,6 +20,7 @@ export const dummySubscriptionForm = { inflightMessageTTL: 3600, retryBackoff: 1000, retryBackoffMultiplier: 1.0, + backoffMaxIntervalInSec: 600, sendingDelay: 0, requestTimeout: 1000, batchSize: null, @@ -60,6 +61,7 @@ export const dummySubscriptionFormValidator = { inflightMessageTTL: [required(), min(0), max(7200)], retryBackoff: [required(), min(0), max(1000000)], retryBackoffMultiplier: [required(), min(1), max(10)], + backoffMaxIntervalInSec: [required(), min(1), max(600)], messageDeliveryTrackingMode: [required()], monitoringSeverity: [required()], }; @@ -143,6 +145,7 @@ export const dummyInitializedSubscriptionForm = { 3600, retryBackoff: 1000, retryBackoffMultiplier: 1.0, + backoffMaxIntervalInSec: 600, sendingDelay: 0, requestTimeout: dummyAppConfig.subscription.defaults.subscriptionPolicy.requestTimeout || @@ -183,6 +186,8 @@ export const dummyInitializedEditSubscriptionForm = { retryBackoff: dummySubscription.subscriptionPolicy.messageBackoff, retryBackoffMultiplier: dummySubscription.subscriptionPolicy.backoffMultiplier, + backoffMaxIntervalInSec: + dummySubscription.subscriptionPolicy.backoffMaxIntervalInSec, sendingDelay: dummySubscription.subscriptionPolicy.sendingDelay, requestTimeout: dummySubscription.subscriptionPolicy.requestTimeout, batchSize: null, diff --git a/hermes-console/src/i18n/en-US/index.ts b/hermes-console/src/i18n/en-US/index.ts index 9386580a1b..da8fd9da5f 100644 --- a/hermes-console/src/i18n/en-US/index.ts +++ b/hermes-console/src/i18n/en-US/index.ts @@ -510,7 +510,7 @@ const en_US = { retryClientErrors: 'Retry on 4xx status', retryBackoff: 'Retry backoff', backoffMultiplier: 'Retry backoff multiplier', - backoffMaxInterval: 'Retry backoff max interval', + backoffMaxIntervalInSec: 'Retry backoff max interval', monitoringSeverity: 'Monitoring severity', monitoringReaction: 'Monitoring reaction', http2: 'Deliver using http/2', diff --git a/hermes-console/src/i18n/en-US/subscription-form.ts b/hermes-console/src/i18n/en-US/subscription-form.ts index 392a6e0956..32dd4d3d5e 100644 --- a/hermes-console/src/i18n/en-US/subscription-form.ts +++ b/hermes-console/src/i18n/en-US/subscription-form.ts @@ -77,6 +77,12 @@ const messages = { placeholder: 'Delay multiplier between consecutive send attempts of failed requests', }, + backoffMaxIntervalInSec: { + label: 'Retry backoff max interval', + placeholder: + 'Max delay between consecutive send attempts of failed requests', + suffix: 'seconds', + }, messageDeliveryTrackingMode: { label: 'Message delivery tracking mode', }, diff --git a/hermes-console/src/mocks/handlers.ts b/hermes-console/src/mocks/handlers.ts index 97355c3d14..d2eceac040 100644 --- a/hermes-console/src/mocks/handlers.ts +++ b/hermes-console/src/mocks/handlers.ts @@ -13,7 +13,7 @@ import { dummyUndeliveredMessages, secondDummySubscription, } from '@/dummy/subscription'; -import { rest } from 'msw'; +import { http, HttpResponse } from 'msw'; import type { AccessTokenResponse } from '@/api/access-token-response'; import type { ConstraintsConfig } from '@/api/constraints'; import type { ConsumerGroup } from '@/api/consumer-group'; @@ -43,8 +43,8 @@ export const fetchTopicHandler = ({ }: { topic?: TopicWithSchema; }) => - rest.get(`${url}/topics/${topic.name}`, (req, res, ctx) => { - return res(ctx.status(200), ctx.json(topic)); + http.get(`${url}/topics/${topic.name}`, () => { + return HttpResponse.json(topic); }); export const fetchTopicErrorHandler = ({ @@ -54,21 +54,20 @@ export const fetchTopicErrorHandler = ({ topicName: string; errorCode?: number; }) => - rest.get(`${url}/topics/${topicName}`, (req, res, ctx) => { - return res(ctx.status(errorCode), ctx.json(undefined)); + http.get(`${url}/topics/${topicName}`, () => { + return new HttpResponse(undefined, { + status: errorCode, + }); }); export const fetchOwnerHandler = ({ owner = dummyOwner }: { owner?: Owner }) => - rest.get( - `${url}/owners/sources/Service%20Catalog/${owner.id}`, - (req, res, ctx) => { - return res(ctx.status(200), ctx.json(owner)); - }, - ); + http.get(`${url}/owners/sources/Service%20Catalog/${owner.id}`, () => { + return HttpResponse.json(owner); + }); export const fetchOwnerSourcesHandler = (body: any) => - rest.get(`${url}/owners/sources`, (req, res, ctx) => { - return res(ctx.status(200), ctx.json(body)); + http.get(`${url}/owners/sources`, () => { + return HttpResponse.json(body); }); export const fetchOwnerErrorHandler = ({ @@ -78,12 +77,11 @@ export const fetchOwnerErrorHandler = ({ owner: string; errorCode?: number; }) => - rest.get( - `${url}/owners/sources/Service%20Catalog/${owner}`, - (req, res, ctx) => { - return res(ctx.status(errorCode), ctx.json(undefined)); - }, - ); + http.get(`${url}/owners/sources/Service%20Catalog/${owner}`, () => { + return new HttpResponse(undefined, { + status: errorCode, + }); + }); export const fetchTopicMessagesPreviewHandler = ({ topicName, @@ -92,8 +90,8 @@ export const fetchTopicMessagesPreviewHandler = ({ topicName: string; messages?: MessagePreview[]; }) => - rest.get(`${url}/topics/${topicName}/preview`, (req, res, ctx) => { - return res(ctx.status(200), ctx.json(messages)); + http.get(`${url}/topics/${topicName}/preview`, () => { + return HttpResponse.json(messages); }); export const fetchTopicMessagesPreviewErrorHandler = ({ @@ -103,8 +101,10 @@ export const fetchTopicMessagesPreviewErrorHandler = ({ topicName: string; errorCode?: number; }) => - rest.get(`${url}/topics/${topicName}/preview`, (req, res, ctx) => { - return res(ctx.status(errorCode), ctx.json(undefined)); + http.get(`${url}/topics/${topicName}/preview`, () => { + return new HttpResponse(undefined, { + status: errorCode, + }); }); export const fetchTopicMetricsHandler = ({ @@ -114,8 +114,8 @@ export const fetchTopicMetricsHandler = ({ topicName: string; metrics?: TopicMetrics; }) => - rest.get(`${url}/topics/${topicName}/metrics`, (req, res, ctx) => { - return res(ctx.status(200), ctx.json(metrics)); + http.get(`${url}/topics/${topicName}/metrics`, () => { + return HttpResponse.json(metrics); }); export const fetchTopicMetricsErrorHandler = ({ @@ -125,8 +125,10 @@ export const fetchTopicMetricsErrorHandler = ({ topicName: string; errorCode?: number; }) => - rest.get(`${url}/topics/${topicName}/metrics`, (req, res, ctx) => { - return res(ctx.status(errorCode), ctx.json(undefined)); + http.get(`${url}/topics/${topicName}/metrics`, () => { + return new HttpResponse(undefined, { + status: errorCode, + }); }); export const fetchTopicSubscriptionsHandler = ({ @@ -136,8 +138,8 @@ export const fetchTopicSubscriptionsHandler = ({ topicName: string; subscriptions?: string[]; }) => - rest.get(`${url}/topics/${topicName}/subscriptions`, (req, res, ctx) => { - return res(ctx.status(200), ctx.json(subscriptions)); + http.get(`${url}/topics/${topicName}/subscriptions`, () => { + return HttpResponse.json(subscriptions); }); export const fetchTopicSubscriptionsErrorHandler = ({ @@ -147,8 +149,10 @@ export const fetchTopicSubscriptionsErrorHandler = ({ topicName: string; errorCode?: number; }) => - rest.get(`${url}/topics/${topicName}/subscriptions`, (req, res, ctx) => { - return res(ctx.status(errorCode), ctx.json(undefined)); + http.get(`${url}/topics/${topicName}/subscriptions`, () => { + return new HttpResponse(undefined, { + status: errorCode, + }); }); export const fetchTopicSubscriptionDetailsHandler = ({ @@ -156,10 +160,10 @@ export const fetchTopicSubscriptionDetailsHandler = ({ }: { subscription?: Subscription; }) => - rest.get( + http.get( `${url}/topics/${subscription.topicName}/subscriptions/${subscription.name}`, - (req, res, ctx) => { - return res(ctx.status(200), ctx.json(subscription)); + () => { + return HttpResponse.json(subscription); }, ); @@ -172,10 +176,12 @@ export const fetchTopicSubscriptionDetailsErrorHandler = ({ subscriptionName: string; errorCode?: number; }) => - rest.get( + http.get( `${url}/topics/${topicName}/subscriptions/${subscriptionName}`, - (req, res, ctx) => { - return res(ctx.status(errorCode), ctx.json(undefined)); + () => { + return new HttpResponse(undefined, { + status: errorCode, + }); }, ); @@ -196,10 +202,10 @@ export const fetchSubscriptionHandler = ({ }: { subscription?: Subscription; }) => - rest.get( + http.get( `${url}/topics/${subscription.topicName}/subscriptions/${subscription.name}`, - (req, res, ctx) => { - return res(ctx.status(200), ctx.json(subscription)); + () => { + return HttpResponse.json(subscription); }, ); @@ -212,10 +218,10 @@ export const fetchSubscriptionMetricsHandler = ({ subscriptionName?: string; subscriptionMetrics?: SubscriptionMetrics; }) => - rest.get( + http.get( `${url}/topics/${topicName}/subscriptions/${subscriptionName}/metrics`, - (req, res, ctx) => { - return res(ctx.status(200), ctx.json(subscriptionMetrics)); + () => { + return HttpResponse.json(subscriptionMetrics); }, ); @@ -228,10 +234,10 @@ export const fetchSubscriptionHealthHandler = ({ subscriptionName?: string; subscriptionHealth?: SubscriptionHealth; }) => - rest.get( + http.get( `${url}/topics/${topicName}/subscriptions/${subscriptionName}/health`, - (req, res, ctx) => { - return res(ctx.status(200), ctx.json(subscriptionHealth)); + () => { + return HttpResponse.json(subscriptionHealth); }, ); @@ -244,10 +250,10 @@ export const fetchSubscriptionUndeliveredMessagesHandler = ({ subscriptionName?: string; subscriptionUndeliveredMessages?: SentMessageTrace[]; }) => - rest.get( + http.get( `${url}/topics/${topicName}/subscriptions/${subscriptionName}/undelivered`, - (req, res, ctx) => { - return res(ctx.status(200), ctx.json(subscriptionUndeliveredMessages)); + () => { + return HttpResponse.json(subscriptionUndeliveredMessages); }, ); @@ -260,10 +266,10 @@ export const fetchSubscriptionLastUndeliveredMessageHandler = ({ subscriptionName?: string; subscriptionLastUndeliveredMessage?: SentMessageTrace; }) => - rest.get( + http.get( `${url}/topics/${topicName}/subscriptions/${subscriptionName}/undelivered/last`, - (req, res, ctx) => { - return res(ctx.status(200), ctx.json(subscriptionLastUndeliveredMessage)); + () => { + return HttpResponse.json(subscriptionLastUndeliveredMessage); }, ); @@ -274,10 +280,12 @@ export const fetchSubscriptionErrorHandler = ({ subscription?: Subscription; errorCode?: number; }) => - rest.get( + http.get( `${url}/topics/${subscription.topicName}/subscriptions/${subscription.name}`, - (req, res, ctx) => { - return res(ctx.status(errorCode), ctx.json(undefined)); + () => { + return new HttpResponse(undefined, { + status: errorCode, + }); }, ); @@ -290,10 +298,12 @@ export const fetchSubscriptionMetricsErrorHandler = ({ subscriptionName?: string; errorCode?: number; }) => - rest.get( + http.get( `${url}/topics/${topicName}/subscriptions/${subscriptionName}/metrics`, - (req, res, ctx) => { - return res(ctx.status(errorCode), ctx.json(undefined)); + () => { + return new HttpResponse(undefined, { + status: errorCode, + }); }, ); @@ -306,10 +316,12 @@ export const fetchSubscriptionHealthErrorHandler = ({ subscriptionName?: string; errorCode?: number; }) => - rest.get( + http.get( `${url}/topics/${topicName}/subscriptions/${subscriptionName}/health`, - (req, res, ctx) => { - return res(ctx.status(errorCode), ctx.json(undefined)); + () => { + return new HttpResponse(undefined, { + status: errorCode, + }); }, ); @@ -322,10 +334,12 @@ export const fetchSubscriptionUndeliveredMessagesErrorHandler = ({ subscriptionName?: string; errorCode?: number; }) => - rest.get( + http.get( `${url}/topics/${topicName}/subscriptions/${subscriptionName}/undelivered`, - (req, res, ctx) => { - return res(ctx.status(errorCode), ctx.json(undefined)); + () => { + return new HttpResponse(undefined, { + status: errorCode, + }); }, ); @@ -338,10 +352,12 @@ export const fetchSubscriptionLastUndeliveredMessageErrorHandler = ({ subscriptionName?: string; errorCode?: number; }) => - rest.get( + http.get( `${url}/topics/${topicName}/subscriptions/${subscriptionName}/undelivered/last`, - (req, res, ctx) => { - return res(ctx.status(errorCode), ctx.json(undefined)); + () => { + return new HttpResponse(undefined, { + status: errorCode, + }); }, ); @@ -359,8 +375,8 @@ export const fetchConstraintsHandler = ({ }: { constraints: ConstraintsConfig; }) => - rest.get(`${url}/workload-constraints`, (req, res, ctx) => { - return res(ctx.status(200), ctx.json(constraints)); + http.get(`${url}/workload-constraints`, () => { + return HttpResponse.json(constraints); }); export const fetchConstraintsErrorHandler = ({ @@ -368,8 +384,10 @@ export const fetchConstraintsErrorHandler = ({ }: { errorCode?: number; }) => - rest.get(`${url}/workload-constraints`, (req, res, ctx) => { - return res(ctx.status(errorCode), ctx.json(undefined)); + http.get(`${url}/workload-constraints`, () => { + return new HttpResponse(undefined, { + status: errorCode, + }); }); export const fetchReadinessHandler = ({ @@ -377,8 +395,8 @@ export const fetchReadinessHandler = ({ }: { datacentersReadiness: DatacenterReadiness[]; }) => - rest.get(`${url}/readiness/datacenters`, (req, res, ctx) => { - return res(ctx.status(200), ctx.json(datacentersReadiness)); + http.get(`${url}/readiness/datacenters`, () => { + return HttpResponse.json(datacentersReadiness); }); export const fetchReadinessErrorHandler = ({ @@ -386,8 +404,10 @@ export const fetchReadinessErrorHandler = ({ }: { errorCode?: number; }) => - rest.get(`${url}/readiness/datacenters`, (req, res, ctx) => { - return res(ctx.status(errorCode), ctx.json(undefined)); + http.get(`${url}/readiness/datacenters`, () => { + return new HttpResponse(undefined, { + status: errorCode, + }); }); export const fetchConsumerGroupsHandler = ({ @@ -399,10 +419,10 @@ export const fetchConsumerGroupsHandler = ({ topicName: string; subscriptionName: string; }) => - rest.get( + http.get( `${url}/topics/${topicName}/subscriptions/${subscriptionName}/consumer-groups`, - (req, res, ctx) => { - return res(ctx.status(200), ctx.json(consumerGroups)); + () => { + return HttpResponse.json(consumerGroups); }, ); @@ -415,10 +435,12 @@ export const fetchConsumerGroupsErrorHandler = ({ topicName: string; subscriptionName: string; }) => - rest.get( + http.get( `${url}/topics/${topicName}/subscriptions/${subscriptionName}/consumer-groups`, - (req, res, ctx) => { - return res(ctx.status(errorCode), ctx.json(undefined)); + () => { + return new HttpResponse(undefined, { + status: errorCode, + }); }, ); @@ -427,8 +449,8 @@ export const fetchInconsistentTopicsHandler = ({ }: { topics: string[]; }) => - rest.get(`${url}/consistency/inconsistencies/topics`, (req, res, ctx) => { - return res(ctx.status(200), ctx.json(topics)); + http.get(`${url}/consistency/inconsistencies/topics`, () => { + return HttpResponse.json(topics); }); export const fetchInconsistentTopicsErrorHandler = ({ @@ -436,8 +458,10 @@ export const fetchInconsistentTopicsErrorHandler = ({ }: { errorCode?: number; }) => - rest.get(`${url}/consistency/inconsistencies/topics`, (req, res, ctx) => { - return res(ctx.status(errorCode), ctx.json(undefined)); + http.get(`${url}/consistency/inconsistencies/topics`, () => { + return new HttpResponse(undefined, { + status: errorCode, + }); }); export const fetchTopicNamesHandler = ({ @@ -445,8 +469,8 @@ export const fetchTopicNamesHandler = ({ }: { topicNames: string[]; }) => - rest.get(`${url}/topics`, (req, res, ctx) => { - return res(ctx.status(200), ctx.json(topicNames)); + http.get(`${url}/topics`, () => { + return HttpResponse.json(topicNames); }); export const fetchTopicNamesErrorHandler = ({ @@ -454,8 +478,10 @@ export const fetchTopicNamesErrorHandler = ({ }: { errorCode?: number; }) => - rest.get(`${url}/topics`, (req, res, ctx) => { - return res(ctx.status(errorCode), ctx.json(undefined)); + http.get(`${url}/topics`, () => { + return new HttpResponse(undefined, { + status: errorCode, + }); }); export const fetchGroupNamesHandler = ({ @@ -463,8 +489,8 @@ export const fetchGroupNamesHandler = ({ }: { groupNames: string[]; }) => - rest.get(`${url}/groups`, (req, res, ctx) => { - return res(ctx.status(200), ctx.json(groupNames)); + http.get(`${url}/groups`, () => { + return HttpResponse.json(groupNames); }); export const fetchGroupNamesErrorHandler = ({ @@ -472,13 +498,15 @@ export const fetchGroupNamesErrorHandler = ({ }: { errorCode?: number; }) => - rest.get(`${url}/groups`, (req, res, ctx) => { - return res(ctx.status(errorCode), ctx.json(undefined)); + http.get(`${url}/groups`, () => { + return new HttpResponse(undefined, { + status: errorCode, + }); }); export const fetchStatsHandler = ({ stats }: { stats: Stats }) => - rest.get(`${url}/stats`, (req, res, ctx) => { - return res(ctx.status(200), ctx.json(stats)); + http.get(`${url}/stats`, () => { + return HttpResponse.json(stats); }); export const fetchStatsErrorHandler = ({ @@ -486,8 +514,10 @@ export const fetchStatsErrorHandler = ({ }: { errorCode?: number; }) => - rest.get(`${url}/stats`, (req, res, ctx) => { - return res(ctx.status(errorCode), ctx.json(undefined)); + http.get(`${url}/stats`, () => { + return new HttpResponse(undefined, { + status: errorCode, + }); }); export const fetchTokenHandler = ({ @@ -495,8 +525,8 @@ export const fetchTokenHandler = ({ }: { accessToken: AccessTokenResponse; }) => - rest.post(`http://localhost:8080/token`, (req, res, ctx) => { - return res(ctx.status(200), ctx.json(accessToken)); + http.post(`http://localhost:8080/token`, () => { + return HttpResponse.json(accessToken); }); export const queryTopicsHandler = ({ @@ -504,8 +534,8 @@ export const queryTopicsHandler = ({ }: { topics?: Topic[]; }) => - rest.post(`${url}/query/topics`, (req, res, ctx) => { - return res(ctx.status(200), ctx.json(topics)); + http.post(`${url}/query/topics`, () => { + return HttpResponse.json(topics); }); export const queryTopicsErrorHandler = ({ @@ -513,8 +543,10 @@ export const queryTopicsErrorHandler = ({ }: { errorCode?: number; }) => - rest.post(`${url}/query/topics`, (req, res, ctx) => { - return res(ctx.status(errorCode), ctx.json(undefined)); + http.post(`${url}/query/topics`, () => { + return new HttpResponse(undefined, { + status: errorCode, + }); }); export const querySubscriptionsHandler = ({ @@ -522,8 +554,8 @@ export const querySubscriptionsHandler = ({ }: { subscriptions?: Subscription[]; }) => - rest.post(`${url}/query/subscriptions`, (req, res, ctx) => { - return res(ctx.status(200), ctx.json(subscriptions)); + http.post(`${url}/query/subscriptions`, () => { + return HttpResponse.json(subscriptions); }); export const querySubscriptionsErrorHandler = ({ @@ -531,8 +563,10 @@ export const querySubscriptionsErrorHandler = ({ }: { errorCode?: number; }) => - rest.post(`${url}/query/subscriptions`, (req, res, ctx) => { - return res(ctx.status(errorCode), ctx.json(undefined)); + http.post(`${url}/query/subscriptions`, () => { + return new HttpResponse(undefined, { + status: errorCode, + }); }); export const fetchRolesHandler = ({ @@ -542,8 +576,8 @@ export const fetchRolesHandler = ({ roles: Role[]; path: string; }) => - rest.get(path, (req, res, ctx) => { - return res(ctx.status(200), ctx.json(roles)); + http.get(path, () => { + return HttpResponse.json(roles); }); export const fetchRolesErrorHandler = ({ @@ -553,8 +587,10 @@ export const fetchRolesErrorHandler = ({ errorCode?: number; path: string; }) => - rest.get(path, (req, res, ctx) => { - return res(ctx.status(errorCode), ctx.json(undefined)); + http.get(path, () => { + return new HttpResponse(undefined, { + status: errorCode, + }); }); export const fetchMetricsDashboardUrlHandler = ({ @@ -564,8 +600,8 @@ export const fetchMetricsDashboardUrlHandler = ({ dashboardUrl: DashboardUrl; path: string; }) => - rest.get(path, (req, res, ctx) => { - return res(ctx.status(200), ctx.json(dashboardUrl)); + http.get(path, () => { + return HttpResponse.json(dashboardUrl); }); export const fetchMetricsDashboardUrlErrorHandler = ({ @@ -575,8 +611,10 @@ export const fetchMetricsDashboardUrlErrorHandler = ({ errorCode?: number; path: string; }) => - rest.get(path, (req, res, ctx) => { - return res(ctx.status(errorCode), ctx.json(undefined)); + http.get(path, () => { + return new HttpResponse(undefined, { + status: errorCode, + }); }); export const fetchConsistencyGroupsHandler = ({ @@ -584,8 +622,8 @@ export const fetchConsistencyGroupsHandler = ({ }: { groups: string[]; }) => - rest.get(`${url}/consistency/groups`, (req, res, ctx) => { - return res(ctx.status(200), ctx.json(groups)); + http.get(`${url}/consistency/groups`, () => { + return HttpResponse.json(groups); }); export const fetchConsistencyGroupsErrorHandler = ({ @@ -593,8 +631,10 @@ export const fetchConsistencyGroupsErrorHandler = ({ }: { errorCode?: number; }) => - rest.get(`${url}/consistency/groups`, (req, res, ctx) => { - return res(ctx.status(errorCode), ctx.json(undefined)); + http.get(`${url}/consistency/groups`, () => { + return new HttpResponse(undefined, { + status: errorCode, + }); }); export const fetchGroupInconsistenciesHandler = ({ @@ -602,8 +642,8 @@ export const fetchGroupInconsistenciesHandler = ({ }: { groupsInconsistency: InconsistentGroup[]; }) => - rest.get(`${url}/consistency/inconsistencies/groups`, (req, res, ctx) => { - return res(ctx.status(200), ctx.json(groupsInconsistency)); + http.get(`${url}/consistency/inconsistencies/groups`, () => { + return HttpResponse.json(groupsInconsistency); }); export const fetchGroupInconsistenciesErrorHandler = ({ @@ -611,13 +651,15 @@ export const fetchGroupInconsistenciesErrorHandler = ({ }: { errorCode?: number; }) => - rest.get(`${url}/consistency/inconsistencies/groups`, (req, res, ctx) => { - return res(ctx.status(errorCode), ctx.json(undefined)); + http.get(`${url}/consistency/inconsistencies/groups`, () => { + return new HttpResponse(undefined, { + status: errorCode, + }); }); export const removeGroupHandler = ({ group }: { group: string }) => - rest.delete(`/groups/${group}`, (req, res, ctx) => { - return res(ctx.status(200), ctx.json(undefined)); + http.delete(`/groups/${group}`, () => { + return HttpResponse.json(undefined); }); export const removeGroupErrorHandler = ({ @@ -627,13 +669,15 @@ export const removeGroupErrorHandler = ({ group: string; errorCode: number; }) => - rest.delete(`/groups/${group}`, (req, res, ctx) => { - return res(ctx.status(errorCode), ctx.json(undefined)); + http.delete(`/groups/${group}`, () => { + return new HttpResponse(undefined, { + status: errorCode, + }); }); export const removeTopicHandler = ({ topic }: { topic: string }) => - rest.delete(`/topics/${topic}`, (req, res, ctx) => { - return res(ctx.status(200), ctx.json(undefined)); + http.delete(`/topics/${topic}`, () => { + return HttpResponse.json(undefined); }); export const removeTopicErrorHandler = ({ @@ -643,13 +687,15 @@ export const removeTopicErrorHandler = ({ topic: string; errorCode: number; }) => - rest.delete(`/topics/${topic}`, (req, res, ctx) => { - return res(ctx.status(errorCode), ctx.json(undefined)); + http.delete(`/topics/${topic}`, () => { + return new HttpResponse(undefined, { + status: errorCode, + }); }); export const removeInconsistentTopicHandler = () => - rest.delete(`/consistency/inconsistencies/topics`, (req, res, ctx) => { - return res(ctx.status(200), ctx.json(undefined)); + http.delete(`/consistency/inconsistencies/topics`, () => { + return HttpResponse.json(undefined); }); export const removeInconsistentTopicErrorHandler = ({ @@ -657,8 +703,10 @@ export const removeInconsistentTopicErrorHandler = ({ }: { errorCode: number; }) => - rest.delete(`/consistency/inconsistencies/topics`, (req, res, ctx) => { - return res(ctx.status(errorCode), ctx.json(undefined)); + http.delete(`/consistency/inconsistencies/topics`, () => { + return new HttpResponse(undefined, { + status: errorCode, + }); }); export const removeSubscriptionHandler = ({ @@ -668,12 +716,9 @@ export const removeSubscriptionHandler = ({ topic: string; subscription: string; }) => - rest.delete( - `/topics/${topic}/subscriptions/${subscription}`, - (req, res, ctx) => { - return res(ctx.status(200), ctx.json(undefined)); - }, - ); + http.delete(`/topics/${topic}/subscriptions/${subscription}`, () => { + return HttpResponse.json(undefined); + }); export const removeSubscriptionErrorHandler = ({ topic, @@ -684,12 +729,11 @@ export const removeSubscriptionErrorHandler = ({ subscription: string; errorCode: number; }) => - rest.delete( - `/topics/${topic}/subscriptions/${subscription}`, - (req, res, ctx) => { - return res(ctx.status(errorCode), ctx.json(undefined)); - }, - ); + http.delete(`/topics/${topic}/subscriptions/${subscription}`, () => { + return new HttpResponse(undefined, { + status: errorCode, + }); + }); export const subscriptionStateHandler = ({ topic, @@ -698,12 +742,9 @@ export const subscriptionStateHandler = ({ topic: string; subscription: string; }) => - rest.put( - `/topics/${topic}/subscriptions/${subscription}/state`, - (req, res, ctx) => { - return res(ctx.status(200), ctx.json(undefined)); - }, - ); + http.put(`/topics/${topic}/subscriptions/${subscription}/state`, () => { + return HttpResponse.json(undefined); + }); export const subscriptionStateErrorHandler = ({ topic, @@ -714,20 +755,19 @@ export const subscriptionStateErrorHandler = ({ subscription: string; errorCode: number; }) => - rest.put( - `/topics/${topic}/subscriptions/${subscription}/state`, - (req, res, ctx) => { - return res(ctx.status(errorCode), ctx.json(undefined)); - }, - ); + http.put(`/topics/${topic}/subscriptions/${subscription}/state`, () => { + return new HttpResponse(undefined, { + status: errorCode, + }); + }); export const switchReadinessHandler = ({ datacenter, }: { datacenter: string; }) => - rest.post(`/readiness/datacenters/${datacenter}`, (req, res, ctx) => { - return res(ctx.status(200), ctx.json(undefined)); + http.post(`/readiness/datacenters/${datacenter}`, () => { + return HttpResponse.json(undefined); }); export const switchReadinessErrorHandler = ({ @@ -737,8 +777,10 @@ export const switchReadinessErrorHandler = ({ datacenter: string; errorCode: number; }) => - rest.post(`/readiness/datacenters/${datacenter}`, (req, res, ctx) => { - return res(ctx.status(errorCode), ctx.json(undefined)); + http.post(`/readiness/datacenters/${datacenter}`, () => { + return new HttpResponse(undefined, { + status: errorCode, + }); }); export const moveSubscriptionOffsetsHandler = ({ @@ -750,10 +792,12 @@ export const moveSubscriptionOffsetsHandler = ({ subscriptionName: string; statusCode: number; }) => - rest.post( + http.post( `${url}/topics/${topicName}/subscriptions/${subscriptionName}/moveOffsetsToTheEnd`, - (req, res, ctx) => { - return res(ctx.status(statusCode), ctx.json(undefined)); + () => { + return new HttpResponse(undefined, { + status: statusCode, + }); }, ); @@ -762,8 +806,10 @@ export const upsertTopicConstraintHandler = ({ }: { statusCode: number; }) => - rest.put(`${url}/workload-constraints/topic`, (req, res, ctx) => { - return res(ctx.status(statusCode), ctx.json(undefined)); + http.put(`${url}/workload-constraints/topic`, () => { + return new HttpResponse(undefined, { + status: statusCode, + }); }); export const upsertSubscriptionConstraintHandler = ({ @@ -771,8 +817,10 @@ export const upsertSubscriptionConstraintHandler = ({ }: { statusCode: number; }) => - rest.put(`${url}/workload-constraints/subscription`, (req, res, ctx) => { - return res(ctx.status(statusCode), ctx.json(undefined)); + http.put(`${url}/workload-constraints/subscription`, () => { + return new HttpResponse(undefined, { + status: statusCode, + }); }); export const deleteTopicConstraintHandler = ({ @@ -782,12 +830,11 @@ export const deleteTopicConstraintHandler = ({ statusCode: number; topicName: string; }) => - rest.delete( - `${url}/workload-constraints/topic/${topicName}`, - (req, res, ctx) => { - return res(ctx.status(statusCode), ctx.json(undefined)); - }, - ); + http.delete(`${url}/workload-constraints/topic/${topicName}`, () => { + return new HttpResponse(undefined, { + status: statusCode, + }); + }); export const deleteSubscriptionConstraintHandler = ({ statusCode, @@ -798,69 +845,73 @@ export const deleteSubscriptionConstraintHandler = ({ topicName: string; subscriptionName: string; }) => - rest.delete( + http.delete( `${url}/workload-constraints/subscription/${topicName}/${subscriptionName}`, - (req, res, ctx) => { - return res(ctx.status(statusCode), ctx.json(undefined)); + () => { + return new HttpResponse(undefined, { + status: statusCode, + }); }, ); export const createSubscriptionHandler = (topic: string) => - rest.post(`${url}/topics/${topic}/subscriptions`, (req, res, ctx) => { - return res(ctx.status(200), ctx.json(undefined)); + http.post(`${url}/topics/${topic}/subscriptions`, () => { + return HttpResponse.json(undefined); }); export const createSubscriptionErrorHandler = ( topic: string, errorCode: number, ) => - rest.post(`${url}/topics/${topic}/subscriptions`, (req, res, ctx) => { - return res(ctx.status(errorCode), ctx.json(undefined)); + http.post(`${url}/topics/${topic}/subscriptions`, () => { + return new HttpResponse(undefined, { + status: errorCode, + }); }); export const createTopicHandler = () => - rest.post(`${url}/topics`, (req, res, ctx) => { - return res(ctx.status(200), ctx.json(undefined)); + http.post(`${url}/topics`, () => { + return HttpResponse.json(undefined); }); export const createTopicErrorHandler = (errorCode: number) => - rest.post(`${url}/topics`, (req, res, ctx) => { - return res(ctx.status(errorCode), ctx.json(undefined)); + http.post(`${url}/topics`, () => { + return new HttpResponse(undefined, { + status: errorCode, + }); }); export const editSubscriptionHandler = (topic: string, subscription: string) => - rest.put( - `${url}/topics/${topic}/subscriptions/${subscription}`, - (req, res, ctx) => { - return res(ctx.status(200), ctx.json(undefined)); - }, - ); + http.put(`${url}/topics/${topic}/subscriptions/${subscription}`, () => { + return HttpResponse.json(undefined); + }); export const editSubscriptionErrorHandler = ( topic: string, subscription: string, errorCode: number, ) => - rest.put( - `${url}/topics/${topic}/subscriptions/${subscription}`, - (req, res, ctx) => { - return res(ctx.status(errorCode), ctx.json(undefined)); - }, - ); + http.put(`${url}/topics/${topic}/subscriptions/${subscription}`, () => { + return new HttpResponse(undefined, { + status: errorCode, + }); + }); export const editTopicHandler = (topic: string) => - rest.put(`${url}/topics/${topic}`, (req, res, ctx) => { - return res(ctx.status(200), ctx.json(undefined)); + http.put(`${url}/topics/${topic}`, () => { + return HttpResponse.json(undefined); }); export const editTopicErrorHandler = (topic: string, errorCode: number) => - rest.put(`${url}/topics/${topic}`, (req, res, ctx) => { - return res(ctx.status(errorCode), ctx.json(undefined)); + http.put(`${url}/topics/${topic}`, () => { + return new HttpResponse(undefined, { + status: errorCode, + }); }); export const createGroupHandler = ({ group }: { group: Group }) => - rest.post(`${url}/groups`, (req, res, ctx) => { - return res(ctx.status(200), ctx.json(group)); + http.post(`${url}/groups`, () => { + return HttpResponse.json(group); }); export const createGroupErrorHandler = ({ @@ -868,8 +919,10 @@ export const createGroupErrorHandler = ({ }: { errorCode?: number; }) => - rest.post(`${url}/groups`, (req, res, ctx) => { - return res(ctx.status(errorCode), ctx.json(undefined)); + http.post(`${url}/groups`, () => { + return new HttpResponse(undefined, { + status: errorCode, + }); }); export const createRetransmissionTaskHandler = ({ @@ -877,8 +930,10 @@ export const createRetransmissionTaskHandler = ({ }: { statusCode: number; }) => - rest.post(`${url}/offline-retransmission/tasks`, (req, res, ctx) => { - return res(ctx.status(statusCode), ctx.json(undefined)); + http.post(`${url}/offline-retransmission/tasks`, () => { + return new HttpResponse(undefined, { + status: statusCode, + }); }); export const createRetransmissionHandler = ({ @@ -890,10 +945,12 @@ export const createRetransmissionHandler = ({ topicName: string; subscriptionName: string; }) => - rest.put( + http.put( `${url}/topics/${topicName}/subscriptions/${subscriptionName}/retransmission`, - (req, res, ctx) => { - return res(ctx.status(statusCode), ctx.json(undefined)); + () => { + return new HttpResponse(undefined, { + status: statusCode, + }); }, ); @@ -904,8 +961,8 @@ export const subscriptionFilterVerificationHandler = ({ topicName: string; response: MessageFiltersVerificationResponse; }) => - rest.post(`${url}/filters/${topicName}`, (req, res, ctx) => { - return res(ctx.status(200), ctx.json(response)); + http.post(`${url}/filters/${topicName}`, () => { + return HttpResponse.json(response); }); export const subscriptionFilterVerificationErrorHandler = ({ @@ -913,6 +970,8 @@ export const subscriptionFilterVerificationErrorHandler = ({ }: { topicName: string; }) => - rest.post(`${url}/filters/${topicName}`, (req, res, ctx) => { - return res(ctx.status(500), ctx.json(undefined)); + http.post(`${url}/filters/${topicName}`, () => { + return new HttpResponse(undefined, { + status: 500, + }); }); diff --git a/hermes-console/src/store/auth/useAuthStore.ts b/hermes-console/src/store/auth/useAuthStore.ts index eed5c9af22..c49f2f1cec 100644 --- a/hermes-console/src/store/auth/useAuthStore.ts +++ b/hermes-console/src/store/auth/useAuthStore.ts @@ -1,9 +1,9 @@ import { encode as base64encode } from 'base64-arraybuffer'; import { defineStore } from 'pinia'; import { fetchToken } from '@/api/hermes-client'; +import { jwtDecode } from 'jwt-decode'; import { useAppConfigStore } from '@/store/app-config/useAppConfigStore'; import axios from '@/utils/axios/axios-instance'; -import decode from 'jwt-decode'; import qs from 'query-string'; import type { AuthStoreState } from '@/store/auth/types'; @@ -79,7 +79,7 @@ export const useAuthStore = defineStore('auth', { }, getters: { userData(state: AuthStoreState): { exp: number } { - return state.accessToken ? decode(state.accessToken) : { exp: 0 }; + return state.accessToken ? jwtDecode(state.accessToken) : { exp: 0 }; }, isUserAuthorized(state: AuthStoreState): boolean { const expiresAt = this.userData.exp * 1000; diff --git a/hermes-console/src/views/admin/constraints/ConstraintsView.vue b/hermes-console/src/views/admin/constraints/ConstraintsView.vue index e662e63eb7..a0da409f01 100644 --- a/hermes-console/src/views/admin/constraints/ConstraintsView.vue +++ b/hermes-console/src/views/admin/constraints/ConstraintsView.vue @@ -67,9 +67,8 @@ }; const onSubscriptionConstraintDeleted = async (subscriptionFqn: string) => { - const constraintChanged = await deleteSubscriptionConstraint( - subscriptionFqn, - ); + const constraintChanged = + await deleteSubscriptionConstraint(subscriptionFqn); refreshOnMutation(constraintChanged); }; diff --git a/hermes-console/src/views/group-topics/GroupTopicsView.vue b/hermes-console/src/views/group-topics/GroupTopicsView.vue index 62cdca499e..9772449af5 100644 --- a/hermes-console/src/views/group-topics/GroupTopicsView.vue +++ b/hermes-console/src/views/group-topics/GroupTopicsView.vue @@ -102,7 +102,7 @@ - +

{{ $t('groupTopics.title') }}

{{ groupId }} diff --git a/hermes-console/src/views/groups/group-form/GroupForm.spec.ts b/hermes-console/src/views/groups/group-form/GroupForm.spec.ts index 34bf7e19db..a21fd4768b 100644 --- a/hermes-console/src/views/groups/group-form/GroupForm.spec.ts +++ b/hermes-console/src/views/groups/group-form/GroupForm.spec.ts @@ -79,8 +79,6 @@ describe('GroupForm', () => { ); // then - expect( - getByText('groups.groupForm.save').closest('button'), - ).not.toBeEnabled(); + expect(getByText('groups.groupForm.save').closest('button')).toBeEnabled(); }); }); diff --git a/hermes-console/src/views/subscription/subscription-form/SubscriptionForm.vue b/hermes-console/src/views/subscription/subscription-form/SubscriptionForm.vue index 7792a1effb..b13a5f342c 100644 --- a/hermes-console/src/views/subscription/subscription-form/SubscriptionForm.vue +++ b/hermes-console/src/views/subscription/subscription-form/SubscriptionForm.vue @@ -286,6 +286,18 @@ " /> + +

diff --git a/hermes-console/vite.config.ts b/hermes-console/vite.config.mts similarity index 100% rename from hermes-console/vite.config.ts rename to hermes-console/vite.config.mts diff --git a/hermes-console/yarn.lock b/hermes-console/yarn.lock index 44ebc52eba..fdec34fe09 100644 --- a/hermes-console/yarn.lock +++ b/hermes-console/yarn.lock @@ -2,12 +2,17 @@ # yarn lockfile v1 -"@adobe/css-tools@^4.0.1": - version "4.1.0" - resolved "https://registry.yarnpkg.com/@adobe/css-tools/-/css-tools-4.1.0.tgz#417fef4a143f4396ad0b3b4351fee21323f15aa8" - integrity sha512-mMVJ/j/GbZ/De4ZHWbQAQO1J6iVnjtZLc9WEdkUQb8S/Bu2cAF2bETXUgMAdvMG3/ngtKmcNBe+Zms9bg6jnQQ== +"@aashutoshrathi/word-wrap@^1.2.3": + version "1.2.6" + resolved "https://registry.yarnpkg.com/@aashutoshrathi/word-wrap/-/word-wrap-1.2.6.tgz#bd9154aec9983f77b3a034ecaa015c2e4201f6cf" + integrity sha512-1Yjs2SvM8TflER/OD3cOjhWWOZb58A2t7wpE2S9XfBYTiIl+XFhQG2bjy4Pu1I+EAlCNUzRDYDdFwFYUKvXcIA== + +"@adobe/css-tools@^4.3.2": + version "4.3.3" + resolved "https://registry.yarnpkg.com/@adobe/css-tools/-/css-tools-4.3.3.tgz#90749bde8b89cd41764224f5aac29cd4138f75ff" + integrity sha512-rE0Pygv0sEZ4vBWHlAgJLGDU7Pm8xoO6p3wsEceb7GYAjScrOHpEo8KK/eVkAcnSM+slAEtXjA2JpdjLp4fJQQ== -"@babel/code-frame@^7.10.4", "@babel/code-frame@^7.12.13": +"@babel/code-frame@^7.10.4": version "7.18.6" resolved "https://registry.yarnpkg.com/@babel/code-frame/-/code-frame-7.18.6.tgz#3b25d38c89600baa2dcc219edfa88a74eb2c427a" integrity sha512-TDCmlK5eOvH+eH7cdAFlNXeVJqWIQ7gW9tY1GJIpUtFb6CmjVyq2VM3u71bOyR8CRihcCgMUYoDNyLXao3+70Q== @@ -28,15 +33,10 @@ chalk "^2.0.0" js-tokens "^4.0.0" -"@babel/parser@^7.16.4": - version "7.20.15" - resolved "https://registry.yarnpkg.com/@babel/parser/-/parser-7.20.15.tgz#eec9f36d8eaf0948bb88c87a46784b5ee9fd0c89" - integrity sha512-DI4a1oZuf8wC+oAJA9RW6ga3Zbe8RZFt7kD9i4qAspz3I/yHet1VvC3DiSy/fsUvv5pvJuNPh0LPOdCcqinDPg== - -"@babel/parser@^7.20.15", "@babel/parser@^7.21.3": - version "7.22.4" - resolved "https://registry.yarnpkg.com/@babel/parser/-/parser-7.22.4.tgz#a770e98fd785c231af9d93f6459d36770993fb32" - integrity sha512-VLLsx06XkEYqBtE5YGPwfSGwfrjnyPP5oiGty3S8pQLFDFLaS8VwWSIxkTXpcvr5zeYLE6+MBNl2npl/YnfofA== +"@babel/parser@^7.24.1": + version "7.24.4" + resolved "https://registry.yarnpkg.com/@babel/parser/-/parser-7.24.4.tgz#234487a110d89ad5a3ed4a8a566c36b9453e8c88" + integrity sha512-zTvEBcghmeBma9QIGunWevvBAp4/Qu9Bdq+2k0Ot4fVMD6v3dsC9WOcRSKk7tRRyBM/53yKMJko9xOatGQAwSg== "@babel/runtime@^7.12.5", "@babel/runtime@^7.9.2": version "7.20.13" @@ -45,131 +45,162 @@ dependencies: regenerator-runtime "^0.13.11" -"@babel/runtime@^7.21.0": - version "7.22.3" - resolved "https://registry.yarnpkg.com/@babel/runtime/-/runtime-7.22.3.tgz#0a7fce51d43adbf0f7b517a71f4c3aaca92ebcbb" - integrity sha512-XsDuspWKLUsxwCp6r7EhsExHtYfbe5oAGQ19kqngTdCPUoPQzOPdUbD/pB9PJiwb2ptYKQDjSJT3R6dC+EPqfQ== +"@babel/runtime@^7.23.2": + version "7.24.4" + resolved "https://registry.yarnpkg.com/@babel/runtime/-/runtime-7.24.4.tgz#de795accd698007a66ba44add6cc86542aff1edd" + integrity sha512-dkxf7+hn8mFBwKjs9bvBlArzLVxVbS8usaPUDd5p2a9JCL9tB8OaOVN1isD4+Xyk4ns89/xeOmbQvgdK7IIVdA== dependencies: - regenerator-runtime "^0.13.11" + regenerator-runtime "^0.14.0" -"@esbuild/android-arm64@0.16.17": - version "0.16.17" - resolved "https://registry.yarnpkg.com/@esbuild/android-arm64/-/android-arm64-0.16.17.tgz#cf91e86df127aa3d141744edafcba0abdc577d23" - integrity sha512-MIGl6p5sc3RDTLLkYL1MyL8BMRN4tLMRCn+yRJJmEDvYZ2M7tmAf80hx1kbNEUX2KJ50RRtxZ4JHLvCfuB6kBg== - -"@esbuild/android-arm@0.16.17": - version "0.16.17" - resolved "https://registry.yarnpkg.com/@esbuild/android-arm/-/android-arm-0.16.17.tgz#025b6246d3f68b7bbaa97069144fb5fb70f2fff2" - integrity sha512-N9x1CMXVhtWEAMS7pNNONyA14f71VPQN9Cnavj1XQh6T7bskqiLLrSca4O0Vr8Wdcga943eThxnVp3JLnBMYtw== - -"@esbuild/android-x64@0.16.17": - version "0.16.17" - resolved "https://registry.yarnpkg.com/@esbuild/android-x64/-/android-x64-0.16.17.tgz#c820e0fef982f99a85c4b8bfdd582835f04cd96e" - integrity sha512-a3kTv3m0Ghh4z1DaFEuEDfz3OLONKuFvI4Xqczqx4BqLyuFaFkuaG4j2MtA6fuWEFeC5x9IvqnX7drmRq/fyAQ== - -"@esbuild/darwin-arm64@0.16.17": - version "0.16.17" - resolved "https://registry.yarnpkg.com/@esbuild/darwin-arm64/-/darwin-arm64-0.16.17.tgz#edef4487af6b21afabba7be5132c26d22379b220" - integrity sha512-/2agbUEfmxWHi9ARTX6OQ/KgXnOWfsNlTeLcoV7HSuSTv63E4DqtAc+2XqGw1KHxKMHGZgbVCZge7HXWX9Vn+w== - -"@esbuild/darwin-x64@0.16.17": - version "0.16.17" - resolved "https://registry.yarnpkg.com/@esbuild/darwin-x64/-/darwin-x64-0.16.17.tgz#42829168730071c41ef0d028d8319eea0e2904b4" - integrity sha512-2By45OBHulkd9Svy5IOCZt376Aa2oOkiE9QWUK9fe6Tb+WDr8hXL3dpqi+DeLiMed8tVXspzsTAvd0jUl96wmg== - -"@esbuild/freebsd-arm64@0.16.17": - version "0.16.17" - resolved "https://registry.yarnpkg.com/@esbuild/freebsd-arm64/-/freebsd-arm64-0.16.17.tgz#1f4af488bfc7e9ced04207034d398e793b570a27" - integrity sha512-mt+cxZe1tVx489VTb4mBAOo2aKSnJ33L9fr25JXpqQqzbUIw/yzIzi+NHwAXK2qYV1lEFp4OoVeThGjUbmWmdw== - -"@esbuild/freebsd-x64@0.16.17": - version "0.16.17" - resolved "https://registry.yarnpkg.com/@esbuild/freebsd-x64/-/freebsd-x64-0.16.17.tgz#636306f19e9bc981e06aa1d777302dad8fddaf72" - integrity sha512-8ScTdNJl5idAKjH8zGAsN7RuWcyHG3BAvMNpKOBaqqR7EbUhhVHOqXRdL7oZvz8WNHL2pr5+eIT5c65kA6NHug== - -"@esbuild/linux-arm64@0.16.17": - version "0.16.17" - resolved "https://registry.yarnpkg.com/@esbuild/linux-arm64/-/linux-arm64-0.16.17.tgz#a003f7ff237c501e095d4f3a09e58fc7b25a4aca" - integrity sha512-7S8gJnSlqKGVJunnMCrXHU9Q8Q/tQIxk/xL8BqAP64wchPCTzuM6W3Ra8cIa1HIflAvDnNOt2jaL17vaW+1V0g== - -"@esbuild/linux-arm@0.16.17": - version "0.16.17" - resolved "https://registry.yarnpkg.com/@esbuild/linux-arm/-/linux-arm-0.16.17.tgz#b591e6a59d9c4fe0eeadd4874b157ab78cf5f196" - integrity sha512-iihzrWbD4gIT7j3caMzKb/RsFFHCwqqbrbH9SqUSRrdXkXaygSZCZg1FybsZz57Ju7N/SHEgPyaR0LZ8Zbe9gQ== - -"@esbuild/linux-ia32@0.16.17": - version "0.16.17" - resolved "https://registry.yarnpkg.com/@esbuild/linux-ia32/-/linux-ia32-0.16.17.tgz#24333a11027ef46a18f57019450a5188918e2a54" - integrity sha512-kiX69+wcPAdgl3Lonh1VI7MBr16nktEvOfViszBSxygRQqSpzv7BffMKRPMFwzeJGPxcio0pdD3kYQGpqQ2SSg== - -"@esbuild/linux-loong64@0.16.17": - version "0.16.17" - resolved "https://registry.yarnpkg.com/@esbuild/linux-loong64/-/linux-loong64-0.16.17.tgz#d5ad459d41ed42bbd4d005256b31882ec52227d8" - integrity sha512-dTzNnQwembNDhd654cA4QhbS9uDdXC3TKqMJjgOWsC0yNCbpzfWoXdZvp0mY7HU6nzk5E0zpRGGx3qoQg8T2DQ== - -"@esbuild/linux-mips64el@0.16.17": - version "0.16.17" - resolved "https://registry.yarnpkg.com/@esbuild/linux-mips64el/-/linux-mips64el-0.16.17.tgz#4e5967a665c38360b0a8205594377d4dcf9c3726" - integrity sha512-ezbDkp2nDl0PfIUn0CsQ30kxfcLTlcx4Foz2kYv8qdC6ia2oX5Q3E/8m6lq84Dj/6b0FrkgD582fJMIfHhJfSw== - -"@esbuild/linux-ppc64@0.16.17": - version "0.16.17" - resolved "https://registry.yarnpkg.com/@esbuild/linux-ppc64/-/linux-ppc64-0.16.17.tgz#206443a02eb568f9fdf0b438fbd47d26e735afc8" - integrity sha512-dzS678gYD1lJsW73zrFhDApLVdM3cUF2MvAa1D8K8KtcSKdLBPP4zZSLy6LFZ0jYqQdQ29bjAHJDgz0rVbLB3g== - -"@esbuild/linux-riscv64@0.16.17": - version "0.16.17" - resolved "https://registry.yarnpkg.com/@esbuild/linux-riscv64/-/linux-riscv64-0.16.17.tgz#c351e433d009bf256e798ad048152c8d76da2fc9" - integrity sha512-ylNlVsxuFjZK8DQtNUwiMskh6nT0vI7kYl/4fZgV1llP5d6+HIeL/vmmm3jpuoo8+NuXjQVZxmKuhDApK0/cKw== - -"@esbuild/linux-s390x@0.16.17": - version "0.16.17" - resolved "https://registry.yarnpkg.com/@esbuild/linux-s390x/-/linux-s390x-0.16.17.tgz#661f271e5d59615b84b6801d1c2123ad13d9bd87" - integrity sha512-gzy7nUTO4UA4oZ2wAMXPNBGTzZFP7mss3aKR2hH+/4UUkCOyqmjXiKpzGrY2TlEUhbbejzXVKKGazYcQTZWA/w== - -"@esbuild/linux-x64@0.16.17": - version "0.16.17" - resolved "https://registry.yarnpkg.com/@esbuild/linux-x64/-/linux-x64-0.16.17.tgz#e4ba18e8b149a89c982351443a377c723762b85f" - integrity sha512-mdPjPxfnmoqhgpiEArqi4egmBAMYvaObgn4poorpUaqmvzzbvqbowRllQ+ZgzGVMGKaPkqUmPDOOFQRUFDmeUw== - -"@esbuild/netbsd-x64@0.16.17": - version "0.16.17" - resolved "https://registry.yarnpkg.com/@esbuild/netbsd-x64/-/netbsd-x64-0.16.17.tgz#7d4f4041e30c5c07dd24ffa295c73f06038ec775" - integrity sha512-/PzmzD/zyAeTUsduZa32bn0ORug+Jd1EGGAUJvqfeixoEISYpGnAezN6lnJoskauoai0Jrs+XSyvDhppCPoKOA== - -"@esbuild/openbsd-x64@0.16.17": - version "0.16.17" - resolved "https://registry.yarnpkg.com/@esbuild/openbsd-x64/-/openbsd-x64-0.16.17.tgz#970fa7f8470681f3e6b1db0cc421a4af8060ec35" - integrity sha512-2yaWJhvxGEz2RiftSk0UObqJa/b+rIAjnODJgv2GbGGpRwAfpgzyrg1WLK8rqA24mfZa9GvpjLcBBg8JHkoodg== - -"@esbuild/sunos-x64@0.16.17": - version "0.16.17" - resolved "https://registry.yarnpkg.com/@esbuild/sunos-x64/-/sunos-x64-0.16.17.tgz#abc60e7c4abf8b89fb7a4fe69a1484132238022c" - integrity sha512-xtVUiev38tN0R3g8VhRfN7Zl42YCJvyBhRKw1RJjwE1d2emWTVToPLNEQj/5Qxc6lVFATDiy6LjVHYhIPrLxzw== - -"@esbuild/win32-arm64@0.16.17": - version "0.16.17" - resolved "https://registry.yarnpkg.com/@esbuild/win32-arm64/-/win32-arm64-0.16.17.tgz#7b0ff9e8c3265537a7a7b1fd9a24e7bd39fcd87a" - integrity sha512-ga8+JqBDHY4b6fQAmOgtJJue36scANy4l/rL97W+0wYmijhxKetzZdKOJI7olaBaMhWt8Pac2McJdZLxXWUEQw== - -"@esbuild/win32-ia32@0.16.17": - version "0.16.17" - resolved "https://registry.yarnpkg.com/@esbuild/win32-ia32/-/win32-ia32-0.16.17.tgz#e90fe5267d71a7b7567afdc403dfd198c292eb09" - integrity sha512-WnsKaf46uSSF/sZhwnqE4L/F89AYNMiD4YtEcYekBt9Q7nj0DiId2XH2Ng2PHM54qi5oPrQ8luuzGszqi/veig== - -"@esbuild/win32-x64@0.16.17": - version "0.16.17" - resolved "https://registry.yarnpkg.com/@esbuild/win32-x64/-/win32-x64-0.16.17.tgz#c5a1a4bfe1b57f0c3e61b29883525c6da3e5c091" - integrity sha512-y+EHuSchhL7FjHgvQL/0fnnFmO4T1bhvWANX6gcnqTjtnKWbTvUMCpGnv2+t+31d7RzyEAYAd4u2fnIhHL6N/Q== - -"@eslint/eslintrc@^1.4.1": - version "1.4.1" - resolved "https://registry.yarnpkg.com/@eslint/eslintrc/-/eslintrc-1.4.1.tgz#af58772019a2d271b7e2d4c23ff4ddcba3ccfb3e" - integrity sha512-XXrH9Uarn0stsyldqDYq8r++mROmWRI1xKMXa640Bb//SY1+ECYX6VzT6Lcx5frD0V30XieqJ0oX9I2Xj5aoMA== +"@bundled-es-modules/cookie@^2.0.0": + version "2.0.0" + resolved "https://registry.yarnpkg.com/@bundled-es-modules/cookie/-/cookie-2.0.0.tgz#c3b82703969a61cf6a46e959a012b2c257f6b164" + integrity sha512-Or6YHg/kamKHpxULAdSqhGqnWFneIXu1NKvvfBBzKGwpVsYuFIQ5aBPHDnnoR3ghW1nvSkALd+EF9iMtY7Vjxw== + dependencies: + cookie "^0.5.0" + +"@bundled-es-modules/statuses@^1.0.1": + version "1.0.1" + resolved "https://registry.yarnpkg.com/@bundled-es-modules/statuses/-/statuses-1.0.1.tgz#761d10f44e51a94902c4da48675b71a76cc98872" + integrity sha512-yn7BklA5acgcBr+7w064fGV+SGIFySjCKpqjcWgBAIfrAkY+4GQTJJHQMeT3V/sgz23VTEVV8TtOmkvJAhFVfg== + dependencies: + statuses "^2.0.1" + +"@esbuild/aix-ppc64@0.20.2": + version "0.20.2" + resolved "https://registry.yarnpkg.com/@esbuild/aix-ppc64/-/aix-ppc64-0.20.2.tgz#a70f4ac11c6a1dfc18b8bbb13284155d933b9537" + integrity sha512-D+EBOJHXdNZcLJRBkhENNG8Wji2kgc9AZ9KiPr1JuZjsNtyHzrsfLRrY0tk2H2aoFu6RANO1y1iPPUCDYWkb5g== + +"@esbuild/android-arm64@0.20.2": + version "0.20.2" + resolved "https://registry.yarnpkg.com/@esbuild/android-arm64/-/android-arm64-0.20.2.tgz#db1c9202a5bc92ea04c7b6840f1bbe09ebf9e6b9" + integrity sha512-mRzjLacRtl/tWU0SvD8lUEwb61yP9cqQo6noDZP/O8VkwafSYwZ4yWy24kan8jE/IMERpYncRt2dw438LP3Xmg== + +"@esbuild/android-arm@0.20.2": + version "0.20.2" + resolved "https://registry.yarnpkg.com/@esbuild/android-arm/-/android-arm-0.20.2.tgz#3b488c49aee9d491c2c8f98a909b785870d6e995" + integrity sha512-t98Ra6pw2VaDhqNWO2Oph2LXbz/EJcnLmKLGBJwEwXX/JAN83Fym1rU8l0JUWK6HkIbWONCSSatf4sf2NBRx/w== + +"@esbuild/android-x64@0.20.2": + version "0.20.2" + resolved "https://registry.yarnpkg.com/@esbuild/android-x64/-/android-x64-0.20.2.tgz#3b1628029e5576249d2b2d766696e50768449f98" + integrity sha512-btzExgV+/lMGDDa194CcUQm53ncxzeBrWJcncOBxuC6ndBkKxnHdFJn86mCIgTELsooUmwUm9FkhSp5HYu00Rg== + +"@esbuild/darwin-arm64@0.20.2": + version "0.20.2" + resolved "https://registry.yarnpkg.com/@esbuild/darwin-arm64/-/darwin-arm64-0.20.2.tgz#6e8517a045ddd86ae30c6608c8475ebc0c4000bb" + integrity sha512-4J6IRT+10J3aJH3l1yzEg9y3wkTDgDk7TSDFX+wKFiWjqWp/iCfLIYzGyasx9l0SAFPT1HwSCR+0w/h1ES/MjA== + +"@esbuild/darwin-x64@0.20.2": + version "0.20.2" + resolved "https://registry.yarnpkg.com/@esbuild/darwin-x64/-/darwin-x64-0.20.2.tgz#90ed098e1f9dd8a9381695b207e1cff45540a0d0" + integrity sha512-tBcXp9KNphnNH0dfhv8KYkZhjc+H3XBkF5DKtswJblV7KlT9EI2+jeA8DgBjp908WEuYll6pF+UStUCfEpdysA== + +"@esbuild/freebsd-arm64@0.20.2": + version "0.20.2" + resolved "https://registry.yarnpkg.com/@esbuild/freebsd-arm64/-/freebsd-arm64-0.20.2.tgz#d71502d1ee89a1130327e890364666c760a2a911" + integrity sha512-d3qI41G4SuLiCGCFGUrKsSeTXyWG6yem1KcGZVS+3FYlYhtNoNgYrWcvkOoaqMhwXSMrZRl69ArHsGJ9mYdbbw== + +"@esbuild/freebsd-x64@0.20.2": + version "0.20.2" + resolved "https://registry.yarnpkg.com/@esbuild/freebsd-x64/-/freebsd-x64-0.20.2.tgz#aa5ea58d9c1dd9af688b8b6f63ef0d3d60cea53c" + integrity sha512-d+DipyvHRuqEeM5zDivKV1KuXn9WeRX6vqSqIDgwIfPQtwMP4jaDsQsDncjTDDsExT4lR/91OLjRo8bmC1e+Cw== + +"@esbuild/linux-arm64@0.20.2": + version "0.20.2" + resolved "https://registry.yarnpkg.com/@esbuild/linux-arm64/-/linux-arm64-0.20.2.tgz#055b63725df678379b0f6db9d0fa85463755b2e5" + integrity sha512-9pb6rBjGvTFNira2FLIWqDk/uaf42sSyLE8j1rnUpuzsODBq7FvpwHYZxQ/It/8b+QOS1RYfqgGFNLRI+qlq2A== + +"@esbuild/linux-arm@0.20.2": + version "0.20.2" + resolved "https://registry.yarnpkg.com/@esbuild/linux-arm/-/linux-arm-0.20.2.tgz#76b3b98cb1f87936fbc37f073efabad49dcd889c" + integrity sha512-VhLPeR8HTMPccbuWWcEUD1Az68TqaTYyj6nfE4QByZIQEQVWBB8vup8PpR7y1QHL3CpcF6xd5WVBU/+SBEvGTg== + +"@esbuild/linux-ia32@0.20.2": + version "0.20.2" + resolved "https://registry.yarnpkg.com/@esbuild/linux-ia32/-/linux-ia32-0.20.2.tgz#c0e5e787c285264e5dfc7a79f04b8b4eefdad7fa" + integrity sha512-o10utieEkNPFDZFQm9CoP7Tvb33UutoJqg3qKf1PWVeeJhJw0Q347PxMvBgVVFgouYLGIhFYG0UGdBumROyiig== + +"@esbuild/linux-loong64@0.20.2": + version "0.20.2" + resolved "https://registry.yarnpkg.com/@esbuild/linux-loong64/-/linux-loong64-0.20.2.tgz#a6184e62bd7cdc63e0c0448b83801001653219c5" + integrity sha512-PR7sp6R/UC4CFVomVINKJ80pMFlfDfMQMYynX7t1tNTeivQ6XdX5r2XovMmha/VjR1YN/HgHWsVcTRIMkymrgQ== + +"@esbuild/linux-mips64el@0.20.2": + version "0.20.2" + resolved "https://registry.yarnpkg.com/@esbuild/linux-mips64el/-/linux-mips64el-0.20.2.tgz#d08e39ce86f45ef8fc88549d29c62b8acf5649aa" + integrity sha512-4BlTqeutE/KnOiTG5Y6Sb/Hw6hsBOZapOVF6njAESHInhlQAghVVZL1ZpIctBOoTFbQyGW+LsVYZ8lSSB3wkjA== + +"@esbuild/linux-ppc64@0.20.2": + version "0.20.2" + resolved "https://registry.yarnpkg.com/@esbuild/linux-ppc64/-/linux-ppc64-0.20.2.tgz#8d252f0b7756ffd6d1cbde5ea67ff8fd20437f20" + integrity sha512-rD3KsaDprDcfajSKdn25ooz5J5/fWBylaaXkuotBDGnMnDP1Uv5DLAN/45qfnf3JDYyJv/ytGHQaziHUdyzaAg== + +"@esbuild/linux-riscv64@0.20.2": + version "0.20.2" + resolved "https://registry.yarnpkg.com/@esbuild/linux-riscv64/-/linux-riscv64-0.20.2.tgz#19f6dcdb14409dae607f66ca1181dd4e9db81300" + integrity sha512-snwmBKacKmwTMmhLlz/3aH1Q9T8v45bKYGE3j26TsaOVtjIag4wLfWSiZykXzXuE1kbCE+zJRmwp+ZbIHinnVg== + +"@esbuild/linux-s390x@0.20.2": + version "0.20.2" + resolved "https://registry.yarnpkg.com/@esbuild/linux-s390x/-/linux-s390x-0.20.2.tgz#3c830c90f1a5d7dd1473d5595ea4ebb920988685" + integrity sha512-wcWISOobRWNm3cezm5HOZcYz1sKoHLd8VL1dl309DiixxVFoFe/o8HnwuIwn6sXre88Nwj+VwZUvJf4AFxkyrQ== + +"@esbuild/linux-x64@0.20.2": + version "0.20.2" + resolved "https://registry.yarnpkg.com/@esbuild/linux-x64/-/linux-x64-0.20.2.tgz#86eca35203afc0d9de0694c64ec0ab0a378f6fff" + integrity sha512-1MdwI6OOTsfQfek8sLwgyjOXAu+wKhLEoaOLTjbijk6E2WONYpH9ZU2mNtR+lZ2B4uwr+usqGuVfFT9tMtGvGw== + +"@esbuild/netbsd-x64@0.20.2": + version "0.20.2" + resolved "https://registry.yarnpkg.com/@esbuild/netbsd-x64/-/netbsd-x64-0.20.2.tgz#e771c8eb0e0f6e1877ffd4220036b98aed5915e6" + integrity sha512-K8/DhBxcVQkzYc43yJXDSyjlFeHQJBiowJ0uVL6Tor3jGQfSGHNNJcWxNbOI8v5k82prYqzPuwkzHt3J1T1iZQ== + +"@esbuild/openbsd-x64@0.20.2": + version "0.20.2" + resolved "https://registry.yarnpkg.com/@esbuild/openbsd-x64/-/openbsd-x64-0.20.2.tgz#9a795ae4b4e37e674f0f4d716f3e226dd7c39baf" + integrity sha512-eMpKlV0SThJmmJgiVyN9jTPJ2VBPquf6Kt/nAoo6DgHAoN57K15ZghiHaMvqjCye/uU4X5u3YSMgVBI1h3vKrQ== + +"@esbuild/sunos-x64@0.20.2": + version "0.20.2" + resolved "https://registry.yarnpkg.com/@esbuild/sunos-x64/-/sunos-x64-0.20.2.tgz#7df23b61a497b8ac189def6e25a95673caedb03f" + integrity sha512-2UyFtRC6cXLyejf/YEld4Hajo7UHILetzE1vsRcGL3earZEW77JxrFjH4Ez2qaTiEfMgAXxfAZCm1fvM/G/o8w== + +"@esbuild/win32-arm64@0.20.2": + version "0.20.2" + resolved "https://registry.yarnpkg.com/@esbuild/win32-arm64/-/win32-arm64-0.20.2.tgz#f1ae5abf9ca052ae11c1bc806fb4c0f519bacf90" + integrity sha512-GRibxoawM9ZCnDxnP3usoUDO9vUkpAxIIZ6GQI+IlVmr5kP3zUq+l17xELTHMWTWzjxa2guPNyrpq1GWmPvcGQ== + +"@esbuild/win32-ia32@0.20.2": + version "0.20.2" + resolved "https://registry.yarnpkg.com/@esbuild/win32-ia32/-/win32-ia32-0.20.2.tgz#241fe62c34d8e8461cd708277813e1d0ba55ce23" + integrity sha512-HfLOfn9YWmkSKRQqovpnITazdtquEW8/SoHW7pWpuEeguaZI4QnCRW6b+oZTztdBnZOS2hqJ6im/D5cPzBTTlQ== + +"@esbuild/win32-x64@0.20.2": + version "0.20.2" + resolved "https://registry.yarnpkg.com/@esbuild/win32-x64/-/win32-x64-0.20.2.tgz#9c907b21e30a52db959ba4f80bb01a0cc403d5cc" + integrity sha512-N49X4lJX27+l9jbLKSqZ6bKNjzQvHaT8IIFUy+YIqmXQdjYCToGWwOItDrfby14c78aDd5NHQl29xingXfCdLQ== + +"@eslint-community/eslint-utils@^4.2.0", "@eslint-community/eslint-utils@^4.4.0": + version "4.4.0" + resolved "https://registry.yarnpkg.com/@eslint-community/eslint-utils/-/eslint-utils-4.4.0.tgz#a23514e8fb9af1269d5f7788aa556798d61c6b59" + integrity sha512-1/sA4dwrzBAyeUoQ6oxahHKmrZvsnLCg4RfxW3ZFGGmQkSNQPFNLV9CUEFQP1x9EYXHTo5p6xdhZM1Ne9p/AfA== + dependencies: + eslint-visitor-keys "^3.3.0" + +"@eslint-community/regexpp@^4.10.0", "@eslint-community/regexpp@^4.6.1": + version "4.10.0" + resolved "https://registry.yarnpkg.com/@eslint-community/regexpp/-/regexpp-4.10.0.tgz#548f6de556857c8bb73bbee70c35dc82a2e74d63" + integrity sha512-Cu96Sd2By9mCNTx2iyKOmq10v22jUVQv0lQnlGNy16oE9589yE+QADPbrMGCkA51cKZSg3Pu/aTJVTGfL/qjUA== + +"@eslint/eslintrc@^2.1.4": + version "2.1.4" + resolved "https://registry.yarnpkg.com/@eslint/eslintrc/-/eslintrc-2.1.4.tgz#388a269f0f25c1b6adc317b5a2c55714894c70ad" + integrity sha512-269Z39MS6wVJtsoUl10L60WdkhJVdPG24Q4eZTH3nnF6lpvSShEK3wQjDX9JRWAUPvPh7COouPpU9IrqaZFvtQ== dependencies: ajv "^6.12.4" debug "^4.3.2" - espree "^9.4.0" + espree "^9.6.0" globals "^13.19.0" ignore "^5.2.0" import-fresh "^3.2.1" @@ -177,13 +208,18 @@ minimatch "^3.1.2" strip-json-comments "^3.1.1" -"@humanwhocodes/config-array@^0.11.8": - version "0.11.8" - resolved "https://registry.yarnpkg.com/@humanwhocodes/config-array/-/config-array-0.11.8.tgz#03595ac2075a4dc0f191cc2131de14fbd7d410b9" - integrity sha512-UybHIJzJnR5Qc/MsD9Kr+RpO2h+/P1GhOwdiLPXK5TWk5sgTdu88bTD9UP+CKbPPh5Rni1u0GjAdYQLemG8g+g== +"@eslint/js@8.57.0": + version "8.57.0" + resolved "https://registry.yarnpkg.com/@eslint/js/-/js-8.57.0.tgz#a5417ae8427873f1dd08b70b3574b453e67b5f7f" + integrity sha512-Ys+3g2TaW7gADOJzPt83SJtCDhMjndcDMFVQ/Tj9iA1BfJzFKD9mAUXT3OenpuPHbI6P/myECxRJrofUsDx/5g== + +"@humanwhocodes/config-array@^0.11.14": + version "0.11.14" + resolved "https://registry.yarnpkg.com/@humanwhocodes/config-array/-/config-array-0.11.14.tgz#d78e481a039f7566ecc9660b4ea7fe6b1fec442b" + integrity sha512-3T8LkOmg45BV5FICb15QQMsyUSWrQ8AygVfC7ZG32zOalnqrilm018ZVCw0eapXux8FtA33q8PSRSstjee3jSg== dependencies: - "@humanwhocodes/object-schema" "^1.2.1" - debug "^4.1.1" + "@humanwhocodes/object-schema" "^2.0.2" + debug "^4.3.1" minimatch "^3.0.5" "@humanwhocodes/module-importer@^1.0.1": @@ -191,106 +227,114 @@ resolved "https://registry.yarnpkg.com/@humanwhocodes/module-importer/-/module-importer-1.0.1.tgz#af5b2691a22b44be847b0ca81641c5fb6ad0172c" integrity sha512-bxveV4V8v5Yb4ncFTT3rPSgZBOpCkjfK0y4oVVVJwIuDVBRMDXrPyXRL988i5ap9m9bnyEEjWfm5WkBmtffLfA== -"@humanwhocodes/object-schema@^1.2.1": - version "1.2.1" - resolved "https://registry.yarnpkg.com/@humanwhocodes/object-schema/-/object-schema-1.2.1.tgz#b520529ec21d8e5945a1851dfd1c32e94e39ff45" - integrity sha512-ZnQMnLV4e7hDlUvw8H+U8ASL02SS2Gn6+9Ac3wGGLIe7+je2AeAOxPY+izIPJDfFDb7eDjev0Us8MO1iFRN8hA== +"@humanwhocodes/object-schema@^2.0.2": + version "2.0.3" + resolved "https://registry.yarnpkg.com/@humanwhocodes/object-schema/-/object-schema-2.0.3.tgz#4a2868d75d6d6963e423bcf90b7fd1be343409d3" + integrity sha512-93zYdMES/c1D69yZiKDBj0V24vqNzB/koF26KPaagAfd3P/4gUlh3Dys5ogAK+Exi9QyzlD8x/08Zt7wIKcDcA== -"@intlify/core-base@9.2.2": - version "9.2.2" - resolved "https://registry.yarnpkg.com/@intlify/core-base/-/core-base-9.2.2.tgz#5353369b05cc9fe35cab95fe20afeb8a4481f939" - integrity sha512-JjUpQtNfn+joMbrXvpR4hTF8iJQ2sEFzzK3KIESOx+f+uwIjgw20igOyaIdhfsVVBCds8ZM64MoeNSx+PHQMkA== +"@inquirer/confirm@^3.0.0": + version "3.1.5" + resolved "https://registry.yarnpkg.com/@inquirer/confirm/-/confirm-3.1.5.tgz#21856f937bc8292eca4146c052271107f8ac949a" + integrity sha512-6+dwZrpko5vr5EFEQmUbfBVhtu6IsnB8lQNsLHgO9S9fbfS5J6MuUj+NY0h98pPpYZXEazLR7qzypEDqVzf6aQ== dependencies: - "@intlify/devtools-if" "9.2.2" - "@intlify/message-compiler" "9.2.2" - "@intlify/shared" "9.2.2" - "@intlify/vue-devtools" "9.2.2" + "@inquirer/core" "^8.0.1" + "@inquirer/type" "^1.3.0" -"@intlify/devtools-if@9.2.2": - version "9.2.2" - resolved "https://registry.yarnpkg.com/@intlify/devtools-if/-/devtools-if-9.2.2.tgz#b13d9ac4b4e2fe6d2e7daa556517a8061fe8bd39" - integrity sha512-4ttr/FNO29w+kBbU7HZ/U0Lzuh2cRDhP8UlWOtV9ERcjHzuyXVZmjyleESK6eVP60tGC9QtQW9yZE+JeRhDHkg== - dependencies: - "@intlify/shared" "9.2.2" +"@inquirer/core@^8.0.1": + version "8.0.1" + resolved "https://registry.yarnpkg.com/@inquirer/core/-/core-8.0.1.tgz#ac3d9a34a6826dc193791b2feec19061a9c250ca" + integrity sha512-qJRk1y51Os2ARc11Bg2N6uIwiQ9qBSrmZeuMonaQ/ntFpb4+VlcQ8Gl1TFH67mJLz3HA2nvuave0nbv6Lu8pbg== + dependencies: + "@inquirer/figures" "^1.0.1" + "@inquirer/type" "^1.3.0" + "@types/mute-stream" "^0.0.4" + "@types/node" "^20.12.7" + "@types/wrap-ansi" "^3.0.0" + ansi-escapes "^4.3.2" + chalk "^4.1.2" + cli-spinners "^2.9.2" + cli-width "^4.1.0" + mute-stream "^1.0.0" + signal-exit "^4.1.0" + strip-ansi "^6.0.1" + wrap-ansi "^6.2.0" -"@intlify/message-compiler@9.2.2": - version "9.2.2" - resolved "https://registry.yarnpkg.com/@intlify/message-compiler/-/message-compiler-9.2.2.tgz#e42ab6939b8ae5b3d21faf6a44045667a18bba1c" - integrity sha512-IUrQW7byAKN2fMBe8z6sK6riG1pue95e5jfokn8hA5Q3Bqy4MBJ5lJAofUsawQJYHeoPJ7svMDyBaVJ4d0GTtA== - dependencies: - "@intlify/shared" "9.2.2" - source-map "0.6.1" +"@inquirer/figures@^1.0.1": + version "1.0.1" + resolved "https://registry.yarnpkg.com/@inquirer/figures/-/figures-1.0.1.tgz#d65f0bd0e9511a90b4d3543ee6a3ce7211f29417" + integrity sha512-mtup3wVKia3ZwULPHcbs4Mor8Voi+iIXEWD7wCNbIO6lYR62oPCTQyrddi5OMYVXHzeCSoneZwJuS8sBvlEwDw== -"@intlify/shared@9.2.2": - version "9.2.2" - resolved "https://registry.yarnpkg.com/@intlify/shared/-/shared-9.2.2.tgz#5011be9ca2b4ab86f8660739286e2707f9abb4a5" - integrity sha512-wRwTpsslgZS5HNyM7uDQYZtxnbI12aGiBZURX3BTR9RFIKKRWpllTsgzHWvj3HKm3Y2Sh5LPC1r0PDCKEhVn9Q== +"@inquirer/type@^1.3.0": + version "1.3.0" + resolved "https://registry.yarnpkg.com/@inquirer/type/-/type-1.3.0.tgz#9dcb4e0e8bbec03063aff7806072cc90eea2c61d" + integrity sha512-RW4Zf6RCTnInRaOZuRHTqAUl+v6VJuQGglir7nW2BkT3OXOphMhkIFhvFRjorBx2l0VwtC/M4No8vYR65TdN9Q== -"@intlify/vue-devtools@9.2.2": - version "9.2.2" - resolved "https://registry.yarnpkg.com/@intlify/vue-devtools/-/vue-devtools-9.2.2.tgz#b95701556daf7ebb3a2d45aa3ae9e6415aed8317" - integrity sha512-+dUyqyCHWHb/UcvY1MlIpO87munedm3Gn6E9WWYdWrMuYLcoIoOEVDWSS8xSwtlPU+kA+MEQTP6Q1iI/ocusJg== +"@intlify/core-base@9.13.0": + version "9.13.0" + resolved "https://registry.yarnpkg.com/@intlify/core-base/-/core-base-9.13.0.tgz#b6305c99521a613c4577b2a9e7dc503631ebcd85" + integrity sha512-Lx8+YTrFpom7AtdbbuJHzgmr612/bceHU92v8ZPU9HU9/rczf+TmCs95BxWPIR4K42xh4MVMLsNzLUWiXcNaLg== dependencies: - "@intlify/core-base" "9.2.2" - "@intlify/shared" "9.2.2" + "@intlify/message-compiler" "9.13.0" + "@intlify/shared" "9.13.0" -"@jest/expect-utils@^29.4.2": - version "29.4.2" - resolved "https://registry.yarnpkg.com/@jest/expect-utils/-/expect-utils-29.4.2.tgz#cd0065dfdd8e8a182aa350cc121db97b5eed7b3f" - integrity sha512-Dd3ilDJpBnqa0GiPN7QrudVs0cczMMHtehSo2CSTjm3zdHx0RcpmhFNVEltuEFeqfLIyWKFI224FsMSQ/nsJQA== +"@intlify/message-compiler@9.13.0": + version "9.13.0" + resolved "https://registry.yarnpkg.com/@intlify/message-compiler/-/message-compiler-9.13.0.tgz#3b523a5331d7ac89dde4aefc2f1952e79b49070b" + integrity sha512-zhESuudiDpFQhUOx/qrSMd7ZYHbmgCc0QzBc27cDUxaaAj3olbYJnsx3osiHPQyYnv/LuC+WTqoNOEBoHP6dqQ== dependencies: - jest-get-type "^29.4.2" + "@intlify/shared" "9.13.0" + source-map-js "^1.0.2" + +"@intlify/shared@9.13.0": + version "9.13.0" + resolved "https://registry.yarnpkg.com/@intlify/shared/-/shared-9.13.0.tgz#e723296fdab798d280087b89764aab06791bc7a4" + integrity sha512-fUwWcpDz9Wm4dSaz+6XmjoNXWBjZLJtT1Zf1cpLBELbCAOS8WBRscPtgOSfzm6JCqf5KgMI4g917f5TtEeez3A== -"@jest/schemas@^29.4.2": - version "29.4.2" - resolved "https://registry.yarnpkg.com/@jest/schemas/-/schemas-29.4.2.tgz#cf7cfe97c5649f518452b176c47ed07486270fc1" - integrity sha512-ZrGzGfh31NtdVH8tn0mgJw4khQuNHiKqdzJAFbCaERbyCP9tHlxWuL/mnMu8P7e/+k4puWjI1NOzi/sFsjce/g== +"@isaacs/cliui@^8.0.2": + version "8.0.2" + resolved "https://registry.yarnpkg.com/@isaacs/cliui/-/cliui-8.0.2.tgz#b37667b7bc181c168782259bab42474fbf52b550" + integrity sha512-O8jcjabXaleOG9DQ0+ARXWZBTfnP4WNAqzuiJK7ll44AmxGKv/J2M4TPjxjY3znBCfvBXFzucm1twdyFybFqEA== dependencies: - "@sinclair/typebox" "^0.25.16" + string-width "^5.1.2" + string-width-cjs "npm:string-width@^4.2.0" + strip-ansi "^7.0.1" + strip-ansi-cjs "npm:strip-ansi@^6.0.1" + wrap-ansi "^8.1.0" + wrap-ansi-cjs "npm:wrap-ansi@^7.0.0" -"@jest/types@^29.4.2": - version "29.4.2" - resolved "https://registry.yarnpkg.com/@jest/types/-/types-29.4.2.tgz#8f724a414b1246b2bfd56ca5225d9e1f39540d82" - integrity sha512-CKlngyGP0fwlgC1BRUtPZSiWLBhyS9dKwKmyGxk8Z6M82LBEGB2aLQSg+U1MyLsU+M7UjnlLllBM2BLWKVm/Uw== +"@jest/schemas@^29.6.3": + version "29.6.3" + resolved "https://registry.yarnpkg.com/@jest/schemas/-/schemas-29.6.3.tgz#430b5ce8a4e0044a7e3819663305a7b3091c8e03" + integrity sha512-mo5j5X+jIZmJQveBKeS/clAueipV7KgiX1vMgCxam1RNYiqE1w62n0/tJJnHtjW8ZHcQco5gY85jA3mi0L+nSA== dependencies: - "@jest/schemas" "^29.4.2" - "@types/istanbul-lib-coverage" "^2.0.0" - "@types/istanbul-reports" "^3.0.0" - "@types/node" "*" - "@types/yargs" "^17.0.8" - chalk "^4.0.0" + "@sinclair/typebox" "^0.27.8" -"@jridgewell/sourcemap-codec@^1.4.13": +"@jridgewell/sourcemap-codec@^1.4.15": version "1.4.15" resolved "https://registry.yarnpkg.com/@jridgewell/sourcemap-codec/-/sourcemap-codec-1.4.15.tgz#d7c6e6755c78567a951e04ab52ef0fd26de59f32" integrity sha512-eF2rxCRulEKXHTRiDrDy6erMYWqNw4LPdQ8UQA4huuxaQsVeRPFl2oM8oDGxMFhJUWZf9McpLtJasDDZb/Bpeg== -"@mdi/font@7.1.96": - version "7.1.96" - resolved "https://registry.yarnpkg.com/@mdi/font/-/font-7.1.96.tgz#211ca4acfa31964278e5085de595e8c73967d400" - integrity sha512-Imag6npmfkBDi2Ze2jiZVAPTDIKLxhz2Sx82xJ2zctyAU5LYJejLI5ChnDwiD9bMkQfVuzEsI98Q8toHyC+HCg== +"@mdi/font@7.4.47": + version "7.4.47" + resolved "https://registry.yarnpkg.com/@mdi/font/-/font-7.4.47.tgz#2ae522867da3a5c88b738d54b403eb91471903af" + integrity sha512-43MtGpd585SNzHZPcYowu/84Vz2a2g31TvPMTm9uTiCSWzaheQySUcSyUH/46fPnuPQWof2yd0pGBtzee/IQWw== -"@mswjs/cookies@^0.2.2": - version "0.2.2" - resolved "https://registry.yarnpkg.com/@mswjs/cookies/-/cookies-0.2.2.tgz#b4e207bf6989e5d5427539c2443380a33ebb922b" - integrity sha512-mlN83YSrcFgk7Dm1Mys40DLssI1KdJji2CMKN8eOlBqsTADYzj2+jWzsANsUTFbxDMWPD5e9bfA1RGqBpS3O1g== - dependencies: - "@types/set-cookie-parser" "^2.4.0" - set-cookie-parser "^2.4.6" +"@mswjs/cookies@^1.1.0": + version "1.1.0" + resolved "https://registry.yarnpkg.com/@mswjs/cookies/-/cookies-1.1.0.tgz#1528eb43630caf83a1d75d5332b30e75e9bb1b5b" + integrity sha512-0ZcCVQxifZmhwNBoQIrystCb+2sWBY2Zw8lpfJBPCHGCA/HWqehITeCRVIv4VMy8MPlaHo2w2pTHFV2pFfqKPw== -"@mswjs/interceptors@^0.17.5": - version "0.17.9" - resolved "https://registry.yarnpkg.com/@mswjs/interceptors/-/interceptors-0.17.9.tgz#0096fc88fea63ee42e36836acae8f4ae33651c04" - integrity sha512-4LVGt03RobMH/7ZrbHqRxQrS9cc2uh+iNKSj8UWr8M26A2i793ju+csaB5zaqYltqJmA2jUq4VeYfKmVqvsXQg== +"@mswjs/interceptors@^0.26.14": + version "0.26.15" + resolved "https://registry.yarnpkg.com/@mswjs/interceptors/-/interceptors-0.26.15.tgz#256ad5c89f325c87d972cc27fc7d0d6d382ce804" + integrity sha512-HM47Lu1YFmnYHKMBynFfjCp0U/yRskHj/8QEJW0CBEPOlw8Gkmjfll+S9b8M7V5CNDw2/ciRxjjnWeaCiblSIQ== dependencies: - "@open-draft/until" "^1.0.3" - "@types/debug" "^4.1.7" - "@xmldom/xmldom" "^0.8.3" - debug "^4.3.3" - headers-polyfill "^3.1.0" + "@open-draft/deferred-promise" "^2.2.0" + "@open-draft/logger" "^0.3.0" + "@open-draft/until" "^2.0.0" + is-node-process "^1.2.0" outvariant "^1.2.1" - strict-event-emitter "^0.2.4" - web-encoding "^1.1.5" + strict-event-emitter "^0.5.1" "@nodelib/fs.scandir@2.1.5": version "2.1.5" @@ -313,10 +357,28 @@ "@nodelib/fs.scandir" "2.1.5" fastq "^1.6.0" -"@open-draft/until@^1.0.3": - version "1.0.3" - resolved "https://registry.yarnpkg.com/@open-draft/until/-/until-1.0.3.tgz#db9cc719191a62e7d9200f6e7bab21c5b848adca" - integrity sha512-Aq58f5HiWdyDlFffbbSjAlv596h/cOnt2DO1w3DOC7OJ5EHs0hd/nycJfiu9RJbT6Yk6F1knnRRXNSpxoIVZ9Q== +"@one-ini/wasm@0.1.1": + version "0.1.1" + resolved "https://registry.yarnpkg.com/@one-ini/wasm/-/wasm-0.1.1.tgz#6013659736c9dbfccc96e8a9c2b3de317df39323" + integrity sha512-XuySG1E38YScSJoMlqovLru4KTUNSjgVTIjyh7qMX6aNN5HY5Ct5LhRJdxO79JtTzKfzV/bnWpz+zquYrISsvw== + +"@open-draft/deferred-promise@^2.2.0": + version "2.2.0" + resolved "https://registry.yarnpkg.com/@open-draft/deferred-promise/-/deferred-promise-2.2.0.tgz#4a822d10f6f0e316be4d67b4d4f8c9a124b073bd" + integrity sha512-CecwLWx3rhxVQF6V4bAgPS5t+So2sTbPgAzafKkVizyi7tlwpcFpdFqq+wqF2OwNBmqFuu6tOyouTuxgpMfzmA== + +"@open-draft/logger@^0.3.0": + version "0.3.0" + resolved "https://registry.yarnpkg.com/@open-draft/logger/-/logger-0.3.0.tgz#2b3ab1242b360aa0adb28b85f5d7da1c133a0954" + integrity sha512-X2g45fzhxH238HKO4xbSr7+wBS8Fvw6ixhTDuvLd5mqh6bJJCFAPwU9mPDxbcrRtfxv4u5IHCEH77BmxvXmmxQ== + dependencies: + is-node-process "^1.2.0" + outvariant "^1.4.0" + +"@open-draft/until@^2.0.0", "@open-draft/until@^2.1.0": + version "2.1.0" + resolved "https://registry.yarnpkg.com/@open-draft/until/-/until-2.1.0.tgz#0acf32f470af2ceaf47f095cdecd40d68666efda" + integrity sha512-U69T3ItWHvLwGg5eJ0n3I62nWuE6ilHlmz7zM0npLBRvPRd7e6NYmg54vvRtP5mZG7kZqZCFVdsTWo7BPtBujg== "@pinia/testing@0.1.3": version "0.1.3" @@ -325,596 +387,526 @@ dependencies: vue-demi ">=0.14.5" -"@rushstack/eslint-patch@1.2.0": - version "1.2.0" - resolved "https://registry.yarnpkg.com/@rushstack/eslint-patch/-/eslint-patch-1.2.0.tgz#8be36a1f66f3265389e90b5f9c9962146758f728" - integrity sha512-sXo/qW2/pAcmT43VoRKOJbDOfV3cYpq3szSVfIThQXNt+E4DfKj361vaAt3c88U5tPUxzEswam7GW48PJqtKAg== - -"@sinclair/typebox@^0.25.16": - version "0.25.21" - resolved "https://registry.yarnpkg.com/@sinclair/typebox/-/typebox-0.25.21.tgz#763b05a4b472c93a8db29b2c3e359d55b29ce272" - integrity sha512-gFukHN4t8K4+wVC+ECqeqwzBDeFeTzBXroBTqE6vcWrQGbEUpHO7LYdG0f4xnvYq4VOEwITSlHlp0JBAIFMS/g== +"@pkgjs/parseargs@^0.11.0": + version "0.11.0" + resolved "https://registry.yarnpkg.com/@pkgjs/parseargs/-/parseargs-0.11.0.tgz#a77ea742fab25775145434eb1d2328cf5013ac33" + integrity sha512-+1VkjdD0QBLPodGrJUeqarH8VAIvQODIbwh9XpP5Syisf7YoQgsJKPNFoqqLQlu+VQ/tVSshMR6loPMn8U+dPg== -"@testing-library/dom@^9.0.1": - version "9.3.0" - resolved "https://registry.yarnpkg.com/@testing-library/dom/-/dom-9.3.0.tgz#ed8ce10aa5e05eb6eaf0635b5b8975d889f66075" - integrity sha512-Dffe68pGwI6WlLRYR2I0piIkyole9cSBH5jGQKCGMRpHW5RHCqAUaqc2Kv0tUyd4dU4DLPKhJIjyKOnjv4tuUw== +"@pkgr/core@^0.1.0": + version "0.1.1" + resolved "https://registry.yarnpkg.com/@pkgr/core/-/core-0.1.1.tgz#1ec17e2edbec25c8306d424ecfbf13c7de1aaa31" + integrity sha512-cq8o4cWH0ibXh9VGi5P20Tu9XF/0fFXl9EUinr9QfTM7a7p0oTA4iJRCQWppXR1Pg8dSM0UCItCkPwsk9qWWYA== + +"@rollup/rollup-android-arm-eabi@4.14.3": + version "4.14.3" + resolved "https://registry.yarnpkg.com/@rollup/rollup-android-arm-eabi/-/rollup-android-arm-eabi-4.14.3.tgz#bddf05c3387d02fac04b6b86b3a779337edfed75" + integrity sha512-X9alQ3XM6I9IlSlmC8ddAvMSyG1WuHk5oUnXGw+yUBs3BFoTizmG1La/Gr8fVJvDWAq+zlYTZ9DBgrlKRVY06g== + +"@rollup/rollup-android-arm64@4.14.3": + version "4.14.3" + resolved "https://registry.yarnpkg.com/@rollup/rollup-android-arm64/-/rollup-android-arm64-4.14.3.tgz#b26bd09de58704c0a45e3375b76796f6eda825e4" + integrity sha512-eQK5JIi+POhFpzk+LnjKIy4Ks+pwJ+NXmPxOCSvOKSNRPONzKuUvWE+P9JxGZVxrtzm6BAYMaL50FFuPe0oWMQ== + +"@rollup/rollup-darwin-arm64@4.14.3": + version "4.14.3" + resolved "https://registry.yarnpkg.com/@rollup/rollup-darwin-arm64/-/rollup-darwin-arm64-4.14.3.tgz#c5f3fd1aa285b6d33dda6e3f3ca395f8c37fd5ca" + integrity sha512-Od4vE6f6CTT53yM1jgcLqNfItTsLt5zE46fdPaEmeFHvPs5SjZYlLpHrSiHEKR1+HdRfxuzXHjDOIxQyC3ptBA== + +"@rollup/rollup-darwin-x64@4.14.3": + version "4.14.3" + resolved "https://registry.yarnpkg.com/@rollup/rollup-darwin-x64/-/rollup-darwin-x64-4.14.3.tgz#8e4673734d7dc9d68f6d48e81246055cda0e840f" + integrity sha512-0IMAO21axJeNIrvS9lSe/PGthc8ZUS+zC53O0VhF5gMxfmcKAP4ESkKOCwEi6u2asUrt4mQv2rjY8QseIEb1aw== + +"@rollup/rollup-linux-arm-gnueabihf@4.14.3": + version "4.14.3" + resolved "https://registry.yarnpkg.com/@rollup/rollup-linux-arm-gnueabihf/-/rollup-linux-arm-gnueabihf-4.14.3.tgz#53ed38eb13b58ababdb55a7f66f0538a7f85dcba" + integrity sha512-ge2DC7tHRHa3caVEoSbPRJpq7azhG+xYsd6u2MEnJ6XzPSzQsTKyXvh6iWjXRf7Rt9ykIUWHtl0Uz3T6yXPpKw== + +"@rollup/rollup-linux-arm-musleabihf@4.14.3": + version "4.14.3" + resolved "https://registry.yarnpkg.com/@rollup/rollup-linux-arm-musleabihf/-/rollup-linux-arm-musleabihf-4.14.3.tgz#0706ee38330e267a5c9326956820f009cfb21fcd" + integrity sha512-ljcuiDI4V3ySuc7eSk4lQ9wU8J8r8KrOUvB2U+TtK0TiW6OFDmJ+DdIjjwZHIw9CNxzbmXY39wwpzYuFDwNXuw== + +"@rollup/rollup-linux-arm64-gnu@4.14.3": + version "4.14.3" + resolved "https://registry.yarnpkg.com/@rollup/rollup-linux-arm64-gnu/-/rollup-linux-arm64-gnu-4.14.3.tgz#426fce7b8b242ac5abd48a10a5020f5a468c6cb4" + integrity sha512-Eci2us9VTHm1eSyn5/eEpaC7eP/mp5n46gTRB3Aar3BgSvDQGJZuicyq6TsH4HngNBgVqC5sDYxOzTExSU+NjA== + +"@rollup/rollup-linux-arm64-musl@4.14.3": + version "4.14.3" + resolved "https://registry.yarnpkg.com/@rollup/rollup-linux-arm64-musl/-/rollup-linux-arm64-musl-4.14.3.tgz#65bf944530d759b50d7ffd00dfbdf4125a43406f" + integrity sha512-UrBoMLCq4E92/LCqlh+blpqMz5h1tJttPIniwUgOFJyjWI1qrtrDhhpHPuFxULlUmjFHfloWdixtDhSxJt5iKw== + +"@rollup/rollup-linux-powerpc64le-gnu@4.14.3": + version "4.14.3" + resolved "https://registry.yarnpkg.com/@rollup/rollup-linux-powerpc64le-gnu/-/rollup-linux-powerpc64le-gnu-4.14.3.tgz#494ba3b31095e9a45df9c3f646d21400fb631a95" + integrity sha512-5aRjvsS8q1nWN8AoRfrq5+9IflC3P1leMoy4r2WjXyFqf3qcqsxRCfxtZIV58tCxd+Yv7WELPcO9mY9aeQyAmw== + +"@rollup/rollup-linux-riscv64-gnu@4.14.3": + version "4.14.3" + resolved "https://registry.yarnpkg.com/@rollup/rollup-linux-riscv64-gnu/-/rollup-linux-riscv64-gnu-4.14.3.tgz#8b88ed0a40724cce04aa15374ebe5ba4092d679f" + integrity sha512-sk/Qh1j2/RJSX7FhEpJn8n0ndxy/uf0kI/9Zc4b1ELhqULVdTfN6HL31CDaTChiBAOgLcsJ1sgVZjWv8XNEsAQ== + +"@rollup/rollup-linux-s390x-gnu@4.14.3": + version "4.14.3" + resolved "https://registry.yarnpkg.com/@rollup/rollup-linux-s390x-gnu/-/rollup-linux-s390x-gnu-4.14.3.tgz#09c9e5ec57a0f6ec3551272c860bb9a04b96d70f" + integrity sha512-jOO/PEaDitOmY9TgkxF/TQIjXySQe5KVYB57H/8LRP/ux0ZoO8cSHCX17asMSv3ruwslXW/TLBcxyaUzGRHcqg== + +"@rollup/rollup-linux-x64-gnu@4.14.3": + version "4.14.3" + resolved "https://registry.yarnpkg.com/@rollup/rollup-linux-x64-gnu/-/rollup-linux-x64-gnu-4.14.3.tgz#197f27fd481ad9c861021d5cbbf21793922a631c" + integrity sha512-8ybV4Xjy59xLMyWo3GCfEGqtKV5M5gCSrZlxkPGvEPCGDLNla7v48S662HSGwRd6/2cSneMQWiv+QzcttLrrOA== + +"@rollup/rollup-linux-x64-musl@4.14.3": + version "4.14.3" + resolved "https://registry.yarnpkg.com/@rollup/rollup-linux-x64-musl/-/rollup-linux-x64-musl-4.14.3.tgz#5cc0522f4942f2df625e9bfb6fb02c6580ffbce6" + integrity sha512-s+xf1I46trOY10OqAtZ5Rm6lzHre/UiLA1J2uOhCFXWkbZrJRkYBPO6FhvGfHmdtQ3Bx793MNa7LvoWFAm93bg== + +"@rollup/rollup-win32-arm64-msvc@4.14.3": + version "4.14.3" + resolved "https://registry.yarnpkg.com/@rollup/rollup-win32-arm64-msvc/-/rollup-win32-arm64-msvc-4.14.3.tgz#a648122389d23a7543b261fba082e65fefefe4f6" + integrity sha512-+4h2WrGOYsOumDQ5S2sYNyhVfrue+9tc9XcLWLh+Kw3UOxAvrfOrSMFon60KspcDdytkNDh7K2Vs6eMaYImAZg== + +"@rollup/rollup-win32-ia32-msvc@4.14.3": + version "4.14.3" + resolved "https://registry.yarnpkg.com/@rollup/rollup-win32-ia32-msvc/-/rollup-win32-ia32-msvc-4.14.3.tgz#34727b5c7953c35fc6e1ae4f770ad3a2025f8e03" + integrity sha512-T1l7y/bCeL/kUwh9OD4PQT4aM7Bq43vX05htPJJ46RTI4r5KNt6qJRzAfNfM+OYMNEVBWQzR2Gyk+FXLZfogGw== + +"@rollup/rollup-win32-x64-msvc@4.14.3": + version "4.14.3" + resolved "https://registry.yarnpkg.com/@rollup/rollup-win32-x64-msvc/-/rollup-win32-x64-msvc-4.14.3.tgz#5b2fb4d8cd44c05deef8a7b0e6deb9ccb8939d18" + integrity sha512-/BypzV0H1y1HzgYpxqRaXGBRqfodgoBBCcsrujT6QRcakDQdfU+Lq9PENPh5jB4I44YWq+0C2eHsHya+nZY1sA== + +"@rushstack/eslint-patch@1.10.2": + version "1.10.2" + resolved "https://registry.yarnpkg.com/@rushstack/eslint-patch/-/eslint-patch-1.10.2.tgz#053f1540703faa81dea2966b768ee5581c66aeda" + integrity sha512-hw437iINopmQuxWPSUEvqE56NCPsiU8N4AYtfHmJFckclktzK9YQJieD3XkDCDH4OjL+C7zgPUh73R/nrcHrqw== + +"@sinclair/typebox@^0.27.8": + version "0.27.8" + resolved "https://registry.yarnpkg.com/@sinclair/typebox/-/typebox-0.27.8.tgz#6667fac16c436b5434a387a34dedb013198f6e6e" + integrity sha512-+Fj43pSMwJs4KRrH/938Uf+uAELIgVBmQzg/q1YG10djyfA3TnrU8N8XzqCh/okZdszqBQTZf96idMfE5lnwTA== + +"@testing-library/dom@^9.3.3": + version "9.3.4" + resolved "https://registry.yarnpkg.com/@testing-library/dom/-/dom-9.3.4.tgz#50696ec28376926fec0a1bf87d9dbac5e27f60ce" + integrity sha512-FlS4ZWlp97iiNWig0Muq8p+3rVDjRiYE+YKGbAqXOu9nwJFFOdL00kFpz42M+4huzYi86vAK1sOOfyOG45muIQ== dependencies: "@babel/code-frame" "^7.10.4" "@babel/runtime" "^7.12.5" "@types/aria-query" "^5.0.1" - aria-query "^5.0.0" + aria-query "5.1.3" chalk "^4.1.0" dom-accessibility-api "^0.5.9" lz-string "^1.5.0" pretty-format "^27.0.2" -"@testing-library/jest-dom@5.16.5": - version "5.16.5" - resolved "https://registry.yarnpkg.com/@testing-library/jest-dom/-/jest-dom-5.16.5.tgz#3912846af19a29b2dbf32a6ae9c31ef52580074e" - integrity sha512-N5ixQ2qKpi5OLYfwQmUb/5mSV9LneAcaUfp32pn4yCnpb8r/Yz0pXFPck21dIicKmi+ta5WRAknkZCfA8refMA== +"@testing-library/jest-dom@6.4.2": + version "6.4.2" + resolved "https://registry.yarnpkg.com/@testing-library/jest-dom/-/jest-dom-6.4.2.tgz#38949f6b63722900e2d75ba3c6d9bf8cffb3300e" + integrity sha512-CzqH0AFymEMG48CpzXFriYYkOjk6ZGPCLMhW9e9jg3KMCn5OfJecF8GtGW7yGfR/IgCe3SX8BSwjdzI6BBbZLw== dependencies: - "@adobe/css-tools" "^4.0.1" + "@adobe/css-tools" "^4.3.2" "@babel/runtime" "^7.9.2" - "@types/testing-library__jest-dom" "^5.9.1" aria-query "^5.0.0" chalk "^3.0.0" css.escape "^1.5.1" - dom-accessibility-api "^0.5.6" + dom-accessibility-api "^0.6.3" lodash "^4.17.15" redent "^3.0.0" -"@testing-library/user-event@14.4.3": - version "14.4.3" - resolved "https://registry.yarnpkg.com/@testing-library/user-event/-/user-event-14.4.3.tgz#af975e367743fa91989cd666666aec31a8f50591" - integrity sha512-kCUc5MEwaEMakkO5x7aoD+DLi02ehmEM2QCGWvNqAS1dV/fAvORWEjnjsEIvml59M7Y5kCkWN6fCCyPOe8OL6Q== +"@testing-library/user-event@14.5.2": + version "14.5.2" + resolved "https://registry.yarnpkg.com/@testing-library/user-event/-/user-event-14.5.2.tgz#db7257d727c891905947bd1c1a99da20e03c2ebd" + integrity sha512-YAh82Wh4TIrxYLmfGcixwD18oIjyC1pFQC2Y01F2lzV2HTMiYrI0nze0FD0ocB//CKS/7jIUgae+adPqxK5yCQ== -"@testing-library/vue@7.0.0": - version "7.0.0" - resolved "https://registry.yarnpkg.com/@testing-library/vue/-/vue-7.0.0.tgz#b85957ec7c69069d52c27d10926dbbe842e7dfc4" - integrity sha512-JU/q93HGo2qdm1dCgWymkeQlfpC0/0/DBZ2nAHgEAsVZxX11xVIxT7gbXdI7HACQpUbsUWt1zABGU075Fzt9XQ== +"@testing-library/vue@8.0.3": + version "8.0.3" + resolved "https://registry.yarnpkg.com/@testing-library/vue/-/vue-8.0.3.tgz#088bd8e6272fd5e38e9bf40cc195a1c346c314a6" + integrity sha512-wSsbNlZ69ZFQgVlHMtc/ZC/g9BHO7MhyDrd4nHyfEubtMr3kToN/w4/BsSBknGIF8w9UmPbsgbIuq/CbdBHzCA== dependencies: - "@babel/runtime" "^7.21.0" - "@testing-library/dom" "^9.0.1" - "@vue/test-utils" "^2.3.1" - -"@tootallnate/once@2": - version "2.0.0" - resolved "https://registry.yarnpkg.com/@tootallnate/once/-/once-2.0.0.tgz#f544a148d3ab35801c1f633a7441fd87c2e484bf" - integrity sha512-XCuKFP5PS55gnMVu3dty8KPatLqUoy/ZYzDzAGCQ8JNFCkLXzmI7vNHCR+XpbZaMWQK/vQubr7PkYq8g470J/A== + "@babel/runtime" "^7.23.2" + "@testing-library/dom" "^9.3.3" + "@vue/test-utils" "^2.4.1" "@types/aria-query@^5.0.1": version "5.0.1" resolved "https://registry.yarnpkg.com/@types/aria-query/-/aria-query-5.0.1.tgz#3286741fb8f1e1580ac28784add4c7a1d49bdfbc" integrity sha512-XTIieEY+gvJ39ChLcB4If5zHtPxt3Syj5rgZR+e1ctpmK8NjPf0zFqsz4JpLJT0xla9GFDKjy8Cpu331nrmE1Q== -"@types/chai-subset@^1.3.3": - version "1.3.3" - resolved "https://registry.yarnpkg.com/@types/chai-subset/-/chai-subset-1.3.3.tgz#97893814e92abd2c534de422cb377e0e0bdaac94" - integrity sha512-frBecisrNGz+F4T6bcc+NLeolfiojh5FxW2klu669+8BARtyQv2C/GkNW6FUodVe4BroGMP/wER/YDGc7rEllw== - dependencies: - "@types/chai" "*" - -"@types/chai@*": - version "4.3.4" - resolved "https://registry.yarnpkg.com/@types/chai/-/chai-4.3.4.tgz#e913e8175db8307d78b4e8fa690408ba6b65dee4" - integrity sha512-KnRanxnpfpjUTqTCXslZSEdLfXExwgNxYPdiO2WGUj8+HDjFi8R3k5RVKPeSCzLjCcshCAtVO2QBbVuAV4kTnw== - -"@types/chai@^4.3.5": - version "4.3.5" - resolved "https://registry.yarnpkg.com/@types/chai/-/chai-4.3.5.tgz#ae69bcbb1bebb68c4ac0b11e9d8ed04526b3562b" - integrity sha512-mEo1sAde+UCE6b2hxn332f1g1E8WfYRu6p5SvTKr2ZKC1f7gFJXk4h5PyGP9Dt6gCaG8y8XhwnXWC6Iy2cmBng== - -"@types/cookie@^0.4.1": - version "0.4.1" - resolved "https://registry.yarnpkg.com/@types/cookie/-/cookie-0.4.1.tgz#bfd02c1f2224567676c1545199f87c3a861d878d" - integrity sha512-XW/Aa8APYr6jSVVA1y/DEIZX0/GMKLEVekNG727R8cs56ahETkRAy/3DR7+fJyh7oUgGwNQaRfXCun0+KbWY7Q== - -"@types/debug@^4.1.7": - version "4.1.8" - resolved "https://registry.yarnpkg.com/@types/debug/-/debug-4.1.8.tgz#cef723a5d0a90990313faec2d1e22aee5eecb317" - integrity sha512-/vPO1EPOs306Cvhwv7KfVfYvOJqA/S/AXjaHQiJboCZzcNDb+TIJFN9/2C9DZ//ijSKWioNyUxD792QmDJ+HKQ== - dependencies: - "@types/ms" "*" - -"@types/istanbul-lib-coverage@*", "@types/istanbul-lib-coverage@^2.0.0": - version "2.0.4" - resolved "https://registry.yarnpkg.com/@types/istanbul-lib-coverage/-/istanbul-lib-coverage-2.0.4.tgz#8467d4b3c087805d63580480890791277ce35c44" - integrity sha512-z/QT1XN4K4KYuslS23k62yDIDLwLFkzxOuMplDtObz0+y7VqJCaO2o+SPwHCvLFZh7xazvvoor2tA/hPz9ee7g== - -"@types/istanbul-lib-report@*": - version "3.0.0" - resolved "https://registry.yarnpkg.com/@types/istanbul-lib-report/-/istanbul-lib-report-3.0.0.tgz#c14c24f18ea8190c118ee7562b7ff99a36552686" - integrity sha512-plGgXAPfVKFoYfa9NpYDAkseG+g6Jr294RqeqcqDixSbU34MZVJRi/P+7Y8GDpzkEwLaGZZOpKIEmeVZNtKsrg== - dependencies: - "@types/istanbul-lib-coverage" "*" - -"@types/istanbul-reports@^3.0.0": - version "3.0.1" - resolved "https://registry.yarnpkg.com/@types/istanbul-reports/-/istanbul-reports-3.0.1.tgz#9153fe98bba2bd565a63add9436d6f0d7f8468ff" - integrity sha512-c3mAZEuK0lvBp8tmuL74XRKn1+y2dcwOUpH7x4WrF6gk1GIgiluDRgMYQtw2OFcBvAJWlt6ASU3tSqxp0Uu0Aw== - dependencies: - "@types/istanbul-lib-report" "*" - -"@types/jest@*": - version "29.4.0" - resolved "https://registry.yarnpkg.com/@types/jest/-/jest-29.4.0.tgz#a8444ad1704493e84dbf07bb05990b275b3b9206" - integrity sha512-VaywcGQ9tPorCX/Jkkni7RWGFfI11whqzs8dvxF41P17Z+z872thvEvlIbznjPJ02kl1HMX3LmLOonsj2n7HeQ== - dependencies: - expect "^29.0.0" - pretty-format "^29.0.0" +"@types/cookie@^0.6.0": + version "0.6.0" + resolved "https://registry.yarnpkg.com/@types/cookie/-/cookie-0.6.0.tgz#eac397f28bf1d6ae0ae081363eca2f425bedf0d5" + integrity sha512-4Kh9a6B2bQciAhf7FSuMRRkUWecJgJu9nPnx3yzpsfXX/c50REIqpHY4C82bXP90qrLtXtkDxTZosYO3UpOwlA== -"@types/js-levenshtein@^1.1.1": - version "1.1.1" - resolved "https://registry.yarnpkg.com/@types/js-levenshtein/-/js-levenshtein-1.1.1.tgz#ba05426a43f9e4e30b631941e0aa17bf0c890ed5" - integrity sha512-qC4bCqYGy1y/NP7dDVr7KJarn+PbX1nSpwA7JXdu0HxT3QYjO8MJ+cntENtHFVy2dRAyBV23OZ6MxsW1AM1L8g== +"@types/estree@1.0.5", "@types/estree@^1.0.0": + version "1.0.5" + resolved "https://registry.yarnpkg.com/@types/estree/-/estree-1.0.5.tgz#a6ce3e556e00fd9895dd872dd172ad0d4bd687f4" + integrity sha512-/kYRxGDLWzHOB7q+wtSUQlFrtcdUccpfy+X+9iMBpHK8QLLhx2wIPYuS5DYtR9Wa/YlZAbIovy7qVdB1Aq6Lyw== -"@types/jsdom@21.1.0": - version "21.1.0" - resolved "https://registry.yarnpkg.com/@types/jsdom/-/jsdom-21.1.0.tgz#219f15e3370da3f85d18fe02ae86bda7ff66104a" - integrity sha512-leWreJOdnuIxq9Y70tBVm/bvTuh31DSlF/r4l7Cfi4uhVQqLHD0Q4v301GMisEMwwbMgF7ZKxuZ+Jbd4NcdmRw== +"@types/jsdom@21.1.6": + version "21.1.6" + resolved "https://registry.yarnpkg.com/@types/jsdom/-/jsdom-21.1.6.tgz#bcbc7b245787ea863f3da1ef19aa1dcfb9271a1b" + integrity sha512-/7kkMsC+/kMs7gAYmmBR9P0vGTnOoLhQhyhQJSlXGI5bzTHp6xdo0TtKWQAsz6pmSAeVqKSbqeyP6hytqr9FDw== dependencies: "@types/node" "*" "@types/tough-cookie" "*" parse5 "^7.0.0" -"@types/json-schema@^7.0.9": - version "7.0.11" - resolved "https://registry.yarnpkg.com/@types/json-schema/-/json-schema-7.0.11.tgz#d421b6c527a3037f7c84433fd2c4229e016863d3" - integrity sha512-wOuvG1SN4Us4rez+tylwwwCV1psiNVOkJeM3AUWUNWg/jDQY2+HE/444y5gc+jBmRqASOm2Oeh5c1axHobwRKQ== +"@types/json-schema@^7.0.15": + version "7.0.15" + resolved "https://registry.yarnpkg.com/@types/json-schema/-/json-schema-7.0.15.tgz#596a1747233694d50f6ad8a7869fcb6f56cf5841" + integrity sha512-5+fP8P8MFNC+AyZCDxrB2pkZFPGzqQWUzpSeuuVLvm8VMcorNYavBqoFcxK8bQz4Qsbn4oUEEem4wDLfcysGHA== -"@types/ms@*": - version "0.7.31" - resolved "https://registry.yarnpkg.com/@types/ms/-/ms-0.7.31.tgz#31b7ca6407128a3d2bbc27fe2d21b345397f6197" - integrity sha512-iiUgKzV9AuaEkZqkOLDIvlQiL6ltuZd9tGcW3gwpnX8JbuiuhFlEGmmFXEXkN50Cvq7Os88IY2v0dkDqXYWVgA== +"@types/mute-stream@^0.0.4": + version "0.0.4" + resolved "https://registry.yarnpkg.com/@types/mute-stream/-/mute-stream-0.0.4.tgz#77208e56a08767af6c5e1237be8888e2f255c478" + integrity sha512-CPM9nzrCPPJHQNA9keH9CVkVI+WR5kMa+7XEs5jcGQ0VoAGnLv242w8lIVgwAEfmE4oufJRaTc9PNLQl0ioAow== + dependencies: + "@types/node" "*" -"@types/node@*", "@types/node@18.13.0": +"@types/node@*": version "18.13.0" resolved "https://registry.yarnpkg.com/@types/node/-/node-18.13.0.tgz#0400d1e6ce87e9d3032c19eb6c58205b0d3f7850" integrity sha512-gC3TazRzGoOnoKAhUx+Q0t8S9Tzs74z7m0ipwGpSqQrleP14hKxP4/JUeEQcD3W1/aIpnWl8pHowI7WokuZpXg== -"@types/semver@^7.3.12": - version "7.3.13" - resolved "https://registry.yarnpkg.com/@types/semver/-/semver-7.3.13.tgz#da4bfd73f49bd541d28920ab0e2bf0ee80f71c91" - integrity sha512-21cFJr9z3g5dW8B0CVI9g2O9beqaThGQ6ZFBqHfwhzLDKUxaqTIy3vnfah/UPkfOiF2pLq+tGz+W8RyCskuslw== - -"@types/set-cookie-parser@^2.4.0": - version "2.4.2" - resolved "https://registry.yarnpkg.com/@types/set-cookie-parser/-/set-cookie-parser-2.4.2.tgz#b6a955219b54151bfebd4521170723df5e13caad" - integrity sha512-fBZgytwhYAUkj/jC/FAV4RQ5EerRup1YQsXQCh8rZfiHkc4UahC192oH0smGwsXol3cL3A5oETuAHeQHmhXM4w== +"@types/node@20.12.7", "@types/node@^20.12.7": + version "20.12.7" + resolved "https://registry.yarnpkg.com/@types/node/-/node-20.12.7.tgz#04080362fa3dd6c5822061aa3124f5c152cff384" + integrity sha512-wq0cICSkRLVaf3UGLMGItu/PtdY7oaXaI/RVU+xliKVOtRna3PRY57ZDfztpDL0n11vfymMUnXv8QwYCO7L1wg== dependencies: - "@types/node" "*" + undici-types "~5.26.4" -"@types/stack-utils@^2.0.0": - version "2.0.1" - resolved "https://registry.yarnpkg.com/@types/stack-utils/-/stack-utils-2.0.1.tgz#20f18294f797f2209b5f65c8e3b5c8e8261d127c" - integrity sha512-Hl219/BT5fLAaz6NDkSuhzasy49dwQS/DSdu4MdggFB8zcXv7vflBI3xp7FEmkmdDkBUI2bPUNeMttp2knYdxw== +"@types/semver@^7.5.8": + version "7.5.8" + resolved "https://registry.yarnpkg.com/@types/semver/-/semver-7.5.8.tgz#8268a8c57a3e4abd25c165ecd36237db7948a55e" + integrity sha512-I8EUhyrgfLrcTkzV3TSsGyl1tSuPrEDzr0yd5m90UgNxQkyDXULk3b6MlQqTCpZpNtWe1K0hzclnZkTcLBe2UQ== -"@types/testing-library__jest-dom@^5.9.1": - version "5.14.5" - resolved "https://registry.yarnpkg.com/@types/testing-library__jest-dom/-/testing-library__jest-dom-5.14.5.tgz#d113709c90b3c75fdb127ec338dad7d5f86c974f" - integrity sha512-SBwbxYoyPIvxHbeHxTZX2Pe/74F/tX2/D3mMvzabdeJ25bBojfW0TyB8BHrbq/9zaaKICJZjLP+8r6AeZMFCuQ== - dependencies: - "@types/jest" "*" +"@types/statuses@^2.0.4": + version "2.0.5" + resolved "https://registry.yarnpkg.com/@types/statuses/-/statuses-2.0.5.tgz#f61ab46d5352fd73c863a1ea4e1cef3b0b51ae63" + integrity sha512-jmIUGWrAiwu3dZpxntxieC+1n/5c3mjrImkmOSQ2NC5uP6cYO4aAZDdSmRcI5C1oiTmqlZGHC+/NmJrKogbP5A== "@types/tough-cookie@*": version "4.0.2" resolved "https://registry.yarnpkg.com/@types/tough-cookie/-/tough-cookie-4.0.2.tgz#6286b4c7228d58ab7866d19716f3696e03a09397" integrity sha512-Q5vtl1W5ue16D+nIaW8JWebSSraJVlK+EthKn7e7UcD4KWsaSJ8BqGPXNaPghgtcn/fhvrN17Tv8ksUsQpiplw== -"@types/uuid@9.0.2": - version "9.0.2" - resolved "https://registry.yarnpkg.com/@types/uuid/-/uuid-9.0.2.tgz#ede1d1b1e451548d44919dc226253e32a6952c4b" - integrity sha512-kNnC1GFBLuhImSnV7w4njQkUiJi0ZXUycu1rUaouPqiKlXkh77JKgdRnTAp1x5eBwcIwbtI+3otwzuIDEuDoxQ== - -"@types/yargs-parser@*": - version "21.0.0" - resolved "https://registry.yarnpkg.com/@types/yargs-parser/-/yargs-parser-21.0.0.tgz#0c60e537fa790f5f9472ed2776c2b71ec117351b" - integrity sha512-iO9ZQHkZxHn4mSakYV0vFHAVDyEOIJQrV2uZ06HxEPcx+mt8swXoZHIbaaJ2crJYFfErySgktuTZ3BeLz+XmFA== +"@types/uuid@9.0.8": + version "9.0.8" + resolved "https://registry.yarnpkg.com/@types/uuid/-/uuid-9.0.8.tgz#7545ba4fc3c003d6c756f651f3bf163d8f0f29ba" + integrity sha512-jg+97EGIcY9AGHJJRaaPVgetKDsrTgbRjQ5Msgjh/DQKEFl0DtyRr/VCOyD1T2R1MNeWPK/u7JoGhlDZnKBAfA== -"@types/yargs@^17.0.8": - version "17.0.22" - resolved "https://registry.yarnpkg.com/@types/yargs/-/yargs-17.0.22.tgz#7dd37697691b5f17d020f3c63e7a45971ff71e9a" - integrity sha512-pet5WJ9U8yPVRhkwuEIp5ktAeAqRZOq4UdAyWLWzxbtpyXnzbtLdKiXAjJzi/KLmPGS9wk86lUFWZFN6sISo4g== - dependencies: - "@types/yargs-parser" "*" - -"@typescript-eslint/eslint-plugin@^5.0.0": - version "5.51.0" - resolved "https://registry.yarnpkg.com/@typescript-eslint/eslint-plugin/-/eslint-plugin-5.51.0.tgz#da3f2819633061ced84bb82c53bba45a6fe9963a" - integrity sha512-wcAwhEWm1RgNd7dxD/o+nnLW8oH+6RK1OGnmbmkj/GGoDPV1WWMVP0FXYQBivKHdwM1pwii3bt//RC62EriIUQ== - dependencies: - "@typescript-eslint/scope-manager" "5.51.0" - "@typescript-eslint/type-utils" "5.51.0" - "@typescript-eslint/utils" "5.51.0" +"@types/wrap-ansi@^3.0.0": + version "3.0.0" + resolved "https://registry.yarnpkg.com/@types/wrap-ansi/-/wrap-ansi-3.0.0.tgz#18b97a972f94f60a679fd5c796d96421b9abb9fd" + integrity sha512-ltIpx+kM7g/MLRZfkbL7EsCEjfzCcScLpkg37eXEtx5kmrAKBkTJwd1GIAjDSL8wTpM6Hzn5YO4pSb91BEwu1g== + +"@typescript-eslint/eslint-plugin@^7.1.1": + version "7.7.0" + resolved "https://registry.yarnpkg.com/@typescript-eslint/eslint-plugin/-/eslint-plugin-7.7.0.tgz#bf34a02f221811505b8bf2f31060c8560c1bb0a3" + integrity sha512-GJWR0YnfrKnsRoluVO3PRb9r5aMZriiMMM/RHj5nnTrBy1/wIgk76XCtCKcnXGjpZQJQRFtGV9/0JJ6n30uwpQ== + dependencies: + "@eslint-community/regexpp" "^4.10.0" + "@typescript-eslint/scope-manager" "7.7.0" + "@typescript-eslint/type-utils" "7.7.0" + "@typescript-eslint/utils" "7.7.0" + "@typescript-eslint/visitor-keys" "7.7.0" debug "^4.3.4" - grapheme-splitter "^1.0.4" - ignore "^5.2.0" - natural-compare-lite "^1.4.0" - regexpp "^3.2.0" - semver "^7.3.7" - tsutils "^3.21.0" - -"@typescript-eslint/parser@^5.0.0": - version "5.51.0" - resolved "https://registry.yarnpkg.com/@typescript-eslint/parser/-/parser-5.51.0.tgz#2d74626652096d966ef107f44b9479f02f51f271" - integrity sha512-fEV0R9gGmfpDeRzJXn+fGQKcl0inIeYobmmUWijZh9zA7bxJ8clPhV9up2ZQzATxAiFAECqPQyMDB4o4B81AaA== - dependencies: - "@typescript-eslint/scope-manager" "5.51.0" - "@typescript-eslint/types" "5.51.0" - "@typescript-eslint/typescript-estree" "5.51.0" + graphemer "^1.4.0" + ignore "^5.3.1" + natural-compare "^1.4.0" + semver "^7.6.0" + ts-api-utils "^1.3.0" + +"@typescript-eslint/parser@^7.1.1": + version "7.7.0" + resolved "https://registry.yarnpkg.com/@typescript-eslint/parser/-/parser-7.7.0.tgz#6b1b3ce76c5de002c43af8ae933613b0f2b4bcc6" + integrity sha512-fNcDm3wSwVM8QYL4HKVBggdIPAy9Q41vcvC/GtDobw3c4ndVT3K6cqudUmjHPw8EAp4ufax0o58/xvWaP2FmTg== + dependencies: + "@typescript-eslint/scope-manager" "7.7.0" + "@typescript-eslint/types" "7.7.0" + "@typescript-eslint/typescript-estree" "7.7.0" + "@typescript-eslint/visitor-keys" "7.7.0" debug "^4.3.4" -"@typescript-eslint/scope-manager@5.51.0": - version "5.51.0" - resolved "https://registry.yarnpkg.com/@typescript-eslint/scope-manager/-/scope-manager-5.51.0.tgz#ad3e3c2ecf762d9a4196c0fbfe19b142ac498990" - integrity sha512-gNpxRdlx5qw3yaHA0SFuTjW4rxeYhpHxt491PEcKF8Z6zpq0kMhe0Tolxt0qjlojS+/wArSDlj/LtE69xUJphQ== +"@typescript-eslint/scope-manager@7.7.0": + version "7.7.0" + resolved "https://registry.yarnpkg.com/@typescript-eslint/scope-manager/-/scope-manager-7.7.0.tgz#3f0db079b275bb8b0cb5be7613fb3130cfb5de77" + integrity sha512-/8INDn0YLInbe9Wt7dK4cXLDYp0fNHP5xKLHvZl3mOT5X17rK/YShXaiNmorl+/U4VKCVIjJnx4Ri5b0y+HClw== dependencies: - "@typescript-eslint/types" "5.51.0" - "@typescript-eslint/visitor-keys" "5.51.0" + "@typescript-eslint/types" "7.7.0" + "@typescript-eslint/visitor-keys" "7.7.0" -"@typescript-eslint/type-utils@5.51.0": - version "5.51.0" - resolved "https://registry.yarnpkg.com/@typescript-eslint/type-utils/-/type-utils-5.51.0.tgz#7af48005531700b62a20963501d47dfb27095988" - integrity sha512-QHC5KKyfV8sNSyHqfNa0UbTbJ6caB8uhcx2hYcWVvJAZYJRBo5HyyZfzMdRx8nvS+GyMg56fugMzzWnojREuQQ== +"@typescript-eslint/type-utils@7.7.0": + version "7.7.0" + resolved "https://registry.yarnpkg.com/@typescript-eslint/type-utils/-/type-utils-7.7.0.tgz#36792ff4209a781b058de61631a48df17bdefbc5" + integrity sha512-bOp3ejoRYrhAlnT/bozNQi3nio9tIgv3U5C0mVDdZC7cpcQEDZXvq8inrHYghLVwuNABRqrMW5tzAv88Vy77Sg== dependencies: - "@typescript-eslint/typescript-estree" "5.51.0" - "@typescript-eslint/utils" "5.51.0" + "@typescript-eslint/typescript-estree" "7.7.0" + "@typescript-eslint/utils" "7.7.0" debug "^4.3.4" - tsutils "^3.21.0" + ts-api-utils "^1.3.0" -"@typescript-eslint/types@5.51.0": - version "5.51.0" - resolved "https://registry.yarnpkg.com/@typescript-eslint/types/-/types-5.51.0.tgz#e7c1622f46c7eea7e12bbf1edfb496d4dec37c90" - integrity sha512-SqOn0ANn/v6hFn0kjvLwiDi4AzR++CBZz0NV5AnusT2/3y32jdc0G4woXPWHCumWtUXZKPAS27/9vziSsC9jnw== +"@typescript-eslint/types@7.7.0": + version "7.7.0" + resolved "https://registry.yarnpkg.com/@typescript-eslint/types/-/types-7.7.0.tgz#23af4d24bf9ce15d8d301236e3e3014143604f27" + integrity sha512-G01YPZ1Bd2hn+KPpIbrAhEWOn5lQBrjxkzHkWvP6NucMXFtfXoevK82hzQdpfuQYuhkvFDeQYbzXCjR1z9Z03w== -"@typescript-eslint/typescript-estree@5.51.0": - version "5.51.0" - resolved "https://registry.yarnpkg.com/@typescript-eslint/typescript-estree/-/typescript-estree-5.51.0.tgz#0ec8170d7247a892c2b21845b06c11eb0718f8de" - integrity sha512-TSkNupHvNRkoH9FMA3w7TazVFcBPveAAmb7Sz+kArY6sLT86PA5Vx80cKlYmd8m3Ha2SwofM1KwraF24lM9FvA== +"@typescript-eslint/typescript-estree@7.7.0": + version "7.7.0" + resolved "https://registry.yarnpkg.com/@typescript-eslint/typescript-estree/-/typescript-estree-7.7.0.tgz#b5dd6383b4c6a852d7b256a37af971e8982be97f" + integrity sha512-8p71HQPE6CbxIBy2kWHqM1KGrC07pk6RJn40n0DSc6bMOBBREZxSDJ+BmRzc8B5OdaMh1ty3mkuWRg4sCFiDQQ== dependencies: - "@typescript-eslint/types" "5.51.0" - "@typescript-eslint/visitor-keys" "5.51.0" + "@typescript-eslint/types" "7.7.0" + "@typescript-eslint/visitor-keys" "7.7.0" debug "^4.3.4" globby "^11.1.0" is-glob "^4.0.3" - semver "^7.3.7" - tsutils "^3.21.0" - -"@typescript-eslint/utils@5.51.0": - version "5.51.0" - resolved "https://registry.yarnpkg.com/@typescript-eslint/utils/-/utils-5.51.0.tgz#074f4fabd5b12afe9c8aa6fdee881c050f8b4d47" - integrity sha512-76qs+5KWcaatmwtwsDJvBk4H76RJQBFe+Gext0EfJdC3Vd2kpY2Pf//OHHzHp84Ciw0/rYoGTDnIAr3uWhhJYw== - dependencies: - "@types/json-schema" "^7.0.9" - "@types/semver" "^7.3.12" - "@typescript-eslint/scope-manager" "5.51.0" - "@typescript-eslint/types" "5.51.0" - "@typescript-eslint/typescript-estree" "5.51.0" - eslint-scope "^5.1.1" - eslint-utils "^3.0.0" - semver "^7.3.7" - -"@typescript-eslint/visitor-keys@5.51.0": - version "5.51.0" - resolved "https://registry.yarnpkg.com/@typescript-eslint/visitor-keys/-/visitor-keys-5.51.0.tgz#c0147dd9a36c0de758aaebd5b48cae1ec59eba87" - integrity sha512-Oh2+eTdjHjOFjKA27sxESlA87YPSOJafGCR0md5oeMdh1ZcCfAGCIOL216uTBAkAIptvLIfKQhl7lHxMJet4GQ== - dependencies: - "@typescript-eslint/types" "5.51.0" - eslint-visitor-keys "^3.3.0" + minimatch "^9.0.4" + semver "^7.6.0" + ts-api-utils "^1.3.0" + +"@typescript-eslint/utils@7.7.0": + version "7.7.0" + resolved "https://registry.yarnpkg.com/@typescript-eslint/utils/-/utils-7.7.0.tgz#3d2b6606a60ac34f3c625facfb3b3ab7e126f58d" + integrity sha512-LKGAXMPQs8U/zMRFXDZOzmMKgFv3COlxUQ+2NMPhbqgVm6R1w+nU1i4836Pmxu9jZAuIeyySNrN/6Rc657ggig== + dependencies: + "@eslint-community/eslint-utils" "^4.4.0" + "@types/json-schema" "^7.0.15" + "@types/semver" "^7.5.8" + "@typescript-eslint/scope-manager" "7.7.0" + "@typescript-eslint/types" "7.7.0" + "@typescript-eslint/typescript-estree" "7.7.0" + semver "^7.6.0" + +"@typescript-eslint/visitor-keys@7.7.0": + version "7.7.0" + resolved "https://registry.yarnpkg.com/@typescript-eslint/visitor-keys/-/visitor-keys-7.7.0.tgz#950148cf1ac11562a2d903fdf7acf76714a2dc9e" + integrity sha512-h0WHOj8MhdhY8YWkzIF30R379y0NqyOHExI9N9KCzvmu05EgG4FumeYa3ccfKUSphyWkWQE1ybVrgz/Pbam6YA== + dependencies: + "@typescript-eslint/types" "7.7.0" + eslint-visitor-keys "^3.4.3" + +"@ungap/structured-clone@^1.2.0": + version "1.2.0" + resolved "https://registry.yarnpkg.com/@ungap/structured-clone/-/structured-clone-1.2.0.tgz#756641adb587851b5ccb3e095daf27ae581c8406" + integrity sha512-zuVdFrMJiuCDQUMCzQaD6KL28MjnqqN8XnAqiEq9PNm/hCPTSGfrXCOfwj1ow4LFb/tNymJPwsNbVePc1xFqrQ== -"@vitejs/plugin-vue@4.0.0": - version "4.0.0" - resolved "https://registry.yarnpkg.com/@vitejs/plugin-vue/-/plugin-vue-4.0.0.tgz#93815beffd23db46288c787352a8ea31a0c03e5e" - integrity sha512-e0X4jErIxAB5oLtDqbHvHpJe/uWNkdpYV83AOG2xo2tEVSzCzewgJMtREZM30wXnM5ls90hxiOtAuVU6H5JgbA== - -"@vitest/expect@0.31.4": - version "0.31.4" - resolved "https://registry.yarnpkg.com/@vitest/expect/-/expect-0.31.4.tgz#115c517404488bf3cb6ce4ac411c40d8e86891b8" - integrity sha512-tibyx8o7GUyGHZGyPgzwiaPaLDQ9MMuCOrc03BYT0nryUuhLbL7NV2r/q98iv5STlwMgaKuFJkgBW/8iPKwlSg== - dependencies: - "@vitest/spy" "0.31.4" - "@vitest/utils" "0.31.4" - chai "^4.3.7" - -"@vitest/runner@0.31.4": - version "0.31.4" - resolved "https://registry.yarnpkg.com/@vitest/runner/-/runner-0.31.4.tgz#e99abee89132a500d9726a53b58dfc9160db1078" - integrity sha512-Wgm6UER+gwq6zkyrm5/wbpXGF+g+UBB78asJlFkIOwyse0pz8lZoiC6SW5i4gPnls/zUcPLWS7Zog0LVepXnpg== - dependencies: - "@vitest/utils" "0.31.4" - concordance "^5.0.4" - p-limit "^4.0.0" - pathe "^1.1.0" +"@vitejs/plugin-vue@5.0.4": + version "5.0.4" + resolved "https://registry.yarnpkg.com/@vitejs/plugin-vue/-/plugin-vue-5.0.4.tgz#508d6a0f2440f86945835d903fcc0d95d1bb8a37" + integrity sha512-WS3hevEszI6CEVEx28F8RjTX97k3KsrcY6kvTg7+Whm5y3oYvcqzVeGCU3hxSAn4uY2CLCkeokkGKpoctccilQ== -"@vitest/snapshot@0.31.4": - version "0.31.4" - resolved "https://registry.yarnpkg.com/@vitest/snapshot/-/snapshot-0.31.4.tgz#59a42046fec4950a1ac70cf0ec64aada3b995559" - integrity sha512-LemvNumL3NdWSmfVAMpXILGyaXPkZbG5tyl6+RQSdcHnTj6hvA49UAI8jzez9oQyE/FWLKRSNqTGzsHuk89LRA== +"@vitest/expect@1.5.0": + version "1.5.0" + resolved "https://registry.yarnpkg.com/@vitest/expect/-/expect-1.5.0.tgz#961190510a2723bd4abf5540bcec0a4dfd59ef14" + integrity sha512-0pzuCI6KYi2SIC3LQezmxujU9RK/vwC1U9R0rLuGlNGcOuDWxqWKu6nUdFsX9tH1WU0SXtAxToOsEjeUn1s3hA== dependencies: - magic-string "^0.30.0" - pathe "^1.1.0" - pretty-format "^27.5.1" + "@vitest/spy" "1.5.0" + "@vitest/utils" "1.5.0" + chai "^4.3.10" -"@vitest/spy@0.31.4": - version "0.31.4" - resolved "https://registry.yarnpkg.com/@vitest/spy/-/spy-0.31.4.tgz#fce8e348cea32deff79996d116c67893b19cc47d" - integrity sha512-3ei5ZH1s3aqbEyftPAzSuunGICRuhE+IXOmpURFdkm5ybUADk+viyQfejNk6q8M5QGX8/EVKw+QWMEP3DTJDag== +"@vitest/runner@1.5.0": + version "1.5.0" + resolved "https://registry.yarnpkg.com/@vitest/runner/-/runner-1.5.0.tgz#1f7cb78ee4064e73e53d503a19c1b211c03dfe0c" + integrity sha512-7HWwdxXP5yDoe7DTpbif9l6ZmDwCzcSIK38kTSIt6CFEpMjX4EpCgT6wUmS0xTXqMI6E/ONmfgRKmaujpabjZQ== dependencies: - tinyspy "^2.1.0" + "@vitest/utils" "1.5.0" + p-limit "^5.0.0" + pathe "^1.1.1" -"@vitest/utils@0.31.4": - version "0.31.4" - resolved "https://registry.yarnpkg.com/@vitest/utils/-/utils-0.31.4.tgz#5cfdcecfd604a7dbe3972cfe0f2b1e0af1246ad2" - integrity sha512-DobZbHacWznoGUfYU8XDPY78UubJxXfMNY1+SUdOp1NsI34eopSA6aZMeaGu10waSOeYwE8lxrd/pLfT0RMxjQ== +"@vitest/snapshot@1.5.0": + version "1.5.0" + resolved "https://registry.yarnpkg.com/@vitest/snapshot/-/snapshot-1.5.0.tgz#cd2d611fd556968ce8fb6b356a09b4593c525947" + integrity sha512-qpv3fSEuNrhAO3FpH6YYRdaECnnRjg9VxbhdtPwPRnzSfHVXnNzzrpX4cJxqiwgRMo7uRMWDFBlsBq4Cr+rO3A== dependencies: - concordance "^5.0.4" - loupe "^2.3.6" - pretty-format "^27.5.1" - -"@volar/language-core@1.0.24": - version "1.0.24" - resolved "https://registry.yarnpkg.com/@volar/language-core/-/language-core-1.0.24.tgz#5d767571e77728464635e61af1debca944811fe0" - integrity sha512-vTN+alJiWwK0Pax6POqrmevbtFW2dXhjwWiW/MW4f48eDYPLdyURWcr8TixO7EN/nHsUBj2udT7igFKPtjyAKg== - dependencies: - "@volar/source-map" "1.0.24" - muggle-string "^0.1.0" - -"@volar/source-map@1.0.24": - version "1.0.24" - resolved "https://registry.yarnpkg.com/@volar/source-map/-/source-map-1.0.24.tgz#ad4c827fea5c26b4bf38a86d983e7deb65b1c61e" - integrity sha512-Qsv/tkplx18pgBr8lKAbM1vcDqgkGKQzbChg6NW+v0CZc3G7FLmK+WrqEPzKlN7Cwdc6XVL559Nod8WKAfKr4A== - dependencies: - muggle-string "^0.1.0" - -"@volar/typescript@1.0.24": - version "1.0.24" - resolved "https://registry.yarnpkg.com/@volar/typescript/-/typescript-1.0.24.tgz#f934eda9774b31abdff53efc56782cd2623723d5" - integrity sha512-f8hCSk+PfKR1/RQHxZ79V1NpDImHoivqoizK+mstphm25tn/YJ/JnKNjZHB+o21fuW0yKlI26NV3jkVb2Cc/7A== - dependencies: - "@volar/language-core" "1.0.24" - -"@volar/vue-language-core@1.0.24": - version "1.0.24" - resolved "https://registry.yarnpkg.com/@volar/vue-language-core/-/vue-language-core-1.0.24.tgz#81d180a8e09a53cb575e83acb79a31493891a1a4" - integrity sha512-2NTJzSgrwKu6uYwPqLiTMuAzi7fAY3yFy5PJ255bGJc82If0Xr+cW8pC80vpjG0D/aVLmlwAdO4+Ya2BI8GdDg== - dependencies: - "@volar/language-core" "1.0.24" - "@volar/source-map" "1.0.24" - "@vue/compiler-dom" "^3.2.45" - "@vue/compiler-sfc" "^3.2.45" - "@vue/reactivity" "^3.2.45" - "@vue/shared" "^3.2.45" - minimatch "^5.1.1" - vue-template-compiler "^2.7.14" + magic-string "^0.30.5" + pathe "^1.1.1" + pretty-format "^29.7.0" -"@volar/vue-typescript@1.0.24": - version "1.0.24" - resolved "https://registry.yarnpkg.com/@volar/vue-typescript/-/vue-typescript-1.0.24.tgz#bef9b2bfb1b108c0f6cb12ec6fbf449b43fc8257" - integrity sha512-9a25oHDvGaNC0okRS47uqJI6FxY4hUQZUsxeOUFHcqVxZEv8s17LPuP/pMMXyz7jPygrZubB/qXqHY5jEu/akA== +"@vitest/spy@1.5.0": + version "1.5.0" + resolved "https://registry.yarnpkg.com/@vitest/spy/-/spy-1.5.0.tgz#1369a1bec47f46f18eccfa45f1e8fbb9b5e15e77" + integrity sha512-vu6vi6ew5N5MMHJjD5PoakMRKYdmIrNJmyfkhRpQt5d9Ewhw9nZ5Aqynbi3N61bvk9UvZ5UysMT6ayIrZ8GA9w== dependencies: - "@volar/typescript" "1.0.24" - "@volar/vue-language-core" "1.0.24" + tinyspy "^2.2.0" -"@vue/compiler-core@3.2.47": - version "3.2.47" - resolved "https://registry.yarnpkg.com/@vue/compiler-core/-/compiler-core-3.2.47.tgz#3e07c684d74897ac9aa5922c520741f3029267f8" - integrity sha512-p4D7FDnQb7+YJmO2iPEv0SQNeNzcbHdGByJDsT4lynf63AFkOTFN07HsiRSvjGo0QrxR/o3d0hUyNCUnBU2Tig== +"@vitest/utils@1.5.0": + version "1.5.0" + resolved "https://registry.yarnpkg.com/@vitest/utils/-/utils-1.5.0.tgz#90c9951f4516f6d595da24876b58e615f6c99863" + integrity sha512-BDU0GNL8MWkRkSRdNFvCUCAVOeHaUlVJ9Tx0TYBZyXaaOTmGtUFObzchCivIBrIwKzvZA7A9sCejVhXM2aY98A== dependencies: - "@babel/parser" "^7.16.4" - "@vue/shared" "3.2.47" - estree-walker "^2.0.2" - source-map "^0.6.1" + diff-sequences "^29.6.3" + estree-walker "^3.0.3" + loupe "^2.3.7" + pretty-format "^29.7.0" -"@vue/compiler-core@3.3.4": - version "3.3.4" - resolved "https://registry.yarnpkg.com/@vue/compiler-core/-/compiler-core-3.3.4.tgz#7fbf591c1c19e1acd28ffd284526e98b4f581128" - integrity sha512-cquyDNvZ6jTbf/+x+AgM2Arrp6G4Dzbb0R64jiG804HRMfRiFXWI6kqUVqZ6ZR0bQhIoQjB4+2bhNtVwndW15g== +"@volar/language-core@2.2.0-alpha.8": + version "2.2.0-alpha.8" + resolved "https://registry.yarnpkg.com/@volar/language-core/-/language-core-2.2.0-alpha.8.tgz#74120a27ff2498ad297e86d17be95a9c7e1b46f5" + integrity sha512-Ew1Iw7/RIRNuDLn60fWJdOLApAlfTVPxbPiSLzc434PReC9kleYtaa//Wo2WlN1oiRqneW0pWQQV0CwYqaimLQ== dependencies: - "@babel/parser" "^7.21.3" - "@vue/shared" "3.3.4" - estree-walker "^2.0.2" - source-map-js "^1.0.2" + "@volar/source-map" "2.2.0-alpha.8" -"@vue/compiler-dom@3.2.47", "@vue/compiler-dom@^3.0.1", "@vue/compiler-dom@^3.2.45": - version "3.2.47" - resolved "https://registry.yarnpkg.com/@vue/compiler-dom/-/compiler-dom-3.2.47.tgz#a0b06caf7ef7056939e563dcaa9cbde30794f305" - integrity sha512-dBBnEHEPoftUiS03a4ggEig74J2YBZ2UIeyfpcRM2tavgMWo4bsEfgCGsu+uJIL/vax9S+JztH8NmQerUo7shQ== +"@volar/source-map@2.2.0-alpha.8": + version "2.2.0-alpha.8" + resolved "https://registry.yarnpkg.com/@volar/source-map/-/source-map-2.2.0-alpha.8.tgz#ca090f828fbef7e09ea06a636c41a06aa2afe153" + integrity sha512-E1ZVmXFJ5DU4fWDcWHzi8OLqqReqIDwhXvIMhVdk6+VipfMVv4SkryXu7/rs4GA/GsebcRyJdaSkKBB3OAkIcA== dependencies: - "@vue/compiler-core" "3.2.47" - "@vue/shared" "3.2.47" + muggle-string "^0.4.0" -"@vue/compiler-dom@3.3.4": - version "3.3.4" - resolved "https://registry.yarnpkg.com/@vue/compiler-dom/-/compiler-dom-3.3.4.tgz#f56e09b5f4d7dc350f981784de9713d823341151" - integrity sha512-wyM+OjOVpuUukIq6p5+nwHYtj9cFroz9cwkfmP9O1nzH68BenTTv0u7/ndggT8cIQlnBeOo6sUT/gvHcIkLA5w== +"@volar/typescript@2.2.0-alpha.8": + version "2.2.0-alpha.8" + resolved "https://registry.yarnpkg.com/@volar/typescript/-/typescript-2.2.0-alpha.8.tgz#83a056c52995b4142364be3dda41d955a96f7356" + integrity sha512-RLbRDI+17CiayHZs9HhSzlH0FhLl/+XK6o2qoiw2o2GGKcyD1aDoY6AcMd44acYncTOrqoTNoY6LuCiRyiJiGg== dependencies: - "@vue/compiler-core" "3.3.4" - "@vue/shared" "3.3.4" + "@volar/language-core" "2.2.0-alpha.8" + path-browserify "^1.0.1" -"@vue/compiler-sfc@3.3.4": - version "3.3.4" - resolved "https://registry.yarnpkg.com/@vue/compiler-sfc/-/compiler-sfc-3.3.4.tgz#b19d942c71938893535b46226d602720593001df" - integrity sha512-6y/d8uw+5TkCuzBkgLS0v3lSM3hJDntFEiUORM11pQ/hKvkhSKZrXW6i69UyXlJQisJxuUEJKAWEqWbWsLeNKQ== - dependencies: - "@babel/parser" "^7.20.15" - "@vue/compiler-core" "3.3.4" - "@vue/compiler-dom" "3.3.4" - "@vue/compiler-ssr" "3.3.4" - "@vue/reactivity-transform" "3.3.4" - "@vue/shared" "3.3.4" +"@vue/compiler-core@3.4.23": + version "3.4.23" + resolved "https://registry.yarnpkg.com/@vue/compiler-core/-/compiler-core-3.4.23.tgz#a08f5998e391ad75e602a66dd7255af9054df2f3" + integrity sha512-HAFmuVEwNqNdmk+w4VCQ2pkLk1Vw4XYiiyxEp3z/xvl14aLTUBw2OfVH3vBcx+FtGsynQLkkhK410Nah1N2yyQ== + dependencies: + "@babel/parser" "^7.24.1" + "@vue/shared" "3.4.23" + entities "^4.5.0" estree-walker "^2.0.2" - magic-string "^0.30.0" - postcss "^8.1.10" - source-map-js "^1.0.2" - -"@vue/compiler-sfc@^3.2.45": - version "3.2.47" - resolved "https://registry.yarnpkg.com/@vue/compiler-sfc/-/compiler-sfc-3.2.47.tgz#1bdc36f6cdc1643f72e2c397eb1a398f5004ad3d" - integrity sha512-rog05W+2IFfxjMcFw10tM9+f7i/+FFpZJJ5XHX72NP9eC2uRD+42M3pYcQqDXVYoj74kHMSEdQ/WmCjt8JFksQ== - dependencies: - "@babel/parser" "^7.16.4" - "@vue/compiler-core" "3.2.47" - "@vue/compiler-dom" "3.2.47" - "@vue/compiler-ssr" "3.2.47" - "@vue/reactivity-transform" "3.2.47" - "@vue/shared" "3.2.47" + source-map-js "^1.2.0" + +"@vue/compiler-dom@3.4.23", "@vue/compiler-dom@^3.4.0": + version "3.4.23" + resolved "https://registry.yarnpkg.com/@vue/compiler-dom/-/compiler-dom-3.4.23.tgz#6fa622d1e5c8508551564c5dc5948e9cddf60867" + integrity sha512-t0b9WSTnCRrzsBGrDd1LNR5HGzYTr7LX3z6nNBG+KGvZLqrT0mY6NsMzOqlVMBKKXKVuusbbB5aOOFgTY+senw== + dependencies: + "@vue/compiler-core" "3.4.23" + "@vue/shared" "3.4.23" + +"@vue/compiler-sfc@3.4.23": + version "3.4.23" + resolved "https://registry.yarnpkg.com/@vue/compiler-sfc/-/compiler-sfc-3.4.23.tgz#7041517b9bbd1b304f0db33bfa424e9a899fda8d" + integrity sha512-fSDTKTfzaRX1kNAUiaj8JB4AokikzStWgHooMhaxyjZerw624L+IAP/fvI4ZwMpwIh8f08PVzEnu4rg8/Npssw== + dependencies: + "@babel/parser" "^7.24.1" + "@vue/compiler-core" "3.4.23" + "@vue/compiler-dom" "3.4.23" + "@vue/compiler-ssr" "3.4.23" + "@vue/shared" "3.4.23" estree-walker "^2.0.2" - magic-string "^0.25.7" - postcss "^8.1.10" - source-map "^0.6.1" + magic-string "^0.30.8" + postcss "^8.4.38" + source-map-js "^1.2.0" -"@vue/compiler-ssr@3.2.47": - version "3.2.47" - resolved "https://registry.yarnpkg.com/@vue/compiler-ssr/-/compiler-ssr-3.2.47.tgz#35872c01a273aac4d6070ab9d8da918ab13057ee" - integrity sha512-wVXC+gszhulcMD8wpxMsqSOpvDZ6xKXSVWkf50Guf/S+28hTAXPDYRTbLQ3EDkOP5Xz/+SY37YiwDquKbJOgZw== +"@vue/compiler-ssr@3.4.23": + version "3.4.23" + resolved "https://registry.yarnpkg.com/@vue/compiler-ssr/-/compiler-ssr-3.4.23.tgz#1ae4afe962a9e156b1a79eff909c37cd423dd4c2" + integrity sha512-hb6Uj2cYs+tfqz71Wj6h3E5t6OKvb4MVcM2Nl5i/z1nv1gjEhw+zYaNOV+Xwn+SSN/VZM0DgANw5TuJfxfezPg== dependencies: - "@vue/compiler-dom" "3.2.47" - "@vue/shared" "3.2.47" + "@vue/compiler-dom" "3.4.23" + "@vue/shared" "3.4.23" -"@vue/compiler-ssr@3.3.4": - version "3.3.4" - resolved "https://registry.yarnpkg.com/@vue/compiler-ssr/-/compiler-ssr-3.3.4.tgz#9d1379abffa4f2b0cd844174ceec4a9721138777" - integrity sha512-m0v6oKpup2nMSehwA6Uuu+j+wEwcy7QmwMkVNVfrV9P2qE5KshC6RwOCq8fjGS/Eak/uNb8AaWekfiXxbBB6gQ== - dependencies: - "@vue/compiler-dom" "3.3.4" - "@vue/shared" "3.3.4" - -"@vue/devtools-api@^6.2.1", "@vue/devtools-api@^6.5.0": +"@vue/devtools-api@^6.5.0": version "6.5.0" resolved "https://registry.yarnpkg.com/@vue/devtools-api/-/devtools-api-6.5.0.tgz#98b99425edee70b4c992692628fa1ea2c1e57d07" integrity sha512-o9KfBeaBmCKl10usN4crU53fYtC1r7jJwdGKjPT24t348rHxgfpZ0xL3Xm/gLUYnc0oTp8LAmrxOeLyu6tbk2Q== -"@vue/eslint-config-prettier@7.0.0": - version "7.0.0" - resolved "https://registry.yarnpkg.com/@vue/eslint-config-prettier/-/eslint-config-prettier-7.0.0.tgz#44ab55ca22401102b57795c59428e9dade72be34" - integrity sha512-/CTc6ML3Wta1tCe1gUeO0EYnVXfo3nJXsIhZ8WJr3sov+cGASr6yuiibJTL6lmIBm7GobopToOuB3B6AWyV0Iw== - dependencies: - eslint-config-prettier "^8.3.0" - eslint-plugin-prettier "^4.0.0" - -"@vue/eslint-config-typescript@11.0.2": - version "11.0.2" - resolved "https://registry.yarnpkg.com/@vue/eslint-config-typescript/-/eslint-config-typescript-11.0.2.tgz#03353f404d4472900794e653450bb6623de3c642" - integrity sha512-EiKud1NqlWmSapBFkeSrE994qpKx7/27uCGnhdqzllYDpQZroyX/O6bwjEpeuyKamvLbsGdO6PMR2faIf+zFnw== - dependencies: - "@typescript-eslint/eslint-plugin" "^5.0.0" - "@typescript-eslint/parser" "^5.0.0" - vue-eslint-parser "^9.0.0" - -"@vue/reactivity-transform@3.2.47": - version "3.2.47" - resolved "https://registry.yarnpkg.com/@vue/reactivity-transform/-/reactivity-transform-3.2.47.tgz#e45df4d06370f8abf29081a16afd25cffba6d84e" - integrity sha512-m8lGXw8rdnPVVIdIFhf0LeQ/ixyHkH5plYuS83yop5n7ggVJU+z5v0zecwEnX7fa7HNLBhh2qngJJkxpwEEmYA== - dependencies: - "@babel/parser" "^7.16.4" - "@vue/compiler-core" "3.2.47" - "@vue/shared" "3.2.47" - estree-walker "^2.0.2" - magic-string "^0.25.7" +"@vue/devtools-api@^6.5.1": + version "6.6.1" + resolved "https://registry.yarnpkg.com/@vue/devtools-api/-/devtools-api-6.6.1.tgz#7c14346383751d9f6ad4bea0963245b30220ef83" + integrity sha512-LgPscpE3Vs0x96PzSSB4IGVSZXZBZHpfxs+ZA1d+VEPwHdOXowy/Y2CsvCAIFrf+ssVU1pD1jidj505EpUnfbA== -"@vue/reactivity-transform@3.3.4": - version "3.3.4" - resolved "https://registry.yarnpkg.com/@vue/reactivity-transform/-/reactivity-transform-3.3.4.tgz#52908476e34d6a65c6c21cd2722d41ed8ae51929" - integrity sha512-MXgwjako4nu5WFLAjpBnCj/ieqcjE2aJBINUNQzkZQfzIZA4xn+0fV1tIYBJvvva3N3OvKGofRLvQIwEQPpaXw== - dependencies: - "@babel/parser" "^7.20.15" - "@vue/compiler-core" "3.3.4" - "@vue/shared" "3.3.4" - estree-walker "^2.0.2" - magic-string "^0.30.0" +"@vue/eslint-config-prettier@9.0.0": + version "9.0.0" + resolved "https://registry.yarnpkg.com/@vue/eslint-config-prettier/-/eslint-config-prettier-9.0.0.tgz#f63394f8f7759d92b6ef3f3e1d30ff6b0c0b97c1" + integrity sha512-z1ZIAAUS9pKzo/ANEfd2sO+v2IUalz7cM/cTLOZ7vRFOPk5/xuRKQteOu1DErFLAh/lYGXMVZ0IfYKlyInuDVg== + dependencies: + eslint-config-prettier "^9.0.0" + eslint-plugin-prettier "^5.0.0" + +"@vue/eslint-config-typescript@13.0.0": + version "13.0.0" + resolved "https://registry.yarnpkg.com/@vue/eslint-config-typescript/-/eslint-config-typescript-13.0.0.tgz#f5f3d986ace34a10f403921d5044831b89a1b679" + integrity sha512-MHh9SncG/sfqjVqjcuFLOLD6Ed4dRAis4HNt0dXASeAuLqIAx4YMB1/m2o4pUKK1vCt8fUvYG8KKX2Ot3BVZTg== + dependencies: + "@typescript-eslint/eslint-plugin" "^7.1.1" + "@typescript-eslint/parser" "^7.1.1" + vue-eslint-parser "^9.3.1" + +"@vue/language-core@2.0.13": + version "2.0.13" + resolved "https://registry.yarnpkg.com/@vue/language-core/-/language-core-2.0.13.tgz#2d1638b882011187b4b57115425d52b0901acab5" + integrity sha512-oQgM+BM66SU5GKtUMLQSQN0bxHFkFpLSSAiY87wVziPaiNQZuKVDt/3yA7GB9PiQw0y/bTNL0bOc0jM/siYjKg== + dependencies: + "@volar/language-core" "2.2.0-alpha.8" + "@vue/compiler-dom" "^3.4.0" + "@vue/shared" "^3.4.0" + computeds "^0.0.1" + minimatch "^9.0.3" + path-browserify "^1.0.1" + vue-template-compiler "^2.7.14" -"@vue/reactivity@3.3.4": - version "3.3.4" - resolved "https://registry.yarnpkg.com/@vue/reactivity/-/reactivity-3.3.4.tgz#a27a29c6cd17faba5a0e99fbb86ee951653e2253" - integrity sha512-kLTDLwd0B1jG08NBF3R5rqULtv/f8x3rOFByTDz4J53ttIQEDmALqKqXY0J+XQeN0aV2FBxY8nJDf88yvOPAqQ== +"@vue/reactivity@3.4.23": + version "3.4.23" + resolved "https://registry.yarnpkg.com/@vue/reactivity/-/reactivity-3.4.23.tgz#f29044a90a24994af075e4368790f31fa29ed747" + integrity sha512-GlXR9PL+23fQ3IqnbSQ8OQKLodjqCyoCrmdLKZk3BP7jN6prWheAfU7a3mrltewTkoBm+N7qMEb372VHIkQRMQ== dependencies: - "@vue/shared" "3.3.4" + "@vue/shared" "3.4.23" -"@vue/reactivity@^3.2.45": - version "3.2.47" - resolved "https://registry.yarnpkg.com/@vue/reactivity/-/reactivity-3.2.47.tgz#1d6399074eadfc3ed35c727e2fd707d6881140b6" - integrity sha512-7khqQ/75oyyg+N/e+iwV6lpy1f5wq759NdlS1fpAhFXa8VeAIKGgk2E/C4VF59lx5b+Ezs5fpp/5WsRYXQiKxQ== +"@vue/runtime-core@3.4.23": + version "3.4.23" + resolved "https://registry.yarnpkg.com/@vue/runtime-core/-/runtime-core-3.4.23.tgz#21f6c7153c33f56081145c2819c2f194cbe3eb22" + integrity sha512-FeQ9MZEXoFzFkFiw9MQQ/FWs3srvrP+SjDKSeRIiQHIhtkzoj0X4rWQlRNHbGuSwLra6pMyjAttwixNMjc/xLw== dependencies: - "@vue/shared" "3.2.47" + "@vue/reactivity" "3.4.23" + "@vue/shared" "3.4.23" -"@vue/runtime-core@3.3.4": - version "3.3.4" - resolved "https://registry.yarnpkg.com/@vue/runtime-core/-/runtime-core-3.3.4.tgz#4bb33872bbb583721b340f3088888394195967d1" - integrity sha512-R+bqxMN6pWO7zGI4OMlmvePOdP2c93GsHFM/siJI7O2nxFRzj55pLwkpCedEY+bTMgp5miZ8CxfIZo3S+gFqvA== +"@vue/runtime-dom@3.4.23": + version "3.4.23" + resolved "https://registry.yarnpkg.com/@vue/runtime-dom/-/runtime-dom-3.4.23.tgz#d5a9cca88b445de125c57e6b0d73faa2491e853f" + integrity sha512-RXJFwwykZWBkMiTPSLEWU3kgVLNAfActBfWFlZd0y79FTUxexogd0PLG4HH2LfOktjRxV47Nulygh0JFXe5f9A== dependencies: - "@vue/reactivity" "3.3.4" - "@vue/shared" "3.3.4" + "@vue/runtime-core" "3.4.23" + "@vue/shared" "3.4.23" + csstype "^3.1.3" -"@vue/runtime-dom@3.3.4": - version "3.3.4" - resolved "https://registry.yarnpkg.com/@vue/runtime-dom/-/runtime-dom-3.3.4.tgz#992f2579d0ed6ce961f47bbe9bfe4b6791251566" - integrity sha512-Aj5bTJ3u5sFsUckRghsNjVTtxZQ1OyMWCr5dZRAPijF/0Vy4xEoRCwLyHXcj4D0UFbJ4lbx3gPTgg06K/GnPnQ== +"@vue/server-renderer@3.4.23": + version "3.4.23" + resolved "https://registry.yarnpkg.com/@vue/server-renderer/-/server-renderer-3.4.23.tgz#e605872e26d995f5ba9382e8758cd8cc7fa2e16d" + integrity sha512-LDwGHtnIzvKFNS8dPJ1SSU5Gvm36p2ck8wCZc52fc3k/IfjKcwCyrWEf0Yag/2wTFUBXrqizfhK9c/mC367dXQ== dependencies: - "@vue/runtime-core" "3.3.4" - "@vue/shared" "3.3.4" - csstype "^3.1.1" + "@vue/compiler-ssr" "3.4.23" + "@vue/shared" "3.4.23" -"@vue/server-renderer@3.3.4", "@vue/server-renderer@^3.0.1": - version "3.3.4" - resolved "https://registry.yarnpkg.com/@vue/server-renderer/-/server-renderer-3.3.4.tgz#ea46594b795d1536f29bc592dd0f6655f7ea4c4c" - integrity sha512-Q6jDDzR23ViIb67v+vM1Dqntu+HUexQcsWKhhQa4ARVzxOY2HbC7QRW/ggkDBd5BU+uM1sV6XOAP0b216o34JQ== - dependencies: - "@vue/compiler-ssr" "3.3.4" - "@vue/shared" "3.3.4" - -"@vue/shared@3.2.47", "@vue/shared@^3.2.45": - version "3.2.47" - resolved "https://registry.yarnpkg.com/@vue/shared/-/shared-3.2.47.tgz#e597ef75086c6e896ff5478a6bfc0a7aa4bbd14c" - integrity sha512-BHGyyGN3Q97EZx0taMQ+OLNuZcW3d37ZEVmEAyeoA9ERdGvm9Irc/0Fua8SNyOtV1w6BS4q25wbMzJujO9HIfQ== - -"@vue/shared@3.3.4": - version "3.3.4" - resolved "https://registry.yarnpkg.com/@vue/shared/-/shared-3.3.4.tgz#06e83c5027f464eef861c329be81454bc8b70780" - integrity sha512-7OjdcV8vQ74eiz1TZLzZP4JwqM5fA94K6yntPS5Z25r9HDuGNzaGdgvwKYq6S+MxwF0TFRwe50fIR/MYnakdkQ== +"@vue/shared@3.4.23", "@vue/shared@^3.4.0": + version "3.4.23" + resolved "https://registry.yarnpkg.com/@vue/shared/-/shared-3.4.23.tgz#e536a6dfd2f5f950d08c2e8ebcfe7e5329a851a1" + integrity sha512-wBQ0gvf+SMwsCQOyusNw/GoXPV47WGd1xB5A1Pgzy0sQ3Bi5r5xm3n+92y3gCnB3MWqnRDdvfkRGxhKtbBRNgg== -"@vue/test-utils@2.3.2", "@vue/test-utils@^2.3.1": - version "2.3.2" - resolved "https://registry.yarnpkg.com/@vue/test-utils/-/test-utils-2.3.2.tgz#b47ce6d7be7bf9700e0bb4322410e4cd8724d277" - integrity sha512-hJnVaYhbrIm0yBS0+e1Y0Sj85cMyAi+PAbK4JHqMRUZ6S622Goa+G7QzkRSyvCteG8wop7tipuEbHoZo26wsSA== +"@vue/test-utils@2.4.5", "@vue/test-utils@^2.4.1": + version "2.4.5" + resolved "https://registry.yarnpkg.com/@vue/test-utils/-/test-utils-2.4.5.tgz#010aa4debe6602d83dc75f233b397092742105a2" + integrity sha512-oo2u7vktOyKUked36R93NB7mg2B+N7Plr8lxp2JBGwr18ch6EggFjixSCdIVVLkT6Qr0z359Xvnafc9dcKyDUg== dependencies: - js-beautify "1.14.6" - optionalDependencies: - "@vue/compiler-dom" "^3.0.1" - "@vue/server-renderer" "^3.0.1" + js-beautify "^1.14.9" + vue-component-type-helpers "^2.0.0" "@vue/tsconfig@0.1.3": version "0.1.3" resolved "https://registry.yarnpkg.com/@vue/tsconfig/-/tsconfig-0.1.3.tgz#4a61dbd29783d01ddab504276dcf0c2b6988654f" integrity sha512-kQVsh8yyWPvHpb8gIc9l/HIDiiVUy1amynLNpCy8p+FoCiZXCo6fQos5/097MmnNZc9AtseDsCrfkhqCrJ8Olg== -"@vuetify/loader-shared@^1.7.1": - version "1.7.1" - resolved "https://registry.yarnpkg.com/@vuetify/loader-shared/-/loader-shared-1.7.1.tgz#0f63a3d41b6df29a2db1ff438aa1819b237c37a3" - integrity sha512-kLUvuAed6RCvkeeTNJzuy14pqnkur8lTuner7v7pNE/kVhPR97TuyXwBSBMR1cJeiLiOfu6SF5XlCYbXByEx1g== +"@vuetify/loader-shared@^2.0.3": + version "2.0.3" + resolved "https://registry.yarnpkg.com/@vuetify/loader-shared/-/loader-shared-2.0.3.tgz#11451c717e4a352ec311da52a79c857cd256c92f" + integrity sha512-Ss3GC7eJYkp2SF6xVzsT7FAruEmdihmn4OCk2+UocREerlXKWgOKKzTN5PN3ZVN5q05jHHrsNhTuWbhN61Bpdg== dependencies: - find-cache-dir "^3.3.2" upath "^2.0.1" -"@xmldom/xmldom@^0.8.3": - version "0.8.8" - resolved "https://registry.yarnpkg.com/@xmldom/xmldom/-/xmldom-0.8.8.tgz#d0d11511cbc1de77e53342ad1546a4d487d6ea72" - integrity sha512-0LNz4EY8B/8xXY86wMrQ4tz6zEHZv9ehFMJPm8u2gq5lQ71cfRKdaKyxfJAx5aUoyzx0qzgURblTisPGgz3d+Q== - -"@zxing/text-encoding@0.9.0": - version "0.9.0" - resolved "https://registry.yarnpkg.com/@zxing/text-encoding/-/text-encoding-0.9.0.tgz#fb50ffabc6c7c66a0c96b4c03e3d9be74864b70b" - integrity sha512-U/4aVJ2mxI0aDNI8Uq0wEhMgY+u4CNtEb0om3+y3+niDAsoTCOB33UF0sxpzqzdqXLqmvc+vZyAt4O8pPdfkwA== - -abab@^2.0.6: - version "2.0.6" - resolved "https://registry.yarnpkg.com/abab/-/abab-2.0.6.tgz#41b80f2c871d19686216b82309231cfd3cb3d291" - integrity sha512-j2afSsaIENvHZN2B8GOpF566vZ5WVk5opAiMTvWgaQT8DkbOqsTfvNAvHoRGU2zzP8cPoqys+xHTRDWW8L+/BA== - -abbrev@^1.0.0: - version "1.1.1" - resolved "https://registry.yarnpkg.com/abbrev/-/abbrev-1.1.1.tgz#f8f2c887ad10bf67f634f005b6987fed3179aac8" - integrity sha512-nne9/IiQ/hzIhY6pdDnbBtz7DjPTKrY00P/zvPSm5pOFkl6xuGrGnXn/VtTNNfNtAfZ9/1RtehkszU9qcTii0Q== +abbrev@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/abbrev/-/abbrev-2.0.0.tgz#cf59829b8b4f03f89dda2771cb7f3653828c89bf" + integrity sha512-6/mh1E2u2YgEsCHdY0Yx5oW+61gZU+1vXaoiHHrpKeuRNNgFvS+/jrwHiQhB5apAf5oB7UB7E19ol2R2LKH8hQ== accepts@~1.3.5, accepts@~1.3.7, accepts@~1.3.8: version "1.3.8" @@ -924,34 +916,39 @@ accepts@~1.3.5, accepts@~1.3.7, accepts@~1.3.8: mime-types "~2.1.34" negotiator "0.6.3" -ace-builds@1.28.0: - version "1.28.0" - resolved "https://registry.yarnpkg.com/ace-builds/-/ace-builds-1.28.0.tgz#b606702c33dec470393531d9808a1082add8302b" - integrity sha512-wkJp+Wz8MRHtCVdt65L/jPFLAQ0iqJZ2EeD2XWOvKGbIi4mZNwHlpHRLRB8ZnQ07VoiB0TLFWwIjjm2FL9gUcQ== +ace-builds@1.33.0: + version "1.33.0" + resolved "https://registry.yarnpkg.com/ace-builds/-/ace-builds-1.33.0.tgz#3b6fb993a4b1d3f2b5923aecded098e13e6527d8" + integrity sha512-PDvytkZNvAfuh+PaP5Oy3l3sBGd7xMk4NsB+4w/w1e3gjBqEOGeJwcX+wF/SB6mLtT3VfJLrhDNPT3eaCjtR3w== acorn-jsx@^5.3.2: version "5.3.2" resolved "https://registry.yarnpkg.com/acorn-jsx/-/acorn-jsx-5.3.2.tgz#7ed5bb55908b3b2f1bc55c6af1653bada7f07937" integrity sha512-rq9s+JNhf0IChjtDXxllJ7g41oZk5SlXtp0LHwyA5cejwn7vKmKp4pPri6YEePv2PU65sAsegbXtIinmDFDXgQ== -acorn-walk@^8.2.0: - version "8.2.0" - resolved "https://registry.yarnpkg.com/acorn-walk/-/acorn-walk-8.2.0.tgz#741210f2e2426454508853a2f44d0ab83b7f69c1" - integrity sha512-k+iyHEuPgSw6SbuDpGQM+06HQUa04DZ3o+F6CSzXMvvI5KMvnaEqXe+YVe555R9nn6GPt404fos4wcgpw12SDA== +acorn-walk@^8.3.2: + version "8.3.2" + resolved "https://registry.yarnpkg.com/acorn-walk/-/acorn-walk-8.3.2.tgz#7703af9415f1b6db9315d6895503862e231d34aa" + integrity sha512-cjkyv4OtNCIeqhHrfS81QWXoCBPExR/J62oyEqepVw8WaQeSqpW2uhuLPh1m9eWhDuOo/jUXVTlifvesOWp/4A== + +acorn@^8.11.3, acorn@^8.9.0: + version "8.11.3" + resolved "https://registry.yarnpkg.com/acorn/-/acorn-8.11.3.tgz#71e0b14e13a4ec160724b38fb7b0f233b1b81d7a" + integrity sha512-Y9rRfJG5jcKOE0CLisYbojUjIrIEE7AGMzA/Sm4BslANhbS+cDMpgBdcPT91oJ7OuJ9hYJBx59RjbhxVnrF8Xg== acorn@^8.8.0, acorn@^8.8.2: version "8.8.2" resolved "https://registry.yarnpkg.com/acorn/-/acorn-8.8.2.tgz#1b2f25db02af965399b9776b0c2c391276d37c4a" integrity sha512-xjIYgE8HBrkpd/sJqOGNspf8uHG+NOHGOw6a/Urj8taM2EXfdNAH2oFcPeIFfsv3+kz/mJrS5VuMqbNLjCa2vw== -agent-base@6: - version "6.0.2" - resolved "https://registry.yarnpkg.com/agent-base/-/agent-base-6.0.2.tgz#49fff58577cfee3f37176feab4c22e00f86d7f77" - integrity sha512-RZNwNclF7+MS/8bDg70amg32dyeZGZxiDuQmZxKLAlQjr3jGyLx+4Kkk58UO7D2QdgFIQCovuSuZESne6RG6XQ== +agent-base@^7.0.2, agent-base@^7.1.0: + version "7.1.1" + resolved "https://registry.yarnpkg.com/agent-base/-/agent-base-7.1.1.tgz#bdbded7dfb096b751a2a087eeeb9664725b2e317" + integrity sha512-H0TSyFNDMomMNJQBn8wFV5YC/2eJ+VXECwOadZJT554xP6cODZHPX3H9QMQECxvrgiSOP1pHjy1sMWQVYJOUOA== dependencies: - debug "4" + debug "^4.3.4" -ajv@^6.10.0, ajv@^6.12.4: +ajv@^6.12.4: version "6.12.6" resolved "https://registry.yarnpkg.com/ajv/-/ajv-6.12.6.tgz#baf5a62e802b07d977034586f8c3baf5adf26df4" integrity sha512-j3fVLgvTo527anyYyJOGTYJbG+vnnQYvE0m5mmkc1TK+nxAppkCLMIL0aZ4dblVCNoGShhm+kzE4ZUykBoMg4g== @@ -961,7 +958,7 @@ ajv@^6.10.0, ajv@^6.12.4: json-schema-traverse "^0.4.1" uri-js "^4.2.2" -ansi-escapes@^4.2.1: +ansi-escapes@^4.3.2: version "4.3.2" resolved "https://registry.yarnpkg.com/ansi-escapes/-/ansi-escapes-4.3.2.tgz#6b2291d1db7d98b6521d5f1efa42d0f3a9feb65e" integrity sha512-gKXj5ALrKWQLsYG9jlTRmR/xKluxHV+Z9QEwNIgCfM1/uwPMCuzVVnh5mwTd+OuBZcwSIMbqssNWRm1lE51QaQ== @@ -973,6 +970,11 @@ ansi-regex@^5.0.1: resolved "https://registry.yarnpkg.com/ansi-regex/-/ansi-regex-5.0.1.tgz#082cb2c89c9fe8659a311a53bd6a4dc5301db304" integrity sha512-quJQXlTSUGL2LH9SUXo8VwsY4soanhgo6LNSm84E1LBcE8s3O0wpdiRzyR9z/ZZJMlMWv37qOOb9pdJlMUEKFQ== +ansi-regex@^6.0.1: + version "6.0.1" + resolved "https://registry.yarnpkg.com/ansi-regex/-/ansi-regex-6.0.1.tgz#3183e38fae9a65d7cb5e53945cd5897d0260a06a" + integrity sha512-n5M855fKb2SsfMIiFFoVrABHJC8QtHwVx+mHWP3QcEqBHYienj5dHSgjbxtC0WEZXYt4wcD6zrQElDPhFuZgfA== + ansi-styles@^3.2.1: version "3.2.1" resolved "https://registry.yarnpkg.com/ansi-styles/-/ansi-styles-3.2.1.tgz#41fbb20243e50b12be0f04b8dedbf07520ce841d" @@ -992,6 +994,11 @@ ansi-styles@^5.0.0: resolved "https://registry.yarnpkg.com/ansi-styles/-/ansi-styles-5.2.0.tgz#07449690ad45777d1924ac2abb2fc8895dba836b" integrity sha512-Cxwpt2SfTzTtXcfOlzGEee8O+c+MmUgGrNiBcXnuWxuFJHe6a5Hz7qwhwe5OgaSYI0IJvkLqWX1ASG+cJOkEiA== +ansi-styles@^6.1.0: + version "6.2.1" + resolved "https://registry.yarnpkg.com/ansi-styles/-/ansi-styles-6.2.1.tgz#0e62320cf99c21afff3b3012192546aacbfb05c5" + integrity sha512-bN798gFfQX+viw3R7yrGWRqnrN2oRkEkUjjl4JNn4E8GxxbjtG3FbrEIIY3l8/hrwUwIeCZvi4QuOTP4MErVug== + anymatch@~3.1.2: version "3.1.3" resolved "https://registry.yarnpkg.com/anymatch/-/anymatch-3.1.3.tgz#790c58b19ba1720a84205b57c618d5ad8524973e" @@ -1005,7 +1012,7 @@ argparse@^2.0.1: resolved "https://registry.yarnpkg.com/argparse/-/argparse-2.0.1.tgz#246f50f3ca78a3240f6c997e8a9bd1eac49e4b38" integrity sha512-8+9WqebbFzpX9OR+Wa6O29asIogeRMzcGtAINdpMHHyAg10f05aSFVBbcEqGf/PXw1EjAZ+q2/bEBg3DvurK3Q== -aria-query@^5.0.0: +aria-query@5.1.3, aria-query@^5.0.0: version "5.1.3" resolved "https://registry.yarnpkg.com/aria-query/-/aria-query-5.1.3.tgz#19db27cd101152773631396f7a95a3b58c22c35e" integrity sha512-R5iJ5lkuHybztUfuOAznmboyjWq8O6sqNqtK7CLOqdydi54VNbORp49mb14KbWgG1QD3JFO9hJdZ+y4KutfdOQ== @@ -1037,12 +1044,12 @@ available-typed-arrays@^1.0.5: resolved "https://registry.yarnpkg.com/available-typed-arrays/-/available-typed-arrays-1.0.5.tgz#92f95616501069d07d10edb2fc37d3e1c65123b7" integrity sha512-DMD0KiN46eipeziST1LPP/STfDU0sufISXmjSgvVsoU2tqxctQeASejWcfNtxYKqETM1UxQ8sp2OrSBWpHY6sw== -axios@1.4.0: - version "1.4.0" - resolved "https://registry.yarnpkg.com/axios/-/axios-1.4.0.tgz#38a7bf1224cd308de271146038b551d725f0be1f" - integrity sha512-S4XCWMEmzvo64T9GfvQDOXgYRDJ/wsSZc7Jvdgx5u1sd0JwsuPLqb3SYmusag+edF6ziyMensPVqLTSc1PiSEA== +axios@1.6.8: + version "1.6.8" + resolved "https://registry.yarnpkg.com/axios/-/axios-1.6.8.tgz#66d294951f5d988a00e87a0ffb955316a619ea66" + integrity sha512-v/ZHtJDU39mDpyBoFVkETcd/uNdxrWRrg3bKpOKzXFA6Bvqopts6ALSMU3y6ijYxbw2B+wPrIv46egTzJXCLGQ== dependencies: - follow-redirects "^1.15.0" + follow-redirects "^1.15.6" form-data "^4.0.0" proxy-from-env "^1.1.0" @@ -1056,11 +1063,6 @@ base64-arraybuffer@1.0.2: resolved "https://registry.yarnpkg.com/base64-arraybuffer/-/base64-arraybuffer-1.0.2.tgz#1c37589a7c4b0746e34bd1feb951da2df01c1bdc" integrity sha512-I3yl4r9QB5ZRY3XuJVEPfc2XhZO6YweFPI+UovAzn+8/hb3oJ6lnysaFcjVpkCPfVWFUDvoZ8kmVDP7WyRtYtQ== -base64-js@^1.3.1: - version "1.5.1" - resolved "https://registry.yarnpkg.com/base64-js/-/base64-js-1.5.1.tgz#1b1b440160a5bf7ad40b650f095963481903930a" - integrity sha512-AKpaYlHn8t4SVbOHCy+b5+KKgvR4vrsD8vbvrbiQJps7fKDTkjkDry6ji0rUJjC0kzbNePLwzxq8iypo41qeWA== - basic-auth@~2.0.1: version "2.0.1" resolved "https://registry.yarnpkg.com/basic-auth/-/basic-auth-2.0.1.tgz#b998279bf47ce38344b4f3cf916d4679bbf51e3a" @@ -1073,20 +1075,6 @@ binary-extensions@^2.0.0: resolved "https://registry.yarnpkg.com/binary-extensions/-/binary-extensions-2.2.0.tgz#75f502eeaf9ffde42fc98829645be4ea76bd9e2d" integrity sha512-jDctJ/IVQbZoJykoeHbhXpOlNBqGNcwXJKJog42E5HDPUwQTSdjCHdihjj0DlnheQ7blbT6dHOafNAiS8ooQKA== -bl@^4.1.0: - version "4.1.0" - resolved "https://registry.yarnpkg.com/bl/-/bl-4.1.0.tgz#451535264182bec2fbbc83a62ab98cf11d9f7b3a" - integrity sha512-1W07cM9gS6DcLperZfFSj+bWLtaPGSOHWhPiGzXmvVJbRLdG82sH/Kn8EtW1VqWVA54AKf2h5k5BbnIbwF3h6w== - dependencies: - buffer "^5.5.0" - inherits "^2.0.4" - readable-stream "^3.4.0" - -blueimp-md5@^2.10.0: - version "2.19.0" - resolved "https://registry.yarnpkg.com/blueimp-md5/-/blueimp-md5-2.19.0.tgz#b53feea5498dcb53dc6ec4b823adb84b729c4af0" - integrity sha512-DRQrD6gJyy8FbiE4s+bDoXS9hiW3Vbx5uCdwvcCf3zLHL+Iv7LtGHLpr+GZV8rHG8tK766FGYBwRbu8pELTt+w== - body-parser@1.20.1, body-parser@^1.19.0: version "1.20.1" resolved "https://registry.yarnpkg.com/body-parser/-/body-parser-1.20.1.tgz#b1812a8912c195cd371a3ee5e66faa2338a5c668" @@ -1132,14 +1120,6 @@ braces@^3.0.2, braces@~3.0.2: dependencies: fill-range "^7.0.1" -buffer@^5.5.0: - version "5.7.1" - resolved "https://registry.yarnpkg.com/buffer/-/buffer-5.7.1.tgz#ba62e7c13133053582197160851a8f648e99eed0" - integrity sha512-EHcyIPBQ4BSGlvjB16k5KgAJ27CIsHY/2JBmCRReo48y9rQ3MaUzWX3KVlBa4U7MyX02HdVj0K7C3WaB3ju7FQ== - dependencies: - base64-js "^1.3.1" - ieee754 "^1.1.13" - bytes@3.0.0: version "3.0.0" resolved "https://registry.yarnpkg.com/bytes/-/bytes-3.0.0.tgz#d32815404d689699f85a4ea4fa8755dd13a96048" @@ -1168,26 +1148,18 @@ callsites@^3.0.0: resolved "https://registry.yarnpkg.com/callsites/-/callsites-3.1.0.tgz#b3630abd8943432f54b3f0519238e33cd7df2f73" integrity sha512-P8BjAsXvZS+VIDUI11hHCQEv74YT67YUi5JJFNWIqL235sBmjX4+qx9Muvls5ivyNENctx46xQLQ3aTuE7ssaQ== -chai@^4.3.7: - version "4.3.7" - resolved "https://registry.yarnpkg.com/chai/-/chai-4.3.7.tgz#ec63f6df01829088e8bf55fca839bcd464a8ec51" - integrity sha512-HLnAzZ2iupm25PlN0xFreAlBA5zaBSv3og0DdeGA4Ar6h6rJ3A0rolRUKJhSF2V10GZKDgWF/VmAEsNWjCRB+A== +chai@^4.3.10: + version "4.4.1" + resolved "https://registry.yarnpkg.com/chai/-/chai-4.4.1.tgz#3603fa6eba35425b0f2ac91a009fe924106e50d1" + integrity sha512-13sOfMv2+DWduEU+/xbun3LScLoqN17nBeTLUsmDfKdoiC1fr0n9PU4guu4AhRcOVFk/sW8LyZWHuhWtQZiF+g== dependencies: assertion-error "^1.1.0" - check-error "^1.0.2" - deep-eql "^4.1.2" - get-func-name "^2.0.0" - loupe "^2.3.1" + check-error "^1.0.3" + deep-eql "^4.1.3" + get-func-name "^2.0.2" + loupe "^2.3.6" pathval "^1.1.1" - type-detect "^4.0.5" - -chalk@4.1.1: - version "4.1.1" - resolved "https://registry.yarnpkg.com/chalk/-/chalk-4.1.1.tgz#c80b3fab28bf6371e6863325eee67e618b77e6ad" - integrity sha512-diHzdDKxcU+bAsUboHLPEDQiw0qEe0qd7SYUn3HgcFlWgbDcfLGswOHYeGrHKzG9z6UYf01d9VFMfZxPM1xZSg== - dependencies: - ansi-styles "^4.1.0" - supports-color "^7.1.0" + type-detect "^4.0.8" chalk@^2.0.0, chalk@^2.4.1: version "2.4.2" @@ -1206,7 +1178,7 @@ chalk@^3.0.0: ansi-styles "^4.1.0" supports-color "^7.1.0" -chalk@^4.0.0, chalk@^4.1.0, chalk@^4.1.1, chalk@^4.1.2: +chalk@^4.0.0, chalk@^4.1.0, chalk@^4.1.2: version "4.1.2" resolved "https://registry.yarnpkg.com/chalk/-/chalk-4.1.2.tgz#aac4e2b7734a740867aeb16bf02aad556a1e7a01" integrity sha512-oKnbhFyRIXpUuez8iBMmyEa4nbj4IOQyuhc/wy9kY7/WVPcwIO9VA668Pu8RkO7+0G76SLROeyw9CpQ061i4mA== @@ -1214,17 +1186,14 @@ chalk@^4.0.0, chalk@^4.1.0, chalk@^4.1.1, chalk@^4.1.2: ansi-styles "^4.1.0" supports-color "^7.1.0" -chardet@^0.7.0: - version "0.7.0" - resolved "https://registry.yarnpkg.com/chardet/-/chardet-0.7.0.tgz#90094849f0937f2eedc2425d0d28a9e5f0cbad9e" - integrity sha512-mT8iDcrh03qDGRRmoA2hmBJnxpllMR+0/0qlzjqZES6NdiWDcZkCNAk4rPFZ9Q85r27unkiNNg8ZOiwZXBHwcA== - -check-error@^1.0.2: - version "1.0.2" - resolved "https://registry.yarnpkg.com/check-error/-/check-error-1.0.2.tgz#574d312edd88bb5dd8912e9286dd6c0aed4aac82" - integrity sha512-BrgHpW9NURQgzoNyjfq0Wu6VFO6D7IZEmJNdtgNqpzGG8RuNFHt2jQxWlAs4HMe119chBnv+34syEZtc6IhLtA== +check-error@^1.0.3: + version "1.0.3" + resolved "https://registry.yarnpkg.com/check-error/-/check-error-1.0.3.tgz#a6502e4312a7ee969f646e83bb3ddd56281bd694" + integrity sha512-iKEoDYaRmd1mxM90a2OEfWhjsjPpYPuQ+lMYsoxB126+t8fw7ySEO48nmDg5COTjxDI65/Y2OWpeEHk3ZOe8zg== + dependencies: + get-func-name "^2.0.2" -"chokidar@>=3.0.0 <4.0.0", chokidar@^3.4.2: +"chokidar@>=3.0.0 <4.0.0": version "3.5.3" resolved "https://registry.yarnpkg.com/chokidar/-/chokidar-3.5.3.tgz#1cf37c8707b932bd1af1ae22c0432e2acd1903bd" integrity sha512-Dr3sfKRP6oTcjf2JmUmFJfeVMvXBdegxB0iVQ5eb2V10uFJUCAS8OByZdVAyVb8xXNz3GjjTgj9kLWsZTqE6kw== @@ -1239,27 +1208,15 @@ check-error@^1.0.2: optionalDependencies: fsevents "~2.3.2" -ci-info@^3.2.0: - version "3.8.0" - resolved "https://registry.yarnpkg.com/ci-info/-/ci-info-3.8.0.tgz#81408265a5380c929f0bc665d62256628ce9ef91" - integrity sha512-eXTggHWSooYhq49F2opQhuHWgzucfF2YgODK4e1566GQs5BIfP30B0oenwBJHfWxAs2fyPB1s7Mg949zLf61Yw== - -cli-cursor@^3.1.0: - version "3.1.0" - resolved "https://registry.yarnpkg.com/cli-cursor/-/cli-cursor-3.1.0.tgz#264305a7ae490d1d03bf0c9ba7c925d1753af307" - integrity sha512-I/zHAwsKf9FqGoXM4WWRACob9+SNukZTd94DWF57E4toouRulbCxcUh6RKUEOQlYTHJnzkPMySvPNaaSLNfLZw== - dependencies: - restore-cursor "^3.1.0" - -cli-spinners@^2.5.0: - version "2.9.0" - resolved "https://registry.yarnpkg.com/cli-spinners/-/cli-spinners-2.9.0.tgz#5881d0ad96381e117bbe07ad91f2008fe6ffd8db" - integrity sha512-4/aL9X3Wh0yiMQlE+eeRhWP6vclO3QRtw1JHKIT0FFUs5FjpFmESqtMvYZ0+lbzBw900b95mS0hohy+qn2VK/g== +cli-spinners@^2.9.2: + version "2.9.2" + resolved "https://registry.yarnpkg.com/cli-spinners/-/cli-spinners-2.9.2.tgz#1773a8f4b9c4d6ac31563df53b3fc1d79462fe41" + integrity sha512-ywqV+5MmyL4E7ybXgKys4DugZbX0FC6LnwrhjuykIjnK9k8OQacQ7axGKnjDXWNhns0xot3bZI5h55H8yo9cJg== -cli-width@^3.0.0: - version "3.0.0" - resolved "https://registry.yarnpkg.com/cli-width/-/cli-width-3.0.0.tgz#a2f48437a2caa9a22436e794bf071ec9e61cedf6" - integrity sha512-FxqpkPPwu1HjuN93Omfm4h8uIanXofW0RxVEW3k5RKx+mJJYSthzNhp32Kzxxy3YAEZ/Dc/EWN1vZRY0+kOhbw== +cli-width@^4.1.0: + version "4.1.0" + resolved "https://registry.yarnpkg.com/cli-width/-/cli-width-4.1.0.tgz#42daac41d3c254ef38ad8ac037672130173691c5" + integrity sha512-ouuZd4/dm2Sw5Gmqy6bGyNNNe1qt9RpmxveLSO7KcgsTnU7RXfsw+/bukWGo1abgBiMAic068rclZsO4IWmmxQ== cliui@^8.0.1: version "8.0.1" @@ -1270,11 +1227,6 @@ cliui@^8.0.1: strip-ansi "^6.0.1" wrap-ansi "^7.0.0" -clone@^1.0.2: - version "1.0.4" - resolved "https://registry.yarnpkg.com/clone/-/clone-1.0.4.tgz#da309cc263df15994c688ca902179ca3c7cd7c7e" - integrity sha512-JQHZ2QMW6l3aH/j6xCqQThY/9OH4D/9ls34cgkUBiEeocRTU04tHfKPBsUK1PqZCUQM7GiA0IIXJSuXHI64Kbg== - color-convert@^1.9.0: version "1.9.3" resolved "https://registry.yarnpkg.com/color-convert/-/color-convert-1.9.3.tgz#bb71850690e1f136567de629d2d5471deda4c1e8" @@ -1306,15 +1258,10 @@ combined-stream@^1.0.8: dependencies: delayed-stream "~1.0.0" -commander@^2.19.0: - version "2.20.3" - resolved "https://registry.yarnpkg.com/commander/-/commander-2.20.3.tgz#fd485e84c03eb4881c20722ba48035e8531aeb33" - integrity sha512-GpVkmM8vF2vQUkj2LvZmD35JxeJOLCwJ9cUkugyk2nuhbv3+mJvpLYYt+0+USMxE+oj+ey/lJEnhZw75x/OMcQ== - -commondir@^1.0.1: - version "1.0.1" - resolved "https://registry.yarnpkg.com/commondir/-/commondir-1.0.1.tgz#ddd800da0c66127393cca5950ea968a3aaf1253b" - integrity sha512-W9pAhw0ja1Edb5GVdIF1mjZw/ASI0AlShXM83UUGe2DVr5TdAPEA1OA8m/g8zWp9x6On7gqufY+FatDbC3MDQg== +commander@^10.0.0: + version "10.0.1" + resolved "https://registry.yarnpkg.com/commander/-/commander-10.0.1.tgz#881ee46b4f77d1c1dccc5823433aa39b022cbe06" + integrity sha512-y4Mg2tXshplEbSGzx7amzPwKKOCGuoSRP/CjEdwwk0FOGlUbq6lKuoyDZTNZkmxHdJtp54hdfY/JUrdL7Xfdug== compressible@~2.0.16: version "2.0.18" @@ -1336,25 +1283,16 @@ compression@^1.7.4: safe-buffer "5.1.2" vary "~1.1.2" +computeds@^0.0.1: + version "0.0.1" + resolved "https://registry.yarnpkg.com/computeds/-/computeds-0.0.1.tgz#215b08a4ba3e08a11ff6eee5d6d8d7166a97ce2e" + integrity sha512-7CEBgcMjVmitjYo5q8JTJVra6X5mQ20uTThdK+0kR7UEaDrAWEQcRiBtWJzga4eRpP6afNwwLsX2SET2JhVB1Q== + concat-map@0.0.1: version "0.0.1" resolved "https://registry.yarnpkg.com/concat-map/-/concat-map-0.0.1.tgz#d8a96bd77fd68df7793a73036a3ba0d5405d477b" integrity sha512-/Srv4dswyQNBfohGpz9o6Yb3Gz3SrUDqBH5rTuhGR7ahtlbYKnVxw2bCFMRljaA7EXHaXZ8wsHdodFvbkhKmqg== -concordance@^5.0.4: - version "5.0.4" - resolved "https://registry.yarnpkg.com/concordance/-/concordance-5.0.4.tgz#9896073261adced72f88d60e4d56f8efc4bbbbd2" - integrity sha512-OAcsnTEYu1ARJqWVGwf4zh4JDfHZEaSNlNccFmt8YjB2l/n19/PF2viLINHc57vO4FKIAFl2FWASIGZZWZ2Kxw== - dependencies: - date-time "^3.1.0" - esutils "^2.0.3" - fast-diff "^1.2.0" - js-string-escape "^1.0.1" - lodash "^4.17.15" - md5-hex "^3.0.1" - semver "^7.3.2" - well-known-symbols "^2.0.0" - config-chain@^1.1.13: version "1.1.13" resolved "https://registry.yarnpkg.com/config-chain/-/config-chain-1.1.13.tgz#fad0795aa6a6cdaff9ed1b68e9dff94372c232f4" @@ -1390,16 +1328,11 @@ cookie-signature@1.0.6: resolved "https://registry.yarnpkg.com/cookie-signature/-/cookie-signature-1.0.6.tgz#e303a882b342cc3ee8ca513a79999734dab3ae2c" integrity sha512-QADzlaHc8icV8I7vbaJXJwod9HWYp8uCqf1xa4OfNu1T7JVxQIrUgOWtHdNDtPiywmFbiS12VjotIXLrKM3orQ== -cookie@0.5.0: +cookie@0.5.0, cookie@^0.5.0: version "0.5.0" resolved "https://registry.yarnpkg.com/cookie/-/cookie-0.5.0.tgz#d1f5d71adec6558c58f389987c366aa47e994f8b" integrity sha512-YZ3GUyn/o8gfKJlnlX7g7xq4gyO6OSuhGPKaaGssGB2qgDUS0gPgtTvoyZLTt9Ab6dC4hfc9dV5arkvc/OCmrw== -cookie@^0.4.2: - version "0.4.2" - resolved "https://registry.yarnpkg.com/cookie/-/cookie-0.4.2.tgz#0e41f24de5ecf317947c82fc789e06a884824432" - integrity sha512-aSWTXFzaKWkvHO1Ny/s+ePFpvKsPnjc551iI41v3ny/ow6tBG5Vd+FuqGNhh1LxOmVzOlGUriIlOaokOvhaStA== - cors@^2.8.5: version "2.8.5" resolved "https://registry.yarnpkg.com/cors/-/cors-2.8.5.tgz#eac11da51592dd86b9f06f6e7ac293b3df875d29" @@ -1419,7 +1352,7 @@ cross-spawn@^6.0.5: shebang-command "^1.2.0" which "^1.2.9" -cross-spawn@^7.0.2: +cross-spawn@^7.0.0, cross-spawn@^7.0.2, cross-spawn@^7.0.3: version "7.0.3" resolved "https://registry.yarnpkg.com/cross-spawn/-/cross-spawn-7.0.3.tgz#f73a85b9d5d41d045551c177e2882d4ac85728a6" integrity sha512-iRDPJKUPVEND7dHPO8rkbOnPpyDygcDFtWjpeWNCgy8WP2rXcxXL8TskReQl6OrB2G7+UJrags1q15Fudc7G6w== @@ -1438,40 +1371,32 @@ cssesc@^3.0.0: resolved "https://registry.yarnpkg.com/cssesc/-/cssesc-3.0.0.tgz#37741919903b868565e1c09ea747445cd18983ee" integrity sha512-/Tb/JcjK111nNScGob5MNtsntNM1aCNUDipB/TkwZFhyDrrE47SOx/18wF2bbjgc3ZzCSKW1T5nt5EbFoAz/Vg== -cssstyle@^3.0.0: - version "3.0.0" - resolved "https://registry.yarnpkg.com/cssstyle/-/cssstyle-3.0.0.tgz#17ca9c87d26eac764bb8cfd00583cff21ce0277a" - integrity sha512-N4u2ABATi3Qplzf0hWbVCdjenim8F3ojEXpBDF5hBpjzW182MjNGLqfmQ0SkSPeQ+V86ZXgeH8aXj6kayd4jgg== +cssstyle@^4.0.1: + version "4.0.1" + resolved "https://registry.yarnpkg.com/cssstyle/-/cssstyle-4.0.1.tgz#ef29c598a1e90125c870525490ea4f354db0660a" + integrity sha512-8ZYiJ3A/3OkDd093CBT/0UKDWry7ak4BdPTFP2+QEP7cmhouyq/Up709ASSj2cK02BbZiMgk7kYjZNS4QP5qrQ== dependencies: rrweb-cssom "^0.6.0" -csstype@^3.1.1: - version "3.1.2" - resolved "https://registry.yarnpkg.com/csstype/-/csstype-3.1.2.tgz#1d4bf9d572f11c14031f0436e1c10bc1f571f50b" - integrity sha512-I7K1Uu0MBPzaFKg4nI5Q7Vs2t+3gWWW648spaF+Rg7pI9ds18Ugn+lvg4SHczUdKlHI5LWBXyqfS8+DufyBsgQ== - -data-urls@^4.0.0: - version "4.0.0" - resolved "https://registry.yarnpkg.com/data-urls/-/data-urls-4.0.0.tgz#333a454eca6f9a5b7b0f1013ff89074c3f522dd4" - integrity sha512-/mMTei/JXPqvFqQtfyTowxmJVwr2PVAeCcDxyFf6LhoOu/09TX2OX3kb2wzi4DMXcfj4OItwDOnhl5oziPnT6g== - dependencies: - abab "^2.0.6" - whatwg-mimetype "^3.0.0" - whatwg-url "^12.0.0" +csstype@^3.1.3: + version "3.1.3" + resolved "https://registry.yarnpkg.com/csstype/-/csstype-3.1.3.tgz#d80ff294d114fb0e6ac500fbf85b60137d7eff81" + integrity sha512-M1uQkMl8rQK/szD0LNhtqxIPLpimGm8sOBwU7lLnCpSbTyY3yeU1Vc7l4KT5zT4s/yOxHH5O7tIuuLOCnLADRw== -date-time@^3.1.0: - version "3.1.0" - resolved "https://registry.yarnpkg.com/date-time/-/date-time-3.1.0.tgz#0d1e934d170579f481ed8df1e2b8ff70ee845e1e" - integrity sha512-uqCUKXE5q1PNBXjPqvwhwJf9SwMoAHBgWJ6DcrnS5o+W2JOiIILl0JEdVD8SGujrNS02GGxgwAg2PN2zONgtjg== +data-urls@^5.0.0: + version "5.0.0" + resolved "https://registry.yarnpkg.com/data-urls/-/data-urls-5.0.0.tgz#2f76906bce1824429ffecb6920f45a0b30f00dde" + integrity sha512-ZYP5VBHshaDAiVZxjbRVcFJpc+4xGgT0bK3vzy1HLN8jTO975HEbuYzZJcHoQEY5K1a0z8YayJkyVETa08eNTg== dependencies: - time-zone "^1.0.0" + whatwg-mimetype "^4.0.0" + whatwg-url "^14.0.0" de-indent@^1.0.2: version "1.0.2" resolved "https://registry.yarnpkg.com/de-indent/-/de-indent-1.0.2.tgz#b2038e846dc33baa5796128d0804b455b8c1e21d" integrity sha512-e/1zu3xH5MQryN2zdVaF0OrdNLUbvWxzMbi+iNA6Bky7l1RoP8a2fIbRocyHclXt/arDrrR6lL3TqFD9pMQTsg== -debug@*, debug@4, debug@^4.1.1, debug@^4.3.2, debug@^4.3.3, debug@^4.3.4: +debug@*, debug@4, debug@^4.3.1, debug@^4.3.2, debug@^4.3.3, debug@^4.3.4: version "4.3.4" resolved "https://registry.yarnpkg.com/debug/-/debug-4.3.4.tgz#1319f6579357f2338d3337d2cdd4914bb5dcc865" integrity sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ== @@ -1502,7 +1427,7 @@ decode-uri-component@^0.4.1: resolved "https://registry.yarnpkg.com/decode-uri-component/-/decode-uri-component-0.4.1.tgz#2ac4859663c704be22bf7db760a1494a49ab2cc5" integrity sha512-+8VxcR21HhTy8nOt6jf20w0c9CADrw1O8d+VZ/YzzCt4bJ3uBjw+D1q2osAB8RnpwwaeYBxy0HyKQxD5JBMuuQ== -deep-eql@^4.1.2: +deep-eql@^4.1.3: version "4.1.3" resolved "https://registry.yarnpkg.com/deep-eql/-/deep-eql-4.1.3.tgz#7c7775513092f7df98d8df9996dd085eb668cc6d" integrity sha512-WaEtAOpRA1MQ0eohqZjpGD8zdI0Ovsm8mmFhaDN8dvDZzyoUMcYDnf5Y6iu7HTXxf8JDS23qWa4a+hKCDyOPzw== @@ -1537,13 +1462,6 @@ deep-is@^0.1.3: resolved "https://registry.yarnpkg.com/deep-is/-/deep-is-0.1.4.tgz#a6f2dce612fadd2ef1f519b73551f17e85199831" integrity sha512-oIPzksmTg4/MriiaYGO+okXDT7ztn/w3Eptv/+gSIdMdKsJo0u4CfYNFJPy+4SKMuCqGw2wxnA+URMg3t8a/bQ== -defaults@^1.0.3: - version "1.0.4" - resolved "https://registry.yarnpkg.com/defaults/-/defaults-1.0.4.tgz#b0b02062c1e2aa62ff5d9528f0f98baa90978d7a" - integrity sha512-eFuaLoy/Rxalv2kr+lqMlUnrDWV+3j4pljOIJgLIhI058IQfWJ7vXhyEIHu+HtC738klGALYxOKDO0bQP3tg8A== - dependencies: - clone "^1.0.2" - define-properties@^1.1.3, define-properties@^1.1.4: version "1.1.4" resolved "https://registry.yarnpkg.com/define-properties/-/define-properties-1.1.4.tgz#0b14d7bd7fbeb2f3572c3a7eda80ea5d57fb05b1" @@ -1567,10 +1485,10 @@ destroy@1.2.0: resolved "https://registry.yarnpkg.com/destroy/-/destroy-1.2.0.tgz#4803735509ad8be552934c67df614f94e66fa015" integrity sha512-2sJGJTaXIIaR1w4iJSNoN0hnMY7Gpc/n8D4qSCJw8QqFWXf7cuAgnEHxBpweaVcPevC2l3KpjYCx3NypQQgaJg== -diff-sequences@^29.4.2: - version "29.4.2" - resolved "https://registry.yarnpkg.com/diff-sequences/-/diff-sequences-29.4.2.tgz#711fe6bd8a5869fe2539cee4a5152425ff671fda" - integrity sha512-R6P0Y6PrsH3n4hUXxL3nns0rbRk6Q33js3ygJBeEpbzLzgcNuJ61+u0RXasFpTKISw99TxUzFnumSnRLsjhLaw== +diff-sequences@^29.6.3: + version "29.6.3" + resolved "https://registry.yarnpkg.com/diff-sequences/-/diff-sequences-29.6.3.tgz#4deaf894d11407c51efc8418012f9e70b84ea921" + integrity sha512-EjePK1srD3P08o2j4f0ExnylqRs5B9tJjcp9t1krH2qRi8CCdsYfwe9JgSLurFBWwq4uOlipzfk5fHNvwFKr8Q== dir-glob@^3.0.1: version "3.0.1" @@ -1586,27 +1504,30 @@ doctrine@^3.0.0: dependencies: esutils "^2.0.2" -dom-accessibility-api@^0.5.6, dom-accessibility-api@^0.5.9: +dom-accessibility-api@^0.5.9: version "0.5.16" resolved "https://registry.yarnpkg.com/dom-accessibility-api/-/dom-accessibility-api-0.5.16.tgz#5a7429e6066eb3664d911e33fb0e45de8eb08453" integrity sha512-X7BJ2yElsnOJ30pZF4uIIDfBEVgF4XEBxL9Bxhy6dnrm5hkzqmsWHGTiHqRiITNhMyFLyAiWndIJP7Z1NTteDg== -domexception@^4.0.0: - version "4.0.0" - resolved "https://registry.yarnpkg.com/domexception/-/domexception-4.0.0.tgz#4ad1be56ccadc86fc76d033353999a8037d03673" - integrity sha512-A2is4PLG+eeSfoTMA95/s4pvAoSo2mKtiM5jlHkAVewmiO8ISFTFKZjH7UAM1Atli/OT/7JHOrJRJiMKUZKYBw== - dependencies: - webidl-conversions "^7.0.0" +dom-accessibility-api@^0.6.3: + version "0.6.3" + resolved "https://registry.yarnpkg.com/dom-accessibility-api/-/dom-accessibility-api-0.6.3.tgz#993e925cc1d73f2c662e7d75dd5a5445259a8fd8" + integrity sha512-7ZgogeTnjuHbo+ct10G9Ffp0mif17idi0IyWNVA/wcwcm7NPOD/WEHVP3n7n3MhXqxoIYm8d6MuZohYWIZ4T3w== -editorconfig@^0.15.3: - version "0.15.3" - resolved "https://registry.yarnpkg.com/editorconfig/-/editorconfig-0.15.3.tgz#bef84c4e75fb8dcb0ce5cee8efd51c15999befc5" - integrity sha512-M9wIMFx96vq0R4F+gRpY3o2exzb8hEj/n9S8unZtHSvYjibBp/iMufSzvmOcV/laG0ZtuTVGtiJggPOSW2r93g== +eastasianwidth@^0.2.0: + version "0.2.0" + resolved "https://registry.yarnpkg.com/eastasianwidth/-/eastasianwidth-0.2.0.tgz#696ce2ec0aa0e6ea93a397ffcf24aa7840c827cb" + integrity sha512-I88TYZWc9XiYHRQ4/3c5rjjfgkjhLyW2luGIheGERbNQ6OY7yTybanSpDXZa8y7VUP9YmDcYa+eyq4ca7iLqWA== + +editorconfig@^1.0.4: + version "1.0.4" + resolved "https://registry.yarnpkg.com/editorconfig/-/editorconfig-1.0.4.tgz#040c9a8e9a6c5288388b87c2db07028aa89f53a3" + integrity sha512-L9Qe08KWTlqYMVvMcTIvMAdl1cDUubzRNYL+WfA4bLDMHe4nemKkpmYzkznE1FwLKu0EEmy6obgQKzMJrg4x9Q== dependencies: - commander "^2.19.0" - lru-cache "^4.1.5" - semver "^5.6.0" - sigmund "^1.0.1" + "@one-ini/wasm" "0.1.1" + commander "^10.0.0" + minimatch "9.0.1" + semver "^7.5.3" ee-first@1.1.1: version "1.1.1" @@ -1618,6 +1539,11 @@ emoji-regex@^8.0.0: resolved "https://registry.yarnpkg.com/emoji-regex/-/emoji-regex-8.0.0.tgz#e818fd69ce5ccfcb404594f842963bf53164cc37" integrity sha512-MSjYzcWNOA0ewAHpz0MxpYFvwg6yjy1NG3xteoqz644VCo/RPgnr1/GGt+ic3iJTzQ8Eu3TdM14SawnVUmGE6A== +emoji-regex@^9.2.2: + version "9.2.2" + resolved "https://registry.yarnpkg.com/emoji-regex/-/emoji-regex-9.2.2.tgz#840c8803b0d8047f4ff0cf963176b32d4ef3ed72" + integrity sha512-L18DaJsXSUk2+42pv8mLs5jJT2hqFkFE4j21wOmgbUqsZ2hL72NsUU785g9RXgo3s0ZNgVl42TiHp3ZtOv/Vyg== + encodeurl@~1.0.2: version "1.0.2" resolved "https://registry.yarnpkg.com/encodeurl/-/encodeurl-1.0.2.tgz#ad3ff4c86ec2d029322f5a02c3a9a606c95b3f59" @@ -1628,6 +1554,11 @@ entities@^4.4.0: resolved "https://registry.yarnpkg.com/entities/-/entities-4.4.0.tgz#97bdaba170339446495e653cfd2db78962900174" integrity sha512-oYp7156SP8LkeGD0GF85ad1X9Ai79WtRsZ2gxJqtBuzH+98YUV6jkHEKlZkMbcrjJjIVJNIDP/3WL9wQkoPbWA== +entities@^4.5.0: + version "4.5.0" + resolved "https://registry.yarnpkg.com/entities/-/entities-4.5.0.tgz#5d268ea5e7113ec74c4d033b79ea5a35a488fb48" + integrity sha512-V0hjH4dGPh9Ao5p0MoRY6BVqtwCjhz6vI5LT8AJ55H+4g9/4vbHx1I54fS0XuclLhDHArPQCiMjDxjaL8fPxhw== + error-ex@^1.3.1: version "1.3.2" resolved "https://registry.yarnpkg.com/error-ex/-/error-ex-1.3.2.tgz#b4ac40648107fdcdcfae242f428bea8a14d4f1bf" @@ -1715,33 +1646,34 @@ es-to-primitive@^1.2.1: is-date-object "^1.0.1" is-symbol "^1.0.2" -esbuild@^0.16.14: - version "0.16.17" - resolved "https://registry.yarnpkg.com/esbuild/-/esbuild-0.16.17.tgz#fc2c3914c57ee750635fee71b89f615f25065259" - integrity sha512-G8LEkV0XzDMNwXKgM0Jwu3nY3lSTwSGY6XbxM9cr9+s0T/qSV1q1JVPBGzm3dcjhCic9+emZDmMffkwgPeOeLg== +esbuild@^0.20.1: + version "0.20.2" + resolved "https://registry.yarnpkg.com/esbuild/-/esbuild-0.20.2.tgz#9d6b2386561766ee6b5a55196c6d766d28c87ea1" + integrity sha512-WdOOppmUNU+IbZ0PaDiTst80zjnrOkyJNHoKupIcVyU8Lvla3Ugx94VzkQ32Ijqd7UhHJy75gNWDMUekcrSJ6g== optionalDependencies: - "@esbuild/android-arm" "0.16.17" - "@esbuild/android-arm64" "0.16.17" - "@esbuild/android-x64" "0.16.17" - "@esbuild/darwin-arm64" "0.16.17" - "@esbuild/darwin-x64" "0.16.17" - "@esbuild/freebsd-arm64" "0.16.17" - "@esbuild/freebsd-x64" "0.16.17" - "@esbuild/linux-arm" "0.16.17" - "@esbuild/linux-arm64" "0.16.17" - "@esbuild/linux-ia32" "0.16.17" - "@esbuild/linux-loong64" "0.16.17" - "@esbuild/linux-mips64el" "0.16.17" - "@esbuild/linux-ppc64" "0.16.17" - "@esbuild/linux-riscv64" "0.16.17" - "@esbuild/linux-s390x" "0.16.17" - "@esbuild/linux-x64" "0.16.17" - "@esbuild/netbsd-x64" "0.16.17" - "@esbuild/openbsd-x64" "0.16.17" - "@esbuild/sunos-x64" "0.16.17" - "@esbuild/win32-arm64" "0.16.17" - "@esbuild/win32-ia32" "0.16.17" - "@esbuild/win32-x64" "0.16.17" + "@esbuild/aix-ppc64" "0.20.2" + "@esbuild/android-arm" "0.20.2" + "@esbuild/android-arm64" "0.20.2" + "@esbuild/android-x64" "0.20.2" + "@esbuild/darwin-arm64" "0.20.2" + "@esbuild/darwin-x64" "0.20.2" + "@esbuild/freebsd-arm64" "0.20.2" + "@esbuild/freebsd-x64" "0.20.2" + "@esbuild/linux-arm" "0.20.2" + "@esbuild/linux-arm64" "0.20.2" + "@esbuild/linux-ia32" "0.20.2" + "@esbuild/linux-loong64" "0.20.2" + "@esbuild/linux-mips64el" "0.20.2" + "@esbuild/linux-ppc64" "0.20.2" + "@esbuild/linux-riscv64" "0.20.2" + "@esbuild/linux-s390x" "0.20.2" + "@esbuild/linux-x64" "0.20.2" + "@esbuild/netbsd-x64" "0.20.2" + "@esbuild/openbsd-x64" "0.20.2" + "@esbuild/sunos-x64" "0.20.2" + "@esbuild/win32-arm64" "0.20.2" + "@esbuild/win32-ia32" "0.20.2" + "@esbuild/win32-x64" "0.20.2" escalade@^3.1.1: version "3.1.1" @@ -1758,54 +1690,43 @@ escape-string-regexp@^1.0.5: resolved "https://registry.yarnpkg.com/escape-string-regexp/-/escape-string-regexp-1.0.5.tgz#1b61c0562190a8dff6ae3bb2cf0200ca130b86d4" integrity sha512-vbRorB5FUQWvla16U8R/qgaFIya2qGzwDrNmCZuYKrbdSUMG6I1ZCGQRefkRVhuOkIGVne7BQ35DSfo1qvJqFg== -escape-string-regexp@^2.0.0: - version "2.0.0" - resolved "https://registry.yarnpkg.com/escape-string-regexp/-/escape-string-regexp-2.0.0.tgz#a30304e99daa32e23b2fd20f51babd07cffca344" - integrity sha512-UpzcLCXolUWcNu5HtVMHYdXJjArjsF9C0aNnquZYY4uW/Vu0miy5YoWvbV345HauVvcAUnpRuhMMcqTcGOY2+w== - escape-string-regexp@^4.0.0: version "4.0.0" resolved "https://registry.yarnpkg.com/escape-string-regexp/-/escape-string-regexp-4.0.0.tgz#14ba83a5d373e3d311e5afca29cf5bfad965bf34" integrity sha512-TtpcNJ3XAzx3Gq8sWRzJaVajRs0uVxA2YAkdb1jm2YkPz4G6egUFAyA3n5vtEIZefPk5Wa4UXbKuS5fKkJWdgA== -eslint-config-prettier@^8.3.0: - version "8.6.0" - resolved "https://registry.yarnpkg.com/eslint-config-prettier/-/eslint-config-prettier-8.6.0.tgz#dec1d29ab728f4fa63061774e1672ac4e363d207" - integrity sha512-bAF0eLpLVqP5oEVUFKpMA+NnRFICwn9X8B5jrR9FcqnYBuPbqWEjTEspPWMj5ye6czoSLDweCzSo3Ko7gGrZaA== +eslint-config-prettier@^9.0.0: + version "9.1.0" + resolved "https://registry.yarnpkg.com/eslint-config-prettier/-/eslint-config-prettier-9.1.0.tgz#31af3d94578645966c082fcb71a5846d3c94867f" + integrity sha512-NSWl5BFQWEPi1j4TjVNItzYV7dZXZ+wP6I6ZhrBGpChQhZRUaElihE9uRRkcbRnNb76UMKDF3r+WTmNcGPKsqw== -eslint-plugin-prettier@^4.0.0: - version "4.2.1" - resolved "https://registry.yarnpkg.com/eslint-plugin-prettier/-/eslint-plugin-prettier-4.2.1.tgz#651cbb88b1dab98bfd42f017a12fa6b2d993f94b" - integrity sha512-f/0rXLXUt0oFYs8ra4w49wYZBG5GKZpAYsJSm6rnYL5uVDjd+zowwMwVZHnAjf4edNrKpCDYfXDgmRE/Ak7QyQ== +eslint-plugin-prettier@^5.0.0: + version "5.1.3" + resolved "https://registry.yarnpkg.com/eslint-plugin-prettier/-/eslint-plugin-prettier-5.1.3.tgz#17cfade9e732cef32b5f5be53bd4e07afd8e67e1" + integrity sha512-C9GCVAs4Eq7ZC/XFQHITLiHJxQngdtraXaM+LoUFoFp/lHNl2Zn8f3WQbe9HvTBBQ9YnKFB0/2Ajdqwo5D1EAw== dependencies: prettier-linter-helpers "^1.0.0" + synckit "^0.8.6" eslint-plugin-sort-imports-es6-autofix@0.6.0: version "0.6.0" resolved "https://registry.yarnpkg.com/eslint-plugin-sort-imports-es6-autofix/-/eslint-plugin-sort-imports-es6-autofix-0.6.0.tgz#b8cd8639d7a54cefce6b17898b102fd5ec31e52b" integrity sha512-2NVaBGF9NN+727Fyq+jJYihdIeegjXeUUrZED9Q8FVB8MsV3YQEyXG96GVnXqWt0pmn7xfCZOZf3uKnIhBrfeQ== -eslint-plugin-vue@9.9.0: - version "9.9.0" - resolved "https://registry.yarnpkg.com/eslint-plugin-vue/-/eslint-plugin-vue-9.9.0.tgz#ac788ebccd2eb94d846a507df55da50693b80c91" - integrity sha512-YbubS7eK0J7DCf0U2LxvVP7LMfs6rC6UltihIgval3azO3gyDwEGVgsCMe1TmDiEkl6GdMKfRpaME6QxIYtzDQ== +eslint-plugin-vue@9.25.0: + version "9.25.0" + resolved "https://registry.yarnpkg.com/eslint-plugin-vue/-/eslint-plugin-vue-9.25.0.tgz#615cb7bb6d0e2140d21840b9aa51dce69e803e7a" + integrity sha512-tDWlx14bVe6Bs+Nnh3IGrD+hb11kf2nukfm6jLsmJIhmiRQ1SUaksvwY9U5MvPB0pcrg0QK0xapQkfITs3RKOA== dependencies: - eslint-utils "^3.0.0" + "@eslint-community/eslint-utils" "^4.4.0" + globals "^13.24.0" natural-compare "^1.4.0" - nth-check "^2.0.1" - postcss-selector-parser "^6.0.9" - semver "^7.3.5" - vue-eslint-parser "^9.0.1" + nth-check "^2.1.1" + postcss-selector-parser "^6.0.15" + semver "^7.6.0" + vue-eslint-parser "^9.4.2" xml-name-validator "^4.0.0" -eslint-scope@^5.1.1: - version "5.1.1" - resolved "https://registry.yarnpkg.com/eslint-scope/-/eslint-scope-5.1.1.tgz#e786e59a66cb92b3f6c1fb0d508aab174848f48c" - integrity sha512-2NxwbF/hZ0KpepYN0cNbo+FN6XoK7GaHlQhgx/hIZl6Va0bF45RQOOwhLIy8lQDbuCiadSLCBnH2CFYquit5bw== - dependencies: - esrecurse "^4.3.0" - estraverse "^4.1.1" - eslint-scope@^7.1.1: version "7.1.1" resolved "https://registry.yarnpkg.com/eslint-scope/-/eslint-scope-7.1.1.tgz#fff34894c2f65e5226d3041ac480b4513a163642" @@ -1814,69 +1735,69 @@ eslint-scope@^7.1.1: esrecurse "^4.3.0" estraverse "^5.2.0" -eslint-utils@^3.0.0: - version "3.0.0" - resolved "https://registry.yarnpkg.com/eslint-utils/-/eslint-utils-3.0.0.tgz#8aebaface7345bb33559db0a1f13a1d2d48c3672" - integrity sha512-uuQC43IGctw68pJA1RgbQS8/NP7rch6Cwd4j3ZBtgo4/8Flj4eGE7ZYSZRN3iq5pVUv6GPdW5Z1RFleo84uLDA== +eslint-scope@^7.2.2: + version "7.2.2" + resolved "https://registry.yarnpkg.com/eslint-scope/-/eslint-scope-7.2.2.tgz#deb4f92563390f32006894af62a22dba1c46423f" + integrity sha512-dOt21O7lTMhDM+X9mB4GX+DZrZtCUJPL/wlcTqxyrx5IvO0IYtILdtrQGQp+8n5S0gwSVmOf9NQrjMOgfQZlIg== dependencies: - eslint-visitor-keys "^2.0.0" - -eslint-visitor-keys@^2.0.0: - version "2.1.0" - resolved "https://registry.yarnpkg.com/eslint-visitor-keys/-/eslint-visitor-keys-2.1.0.tgz#f65328259305927392c938ed44eb0a5c9b2bd303" - integrity sha512-0rSmRBzXgDzIsD6mGdJgevzgezI534Cer5L/vyMX0kHzT/jiB43jRhd9YUlMGYLQy2zprNmoT8qasCGtY+QaKw== + esrecurse "^4.3.0" + estraverse "^5.2.0" eslint-visitor-keys@^3.3.0: version "3.3.0" resolved "https://registry.yarnpkg.com/eslint-visitor-keys/-/eslint-visitor-keys-3.3.0.tgz#f6480fa6b1f30efe2d1968aa8ac745b862469826" integrity sha512-mQ+suqKJVyeuwGYHAdjMFqjCyfl8+Ldnxuyp3ldiMBFKkvytrXUZWaiPCEav8qDHKty44bD+qV1IP4T+w+xXRA== -eslint@8.34.0: - version "8.34.0" - resolved "https://registry.yarnpkg.com/eslint/-/eslint-8.34.0.tgz#fe0ab0ef478104c1f9ebc5537e303d25a8fb22d6" - integrity sha512-1Z8iFsucw+7kSqXNZVslXS8Ioa4u2KM7GPwuKtkTFAqZ/cHMcEaR+1+Br0wLlot49cNxIiZk5wp8EAbPcYZxTg== - dependencies: - "@eslint/eslintrc" "^1.4.1" - "@humanwhocodes/config-array" "^0.11.8" +eslint-visitor-keys@^3.4.1, eslint-visitor-keys@^3.4.3: + version "3.4.3" + resolved "https://registry.yarnpkg.com/eslint-visitor-keys/-/eslint-visitor-keys-3.4.3.tgz#0cd72fe8550e3c2eae156a96a4dddcd1c8ac5800" + integrity sha512-wpc+LXeiyiisxPlEkUzU6svyS1frIO3Mgxj1fdy7Pm8Ygzguax2N3Fa/D/ag1WqbOprdI+uY6wMUl8/a2G+iag== + +eslint@8.57.0: + version "8.57.0" + resolved "https://registry.yarnpkg.com/eslint/-/eslint-8.57.0.tgz#c786a6fd0e0b68941aaf624596fb987089195668" + integrity sha512-dZ6+mexnaTIbSBZWgou51U6OmzIhYM2VcNdtiTtI7qPNZm35Akpr0f6vtw3w1Kmn5PYo+tZVfh13WrhpS6oLqQ== + dependencies: + "@eslint-community/eslint-utils" "^4.2.0" + "@eslint-community/regexpp" "^4.6.1" + "@eslint/eslintrc" "^2.1.4" + "@eslint/js" "8.57.0" + "@humanwhocodes/config-array" "^0.11.14" "@humanwhocodes/module-importer" "^1.0.1" "@nodelib/fs.walk" "^1.2.8" - ajv "^6.10.0" + "@ungap/structured-clone" "^1.2.0" + ajv "^6.12.4" chalk "^4.0.0" cross-spawn "^7.0.2" debug "^4.3.2" doctrine "^3.0.0" escape-string-regexp "^4.0.0" - eslint-scope "^7.1.1" - eslint-utils "^3.0.0" - eslint-visitor-keys "^3.3.0" - espree "^9.4.0" - esquery "^1.4.0" + eslint-scope "^7.2.2" + eslint-visitor-keys "^3.4.3" + espree "^9.6.1" + esquery "^1.4.2" esutils "^2.0.2" fast-deep-equal "^3.1.3" file-entry-cache "^6.0.1" find-up "^5.0.0" glob-parent "^6.0.2" globals "^13.19.0" - grapheme-splitter "^1.0.4" + graphemer "^1.4.0" ignore "^5.2.0" - import-fresh "^3.0.0" imurmurhash "^0.1.4" is-glob "^4.0.0" is-path-inside "^3.0.3" - js-sdsl "^4.1.4" js-yaml "^4.1.0" json-stable-stringify-without-jsonify "^1.0.1" levn "^0.4.1" lodash.merge "^4.6.2" minimatch "^3.1.2" natural-compare "^1.4.0" - optionator "^0.9.1" - regexpp "^3.2.0" + optionator "^0.9.3" strip-ansi "^6.0.1" - strip-json-comments "^3.1.0" text-table "^0.2.0" -espree@^9.3.1, espree@^9.4.0: +espree@^9.3.1: version "9.4.1" resolved "https://registry.yarnpkg.com/espree/-/espree-9.4.1.tgz#51d6092615567a2c2cff7833445e37c28c0065bd" integrity sha512-XwctdmTO6SIvCzd9810yyNzIrOrqNYV9Koizx4C/mRhf9uq0o4yHoCEU/670pOxOL/MSraektvSAji79kX90Vg== @@ -1885,6 +1806,15 @@ espree@^9.3.1, espree@^9.4.0: acorn-jsx "^5.3.2" eslint-visitor-keys "^3.3.0" +espree@^9.6.0, espree@^9.6.1: + version "9.6.1" + resolved "https://registry.yarnpkg.com/espree/-/espree-9.6.1.tgz#a2a17b8e434690a5432f2f8018ce71d331a48c6f" + integrity sha512-oruZaFkjorTpF32kDSI5/75ViwGeZginGGy2NoOSg3Q9bnwlnmDm4HLnkl0RE3n+njDXR037aY1+x58Z/zFdwQ== + dependencies: + acorn "^8.9.0" + acorn-jsx "^5.3.2" + eslint-visitor-keys "^3.4.1" + esquery@^1.4.0: version "1.4.0" resolved "https://registry.yarnpkg.com/esquery/-/esquery-1.4.0.tgz#2148ffc38b82e8c7057dfed48425b3e61f0f24a5" @@ -1892,6 +1822,13 @@ esquery@^1.4.0: dependencies: estraverse "^5.1.0" +esquery@^1.4.2: + version "1.5.0" + resolved "https://registry.yarnpkg.com/esquery/-/esquery-1.5.0.tgz#6ce17738de8577694edd7361c57182ac8cb0db0b" + integrity sha512-YQLXUplAwJgCydQ78IMJywZCceoqk1oH01OERdSAJc/7U2AylwjhSCLDEtqwg811idIS/9fIU5GjG73IgjKMVg== + dependencies: + estraverse "^5.1.0" + esrecurse@^4.3.0: version "4.3.0" resolved "https://registry.yarnpkg.com/esrecurse/-/esrecurse-4.3.0.tgz#7ad7964d679abb28bee72cec63758b1c5d2c9921" @@ -1899,11 +1836,6 @@ esrecurse@^4.3.0: dependencies: estraverse "^5.2.0" -estraverse@^4.1.1: - version "4.3.0" - resolved "https://registry.yarnpkg.com/estraverse/-/estraverse-4.3.0.tgz#398ad3f3c5a24948be7725e83d11a7de28cdbd1d" - integrity sha512-39nnKffWz8xN1BU/2c79n9nB9HDzo0niYUqx6xyqUnyoAnQyyWpOTdZEeiCch8BBu515t4wp9ZmgVfVhn9EBpw== - estraverse@^5.1.0, estraverse@^5.2.0: version "5.3.0" resolved "https://registry.yarnpkg.com/estraverse/-/estraverse-5.3.0.tgz#2eea5290702f26ab8fe5370370ff86c965d21123" @@ -1914,7 +1846,14 @@ estree-walker@^2.0.2: resolved "https://registry.yarnpkg.com/estree-walker/-/estree-walker-2.0.2.tgz#52f010178c2a4c117a7757cfe942adb7d2da4cac" integrity sha512-Rfkk/Mp/DL7JVje3u18FxFujQlTNR2q6QfMSMB7AvCBx91NGj/ba3kCfza0f6dVDbw7YlRf/nDrn7pQrCCyQ/w== -esutils@^2.0.2, esutils@^2.0.3: +estree-walker@^3.0.3: + version "3.0.3" + resolved "https://registry.yarnpkg.com/estree-walker/-/estree-walker-3.0.3.tgz#67c3e549ec402a487b4fc193d1953a524752340d" + integrity sha512-7RUKfXgSMMkzt6ZuXmqapOurLGPPfgj6l9uRZ7lRGolvk0y2yocc35LdcxKC5PQZdn2DMqioAQ2NoWcrTKmm6g== + dependencies: + "@types/estree" "^1.0.0" + +esutils@^2.0.2: version "2.0.3" resolved "https://registry.yarnpkg.com/esutils/-/esutils-2.0.3.tgz#74d2eb4de0b8da1293711910d50775b9b710ef64" integrity sha512-kVscqXk4OCp68SZ0dkgEKVi6/8ij300KBWTJq32P/dYeWTSwK41WyTxalN1eRmA5Z9UU/LX9D7FWSmV9SAYx6g== @@ -1924,21 +1863,20 @@ etag@~1.8.1: resolved "https://registry.yarnpkg.com/etag/-/etag-1.8.1.tgz#41ae2eeb65efa62268aebfea83ac7d79299b0887" integrity sha512-aIL5Fx7mawVa300al2BnEE4iNvo1qETxLrPI/o05L7z6go7fCw1J6EQmbK4FmJ2AS7kgVF/KEZWufBfdClMcPg== -events@^3.3.0: - version "3.3.0" - resolved "https://registry.yarnpkg.com/events/-/events-3.3.0.tgz#31a95ad0a924e2d2c419a813aeb2c4e878ea7400" - integrity sha512-mQw+2fkQbALzQ7V0MY0IqdnXNOeTtP4r0lN9z7AAawCXgqea7bDii20AYrIBrFd/Hx0M2Ocz6S111CaFkUcb0Q== - -expect@^29.0.0: - version "29.4.2" - resolved "https://registry.yarnpkg.com/expect/-/expect-29.4.2.tgz#2ae34eb88de797c64a1541ad0f1e2ea8a7a7b492" - integrity sha512-+JHYg9O3hd3RlICG90OPVjRkPBoiUH7PxvDVMnRiaq1g6JUgZStX514erMl0v2Dc5SkfVbm7ztqbd6qHHPn+mQ== - dependencies: - "@jest/expect-utils" "^29.4.2" - jest-get-type "^29.4.2" - jest-matcher-utils "^29.4.2" - jest-message-util "^29.4.2" - jest-util "^29.4.2" +execa@^8.0.1: + version "8.0.1" + resolved "https://registry.yarnpkg.com/execa/-/execa-8.0.1.tgz#51f6a5943b580f963c3ca9c6321796db8cc39b8c" + integrity sha512-VyhnebXciFV2DESc+p6B+y0LjSm0krU4OgJN44qFAhBY0TJ+1V61tYD2+wHusZ6F9n5K+vl8k0sTy7PEfV4qpg== + dependencies: + cross-spawn "^7.0.3" + get-stream "^8.0.1" + human-signals "^5.0.0" + is-stream "^3.0.0" + merge-stream "^2.0.0" + npm-run-path "^5.1.0" + onetime "^6.0.0" + signal-exit "^4.1.0" + strip-final-newline "^3.0.0" express-urlrewrite@^1.4.0: version "1.4.0" @@ -1985,15 +1923,6 @@ express@^4.17.1: utils-merge "1.0.1" vary "~1.1.2" -external-editor@^3.0.3: - version "3.1.0" - resolved "https://registry.yarnpkg.com/external-editor/-/external-editor-3.1.0.tgz#cb03f740befae03ea4d283caed2741a83f335495" - integrity sha512-hMQ4CX1p1izmuLYyZqLMO/qGNw10wSv9QDCPfzXfyFrOaCSSoRfqE1Kf1s5an66J5JZC62NewG+mK49jOCtQew== - dependencies: - chardet "^0.7.0" - iconv-lite "^0.4.24" - tmp "^0.0.33" - fast-deep-equal@^3.1.1, fast-deep-equal@^3.1.3: version "3.1.3" resolved "https://registry.yarnpkg.com/fast-deep-equal/-/fast-deep-equal-3.1.3.tgz#3a7d56b559d6cbc3eb512325244e619a65c6c525" @@ -2004,11 +1933,6 @@ fast-diff@^1.1.2: resolved "https://registry.yarnpkg.com/fast-diff/-/fast-diff-1.2.0.tgz#73ee11982d86caaf7959828d519cfe927fac5f03" integrity sha512-xJuoT5+L99XlZ8twedaRf6Ax2TgQVxvgZOYoPKqZufmJib0tL2tegPBOZb1pVNgIhlqDlA0eO0c3wBvQcmzx4w== -fast-diff@^1.2.0: - version "1.3.0" - resolved "https://registry.yarnpkg.com/fast-diff/-/fast-diff-1.3.0.tgz#ece407fa550a64d638536cd727e129c61616e0f0" - integrity sha512-VxPP4NqbUjj6MaAOafWeUn2cXWLcCtljklUtZf0Ind4XQ+QPtmA0b18zZy0jIQx+ExRVCR/ZQpBmik5lXshNsw== - fast-glob@^3.2.9: version "3.2.12" resolved "https://registry.yarnpkg.com/fast-glob/-/fast-glob-3.2.12.tgz#7f39ec99c2e6ab030337142da9e0c18f37afae80" @@ -2037,13 +1961,6 @@ fastq@^1.6.0: dependencies: reusify "^1.0.4" -figures@^3.0.0: - version "3.2.0" - resolved "https://registry.yarnpkg.com/figures/-/figures-3.2.0.tgz#625c18bd293c604dc4a8ddb2febf0c88341746af" - integrity sha512-yaduQFRKLXYOGgEn6AZau90j3ggSOyiqXU0F9JZfeXYhNa+Jk4X+s45A2zg5jns87GAFa34BBm2kXw4XpNcbdg== - dependencies: - escape-string-regexp "^1.0.5" - file-entry-cache@^6.0.1: version "6.0.1" resolved "https://registry.yarnpkg.com/file-entry-cache/-/file-entry-cache-6.0.1.tgz#211b2dd9659cb0394b073e7323ac3c933d522027" @@ -2076,23 +1993,6 @@ finalhandler@1.2.0: statuses "2.0.1" unpipe "~1.0.0" -find-cache-dir@^3.3.2: - version "3.3.2" - resolved "https://registry.yarnpkg.com/find-cache-dir/-/find-cache-dir-3.3.2.tgz#b30c5b6eff0730731aea9bbd9dbecbd80256d64b" - integrity sha512-wXZV5emFEjrridIgED11OoUKLxiYjAcqot/NJdAkOhlJ+vGzwhOAfcG5OX1jP+S0PcjEn8bdMJv+g2jwQ3Onig== - dependencies: - commondir "^1.0.1" - make-dir "^3.0.2" - pkg-dir "^4.1.0" - -find-up@^4.0.0: - version "4.1.0" - resolved "https://registry.yarnpkg.com/find-up/-/find-up-4.1.0.tgz#97afe7d6cdc0bc5928584b7c8d7b16e8a9aa5d19" - integrity sha512-PpOwAdQ/YlXQ2vj8a3h8IipDuYRi3wceVQQGYWxNINccq40Anw7BlsEXCMbt1Zt+OLA6Fq9suIpIWD0OsnISlw== - dependencies: - locate-path "^5.0.0" - path-exists "^4.0.0" - find-up@^5.0.0: version "5.0.0" resolved "https://registry.yarnpkg.com/find-up/-/find-up-5.0.0.tgz#4c92819ecb7083561e4f4a240a86be5198f536fc" @@ -2114,10 +2014,10 @@ flatted@^3.1.0: resolved "https://registry.yarnpkg.com/flatted/-/flatted-3.2.7.tgz#609f39207cb614b89d0765b477cb2d437fbf9787" integrity sha512-5nqDSxl8nn5BSNxyR3n4I6eDmbolI6WT+QqR547RwxQapgjQBmtktdP+HTBb/a/zLsbzERTONyUB5pefh5TtjQ== -follow-redirects@^1.15.0: - version "1.15.2" - resolved "https://registry.yarnpkg.com/follow-redirects/-/follow-redirects-1.15.2.tgz#b460864144ba63f2681096f274c4e57026da2c13" - integrity sha512-VQLG33o04KaQ8uYi2tVNbdrWp1QWxNNea+nmIB4EVM28v0hmP17z7aG1+wAkNzVq4KeXTq3221ye5qTJP91JwA== +follow-redirects@^1.15.6: + version "1.15.6" + resolved "https://registry.yarnpkg.com/follow-redirects/-/follow-redirects-1.15.6.tgz#7f815c0cda4249c74ff09e95ef97c23b5fd0399b" + integrity sha512-wWN62YITEaOpSK584EZXJafH1AGpO8RVgElfkuXbTOrPX4fIfOyEpW/CsiNd8JdYrAoOvafRTOEnvsO++qCqFA== for-each@^0.3.3: version "0.3.3" @@ -2126,6 +2026,14 @@ for-each@^0.3.3: dependencies: is-callable "^1.1.3" +foreground-child@^3.1.0: + version "3.1.1" + resolved "https://registry.yarnpkg.com/foreground-child/-/foreground-child-3.1.1.tgz#1d173e776d75d2772fed08efe4a0de1ea1b12d0d" + integrity sha512-TMKDUnIte6bfb5nWv7V/caI169OHgvwjb7V4WkeUvbQQdjr5rWKqHFiKWb/fcOwB+CzBT+qbWjvj+DVwRskpIg== + dependencies: + cross-spawn "^7.0.0" + signal-exit "^4.0.1" + form-data@^4.0.0: version "4.0.0" resolved "https://registry.yarnpkg.com/form-data/-/form-data-4.0.0.tgz#93919daeaf361ee529584b9b31664dc12c9fa452" @@ -2155,6 +2063,11 @@ fsevents@~2.3.2: resolved "https://registry.yarnpkg.com/fsevents/-/fsevents-2.3.2.tgz#8a526f78b8fdf4623b709e0b975c52c24c02fd1a" integrity sha512-xiqMQR4xAeHTuB9uWm+fFRcIOgKBMiOBP+eXiyT7jsgVCq1bkVygt00oASowB7EdtpOHaaPgKt812P9ab+DDKA== +fsevents@~2.3.3: + version "2.3.3" + resolved "https://registry.yarnpkg.com/fsevents/-/fsevents-2.3.3.tgz#cac6407785d03675a2a5e1a5305c697b347d90d6" + integrity sha512-5xoDfX+fL7faATnagmWPpbFtwh/R77WmMMqqHGS65C3vvB0YHrgF+B1YmZ3441tMj5n63k0212XNoJwzlhffQw== + function-bind@^1.1.1: version "1.1.1" resolved "https://registry.yarnpkg.com/function-bind/-/function-bind-1.1.1.tgz#a56899d3ea3c9bab874bb9773b7c5ede92f4895d" @@ -2185,6 +2098,11 @@ get-func-name@^2.0.0: resolved "https://registry.yarnpkg.com/get-func-name/-/get-func-name-2.0.0.tgz#ead774abee72e20409433a066366023dd6887a41" integrity sha512-Hm0ixYtaSZ/V7C8FJrtZIuBBI+iSgL+1Aq82zSu8VQNB4S3Gk8e7Qs3VwBDJAhmRZcFqkl3tQu36g/Foh5I5ig== +get-func-name@^2.0.1, get-func-name@^2.0.2: + version "2.0.2" + resolved "https://registry.yarnpkg.com/get-func-name/-/get-func-name-2.0.2.tgz#0d7cf20cd13fda808669ffa88f4ffc7a3943fc41" + integrity sha512-8vXOvuE167CtIc3OyItco7N/dpRtBbYOsPsXCz7X/PMnlGjYjSGuZJgM1Y7mmew7BKf9BqvLX2tnOVy1BBUsxQ== + get-intrinsic@^1.0.2, get-intrinsic@^1.1.1, get-intrinsic@^1.1.3: version "1.2.0" resolved "https://registry.yarnpkg.com/get-intrinsic/-/get-intrinsic-1.2.0.tgz#7ad1dc0535f3a2904bba075772763e5051f6d05f" @@ -2194,6 +2112,11 @@ get-intrinsic@^1.0.2, get-intrinsic@^1.1.1, get-intrinsic@^1.1.3: has "^1.0.3" has-symbols "^1.0.3" +get-stream@^8.0.1: + version "8.0.1" + resolved "https://registry.yarnpkg.com/get-stream/-/get-stream-8.0.1.tgz#def9dfd71742cd7754a7761ed43749a27d02eca2" + integrity sha512-VaUJspBffn/LMCJVoMvSAdmscJyS1auj5Zulnn5UoYcY531UWmdwhRWkcGKnGU93m5HSXP9LP2usOryrBtQowA== + get-symbol-description@^1.0.0: version "1.0.0" resolved "https://registry.yarnpkg.com/get-symbol-description/-/get-symbol-description-1.0.0.tgz#7fdb81c900101fbd564dd5f1a30af5aadc1e58d6" @@ -2216,6 +2139,17 @@ glob-parent@^6.0.2: dependencies: is-glob "^4.0.3" +glob@^10.3.3: + version "10.3.12" + resolved "https://registry.yarnpkg.com/glob/-/glob-10.3.12.tgz#3a65c363c2e9998d220338e88a5f6ac97302960b" + integrity sha512-TCNv8vJ+xz4QiqTpfOJA7HvYv+tNIRHKfUWw/q+v2jdgN4ebz+KY9tGx5J4rHP0o84mNP+ApH66HRX8us3Khqg== + dependencies: + foreground-child "^3.1.0" + jackspeak "^2.3.6" + minimatch "^9.0.1" + minipass "^7.0.4" + path-scurry "^1.10.2" + glob@^7.1.3: version "7.2.3" resolved "https://registry.yarnpkg.com/glob/-/glob-7.2.3.tgz#b8df0fb802bbfa8e89bd1d938b4e16578ed44f2b" @@ -2228,17 +2162,6 @@ glob@^7.1.3: once "^1.3.0" path-is-absolute "^1.0.0" -glob@^8.0.3: - version "8.1.0" - resolved "https://registry.yarnpkg.com/glob/-/glob-8.1.0.tgz#d388f656593ef708ee3e34640fdfb99a9fd1c33e" - integrity sha512-r8hpEjiQEYlF2QU0df3dS+nxxSIreXQS1qRhMJM0Q5NDdR386C7jb7Hwwod8Fgiuex+k0GFjgft18yvxm5XoCQ== - dependencies: - fs.realpath "^1.0.0" - inflight "^1.0.4" - inherits "2" - minimatch "^5.0.1" - once "^1.3.0" - globals@^13.19.0: version "13.20.0" resolved "https://registry.yarnpkg.com/globals/-/globals-13.20.0.tgz#ea276a1e508ffd4f1612888f9d1bad1e2717bf82" @@ -2246,6 +2169,13 @@ globals@^13.19.0: dependencies: type-fest "^0.20.2" +globals@^13.24.0: + version "13.24.0" + resolved "https://registry.yarnpkg.com/globals/-/globals-13.24.0.tgz#8432a19d78ce0c1e833949c36adb345400bb1171" + integrity sha512-AhO5QUcj8llrbG09iWhPU2B204J1xnPeL8kQmVorSsy+Sjj1sk8gIyh6cUocGmH4L0UuhAJy+hJMRA4mgA4mFQ== + dependencies: + type-fest "^0.20.2" + globalthis@^1.0.3: version "1.0.3" resolved "https://registry.yarnpkg.com/globalthis/-/globalthis-1.0.3.tgz#5852882a52b80dc301b0660273e1ed082f0b6ccf" @@ -2272,20 +2202,20 @@ gopd@^1.0.1: dependencies: get-intrinsic "^1.1.3" -graceful-fs@^4.1.2, graceful-fs@^4.1.3, graceful-fs@^4.2.9: +graceful-fs@^4.1.2, graceful-fs@^4.1.3: version "4.2.10" resolved "https://registry.yarnpkg.com/graceful-fs/-/graceful-fs-4.2.10.tgz#147d3a006da4ca3ce14728c7aefc287c367d7a6c" integrity sha512-9ByhssR2fPVsNZj478qUUbKfmL0+t5BDVyjShtyZZLiK7ZDAArFFfopyOTj0M05wE2tJPisA4iTnnXl2YoPvOA== -grapheme-splitter@^1.0.4: - version "1.0.4" - resolved "https://registry.yarnpkg.com/grapheme-splitter/-/grapheme-splitter-1.0.4.tgz#9cf3a665c6247479896834af35cf1dbb4400767e" - integrity sha512-bzh50DW9kTPM00T8y4o8vQg89Di9oLJVLW/KaOGIXJWP/iqCN6WKYkbNOF04vFLJhwcpYUh9ydh/+5vpOqV4YQ== +graphemer@^1.4.0: + version "1.4.0" + resolved "https://registry.yarnpkg.com/graphemer/-/graphemer-1.4.0.tgz#fb2f1d55e0e3a1849aeffc90c4fa0dd53a0e66c6" + integrity sha512-EtKwoO6kxCL9WO5xipiHTZlSzBm7WLT627TqC/uVRd0HKmq8NXyebnNYxDoBi7wt8eTWrUrKXCOVaFq9x1kgag== -"graphql@^15.0.0 || ^16.0.0": - version "16.7.1" - resolved "https://registry.yarnpkg.com/graphql/-/graphql-16.7.1.tgz#11475b74a7bff2aefd4691df52a0eca0abd9b642" - integrity sha512-DRYR9tf+UGU0KOsMcKAlXeFfX89UiiIZ0dRU3mR0yJfu6OjZqUcp68NnFLnqQU5RexygFoDy1EW+ccOYcPfmHg== +graphql@^16.8.1: + version "16.8.1" + resolved "https://registry.yarnpkg.com/graphql/-/graphql-16.8.1.tgz#1930a965bef1170603702acdb68aedd3f3cf6f07" + integrity sha512-59LZHPdGZVh695Ud9lRzPBVTtlX9ZCV150Er2W43ro37wVof0ctenSaskPPjN7lVTIN8mSZt8PHUNKZuNQUuxw== has-bigints@^1.0.1, has-bigints@^1.0.2: version "1.0.2" @@ -2338,22 +2268,22 @@ he@^1.2.0: resolved "https://registry.yarnpkg.com/he/-/he-1.2.0.tgz#84ae65fa7eafb165fddb61566ae14baf05664f0f" integrity sha512-F/1DnUGPopORZi0ni+CvrCgHQ5FyEAHRLSApuYWMmrbSwoN2Mn/7k+Gl38gJnR7yyDZk6WLXwiGod1JOWNDKGw== -headers-polyfill@^3.1.0, headers-polyfill@^3.1.2: - version "3.1.2" - resolved "https://registry.yarnpkg.com/headers-polyfill/-/headers-polyfill-3.1.2.tgz#9a4dcb545c5b95d9569592ef7ec0708aab763fbe" - integrity sha512-tWCK4biJ6hcLqTviLXVR9DTRfYGQMXEIUj3gwJ2rZ5wO/at3XtkI4g8mCvFdUF9l1KMBNCfmNAdnahm1cgavQA== +headers-polyfill@^4.0.2: + version "4.0.3" + resolved "https://registry.yarnpkg.com/headers-polyfill/-/headers-polyfill-4.0.3.tgz#922a0155de30ecc1f785bcf04be77844ca95ad07" + integrity sha512-IScLbePpkvO846sIwOtOTDjutRMWdXdJmXdMvk6gCBHxFO8d+QKOQedyZSxFTTFYRSmlgSTDtXqqq4pcenBXLQ== hosted-git-info@^2.1.4: version "2.8.9" resolved "https://registry.yarnpkg.com/hosted-git-info/-/hosted-git-info-2.8.9.tgz#dffc0bf9a21c02209090f2aa69429e1414daf3f9" integrity sha512-mxIDAb9Lsm6DoOJ7xH+5+X4y1LU/4Hi50L9C5sIswK3JzULS4bwk1FvjdBgvYR4bzT4tuUQiC15FE2f5HbLvYw== -html-encoding-sniffer@^3.0.0: - version "3.0.0" - resolved "https://registry.yarnpkg.com/html-encoding-sniffer/-/html-encoding-sniffer-3.0.0.tgz#2cb1a8cf0db52414776e5b2a7a04d5dd98158de9" - integrity sha512-oWv4T4yJ52iKrufjnyZPkrN0CH3QnrUqdB6In1g5Fe1mia8GmF36gnfNySxoZtxD5+NmYw1EElVXiBk93UeskA== +html-encoding-sniffer@^4.0.0: + version "4.0.0" + resolved "https://registry.yarnpkg.com/html-encoding-sniffer/-/html-encoding-sniffer-4.0.0.tgz#696df529a7cfd82446369dc5193e590a3735b448" + integrity sha512-Y22oTqIU4uuPgEemfz7NDJz6OeKf12Lsu+QC+s3BVpda64lTiMYCyGwg5ki4vFxkMwQdeZDl2adZoqUgdFuTgQ== dependencies: - whatwg-encoding "^2.0.0" + whatwg-encoding "^3.1.1" http-errors@2.0.0: version "2.0.0" @@ -2366,24 +2296,28 @@ http-errors@2.0.0: statuses "2.0.1" toidentifier "1.0.1" -http-proxy-agent@^5.0.0: - version "5.0.0" - resolved "https://registry.yarnpkg.com/http-proxy-agent/-/http-proxy-agent-5.0.0.tgz#5129800203520d434f142bc78ff3c170800f2b43" - integrity sha512-n2hY8YdoRE1i7r6M0w9DIw5GgZN0G25P8zLCRQ8rjXtTU3vsNFBI/vWK/UIeE6g5MUUz6avwAPXmL6Fy9D/90w== +http-proxy-agent@^7.0.0: + version "7.0.2" + resolved "https://registry.yarnpkg.com/http-proxy-agent/-/http-proxy-agent-7.0.2.tgz#9a8b1f246866c028509486585f62b8f2c18c270e" + integrity sha512-T1gkAiYYDWYx3V5Bmyu7HcfcvL7mUrTWiM6yOfa3PIphViJ/gFPbvidQ+veqSOHci/PxBcDabeUNCzpOODJZig== dependencies: - "@tootallnate/once" "2" - agent-base "6" - debug "4" + agent-base "^7.1.0" + debug "^4.3.4" -https-proxy-agent@^5.0.1: - version "5.0.1" - resolved "https://registry.yarnpkg.com/https-proxy-agent/-/https-proxy-agent-5.0.1.tgz#c59ef224a04fe8b754f3db0063a25ea30d0005d6" - integrity sha512-dFcAjpTQFgoLMzC2VwU+C/CbS7uRL0lWmxDITmqm7C+7F0Odmj6s9l6alZc6AELXhrnggM2CeWSXHGOdX2YtwA== +https-proxy-agent@^7.0.2: + version "7.0.4" + resolved "https://registry.yarnpkg.com/https-proxy-agent/-/https-proxy-agent-7.0.4.tgz#8e97b841a029ad8ddc8731f26595bad868cb4168" + integrity sha512-wlwpilI7YdjSkWaQ/7omYBMTliDcmCN8OLihO6I9B86g06lMyAoqgoDpV0XqoaPOKj+0DIdAvnsWfyAAhmimcg== dependencies: - agent-base "6" + agent-base "^7.0.2" debug "4" -iconv-lite@0.4.24, iconv-lite@^0.4.24: +human-signals@^5.0.0: + version "5.0.0" + resolved "https://registry.yarnpkg.com/human-signals/-/human-signals-5.0.0.tgz#42665a284f9ae0dade3ba41ebc37eb4b852f3a28" + integrity sha512-AXcZb6vzzrFAUE61HnN4mpLqd/cSIwNQjtNWR0euPm6y0iqx3G4gOXaIDdtdDwZmhwe82LA6+zinmW4UBWVePQ== + +iconv-lite@0.4.24: version "0.4.24" resolved "https://registry.yarnpkg.com/iconv-lite/-/iconv-lite-0.4.24.tgz#2022b4b25fbddc21d2f524974a474aafe733908b" integrity sha512-v3MXnZAcvnywkTUEZomIActle7RXXeedOR31wwl7VlyoXO4Qi9arvSenNQWne1TcRwhCL1HwLI21bEqdpj8/rA== @@ -2397,22 +2331,22 @@ iconv-lite@0.6.3: dependencies: safer-buffer ">= 2.1.2 < 3.0.0" -ieee754@^1.1.13: - version "1.2.1" - resolved "https://registry.yarnpkg.com/ieee754/-/ieee754-1.2.1.tgz#8eb7a10a63fff25d15a57b001586d177d1b0d352" - integrity sha512-dcyqhDvX1C46lXZcVqCpK+FtMRQVdIMN6/Df5js2zouUsqG7I6sFxitIC+7KYK29KdXOLHdu9zL4sFnoVQnqaA== - ignore@^5.2.0: version "5.2.4" resolved "https://registry.yarnpkg.com/ignore/-/ignore-5.2.4.tgz#a291c0c6178ff1b960befe47fcdec301674a6324" integrity sha512-MAb38BcSbH0eHNBxn7ql2NH/kX33OkB3lZ1BNdh7ENeRChHTYsTvWrMubiIAMNS2llXEEgZ1MUOBtXChP3kaFQ== +ignore@^5.3.1: + version "5.3.1" + resolved "https://registry.yarnpkg.com/ignore/-/ignore-5.3.1.tgz#5073e554cd42c5b33b394375f538b8593e34d4ef" + integrity sha512-5Fytz/IraMjqpwfd34ke28PTVMjZjJG2MPn5t7OE4eUCUNf8BAa7b5WUS9/Qvr6mwOQS7Mk6vdsMno5he+T8Xw== + immutable@^4.0.0: version "4.2.4" resolved "https://registry.yarnpkg.com/immutable/-/immutable-4.2.4.tgz#83260d50889526b4b531a5e293709a77f7c55a2a" integrity sha512-WDxL3Hheb1JkRN3sQkyujNlL/xRjAo3rJtaU5xeufUauG66JdMr32bLj4gF+vWl84DIA3Zxw7tiAjneYzRRw+w== -import-fresh@^3.0.0, import-fresh@^3.2.1: +import-fresh@^3.2.1: version "3.3.0" resolved "https://registry.yarnpkg.com/import-fresh/-/import-fresh-3.3.0.tgz#37162c25fcb9ebaa2e6e53d5b4d88ce17d9e0c2b" integrity sha512-veYYhQa+D1QBKznvhUHxb8faxlrwUnxseDAbAp457E0wLNio2bOSKnjYDhMj+YiAq61xrMGhQk9iXVk5FzgQMw== @@ -2438,7 +2372,7 @@ inflight@^1.0.4: once "^1.3.0" wrappy "1" -inherits@2, inherits@2.0.4, inherits@^2.0.3, inherits@^2.0.4: +inherits@2, inherits@2.0.4: version "2.0.4" resolved "https://registry.yarnpkg.com/inherits/-/inherits-2.0.4.tgz#0fa2c64f932917c3433a0ded55363aae37416b7c" integrity sha512-k/vGaX4/Yla3WzyMCvTQOXYeIHvqOKtnqBduzTHpzpQZzAskKMhZ2K+EnBiSM9zGSoIFeMpXKxa4dYeZIQqewQ== @@ -2448,27 +2382,6 @@ ini@^1.3.4: resolved "https://registry.yarnpkg.com/ini/-/ini-1.3.8.tgz#a29da425b48806f34767a4efce397269af28432c" integrity sha512-JV/yugV2uzW5iMRSiZAyDtQd+nxtUnjeLt0acNdw98kKLrvuRVyB80tsREOE7yvGVgalhZ6RNXCmEHkUKBKxew== -inquirer@^8.2.0: - version "8.2.5" - resolved "https://registry.yarnpkg.com/inquirer/-/inquirer-8.2.5.tgz#d8654a7542c35a9b9e069d27e2df4858784d54f8" - integrity sha512-QAgPDQMEgrDssk1XiwwHoOGYF9BAbUcc1+j+FhEvaOt8/cKRqyLn0U5qA6F74fGhTMGxf92pOvPBeh29jQJDTQ== - dependencies: - ansi-escapes "^4.2.1" - chalk "^4.1.1" - cli-cursor "^3.1.0" - cli-width "^3.0.0" - external-editor "^3.0.3" - figures "^3.0.0" - lodash "^4.17.21" - mute-stream "0.0.8" - ora "^5.4.1" - run-async "^2.4.0" - rxjs "^7.5.5" - string-width "^4.1.0" - strip-ansi "^6.0.0" - through "^2.3.6" - wrap-ansi "^7.0.0" - internal-slot@^1.0.4: version "1.0.4" resolved "https://registry.yarnpkg.com/internal-slot/-/internal-slot-1.0.4.tgz#8551e7baf74a7a6ba5f749cfb16aa60722f0d6f3" @@ -2483,7 +2396,7 @@ ipaddr.js@1.9.1: resolved "https://registry.yarnpkg.com/ipaddr.js/-/ipaddr.js-1.9.1.tgz#bff38543eeb8984825079ff3a2a8e6cbd46781b3" integrity sha512-0KI/607xoxSToH7GjN1FfSbLoU0+btTicjsQSWQlh/hZykN8KpmMf7uYwPW3R+akZ6R/w18ZlXSHBYXiYUPO3g== -is-arguments@^1.0.4, is-arguments@^1.1.1: +is-arguments@^1.1.1: version "1.1.1" resolved "https://registry.yarnpkg.com/is-arguments/-/is-arguments-1.1.1.tgz#15b3f88fda01f2a97fec84ca761a560f123efa9b" integrity sha512-8Q7EARjzEnKpt/PCD7e1cgUS0a6X8u5tdSiMqXhojOdoV9TsMsiO+9VLC5vAmO8N7/GmXn7yjR8qnA6bVAEzfA== @@ -2556,13 +2469,6 @@ is-fullwidth-code-point@^3.0.0: resolved "https://registry.yarnpkg.com/is-fullwidth-code-point/-/is-fullwidth-code-point-3.0.0.tgz#f116f8064fe90b3f7844a38997c0b75051269f1d" integrity sha512-zymm5+u+sCsSWyD9qNaejV3DFvhCKclKdizYaJUuHA83RLjb7nSuGnddCHGv0hk+KY7BMAlsWeK4Ueg6EV6XQg== -is-generator-function@^1.0.7: - version "1.0.10" - resolved "https://registry.yarnpkg.com/is-generator-function/-/is-generator-function-1.0.10.tgz#f1558baf1ac17e0deea7c0415c438351ff2b3c72" - integrity sha512-jsEjy9l3yiXEQ+PsXdmBwEPcOxaXWLspKdplFUVI9vq1iZgIekeC0L167qeu86czQaxed3q/Uzuw0swL0irL8A== - dependencies: - has-tostringtag "^1.0.0" - is-glob@^4.0.0, is-glob@^4.0.1, is-glob@^4.0.3, is-glob@~4.0.1: version "4.0.3" resolved "https://registry.yarnpkg.com/is-glob/-/is-glob-4.0.3.tgz#64f61e42cbbb2eec2071a9dac0b28ba1e65d5084" @@ -2570,11 +2476,6 @@ is-glob@^4.0.0, is-glob@^4.0.1, is-glob@^4.0.3, is-glob@~4.0.1: dependencies: is-extglob "^2.1.1" -is-interactive@^1.0.0: - version "1.0.0" - resolved "https://registry.yarnpkg.com/is-interactive/-/is-interactive-1.0.0.tgz#cea6e6ae5c870a7b0a0004070b7b587e0252912e" - integrity sha512-2HvIEKRoqS62guEC+qBjpvRubdX910WCMuJTZ+I9yvqKU2/12eSL549HMwtabb4oupdj2sMP50k+XJfB/8JE6w== - is-map@^2.0.1, is-map@^2.0.2: version "2.0.2" resolved "https://registry.yarnpkg.com/is-map/-/is-map-2.0.2.tgz#00922db8c9bf73e81b7a335827bc2a43f2b91127" @@ -2637,6 +2538,11 @@ is-shared-array-buffer@^1.0.2: dependencies: call-bind "^1.0.2" +is-stream@^3.0.0: + version "3.0.0" + resolved "https://registry.yarnpkg.com/is-stream/-/is-stream-3.0.0.tgz#e6bfd7aa6bef69f4f472ce9bb681e3e57b4319ac" + integrity sha512-LnQR4bZ9IADDRSkvpqMGvt/tEJWclzklNgSw48V5EAaAeDd6qGvN8ei6k5p0tvxSR171VmGyHuTiAOfxAbr8kA== + is-string@^1.0.5, is-string@^1.0.7: version "1.0.7" resolved "https://registry.yarnpkg.com/is-string/-/is-string-1.0.7.tgz#0dd12bf2006f255bb58f695110eff7491eebc0fd" @@ -2651,7 +2557,7 @@ is-symbol@^1.0.2, is-symbol@^1.0.3: dependencies: has-symbols "^1.0.2" -is-typed-array@^1.1.10, is-typed-array@^1.1.3, is-typed-array@^1.1.9: +is-typed-array@^1.1.10, is-typed-array@^1.1.9: version "1.1.10" resolved "https://registry.yarnpkg.com/is-typed-array/-/is-typed-array-1.1.10.tgz#36a5b5cb4189b575d1a3e4b08536bfb485801e3f" integrity sha512-PJqgEHiWZvMpaFZ3uTc8kHPM4+4ADTlDniuQL7cU/UDA0Ql7F70yGfHph3cLNe+c9toaigv+DFzTJKhc2CtO6A== @@ -2662,11 +2568,6 @@ is-typed-array@^1.1.10, is-typed-array@^1.1.3, is-typed-array@^1.1.9: gopd "^1.0.1" has-tostringtag "^1.0.0" -is-unicode-supported@^0.1.0: - version "0.1.0" - resolved "https://registry.yarnpkg.com/is-unicode-supported/-/is-unicode-supported-0.1.0.tgz#3f26c76a809593b52bfa2ecb5710ed2779b522a7" - integrity sha512-knxG2q4UC3u8stRGyAVJCOdxFmv5DZiRcdlIaAQXAbSfJya+OhopNotLQrstBhququ4ZpuKbDc/8S6mgXgPFPw== - is-weakmap@^2.0.1: version "2.0.1" resolved "https://registry.yarnpkg.com/is-weakmap/-/is-weakmap-2.0.1.tgz#5008b59bdc43b698201d18f62b37b2ca243e8cf2" @@ -2702,93 +2603,46 @@ isexe@^2.0.0: resolved "https://registry.yarnpkg.com/isexe/-/isexe-2.0.0.tgz#e8fbf374dc556ff8947a10dcb0572d633f2cfa10" integrity sha512-RHxMLp9lnKHGHRng9QFhRCMbYAcVpn69smSGcq3f36xjgVVWThj4qqLbTLlq7Ssj8B+fIQ1EuCEGI2lKsyQeIw== -jest-diff@^29.4.2: - version "29.4.2" - resolved "https://registry.yarnpkg.com/jest-diff/-/jest-diff-29.4.2.tgz#b88502d5dc02d97f6512d73c37da8b36f49b4871" - integrity sha512-EK8DSajVtnjx9sa1BkjZq3mqChm2Cd8rIzdXkQMA8e0wuXq53ypz6s5o5V8HRZkoEt2ywJ3eeNWFKWeYr8HK4g== - dependencies: - chalk "^4.0.0" - diff-sequences "^29.4.2" - jest-get-type "^29.4.2" - pretty-format "^29.4.2" - -jest-get-type@^29.4.2: - version "29.4.2" - resolved "https://registry.yarnpkg.com/jest-get-type/-/jest-get-type-29.4.2.tgz#7cb63f154bca8d8f57364d01614477d466fa43fe" - integrity sha512-vERN30V5i2N6lqlFu4ljdTqQAgrkTFMC9xaIIfOPYBw04pufjXRty5RuXBiB1d72tGbURa/UgoiHB90ruOSivg== - -jest-matcher-utils@^29.4.2: - version "29.4.2" - resolved "https://registry.yarnpkg.com/jest-matcher-utils/-/jest-matcher-utils-29.4.2.tgz#08d0bf5abf242e3834bec92c7ef5071732839e85" - integrity sha512-EZaAQy2je6Uqkrm6frnxBIdaWtSYFoR8SVb2sNLAtldswlR/29JAgx+hy67llT3+hXBaLB0zAm5UfeqerioZyg== - dependencies: - chalk "^4.0.0" - jest-diff "^29.4.2" - jest-get-type "^29.4.2" - pretty-format "^29.4.2" - -jest-message-util@^29.4.2: - version "29.4.2" - resolved "https://registry.yarnpkg.com/jest-message-util/-/jest-message-util-29.4.2.tgz#309a2924eae6ca67cf7f25781a2af1902deee717" - integrity sha512-SElcuN4s6PNKpOEtTInjOAA8QvItu0iugkXqhYyguRvQoXapg5gN+9RQxLAkakChZA7Y26j6yUCsFWN+hlKD6g== - dependencies: - "@babel/code-frame" "^7.12.13" - "@jest/types" "^29.4.2" - "@types/stack-utils" "^2.0.0" - chalk "^4.0.0" - graceful-fs "^4.2.9" - micromatch "^4.0.4" - pretty-format "^29.4.2" - slash "^3.0.0" - stack-utils "^2.0.3" - -jest-util@^29.4.2: - version "29.4.2" - resolved "https://registry.yarnpkg.com/jest-util/-/jest-util-29.4.2.tgz#3db8580b295df453a97de4a1b42dd2578dabd2c2" - integrity sha512-wKnm6XpJgzMUSRFB7YF48CuwdzuDIHenVuoIb1PLuJ6F+uErZsuDkU+EiExkChf6473XcawBrSfDSnXl+/YG4g== +jackspeak@^2.3.6: + version "2.3.6" + resolved "https://registry.yarnpkg.com/jackspeak/-/jackspeak-2.3.6.tgz#647ecc472238aee4b06ac0e461acc21a8c505ca8" + integrity sha512-N3yCS/NegsOBokc8GAdM8UcmfsKiSS8cipheD/nivzr700H+nsMOxJjQnvwOcRYVuFkdH0wGUvW2WbXGmrZGbQ== dependencies: - "@jest/types" "^29.4.2" - "@types/node" "*" - chalk "^4.0.0" - ci-info "^3.2.0" - graceful-fs "^4.2.9" - picomatch "^2.2.3" + "@isaacs/cliui" "^8.0.2" + optionalDependencies: + "@pkgjs/parseargs" "^0.11.0" jju@^1.1.0: version "1.4.0" resolved "https://registry.yarnpkg.com/jju/-/jju-1.4.0.tgz#a3abe2718af241a2b2904f84a625970f389ae32a" integrity sha512-8wb9Yw966OSxApiCt0K3yNJL8pnNeIv+OEq2YMidz4FKP6nonSRoOXc80iXY4JaN2FC11B9qsNmDsm+ZOfMROA== -js-beautify@1.14.6: - version "1.14.6" - resolved "https://registry.yarnpkg.com/js-beautify/-/js-beautify-1.14.6.tgz#b23ca5d74a462c282c7711bb51150bcc97f2b507" - integrity sha512-GfofQY5zDp+cuHc+gsEXKPpNw2KbPddreEo35O6jT6i0RVK6LhsoYBhq5TvK4/n74wnA0QbK8gGd+jUZwTMKJw== +js-beautify@^1.14.9: + version "1.15.1" + resolved "https://registry.yarnpkg.com/js-beautify/-/js-beautify-1.15.1.tgz#4695afb508c324e1084ee0b952a102023fc65b64" + integrity sha512-ESjNzSlt/sWE8sciZH8kBF8BPlwXPwhR6pWKAw8bw4Bwj+iZcnKW6ONWUutJ7eObuBZQpiIb8S7OYspWrKt7rA== dependencies: config-chain "^1.1.13" - editorconfig "^0.15.3" - glob "^8.0.3" - nopt "^6.0.0" + editorconfig "^1.0.4" + glob "^10.3.3" + js-cookie "^3.0.5" + nopt "^7.2.0" -js-levenshtein@^1.1.6: - version "1.1.6" - resolved "https://registry.yarnpkg.com/js-levenshtein/-/js-levenshtein-1.1.6.tgz#c6cee58eb3550372df8deb85fad5ce66ce01d59d" - integrity sha512-X2BB11YZtrRqY4EnQcLX5Rh373zbK4alC1FW7D7MBhL2gtcC17cTnr6DmfHZeS0s2rTHjUTMMHfG7gO8SSdw+g== - -js-sdsl@^4.1.4: - version "4.3.0" - resolved "https://registry.yarnpkg.com/js-sdsl/-/js-sdsl-4.3.0.tgz#aeefe32a451f7af88425b11fdb5f58c90ae1d711" - integrity sha512-mifzlm2+5nZ+lEcLJMoBK0/IH/bDg8XnJfd/Wq6IP+xoCjLZsTOnV2QpxlVbX9bMnkl5PdEjNtBJ9Cj1NjifhQ== - -js-string-escape@^1.0.1: - version "1.0.1" - resolved "https://registry.yarnpkg.com/js-string-escape/-/js-string-escape-1.0.1.tgz#e2625badbc0d67c7533e9edc1068c587ae4137ef" - integrity sha512-Smw4xcfIQ5LVjAOuJCvN/zIodzA/BBSsluuoSykP+lUvScIi4U6RJLfwHet5cxFnCswUjISV8oAXaqaJDY3chg== +js-cookie@^3.0.5: + version "3.0.5" + resolved "https://registry.yarnpkg.com/js-cookie/-/js-cookie-3.0.5.tgz#0b7e2fd0c01552c58ba86e0841f94dc2557dcdbc" + integrity sha512-cEiJEAEoIbWfCZYKWhVwFuvPX1gETRYPw6LlaTKoxD3s2AkXzkCjnp6h0V77ozyqj0jakteJ4YqDJT830+lVGw== js-tokens@^4.0.0: version "4.0.0" resolved "https://registry.yarnpkg.com/js-tokens/-/js-tokens-4.0.0.tgz#19203fb59991df98e3a287050d4647cdeaf32499" integrity sha512-RdJUflcE3cUzKiMqQgsCu06FPu9UdIJO0beYbPhHN4k6apgJtifcoCtT9bcxOpYBtpD2kCM6Sbzg4CausW/PKQ== +js-tokens@^9.0.0: + version "9.0.0" + resolved "https://registry.yarnpkg.com/js-tokens/-/js-tokens-9.0.0.tgz#0f893996d6f3ed46df7f0a3b12a03f5fd84223c1" + integrity sha512-WriZw1luRMlmV3LGJaR6QOJjWwgLUTf89OwT2lUOyjX2dJGBwgmIkbcz+7WFZjrZM635JOIR517++e/67CP9dQ== + js-yaml@^4.1.0: version "4.1.0" resolved "https://registry.yarnpkg.com/js-yaml/-/js-yaml-4.1.0.tgz#c1fb65f8f5017901cdd2c951864ba18458a10602" @@ -2796,34 +2650,32 @@ js-yaml@^4.1.0: dependencies: argparse "^2.0.1" -jsdom@22.1.0: - version "22.1.0" - resolved "https://registry.yarnpkg.com/jsdom/-/jsdom-22.1.0.tgz#0fca6d1a37fbeb7f4aac93d1090d782c56b611c8" - integrity sha512-/9AVW7xNbsBv6GfWho4TTNjEo9fe6Zhf9O7s0Fhhr3u+awPwAJMKwAMXnkk5vBxflqLW9hTHX/0cs+P3gW+cQw== +jsdom@24.0.0: + version "24.0.0" + resolved "https://registry.yarnpkg.com/jsdom/-/jsdom-24.0.0.tgz#e2dc04e4c79da368481659818ee2b0cd7c39007c" + integrity sha512-UDS2NayCvmXSXVP6mpTj+73JnNQadZlr9N68189xib2tx5Mls7swlTNao26IoHv46BZJFvXygyRtyXd1feAk1A== dependencies: - abab "^2.0.6" - cssstyle "^3.0.0" - data-urls "^4.0.0" + cssstyle "^4.0.1" + data-urls "^5.0.0" decimal.js "^10.4.3" - domexception "^4.0.0" form-data "^4.0.0" - html-encoding-sniffer "^3.0.0" - http-proxy-agent "^5.0.0" - https-proxy-agent "^5.0.1" + html-encoding-sniffer "^4.0.0" + http-proxy-agent "^7.0.0" + https-proxy-agent "^7.0.2" is-potential-custom-element-name "^1.0.1" - nwsapi "^2.2.4" + nwsapi "^2.2.7" parse5 "^7.1.2" rrweb-cssom "^0.6.0" saxes "^6.0.0" symbol-tree "^3.2.4" - tough-cookie "^4.1.2" - w3c-xmlserializer "^4.0.0" + tough-cookie "^4.1.3" + w3c-xmlserializer "^5.0.0" webidl-conversions "^7.0.0" - whatwg-encoding "^2.0.0" - whatwg-mimetype "^3.0.0" - whatwg-url "^12.0.1" - ws "^8.13.0" - xml-name-validator "^4.0.0" + whatwg-encoding "^3.1.1" + whatwg-mimetype "^4.0.0" + whatwg-url "^14.0.0" + ws "^8.16.0" + xml-name-validator "^5.0.0" json-parse-better-errors@^1.0.1: version "1.0.2" @@ -2842,10 +2694,10 @@ json-schema-traverse@^0.4.1: resolved "https://registry.yarnpkg.com/json-schema-traverse/-/json-schema-traverse-0.4.1.tgz#69f6a87d9513ab8bb8fe63bdb0979c448e684660" integrity sha512-xbbCH5dCYU5T8LcEhhuh7HJ88HXuW3qsI3Y0zOZFKfZEHcpWiHU/Jxzk629Brsab/mMiHQti9wMP+845RPe3Vg== -json-server@0.17.1: - version "0.17.1" - resolved "https://registry.yarnpkg.com/json-server/-/json-server-0.17.1.tgz#7ce793e4ee52e815648b0ce4e0736197bb671e5c" - integrity sha512-2cR/IAL9xX4M1Y5ONMpnvRvJ5o9gJH5GsSQs7fQHdTKTKoN4YRZu1ApQKg/4P0XzwlLyUDqTlwhPq9nsvJ9acw== +json-server@0.17.4: + version "0.17.4" + resolved "https://registry.yarnpkg.com/json-server/-/json-server-0.17.4.tgz#d4ef25a516e26d9ba86fd6db2f9d81a5f405421e" + integrity sha512-bGBb0WtFuAKbgI7JV3A864irWnMZSvBYRJbohaOuatHwKSRFUfqtQlrYMrB6WbalXy/cJabyjlb7JkHli6dYjQ== dependencies: body-parser "^1.19.0" chalk "^4.1.2" @@ -2877,10 +2729,10 @@ jsonc-parser@^3.2.0: resolved "https://registry.yarnpkg.com/jsonc-parser/-/jsonc-parser-3.2.0.tgz#31ff3f4c2b9793f89c67212627c51c6394f88e76" integrity sha512-gfFQZrcTc8CnKXp6Y4/CBT3fTc0OVuDofpre4aEeEpSBPV5X5v4+Vmx+8snU7RLPrNHPKSgLxGo9YuQzz20o+w== -jwt-decode@3.1.2: - version "3.1.2" - resolved "https://registry.yarnpkg.com/jwt-decode/-/jwt-decode-3.1.2.tgz#3fb319f3675a2df0c2895c8f5e9fa4b67b04ed59" - integrity sha512-UfpWE/VZn0iP50d8cz9NrZLM9lSWhcJ+0Gt/nm4by88UL+J1SiKN8/5dkjMmbEzwL2CAe+67GsegCbIKtbp75A== +jwt-decode@4.0.0: + version "4.0.0" + resolved "https://registry.yarnpkg.com/jwt-decode/-/jwt-decode-4.0.0.tgz#2270352425fd413785b2faf11f6e755c5151bd4b" + integrity sha512-+KJGIyHgkGuIq3IEBNftfhW/LfWhXUIY6OmyVWjliu5KH1y0fw7VQ8YndE2O4qZdMSd9SqbnC8GOcZEy0Om7sA== levn@^0.4.1: version "0.4.1" @@ -2900,17 +2752,13 @@ load-json-file@^4.0.0: pify "^3.0.0" strip-bom "^3.0.0" -local-pkg@^0.4.3: - version "0.4.3" - resolved "https://registry.yarnpkg.com/local-pkg/-/local-pkg-0.4.3.tgz#0ff361ab3ae7f1c19113d9bb97b98b905dbc4963" - integrity sha512-SFppqq5p42fe2qcZQqqEOiVRXl+WCP1MdT6k7BDEW1j++sp5fIY+/fdRQitvKgB5BrBcmrs5m/L0v2FrU5MY1g== - -locate-path@^5.0.0: - version "5.0.0" - resolved "https://registry.yarnpkg.com/locate-path/-/locate-path-5.0.0.tgz#1afba396afd676a6d42504d0a67a3a7eb9f62aa0" - integrity sha512-t7hw9pI+WvuwNJXwk5zVHpyhIqzg2qTlklJOf0mVxGSbe3Fp2VieZcduNYjaLDoy6p9uGpQEGWG87WpMKlNq8g== +local-pkg@^0.5.0: + version "0.5.0" + resolved "https://registry.yarnpkg.com/local-pkg/-/local-pkg-0.5.0.tgz#093d25a346bae59a99f80e75f6e9d36d7e8c925c" + integrity sha512-ok6z3qlYyCDS4ZEU27HaU6x/xZa9Whf8jD4ptH5UZTQYZVYeb9bnZ3ojVhiJNLiXK1Hfc0GNbLXcmZ5plLDDBg== dependencies: - p-locate "^4.1.0" + mlly "^1.4.2" + pkg-types "^1.0.3" locate-path@^6.0.0: version "6.0.0" @@ -2934,21 +2782,20 @@ lodash@4, lodash@^4.17.15, lodash@^4.17.21: resolved "https://registry.yarnpkg.com/lodash/-/lodash-4.17.21.tgz#679591c564c3bffaae8454cf0b3df370c3d6911c" integrity sha512-v2kDEe57lecTulaDIuNTPy3Ry4gLGJ6Z1O3vE1krgXZNrsQ+LFTGHVxVjcXPs17LhbZVGedAJv8XZ1tvj5FvSg== -log-symbols@^4.1.0: - version "4.1.0" - resolved "https://registry.yarnpkg.com/log-symbols/-/log-symbols-4.1.0.tgz#3fbdbb95b4683ac9fc785111e792e558d4abd503" - integrity sha512-8XPvpAA8uyhfteu8pIvQxpJZ7SYYdpUivZpGy6sFsBuKRY/7rQGavedeB8aK+Zkyq6upMFVL/9AW6vOYzfRyLg== - dependencies: - chalk "^4.1.0" - is-unicode-supported "^0.1.0" - -loupe@^2.3.1, loupe@^2.3.6: +loupe@^2.3.6: version "2.3.6" resolved "https://registry.yarnpkg.com/loupe/-/loupe-2.3.6.tgz#76e4af498103c532d1ecc9be102036a21f787b53" integrity sha512-RaPMZKiMy8/JruncMU5Bt6na1eftNoo++R4Y+N2FrxkDVTrGvcyzFTsaGif4QTeKESheMGegbhw6iUAq+5A8zA== dependencies: get-func-name "^2.0.0" +loupe@^2.3.7: + version "2.3.7" + resolved "https://registry.yarnpkg.com/loupe/-/loupe-2.3.7.tgz#6e69b7d4db7d3ab436328013d37d1c8c3540c697" + integrity sha512-zSMINGVYkdpYSOBmLi0D1Uo7JU9nVdQKrHxC8eYlV+9YKK9WePqAlL7lSlorG/U2Fw1w0hTBmaa/jrQ3UbPHtA== + dependencies: + get-func-name "^2.0.1" + lowdb@^1.0.0: version "1.0.0" resolved "https://registry.yarnpkg.com/lowdb/-/lowdb-1.0.0.tgz#5243be6b22786ccce30e50c9a33eac36b20c8064" @@ -2960,13 +2807,10 @@ lowdb@^1.0.0: pify "^3.0.0" steno "^0.4.1" -lru-cache@^4.1.5: - version "4.1.5" - resolved "https://registry.yarnpkg.com/lru-cache/-/lru-cache-4.1.5.tgz#8bbe50ea85bed59bc9e33dcab8235ee9bcf443cd" - integrity sha512-sWZlbEP2OsHNkXrMl5GYk/jKk70MBng6UU4YI/qGDYbgf6YbP4EvmqISbXCoJiRKs+1bSpFHVgQxvJ17F2li5g== - dependencies: - pseudomap "^1.0.2" - yallist "^2.1.2" +lru-cache@^10.2.0: + version "10.2.0" + resolved "https://registry.yarnpkg.com/lru-cache/-/lru-cache-10.2.0.tgz#0bd445ca57363465900f4d1f9bd8db343a4d95c3" + integrity sha512-2bIM8x+VAf6JT4bKAljS1qUWgMsqZRPGJS6FSahIMPVvctcNhyVp7AJu7quxOW9jwkryBReKZY5tY5JYv2n/7Q== lru-cache@^6.0.0: version "6.0.0" @@ -2980,33 +2824,12 @@ lz-string@^1.5.0: resolved "https://registry.yarnpkg.com/lz-string/-/lz-string-1.5.0.tgz#c1ab50f77887b712621201ba9fd4e3a6ed099941" integrity sha512-h5bgJWpxJNswbU7qCrV0tIKQCaS3blPDrqKWx+QxzuzL1zGUzij9XCWLrSLsJPu5t+eWA/ycetzYAO5IOMcWAQ== -magic-string@^0.25.7: - version "0.25.9" - resolved "https://registry.yarnpkg.com/magic-string/-/magic-string-0.25.9.tgz#de7f9faf91ef8a1c91d02c2e5314c8277dbcdd1c" - integrity sha512-RmF0AsMzgt25qzqqLc1+MbHmhdx0ojF2Fvs4XnOqz2ZOBXzzkEwc/dJQZCYHAn7v1jbVOjAZfK8msRn4BxO4VQ== - dependencies: - sourcemap-codec "^1.4.8" - -magic-string@^0.30.0: - version "0.30.0" - resolved "https://registry.yarnpkg.com/magic-string/-/magic-string-0.30.0.tgz#fd58a4748c5c4547338a424e90fa5dd17f4de529" - integrity sha512-LA+31JYDJLs82r2ScLrlz1GjSgu66ZV518eyWT+S8VhyQn/JL0u9MeBOvQMGYiPk1DBiSN9DDMOcXvigJZaViQ== - dependencies: - "@jridgewell/sourcemap-codec" "^1.4.13" - -make-dir@^3.0.2: - version "3.1.0" - resolved "https://registry.yarnpkg.com/make-dir/-/make-dir-3.1.0.tgz#415e967046b3a7f1d185277d84aa58203726a13f" - integrity sha512-g3FeP20LNwhALb/6Cz6Dd4F2ngze0jz7tbzrD2wAV+o9FeNHe4rL+yK2md0J/fiSf1sa1ADhXqi5+oVwOM/eGw== - dependencies: - semver "^6.0.0" - -md5-hex@^3.0.1: - version "3.0.1" - resolved "https://registry.yarnpkg.com/md5-hex/-/md5-hex-3.0.1.tgz#be3741b510591434b2784d79e556eefc2c9a8e5c" - integrity sha512-BUiRtTtV39LIJwinWBjqVsU9xhdnz7/i889V859IBFpuqGAj6LuOvHv5XLbgZ2R7ptJoJaEcxkv88/h25T7Ciw== +magic-string@^0.30.5, magic-string@^0.30.8: + version "0.30.10" + resolved "https://registry.yarnpkg.com/magic-string/-/magic-string-0.30.10.tgz#123d9c41a0cb5640c892b041d4cfb3bd0aa4b39e" + integrity sha512-iIRwTIf0QKV3UAnYK4PU8uiEc4SRh5jX0mwpIwETPpHdhVM4f53RSwS/vXvN1JhGX+Cs7B8qIq3d6AH49O5fAQ== dependencies: - blueimp-md5 "^2.10.0" + "@jridgewell/sourcemap-codec" "^1.4.15" media-typer@0.3.0: version "0.3.0" @@ -3023,6 +2846,11 @@ merge-descriptors@1.0.1: resolved "https://registry.yarnpkg.com/merge-descriptors/-/merge-descriptors-1.0.1.tgz#b00aaa556dd8b44568150ec9d1b953f3f90cbb61" integrity sha512-cCi6g3/Zr1iqQi6ySbseM1Xvooa98N0w31jzUYrXPX2xqObmFGHJ0tQ5u74H3mVh7wLouTseZyYIq39g8cNp1w== +merge-stream@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/merge-stream/-/merge-stream-2.0.0.tgz#52823629a14dd00c9770fb6ad47dc6310f2c1f60" + integrity sha512-abv/qOcuPfk3URPfDzmZU1LKmuw8kT+0nIHvKrKgFrwifol/doWcdA4ZqsWQ8ENrFKkd67Mfpo/LovbIUsbt3w== + merge2@^1.3.0, merge2@^1.4.1: version "1.4.1" resolved "https://registry.yarnpkg.com/merge2/-/merge2-1.4.1.tgz#4368892f885e907455a6fd7dc55c0c9d404990ae" @@ -3068,16 +2896,23 @@ mime@1.6.0: resolved "https://registry.yarnpkg.com/mime/-/mime-1.6.0.tgz#32cd9e5c64553bd58d19a568af452acff04981b1" integrity sha512-x0Vn8spI+wuJ1O6S7gnbaQg8Pxh4NNHb7KSINmEWKiPE4RKOplvijn+NkmYmmRgP68mc70j2EbeTFRsrswaQeg== -mimic-fn@^2.1.0: - version "2.1.0" - resolved "https://registry.yarnpkg.com/mimic-fn/-/mimic-fn-2.1.0.tgz#7ed2c2ccccaf84d3ffcb7a69b57711fc2083401b" - integrity sha512-OqbOk5oEQeAZ8WXWydlu9HJjz9WVdEIvamMCcXmuqUYjTknH/sqsWvhQ3vgwKFRR1HpjvNBKQ37nbJgYzGqGcg== +mimic-fn@^4.0.0: + version "4.0.0" + resolved "https://registry.yarnpkg.com/mimic-fn/-/mimic-fn-4.0.0.tgz#60a90550d5cb0b239cca65d893b1a53b29871ecc" + integrity sha512-vqiC06CuhBTUdZH+RYl8sFrL096vA45Ok5ISO6sE/Mr1jRbGH4Csnhi8f3wKVl7x8mO4Au7Ir9D3Oyv1VYMFJw== min-indent@^1.0.0: version "1.0.1" resolved "https://registry.yarnpkg.com/min-indent/-/min-indent-1.0.1.tgz#a63f681673b30571fbe8bc25686ae746eefa9869" integrity sha512-I9jwMn07Sy/IwOj3zVkVik2JTvgpaykDZEigL6Rx6N9LbMywwUSMtxET+7lVoDLLd3O3IXwJwvuuns8UB/HeAg== +minimatch@9.0.1: + version "9.0.1" + resolved "https://registry.yarnpkg.com/minimatch/-/minimatch-9.0.1.tgz#8a555f541cf976c622daf078bb28f29fb927c253" + integrity sha512-0jWhJpD/MdhPXwPuiRkCbfYfSKp2qnn2eOc279qI7f+osl/l+prKSrvhg157zSYvx/1nmgn2NqdT6k2Z7zSH9w== + dependencies: + brace-expansion "^2.0.1" + minimatch@^3.0.4, minimatch@^3.0.5, minimatch@^3.1.1, minimatch@^3.1.2: version "3.1.2" resolved "https://registry.yarnpkg.com/minimatch/-/minimatch-3.1.2.tgz#19cd194bfd3e428f049a70817c038d89ab4be35b" @@ -3085,13 +2920,18 @@ minimatch@^3.0.4, minimatch@^3.0.5, minimatch@^3.1.1, minimatch@^3.1.2: dependencies: brace-expansion "^1.1.7" -minimatch@^5.0.1, minimatch@^5.1.1: - version "5.1.6" - resolved "https://registry.yarnpkg.com/minimatch/-/minimatch-5.1.6.tgz#1cfcb8cf5522ea69952cd2af95ae09477f122a96" - integrity sha512-lKwV/1brpG6mBUFHtb7NUmtABCb2WZZmm2wNiOA5hAb8VdCS4B3dtMWyvcoViccwAW/COERjXLt0zP1zXUN26g== +minimatch@^9.0.1, minimatch@^9.0.3, minimatch@^9.0.4: + version "9.0.4" + resolved "https://registry.yarnpkg.com/minimatch/-/minimatch-9.0.4.tgz#8e49c731d1749cbec05050ee5145147b32496a51" + integrity sha512-KqWh+VchfxcMNRAJjj2tnsSJdNbHsVgnkBhTNrW7AjVo6OvLtxw8zfT9oLw1JSohlFzJ8jCoTgaoXvJ+kHt6fw== dependencies: brace-expansion "^2.0.1" +"minipass@^5.0.0 || ^6.0.2 || ^7.0.0", minipass@^7.0.4: + version "7.0.4" + resolved "https://registry.yarnpkg.com/minipass/-/minipass-7.0.4.tgz#dbce03740f50a4786ba994c1fb908844d27b038c" + integrity sha512-jYofLM5Dam9279rdkWzqHozUo4ybjdZmCsDHePy5V/PbBcVMiSZR97gmAy45aqi8CK1lG2ECd356FU86avfwUQ== + mlly@^1.2.0: version "1.3.0" resolved "https://registry.yarnpkg.com/mlly/-/mlly-1.3.0.tgz#3184cb80c6437bda861a9f452ae74e3434ed9cd1" @@ -3102,6 +2942,16 @@ mlly@^1.2.0: pkg-types "^1.0.3" ufo "^1.1.2" +mlly@^1.4.2: + version "1.6.1" + resolved "https://registry.yarnpkg.com/mlly/-/mlly-1.6.1.tgz#0983067dc3366d6314fc5e12712884e6978d028f" + integrity sha512-vLgaHvaeunuOXHSmEbZ9izxPx3USsk8KCQ8iC+aTlp5sKRSoZvwhHh5L9VbKSaVC6sJDqbyohIS76E2VmHIPAA== + dependencies: + acorn "^8.11.3" + pathe "^1.1.2" + pkg-types "^1.0.3" + ufo "^1.3.2" + morgan@^1.10.0: version "1.10.0" resolved "https://registry.yarnpkg.com/morgan/-/morgan-1.10.0.tgz#091778abc1fc47cd3509824653dae1faab6b17d7" @@ -3128,50 +2978,48 @@ ms@2.1.3: resolved "https://registry.yarnpkg.com/ms/-/ms-2.1.3.tgz#574c8138ce1d2b5861f0b44579dbadd60c6615b2" integrity sha512-6FlzubTLZG3J2a/NVCAleEhjzq5oxgHyaCU9yYXvcLsvoVaHJq/s5xXI6/XXP6tz7R9xAOtHnSO/tXtF3WRTlA== -msw@1.2.2: - version "1.2.2" - resolved "https://registry.yarnpkg.com/msw/-/msw-1.2.2.tgz#126c3150c07f651e97b24fbd405821f3aeaf9397" - integrity sha512-GsW3PE/Es/a1tYThXcM8YHOZ1S1MtivcS3He/LQbbTCx3rbWJYCtWD5XXyJ53KlNPT7O1VI9sCW3xMtgFe8XpQ== - dependencies: - "@mswjs/cookies" "^0.2.2" - "@mswjs/interceptors" "^0.17.5" - "@open-draft/until" "^1.0.3" - "@types/cookie" "^0.4.1" - "@types/js-levenshtein" "^1.1.1" - chalk "4.1.1" - chokidar "^3.4.2" - cookie "^0.4.2" - graphql "^15.0.0 || ^16.0.0" - headers-polyfill "^3.1.2" - inquirer "^8.2.0" +msw@2.2.14: + version "2.2.14" + resolved "https://registry.yarnpkg.com/msw/-/msw-2.2.14.tgz#ed16b89f99ffc105a84b0295a6fcae2ee99f5c62" + integrity sha512-64i8rNCa1xzDK8ZYsTrVMli05D687jty8+Th+PU5VTbJ2/4P7fkQFVyDQ6ZFT5FrNR8z2BHhbY47fKNvfHrumA== + dependencies: + "@bundled-es-modules/cookie" "^2.0.0" + "@bundled-es-modules/statuses" "^1.0.1" + "@inquirer/confirm" "^3.0.0" + "@mswjs/cookies" "^1.1.0" + "@mswjs/interceptors" "^0.26.14" + "@open-draft/until" "^2.1.0" + "@types/cookie" "^0.6.0" + "@types/statuses" "^2.0.4" + chalk "^4.1.2" + graphql "^16.8.1" + headers-polyfill "^4.0.2" is-node-process "^1.2.0" - js-levenshtein "^1.1.6" - node-fetch "^2.6.7" - outvariant "^1.4.0" + outvariant "^1.4.2" path-to-regexp "^6.2.0" - strict-event-emitter "^0.4.3" - type-fest "^2.19.0" - yargs "^17.3.1" + strict-event-emitter "^0.5.1" + type-fest "^4.9.0" + yargs "^17.7.2" -muggle-string@^0.1.0: - version "0.1.0" - resolved "https://registry.yarnpkg.com/muggle-string/-/muggle-string-0.1.0.tgz#1fda8a281c8b27bb8b70466dbc9f27586a8baa6c" - integrity sha512-Tr1knR3d2mKvvWthlk7202rywKbiOm4rVFLsfAaSIhJ6dt9o47W4S+JMtWhd/PW9Wrdew2/S2fSvhz3E2gkfEg== +muggle-string@^0.4.0: + version "0.4.1" + resolved "https://registry.yarnpkg.com/muggle-string/-/muggle-string-0.4.1.tgz#3b366bd43b32f809dc20659534dd30e7c8a0d328" + integrity sha512-VNTrAak/KhO2i8dqqnqnAHOa3cYBwXEZe9h+D5h/1ZqFSTEFHdM65lR7RoIqq3tBBYavsOXV84NoHXZ0AkPyqQ== -mute-stream@0.0.8: - version "0.0.8" - resolved "https://registry.yarnpkg.com/mute-stream/-/mute-stream-0.0.8.tgz#1630c42b2251ff81e2a283de96a5497ea92e5e0d" - integrity sha512-nnbWWOkoWyUsTjKrhgD0dcz22mdkSnpYqbEjIm2nhwhuxlSkpywJmBo8h0ZqJdkp73mb90SssHkN4rsRaBAfAA== +mute-stream@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/mute-stream/-/mute-stream-1.0.0.tgz#e31bd9fe62f0aed23520aa4324ea6671531e013e" + integrity sha512-avsJQhyd+680gKXyG/sQc0nXaC6rBkPOfyHYcFb9+hdkqQkR9bdnkJ0AMZhke0oesPqIO+mFFJ+IdBc7mst4IA== -nanoid@^3.1.23, nanoid@^3.3.4: +nanoid@^3.1.23: version "3.3.4" resolved "https://registry.yarnpkg.com/nanoid/-/nanoid-3.3.4.tgz#730b67e3cd09e2deacf03c027c81c9d9dbc5e8ab" integrity sha512-MqBkQh/OHTS2egovRtLk45wEyNXwF+cokD+1YPf9u5VfJiRdAiRwB2froX5Co9Rh20xs4siNPm8naNotSD6RBw== -natural-compare-lite@^1.4.0: - version "1.4.0" - resolved "https://registry.yarnpkg.com/natural-compare-lite/-/natural-compare-lite-1.4.0.tgz#17b09581988979fddafe0201e931ba933c96cbb4" - integrity sha512-Tj+HTDSJJKaZnfiuw+iaF9skdPpTo2GtEly5JHnWV/hfv2Qj/9RKsGISQtLh2ox3l5EAGw487hnBee0sIJ6v2g== +nanoid@^3.3.7: + version "3.3.7" + resolved "https://registry.yarnpkg.com/nanoid/-/nanoid-3.3.7.tgz#d0c301a691bc8d54efa0a2226ccf3fe2fd656bd8" + integrity sha512-eSRppjcPIatRIMC1U6UngP8XFcz8MQWGQdt1MTBQ7NaAmvXDfvNxbvWV3x2y6CdEUciCSsDHDQZbhYaB8QEo2g== natural-compare@^1.4.0: version "1.4.0" @@ -3188,19 +3036,12 @@ nice-try@^1.0.4: resolved "https://registry.yarnpkg.com/nice-try/-/nice-try-1.0.5.tgz#a3378a7696ce7d223e88fc9b764bd7ef1089e366" integrity sha512-1nh45deeb5olNY7eX82BkPO7SSxR5SSYJiPTrTdFUVYwAl8CKMA5N9PjTYkHiRjisVcxcQ1HXdLhx2qxxJzLNQ== -node-fetch@^2.6.7: - version "2.6.12" - resolved "https://registry.yarnpkg.com/node-fetch/-/node-fetch-2.6.12.tgz#02eb8e22074018e3d5a83016649d04df0e348fba" - integrity sha512-C/fGU2E8ToujUivIO0H+tpQ6HWo4eEmchoPIoXtxCrVghxdKq+QOHqEZW7tuP3KlV3bC8FRMO5nMCC7Zm1VP6g== - dependencies: - whatwg-url "^5.0.0" - -nopt@^6.0.0: - version "6.0.0" - resolved "https://registry.yarnpkg.com/nopt/-/nopt-6.0.0.tgz#245801d8ebf409c6df22ab9d95b65e1309cdb16d" - integrity sha512-ZwLpbTgdhuZUnZzjd7nb1ZV+4DoiC6/sfiVKok72ym/4Tlf+DFdlHYmT2JPmcNNWV6Pi3SDf1kT+A4r9RTuT9g== +nopt@^7.2.0: + version "7.2.0" + resolved "https://registry.yarnpkg.com/nopt/-/nopt-7.2.0.tgz#067378c68116f602f552876194fd11f1292503d7" + integrity sha512-CVDtwCdhYIvnAzFoJ6NJ6dX3oga9/HyciQDnG1vQDjSLMeKLJ4A93ZqYKDrgYSr1FBY5/hMYC+2VCi24pgpkGA== dependencies: - abbrev "^1.0.0" + abbrev "^2.0.0" normalize-package-data@^2.3.2: version "2.5.0" @@ -3232,17 +3073,24 @@ npm-run-all@4.1.5: shell-quote "^1.6.1" string.prototype.padend "^3.0.0" -nth-check@^2.0.1: +npm-run-path@^5.1.0: + version "5.3.0" + resolved "https://registry.yarnpkg.com/npm-run-path/-/npm-run-path-5.3.0.tgz#e23353d0ebb9317f174e93417e4a4d82d0249e9f" + integrity sha512-ppwTtiJZq0O/ai0z7yfudtBpWIoxM8yE6nHi1X47eFR2EWORqfbu6CnPlNsjeN683eT0qG6H/Pyf9fCcvjnnnQ== + dependencies: + path-key "^4.0.0" + +nth-check@^2.1.1: version "2.1.1" resolved "https://registry.yarnpkg.com/nth-check/-/nth-check-2.1.1.tgz#c9eab428effce36cd6b92c924bdb000ef1f1ed1d" integrity sha512-lqjrjmaOoAnWfMmBPL+XNnynZh2+swxiX3WUE0s4yEHI6m+AwrK2UZOimIRl3X/4QctVqS8AiZjFqyOGrMXb/w== dependencies: boolbase "^1.0.0" -nwsapi@^2.2.4: - version "2.2.5" - resolved "https://registry.yarnpkg.com/nwsapi/-/nwsapi-2.2.5.tgz#a52744c61b3889dd44b0a158687add39b8d935e2" - integrity sha512-6xpotnECFy/og7tKSBVmUNft7J3jyXAka4XvG6AUhFWRz+Q/Ljus7znJAA3bxColfQLdS+XsjoodtJfCgeTEFQ== +nwsapi@^2.2.7: + version "2.2.7" + resolved "https://registry.yarnpkg.com/nwsapi/-/nwsapi-2.2.7.tgz#738e0707d3128cb750dddcfe90e4610482df0f30" + integrity sha512-ub5E4+FBPKwAZx0UwIQOjYWGHTEq5sPqHQNRN8Z9e4A7u3Tj1weLJsL59yH9vmvqEtBHaOmT6cYQKIZOxp35FQ== object-assign@^4: version "4.1.1" @@ -3303,56 +3151,34 @@ once@^1.3.0: dependencies: wrappy "1" -onetime@^5.1.0: - version "5.1.2" - resolved "https://registry.yarnpkg.com/onetime/-/onetime-5.1.2.tgz#d0e96ebb56b07476df1dd9c4806e5237985ca45e" - integrity sha512-kbpaSSGJTWdAY5KPVeMOKXSrPtr8C8C7wodJbcsd51jRnmD+GZu8Y0VoU6Dm5Z4vWr0Ig/1NKuWRKf7j5aaYSg== +onetime@^6.0.0: + version "6.0.0" + resolved "https://registry.yarnpkg.com/onetime/-/onetime-6.0.0.tgz#7c24c18ed1fd2e9bca4bd26806a33613c77d34b4" + integrity sha512-1FlR+gjXK7X+AsAHso35MnyN5KqGwJRi/31ft6x0M194ht7S+rWAvd7PHss9xSKMzE0asv1pyIHaJYq+BbacAQ== dependencies: - mimic-fn "^2.1.0" + mimic-fn "^4.0.0" -optionator@^0.9.1: - version "0.9.1" - resolved "https://registry.yarnpkg.com/optionator/-/optionator-0.9.1.tgz#4f236a6373dae0566a6d43e1326674f50c291499" - integrity sha512-74RlY5FCnhq4jRxVUPKDaRwrVNXMqsGsiW6AJw4XK8hmtm10wC0ypZBLw5IIp85NZMr91+qd1RvvENwg7jjRFw== +optionator@^0.9.3: + version "0.9.3" + resolved "https://registry.yarnpkg.com/optionator/-/optionator-0.9.3.tgz#007397d44ed1872fdc6ed31360190f81814e2c64" + integrity sha512-JjCoypp+jKn1ttEFExxhetCKeJt9zhAgAve5FXHixTvFDW/5aEktX9bufBKLRRMdU7bNtpLfcGu94B3cdEJgjg== dependencies: + "@aashutoshrathi/word-wrap" "^1.2.3" deep-is "^0.1.3" fast-levenshtein "^2.0.6" levn "^0.4.1" prelude-ls "^1.2.1" type-check "^0.4.0" - word-wrap "^1.2.3" - -ora@^5.4.1: - version "5.4.1" - resolved "https://registry.yarnpkg.com/ora/-/ora-5.4.1.tgz#1b2678426af4ac4a509008e5e4ac9e9959db9e18" - integrity sha512-5b6Y85tPxZZ7QytO+BQzysW31HJku27cRIlkbAXaNx+BdcVi+LlRFmVXzeF6a7JCwJpyw5c4b+YSVImQIrBpuQ== - dependencies: - bl "^4.1.0" - chalk "^4.1.0" - cli-cursor "^3.1.0" - cli-spinners "^2.5.0" - is-interactive "^1.0.0" - is-unicode-supported "^0.1.0" - log-symbols "^4.1.0" - strip-ansi "^6.0.0" - wcwidth "^1.0.1" - -os-tmpdir@~1.0.2: - version "1.0.2" - resolved "https://registry.yarnpkg.com/os-tmpdir/-/os-tmpdir-1.0.2.tgz#bbe67406c79aa85c5cfec766fe5734555dfa1274" - integrity sha512-D2FR03Vir7FIu45XBY20mTb+/ZSWB00sjU9jdQXt83gDrI4Ztz5Fs7/yy74g2N5SVQY4xY1qDr4rNddwYRVX0g== outvariant@^1.2.1, outvariant@^1.4.0: version "1.4.0" resolved "https://registry.yarnpkg.com/outvariant/-/outvariant-1.4.0.tgz#e742e4bda77692da3eca698ef5bfac62d9fba06e" integrity sha512-AlWY719RF02ujitly7Kk/0QlV+pXGFDHrHf9O2OKqyqgBieaPOIeuSkL8sRK6j2WK+/ZAURq2kZsY0d8JapUiw== -p-limit@^2.2.0: - version "2.3.0" - resolved "https://registry.yarnpkg.com/p-limit/-/p-limit-2.3.0.tgz#3dd33c647a214fdfffd835933eb086da0dc21db1" - integrity sha512-//88mFWSJx8lxCzwdAABTJL2MyWB12+eIY7MDL2SqLmAkeKU9qxRvWuSyTjm3FUmpBEMuFfckAIqEaVGUDxb6w== - dependencies: - p-try "^2.0.0" +outvariant@^1.4.2: + version "1.4.2" + resolved "https://registry.yarnpkg.com/outvariant/-/outvariant-1.4.2.tgz#f54f19240eeb7f15b28263d5147405752d8e2066" + integrity sha512-Ou3dJ6bA/UJ5GVHxah4LnqDwZRwAmWxrG3wtrHrbGnP4RnLCtA64A4F+ae7Y8ww660JaddSoArUR5HjipWSHAQ== p-limit@^3.0.2: version "3.1.0" @@ -3361,20 +3187,13 @@ p-limit@^3.0.2: dependencies: yocto-queue "^0.1.0" -p-limit@^4.0.0: - version "4.0.0" - resolved "https://registry.yarnpkg.com/p-limit/-/p-limit-4.0.0.tgz#914af6544ed32bfa54670b061cafcbd04984b644" - integrity sha512-5b0R4txpzjPWVw/cXXUResoD4hb6U/x9BH08L7nw+GN1sezDzPdxeRvpc9c433fZhBan/wusjbCsqwqm4EIBIQ== +p-limit@^5.0.0: + version "5.0.0" + resolved "https://registry.yarnpkg.com/p-limit/-/p-limit-5.0.0.tgz#6946d5b7140b649b7a33a027d89b4c625b3a5985" + integrity sha512-/Eaoq+QyLSiXQ4lyYV23f14mZRQcXnxfHrN0vCai+ak9G0pp9iEQukIIZq5NccEvwRB8PUnZT0KsOoDCINS1qQ== dependencies: yocto-queue "^1.0.0" -p-locate@^4.1.0: - version "4.1.0" - resolved "https://registry.yarnpkg.com/p-locate/-/p-locate-4.1.0.tgz#a3428bb7088b3a60292f66919278b7c297ad4f07" - integrity sha512-R79ZZ/0wAxKGu3oYMlz8jy/kbhsNrS7SKZ7PxEHBgJ5+F2mtFW2fK2cOtBh1cHYkQsbzFV7I+EoRKe6Yt0oK7A== - dependencies: - p-limit "^2.2.0" - p-locate@^5.0.0: version "5.0.0" resolved "https://registry.yarnpkg.com/p-locate/-/p-locate-5.0.0.tgz#83c8315c6785005e3bd021839411c9e110e6d834" @@ -3382,11 +3201,6 @@ p-locate@^5.0.0: dependencies: p-limit "^3.0.2" -p-try@^2.0.0: - version "2.2.0" - resolved "https://registry.yarnpkg.com/p-try/-/p-try-2.2.0.tgz#cb2868540e313d61de58fafbe35ce9004d5540e6" - integrity sha512-R4nPAVTAU0B9D35/Gk3uJf/7XYbQcyohSKdvAxIRSNghFl4e71hVoGnBNQz9cWaXxO2I10KTC+3jMdvvoKw6dQ== - parent-module@^1.0.0: version "1.0.1" resolved "https://registry.yarnpkg.com/parent-module/-/parent-module-1.0.1.tgz#691d2709e78c79fae3a156622452d00762caaaa2" @@ -3414,6 +3228,11 @@ parseurl@~1.3.2, parseurl@~1.3.3: resolved "https://registry.yarnpkg.com/parseurl/-/parseurl-1.3.3.tgz#9da19e7bee8d12dff0513ed5b76957793bc2e8d4" integrity sha512-CiyeOxFT/JZyN5m0z9PfXw4SCBJ6Sygz1Dpl0wqjlhDEGGBP1GnsUVEL0p63hoG1fcj3fHynXi9NYO4nWOL+qQ== +path-browserify@^1.0.1: + version "1.0.1" + resolved "https://registry.yarnpkg.com/path-browserify/-/path-browserify-1.0.1.tgz#d98454a9c3753d5790860f16f68867b9e46be1fd" + integrity sha512-b7uo2UCUOYZcnF/3ID0lulOJi/bafxa1xPe7ZPsammBSpjSWQkjNxlt635YGS2MiR9GjvuXCtz2emr3jbsz98g== + path-exists@^4.0.0: version "4.0.0" resolved "https://registry.yarnpkg.com/path-exists/-/path-exists-4.0.0.tgz#513bdbe2d3b95d7762e8c1137efa195c6c61b5b3" @@ -3434,11 +3253,24 @@ path-key@^3.1.0: resolved "https://registry.yarnpkg.com/path-key/-/path-key-3.1.1.tgz#581f6ade658cbba65a0d3380de7753295054f375" integrity sha512-ojmeN0qd+y0jszEtoY48r0Peq5dwMEkIlCOu6Q5f41lfkswXuKtYrhgoTpLnyIcHm24Uhqx+5Tqm2InSwLhE6Q== +path-key@^4.0.0: + version "4.0.0" + resolved "https://registry.yarnpkg.com/path-key/-/path-key-4.0.0.tgz#295588dc3aee64154f877adb9d780b81c554bf18" + integrity sha512-haREypq7xkM7ErfgIyA0z+Bj4AGKlMSdlQE2jvJo6huWD1EdkKYV+G/T4nq0YEF2vgTT8kqMFKo1uHn950r4SQ== + path-parse@^1.0.7: version "1.0.7" resolved "https://registry.yarnpkg.com/path-parse/-/path-parse-1.0.7.tgz#fbc114b60ca42b30d9daf5858e4bd68bbedb6735" integrity sha512-LDJzPVEEEPR+y48z93A0Ed0yXb8pAByGWo/k5YYdYgpY2/2EsOsksJrq7lOHxryrVOn1ejG6oAp8ahvOIQD8sw== +path-scurry@^1.10.2: + version "1.10.2" + resolved "https://registry.yarnpkg.com/path-scurry/-/path-scurry-1.10.2.tgz#8f6357eb1239d5fa1da8b9f70e9c080675458ba7" + integrity sha512-7xTavNy5RQXnsjANvVvMkEjvloOinkAjv/Z6Ildz9v2RinZ4SBKTWFOVRbaF8p0vpHnyjV/UwNDdKuUv6M5qcA== + dependencies: + lru-cache "^10.2.0" + minipass "^5.0.0 || ^6.0.2 || ^7.0.0" + path-to-regexp@0.1.7: version "0.1.7" resolved "https://registry.yarnpkg.com/path-to-regexp/-/path-to-regexp-0.1.7.tgz#df604178005f522f15eb4490e7247a1bfaa67f8c" @@ -3473,6 +3305,11 @@ pathe@^1.1.0: resolved "https://registry.yarnpkg.com/pathe/-/pathe-1.1.0.tgz#e2e13f6c62b31a3289af4ba19886c230f295ec03" integrity sha512-ODbEPR0KKHqECXW1GoxdDb+AZvULmXjVPy4rt+pGo2+TnjJTIPJQSVS6N63n8T2Ip+syHhbn52OewKicV0373w== +pathe@^1.1.1, pathe@^1.1.2: + version "1.1.2" + resolved "https://registry.yarnpkg.com/pathe/-/pathe-1.1.2.tgz#6c4cb47a945692e48a1ddd6e4094d170516437ec" + integrity sha512-whLdWMYL2TwI08hn8/ZqAbrVemu0LNaNNJZX73O6qaIdCTfXutsLhMkjdENX0qhsQ9uIimo4/aQOmXkoon2nDQ== + pathval@^1.1.1: version "1.1.1" resolved "https://registry.yarnpkg.com/pathval/-/pathval-1.1.1.tgz#8534e77a77ce7ac5a2512ea21e0fdb8fcf6c3d8d" @@ -3483,7 +3320,7 @@ picocolors@^1.0.0: resolved "https://registry.yarnpkg.com/picocolors/-/picocolors-1.0.0.tgz#cb5bdc74ff3f51892236eaf79d68bc44564ab81c" integrity sha512-1fygroTLlHu66zi26VoTDv8yRgm0Fccecssto+MhsZ0D/DGW2sm8E8AjW7NU5VVTRt5GxbeZ5qBuJr+HyLYkjQ== -picomatch@^2.0.4, picomatch@^2.2.1, picomatch@^2.2.3, picomatch@^2.3.1: +picomatch@^2.0.4, picomatch@^2.2.1, picomatch@^2.3.1: version "2.3.1" resolved "https://registry.yarnpkg.com/picomatch/-/picomatch-2.3.1.tgz#3ba3833733646d9d3e4995946c1365a67fb07a42" integrity sha512-JU3teHTNjmE2VCGFzuY8EXzCDVwEqB2a8fsIvwaStHhAWJEeVd1o1QD80CU6+ZdEXXSLbSsuLwJjkCBWqRQUVA== @@ -3498,26 +3335,19 @@ pify@^3.0.0: resolved "https://registry.yarnpkg.com/pify/-/pify-3.0.0.tgz#e5a4acd2c101fdf3d9a4d07f0dbc4db49dd28176" integrity sha512-C3FsVNH1udSEX48gGX1xfvwTWfsYWj5U+8/uK15BGzIGrKoUpghX8hWZwa/OFnakBiiVNmBvemTJR5mcy7iPcg== -pinia-plugin-persistedstate@3.2.0: - version "3.2.0" - resolved "https://registry.yarnpkg.com/pinia-plugin-persistedstate/-/pinia-plugin-persistedstate-3.2.0.tgz#9932ca2ae88aa6c0d6763bebc6447d7bd1f097d0" - integrity sha512-tZbNGf2vjAQcIm7alK40sE51Qu/m9oWr+rEgNm/2AWr1huFxj72CjvpQcIQzMknDBJEkQznCLAGtJTIcLKrKdw== +pinia-plugin-persistedstate@3.2.1: + version "3.2.1" + resolved "https://registry.yarnpkg.com/pinia-plugin-persistedstate/-/pinia-plugin-persistedstate-3.2.1.tgz#66780602aecd6c7b152dd7e3ddc249a1f7a13fe5" + integrity sha512-MK++8LRUsGF7r45PjBFES82ISnPzyO6IZx3CH5vyPseFLZCk1g2kgx6l/nW8pEBKxxd4do0P6bJw+mUSZIEZUQ== -pinia@2.1.4: - version "2.1.4" - resolved "https://registry.yarnpkg.com/pinia/-/pinia-2.1.4.tgz#a642adfe6208e10c36d3dc16184a91064788142a" - integrity sha512-vYlnDu+Y/FXxv1ABo1vhjC+IbqvzUdiUC3sfDRrRyY2CQSrqqaa+iiHmqtARFxJVqWQMCJfXx1PBvFs9aJVLXQ== +pinia@2.1.7: + version "2.1.7" + resolved "https://registry.yarnpkg.com/pinia/-/pinia-2.1.7.tgz#4cf5420d9324ca00b7b4984d3fbf693222115bbc" + integrity sha512-+C2AHFtcFqjPih0zpYuvof37SFxMQ7OEG2zV9jRI12i9BOy3YQVAHwdKtyyc8pDcDyIc33WCIsZaCFWU7WWxGQ== dependencies: "@vue/devtools-api" "^6.5.0" vue-demi ">=0.14.5" -pkg-dir@^4.1.0: - version "4.2.0" - resolved "https://registry.yarnpkg.com/pkg-dir/-/pkg-dir-4.2.0.tgz#f099133df7ede422e81d1d8448270eeb3e4261f3" - integrity sha512-HRDzbaKjC+AOWVXxAU/x54COGeIv9eb+6CkDSQoNTt4XyWoIJvuPsXizxu/Fr23EiekbtZwmh1IcIG/l/a10GQ== - dependencies: - find-up "^4.0.0" - pkg-types@^1.0.3: version "1.0.3" resolved "https://registry.yarnpkg.com/pkg-types/-/pkg-types-1.0.3.tgz#988b42ab19254c01614d13f4f65a2cfc7880f868" @@ -3539,22 +3369,22 @@ pluralize@^8.0.0: resolved "https://registry.yarnpkg.com/pluralize/-/pluralize-8.0.0.tgz#1a6fa16a38d12a1901e0320fa017051c539ce3b1" integrity sha512-Nc3IT5yHzflTfbjgqWcCPpo7DaKy4FnpB0l/zCAW0Tc7jxAiuqSxHasntB3D7887LSrA93kDJ9IXovxJYxyLCA== -postcss-selector-parser@^6.0.9: - version "6.0.11" - resolved "https://registry.yarnpkg.com/postcss-selector-parser/-/postcss-selector-parser-6.0.11.tgz#2e41dc39b7ad74046e1615185185cd0b17d0c8dc" - integrity sha512-zbARubNdogI9j7WY4nQJBiNqQf3sLS3wCP4WfOidu+p28LofJqDH1tcXypGrcmMHhDk2t9wGhCsYe/+szLTy1g== +postcss-selector-parser@^6.0.15: + version "6.0.16" + resolved "https://registry.yarnpkg.com/postcss-selector-parser/-/postcss-selector-parser-6.0.16.tgz#3b88b9f5c5abd989ef4e2fc9ec8eedd34b20fb04" + integrity sha512-A0RVJrX+IUkVZbW3ClroRWurercFhieevHB38sr2+l9eUClMqome3LmEmnhlNy+5Mr2EYN6B2Kaw9wYdd+VHiw== dependencies: cssesc "^3.0.0" util-deprecate "^1.0.2" -postcss@^8.1.10, postcss@^8.4.21: - version "8.4.21" - resolved "https://registry.yarnpkg.com/postcss/-/postcss-8.4.21.tgz#c639b719a57efc3187b13a1d765675485f4134f4" - integrity sha512-tP7u/Sn/dVxK2NnruI4H9BG+x+Wxz6oeZ1cJ8P6G/PZY0IKk4k/63TDsQf2kQq3+qoJeLm2kIBUNlZe3zgb4Zg== +postcss@^8.4.38: + version "8.4.38" + resolved "https://registry.yarnpkg.com/postcss/-/postcss-8.4.38.tgz#b387d533baf2054288e337066d81c6bee9db9e0e" + integrity sha512-Wglpdk03BSfXkHoQa3b/oulrotAkwrlLDRSOb9D0bN86FdRyE9lppSp33aHNPgBa0JKCoB+drFLZkQoRRYae5A== dependencies: - nanoid "^3.3.4" + nanoid "^3.3.7" picocolors "^1.0.0" - source-map-js "^1.0.2" + source-map-js "^1.2.0" prelude-ls@^1.2.1: version "1.2.1" @@ -3568,12 +3398,12 @@ prettier-linter-helpers@^1.0.0: dependencies: fast-diff "^1.1.2" -prettier@2.8.4: - version "2.8.4" - resolved "https://registry.yarnpkg.com/prettier/-/prettier-2.8.4.tgz#34dd2595629bfbb79d344ac4a91ff948694463c3" - integrity sha512-vIS4Rlc2FNh0BySk3Wkd6xmwxB0FpOndW5fisM5H8hsZSxU2VWVB5CWIkIjWvrHjIhxk2g3bfMKM87zNTrZddw== +prettier@3.2.5: + version "3.2.5" + resolved "https://registry.yarnpkg.com/prettier/-/prettier-3.2.5.tgz#e52bc3090586e824964a8813b09aba6233b28368" + integrity sha512-3/GWa9aOC0YeD7LUfvOG2NiDyhOWRvt1k+rcKhOuYnMY24iiCphgneUfJDyFXd6rZCAnuLBv6UeAULtrhT/F4A== -pretty-format@^27.0.2, pretty-format@^27.5.1: +pretty-format@^27.0.2: version "27.5.1" resolved "https://registry.yarnpkg.com/pretty-format/-/pretty-format-27.5.1.tgz#2181879fdea51a7a5851fb39d920faa63f01d88e" integrity sha512-Qb1gy5OrP5+zDf2Bvnzdl3jsTf1qXVMazbvCoKhtKqVs4/YK4ozX4gKQJJVyNe+cajNPn0KoC0MC3FUmaHWEmQ== @@ -3582,12 +3412,12 @@ pretty-format@^27.0.2, pretty-format@^27.5.1: ansi-styles "^5.0.0" react-is "^17.0.1" -pretty-format@^29.0.0, pretty-format@^29.4.2: - version "29.4.2" - resolved "https://registry.yarnpkg.com/pretty-format/-/pretty-format-29.4.2.tgz#64bf5ccc0d718c03027d94ac957bdd32b3fb2401" - integrity sha512-qKlHR8yFVCbcEWba0H0TOC8dnLlO4vPlyEjRPw31FZ2Rupy9nLa8ZLbYny8gWEl8CkEhJqAE6IzdNELTBVcBEg== +pretty-format@^29.7.0: + version "29.7.0" + resolved "https://registry.yarnpkg.com/pretty-format/-/pretty-format-29.7.0.tgz#ca42c758310f365bfa71a0bda0a807160b776812" + integrity sha512-Pdlw/oPxN+aXdmM9R00JVC9WVFoCLTKJvDVLgmJ+qAffBMxsV85l/Lu7sNx4zSzPyoL2euImuEwHhOXdEgNFZQ== dependencies: - "@jest/schemas" "^29.4.2" + "@jest/schemas" "^29.6.3" ansi-styles "^5.0.0" react-is "^18.0.0" @@ -3609,21 +3439,21 @@ proxy-from-env@^1.1.0: resolved "https://registry.yarnpkg.com/proxy-from-env/-/proxy-from-env-1.1.0.tgz#e102f16ca355424865755d2c9e8ea4f24d58c3e2" integrity sha512-D+zkORCbA9f1tdWRK0RaCR3GPv50cMxcrz4X8k5LTSUD1Dkw47mKJEZQNunItRTkWwgtaUSo1RVFRIG9ZXiFYg== -pseudomap@^1.0.2: - version "1.0.2" - resolved "https://registry.yarnpkg.com/pseudomap/-/pseudomap-1.0.2.tgz#f052a28da70e618917ef0a8ac34c1ae5a68286b3" - integrity sha512-b/YwNhb8lk1Zz2+bXXpS/LK9OisiZZ1SNsSLxN1x2OXVEhW2Ckr/7mWE5vrC1ZTiJlD9g19jWszTmJsB+oEpFQ== - psl@^1.1.33: version "1.9.0" resolved "https://registry.yarnpkg.com/psl/-/psl-1.9.0.tgz#d0df2a137f00794565fcaf3b2c00cd09f8d5a5a7" integrity sha512-E/ZsdU4HLs/68gYzgGTkMicWTLPdAftJLfJFlLUAAKZGkStNU72sZjT66SnMDVOfOWY/YAoiD7Jxa9iHvngcag== -punycode@^2.1.0, punycode@^2.1.1, punycode@^2.3.0: +punycode@^2.1.0, punycode@^2.1.1: version "2.3.0" resolved "https://registry.yarnpkg.com/punycode/-/punycode-2.3.0.tgz#f67fa67c94da8f4d0cfff981aee4118064199b8f" integrity sha512-rRV+zQD8tVFys26lAGR9WUuS4iUAngJScM+ZRSKtvl5tKeZ2t5bvdNFdNHBW9FWR4guGHlgmsZ1G7BSm2wTbuA== +punycode@^2.3.1: + version "2.3.1" + resolved "https://registry.yarnpkg.com/punycode/-/punycode-2.3.1.tgz#027422e2faec0b25e1549c3e1bd8309b9133b6e5" + integrity sha512-vYt7UD1U9Wg6138shLtLOvdAu+8DsC/ilFtEVHcH+wydcSpNE20AfSOduf6MkRFahL5FY7X1oU7nKVZFtfq8Fg== + qs@6.11.0: version "6.11.0" resolved "https://registry.yarnpkg.com/qs/-/qs-6.11.0.tgz#fd0d963446f7a65e1367e01abd85429453f0c37a" @@ -3631,10 +3461,10 @@ qs@6.11.0: dependencies: side-channel "^1.0.4" -query-string@8.1.0: - version "8.1.0" - resolved "https://registry.yarnpkg.com/query-string/-/query-string-8.1.0.tgz#e7f95367737219544cd360a11a4f4ca03836e115" - integrity sha512-BFQeWxJOZxZGix7y+SByG3F36dA0AbTy9o6pSmKFcFz7DAj0re9Frkty3saBn3nHo3D0oZJ/+rx3r8H8r8Jbpw== +query-string@9.0.0: + version "9.0.0" + resolved "https://registry.yarnpkg.com/query-string/-/query-string-9.0.0.tgz#1fe177cd95545600f0deab93f5fb02fd4e3e7273" + integrity sha512-4EWwcRGsO2H+yzq6ddHcVqkCQ2EFUSfDMEjF8ryp8ReymyZhIuaFRGLomeOQLkrzacMHoyky2HW0Qe30UbzkKw== dependencies: decode-uri-component "^0.4.1" filter-obj "^5.1.0" @@ -3684,15 +3514,6 @@ read-pkg@^3.0.0: normalize-package-data "^2.3.2" path-type "^3.0.0" -readable-stream@^3.4.0: - version "3.6.2" - resolved "https://registry.yarnpkg.com/readable-stream/-/readable-stream-3.6.2.tgz#56a9b36ea965c00c5a93ef31eb111a0f11056967" - integrity sha512-9u/sniCrY3D5WdsERHzHE4G2YCXqoG5FTHUiCC4SIbr6XcLZBY05ya9EKjYek9O5xOAwjGq+1JdGBAS7Q9ScoA== - dependencies: - inherits "^2.0.3" - string_decoder "^1.1.1" - util-deprecate "^1.0.1" - readdirp@~3.6.0: version "3.6.0" resolved "https://registry.yarnpkg.com/readdirp/-/readdirp-3.6.0.tgz#74a370bd857116e245b29cc97340cd431a02a6c7" @@ -3713,6 +3534,11 @@ regenerator-runtime@^0.13.11: resolved "https://registry.yarnpkg.com/regenerator-runtime/-/regenerator-runtime-0.13.11.tgz#f6dca3e7ceec20590d07ada785636a90cdca17f9" integrity sha512-kY1AZVr2Ra+t+piVaJ4gxaFaReZVH40AKNo7UCX6W+dEwBo/2oZJzqfuN1qLq1oL45o56cPaTXELwrTh8Fpggg== +regenerator-runtime@^0.14.0: + version "0.14.1" + resolved "https://registry.yarnpkg.com/regenerator-runtime/-/regenerator-runtime-0.14.1.tgz#356ade10263f685dda125100cd862c1db895327f" + integrity sha512-dYnhHh0nJoMfnkZs6GmmhFknAGRrLznOu5nc9ML+EJxGvrx6H7teuevqVqCuPcPK//3eDrrjQhehXVx9cnkGdw== + regexp.prototype.flags@^1.4.3: version "1.4.3" resolved "https://registry.yarnpkg.com/regexp.prototype.flags/-/regexp.prototype.flags-1.4.3.tgz#87cab30f80f66660181a3bb7bf5981a872b367ac" @@ -3722,11 +3548,6 @@ regexp.prototype.flags@^1.4.3: define-properties "^1.1.3" functions-have-names "^1.2.2" -regexpp@^3.2.0: - version "3.2.0" - resolved "https://registry.yarnpkg.com/regexpp/-/regexpp-3.2.0.tgz#0425a2768d8f23bad70ca4b90461fa2f1213e1b2" - integrity sha512-pq2bWo9mVD43nbts2wGv17XLiNLya+GklZ8kaDLV2Z08gDCsGpnKn9BFMepvWuHCbyVvY7J5o5+BVvoQbmlJLg== - require-directory@^2.1.1: version "2.1.1" resolved "https://registry.yarnpkg.com/require-directory/-/require-directory-2.1.1.tgz#8c64ad5fd30dab1c976e2344ffe7f792a6a6df42" @@ -3747,7 +3568,7 @@ resolve-from@^4.0.0: resolved "https://registry.yarnpkg.com/resolve-from/-/resolve-from-4.0.0.tgz#4abcd852ad32dd7baabfe9b40e00a36db5f392e6" integrity sha512-pb/MYmXstAkysRFx8piNI1tGFNQIFA3vkE3Gq4EuA1dF6gHp/+vgZqsCGJapvy8N3Q+4o7FwvquPJcnZ7RYy4g== -resolve@^1.10.0, resolve@^1.22.1: +resolve@^1.10.0: version "1.22.1" resolved "https://registry.yarnpkg.com/resolve/-/resolve-1.22.1.tgz#27cb2ebb53f91abb49470a928bba7558066ac177" integrity sha512-nBpuuYuY5jFsli/JIs1oldw6fOQCBioohqWZg/2hiaOybXOft4lonv85uDOKXdf8rhyK159cxU5cDcK/NKk8zw== @@ -3756,14 +3577,6 @@ resolve@^1.10.0, resolve@^1.22.1: path-parse "^1.0.7" supports-preserve-symlinks-flag "^1.0.0" -restore-cursor@^3.1.0: - version "3.1.0" - resolved "https://registry.yarnpkg.com/restore-cursor/-/restore-cursor-3.1.0.tgz#39f67c54b3a7a58cea5236d95cf0034239631f7e" - integrity sha512-l+sSefzHpj5qimhFSE5a8nufZYAM3sBSVMAPtYkmC+4EH2anSGaEMXSD0izRQbu9nfyQ9y5JrVmp7E8oZrUjvA== - dependencies: - onetime "^5.1.0" - signal-exit "^3.0.2" - reusify@^1.0.4: version "1.0.4" resolved "https://registry.yarnpkg.com/reusify/-/reusify-1.0.4.tgz#90da382b1e126efc02146e90845a88db12925d76" @@ -3776,11 +3589,29 @@ rimraf@^3.0.2: dependencies: glob "^7.1.3" -rollup@^3.10.0: - version "3.15.0" - resolved "https://registry.yarnpkg.com/rollup/-/rollup-3.15.0.tgz#6f4105e8c4b8145229657b74ad660b02fbfacc05" - integrity sha512-F9hrCAhnp5/zx/7HYmftvsNBkMfLfk/dXUh73hPSM2E3CRgap65orDNJbLetoiUFwSAk6iHPLvBrZ5iHYvzqsg== +rollup@^4.13.0: + version "4.14.3" + resolved "https://registry.yarnpkg.com/rollup/-/rollup-4.14.3.tgz#bcbb7784b35826d3164346fa6d5aac95190d8ba9" + integrity sha512-ag5tTQKYsj1bhrFC9+OEWqb5O6VYgtQDO9hPDBMmIbePwhfSr+ExlcU741t8Dhw5DkPCQf6noz0jb36D6W9/hw== + dependencies: + "@types/estree" "1.0.5" optionalDependencies: + "@rollup/rollup-android-arm-eabi" "4.14.3" + "@rollup/rollup-android-arm64" "4.14.3" + "@rollup/rollup-darwin-arm64" "4.14.3" + "@rollup/rollup-darwin-x64" "4.14.3" + "@rollup/rollup-linux-arm-gnueabihf" "4.14.3" + "@rollup/rollup-linux-arm-musleabihf" "4.14.3" + "@rollup/rollup-linux-arm64-gnu" "4.14.3" + "@rollup/rollup-linux-arm64-musl" "4.14.3" + "@rollup/rollup-linux-powerpc64le-gnu" "4.14.3" + "@rollup/rollup-linux-riscv64-gnu" "4.14.3" + "@rollup/rollup-linux-s390x-gnu" "4.14.3" + "@rollup/rollup-linux-x64-gnu" "4.14.3" + "@rollup/rollup-linux-x64-musl" "4.14.3" + "@rollup/rollup-win32-arm64-msvc" "4.14.3" + "@rollup/rollup-win32-ia32-msvc" "4.14.3" + "@rollup/rollup-win32-x64-msvc" "4.14.3" fsevents "~2.3.2" rrweb-cssom@^0.6.0: @@ -3788,11 +3619,6 @@ rrweb-cssom@^0.6.0: resolved "https://registry.yarnpkg.com/rrweb-cssom/-/rrweb-cssom-0.6.0.tgz#ed298055b97cbddcdeb278f904857629dec5e0e1" integrity sha512-APM0Gt1KoXBz0iIkkdB/kfvGOwC4UuJFeG/c+yV7wSc7q96cG/kJ0HiYCnzivD9SB53cLV1MlHFNfOuPaadYSw== -run-async@^2.4.0: - version "2.4.1" - resolved "https://registry.yarnpkg.com/run-async/-/run-async-2.4.1.tgz#8440eccf99ea3e70bd409d49aab88e10c189a455" - integrity sha512-tvVnVv01b8c1RrA6Ep7JkStj85Guv/YrMcwqYQnwjsAS2cTmmPGBBjAjpCW7RrSodNSoE2/qg9O4bceNvUuDgQ== - run-parallel@^1.1.9: version "1.2.0" resolved "https://registry.yarnpkg.com/run-parallel/-/run-parallel-1.2.0.tgz#66d1368da7bdf921eb9d95bd1a9229e7f21a43ee" @@ -3800,19 +3626,12 @@ run-parallel@^1.1.9: dependencies: queue-microtask "^1.2.2" -rxjs@^7.5.5: - version "7.8.1" - resolved "https://registry.yarnpkg.com/rxjs/-/rxjs-7.8.1.tgz#6f6f3d99ea8044291efd92e7c7fcf562c4057543" - integrity sha512-AA3TVj+0A2iuIoQkWEK/tqFjBq2j+6PO6Y0zJcvzLAFhEFIO3HL0vls9hWLncZbAAbK0mar7oZ4V079I/qPMxg== - dependencies: - tslib "^2.1.0" - safe-buffer@5.1.2: version "5.1.2" resolved "https://registry.yarnpkg.com/safe-buffer/-/safe-buffer-5.1.2.tgz#991ec69d296e0313747d59bdfd2b745c35f8828d" integrity sha512-Gd2UZBJDkXlY7GbJxfsE8/nvKkUEU1G38c1siN6QP6a9PT9MmHB8GnpscSmMJSoF8LOIrt8ud/wPtojys4G6+g== -safe-buffer@5.2.1, safe-buffer@~5.2.0: +safe-buffer@5.2.1: version "5.2.1" resolved "https://registry.yarnpkg.com/safe-buffer/-/safe-buffer-5.2.1.tgz#1eaf9fa9bdb1fdd4ec75f58f9cdb4e6b7827eec6" integrity sha512-rp3So07KcdmmKbGvgaNxQSJr7bGVSVk5S9Eq1F+ppbRo70+YeaDxkw5Dd8NPN+GD6bjnYm2VuPuCXmpuYvmCXQ== @@ -3831,10 +3650,10 @@ safe-regex-test@^1.0.0: resolved "https://registry.yarnpkg.com/safer-buffer/-/safer-buffer-2.1.2.tgz#44fa161b0187b9549dd84bb91802f9bd8385cd6a" integrity sha512-YZo3K82SD7Riyi0E1EQPojLz7kpepnSQI9IyPbHHg1XXXevb5dJI7tpyN2ADxGcQbHG7vcyRHk0cbwqcQriUtg== -sass@1.58.1: - version "1.58.1" - resolved "https://registry.yarnpkg.com/sass/-/sass-1.58.1.tgz#17ab0390076a50578ed0733f1cc45429e03405f6" - integrity sha512-bnINi6nPXbP1XNRaranMFEBZWUfdW/AF16Ql5+ypRxfTvCRTTKrLsMIakyDcayUt2t/RZotmL4kgJwNH5xO+bg== +sass@1.75.0: + version "1.75.0" + resolved "https://registry.yarnpkg.com/sass/-/sass-1.75.0.tgz#91bbe87fb02dfcc34e052ddd6ab80f60d392be6c" + integrity sha512-ShMYi3WkrDWxExyxSZPst4/okE9ts46xZmJDSawJQrnte7M1V9fScVB+uNXOVKRBt0PggHOwoZcn8mYX4trnBw== dependencies: chokidar ">=3.0.0 <4.0.0" immutable "^4.0.0" @@ -3852,30 +3671,25 @@ semver-compare@^1.0.0: resolved "https://registry.yarnpkg.com/semver-compare/-/semver-compare-1.0.0.tgz#0dee216a1c941ab37e9efb1788f6afc5ff5537fc" integrity sha512-YM3/ITh2MJ5MtzaM429anh+x2jiLVjqILF4m4oyQB18W7Ggea7BfqdH/wGMK7dDiMghv/6WG7znWMwUDzJiXow== -"semver@2 || 3 || 4 || 5", semver@^5.5.0, semver@^5.6.0: +"semver@2 || 3 || 4 || 5", semver@^5.5.0: version "5.7.1" resolved "https://registry.yarnpkg.com/semver/-/semver-5.7.1.tgz#a954f931aeba508d307bbf069eff0c01c96116f7" integrity sha512-sauaDf/PZdVgrLTNYHRtpXa1iRiKcaebiKQ1BJdpQlWH2lCvexQdX55snPFyK7QzpudqbCI0qXFfOasHdyNDGQ== -semver@^6.0.0: - version "6.3.0" - resolved "https://registry.yarnpkg.com/semver/-/semver-6.3.0.tgz#ee0a64c8af5e8ceea67687b133761e1becbd1d3d" - integrity sha512-b39TBaTSfV6yBrapU89p5fKekE2m/NwnDocOVruQFS1/veMgdzuPcnOM34M6CwxW8jH/lxEa5rBoDeUwu5HHTw== - -semver@^7.3.2: - version "7.5.1" - resolved "https://registry.yarnpkg.com/semver/-/semver-7.5.1.tgz#c90c4d631cf74720e46b21c1d37ea07edfab91ec" - integrity sha512-Wvss5ivl8TMRZXXESstBA4uR5iXgEN/VC5/sOcuXdVLzcdkz4HWetIoRfG5gb5X+ij/G9rw9YoGn3QoQ8OCSpw== - dependencies: - lru-cache "^6.0.0" - -semver@^7.3.5, semver@^7.3.6, semver@^7.3.7: +semver@^7.3.6: version "7.3.8" resolved "https://registry.yarnpkg.com/semver/-/semver-7.3.8.tgz#07a78feafb3f7b32347d725e33de7e2a2df67798" integrity sha512-NB1ctGL5rlHrPJtFDVIVzTyQylMLu9N9VICA6HSFJo8MCGVTMW6gfpicwKmmK/dAjTOrqu5l63JJOpDSrAis3A== dependencies: lru-cache "^6.0.0" +semver@^7.5.3, semver@^7.5.4, semver@^7.6.0: + version "7.6.0" + resolved "https://registry.yarnpkg.com/semver/-/semver-7.6.0.tgz#1a46a4db4bffcccd97b743b5005c8325f23d4e2d" + integrity sha512-EnwXhrlwXMk9gKu5/flx5sv/an57AkRplG3hTK68W7FRDN+k+OWBj65M7719OkA82XLBxrcX0KSHj+X5COhOVg== + dependencies: + lru-cache "^6.0.0" + send@0.18.0: version "0.18.0" resolved "https://registry.yarnpkg.com/send/-/send-0.18.0.tgz#670167cc654b05f5aa4a767f9113bb371bc706be" @@ -3910,11 +3724,6 @@ server-destroy@^1.0.1: resolved "https://registry.yarnpkg.com/server-destroy/-/server-destroy-1.0.1.tgz#f13bf928e42b9c3e79383e61cc3998b5d14e6cdd" integrity sha512-rb+9B5YBIEzYcD6x2VKidaa+cqYBJQKnU4oe4E3ANwRRN56yk/ua1YCJT1n21NTS8w6CcOclAKNP3PhdCXKYtQ== -set-cookie-parser@^2.4.6: - version "2.6.0" - resolved "https://registry.yarnpkg.com/set-cookie-parser/-/set-cookie-parser-2.6.0.tgz#131921e50f62ff1a66a461d7d62d7b21d5d15a51" - integrity sha512-RVnVQxTXuerk653XfuliOxBP81Sf0+qfQE73LIYKcyMYHG94AuH0kgrQpRDuTZnSmjpysHmzxJXKNfa6PjFhyQ== - setprototypeof@1.2.0: version "1.2.0" resolved "https://registry.yarnpkg.com/setprototypeof/-/setprototypeof-1.2.0.tgz#66c9a24a73f9fc28cbe66b09fed3d33dcaf1b424" @@ -3963,15 +3772,10 @@ siginfo@^2.0.0: resolved "https://registry.yarnpkg.com/siginfo/-/siginfo-2.0.0.tgz#32e76c70b79724e3bb567cb9d543eb858ccfaf30" integrity sha512-ybx0WO1/8bSBLEWXZvEd7gMW3Sn3JFlW3TvX1nREbDLRNQNaeNN8WK0meBwPdAaOI7TtRRRJn/Es1zhrrCHu7g== -sigmund@^1.0.1: - version "1.0.1" - resolved "https://registry.yarnpkg.com/sigmund/-/sigmund-1.0.1.tgz#3ff21f198cad2175f9f3b781853fd94d0d19b590" - integrity sha512-fCvEXfh6NWpm+YSuY2bpXb/VIihqWA6hLsgboC+0nl71Q7N7o2eaCW8mJa/NLvQhs6jpd3VZV4UiUQlV6+lc8g== - -signal-exit@^3.0.2: - version "3.0.7" - resolved "https://registry.yarnpkg.com/signal-exit/-/signal-exit-3.0.7.tgz#a9a1767f8af84155114eaabd73f99273c8f59ad9" - integrity sha512-wnD2ZE+l+SPC/uoS0vXeE9L1+0wuaMqKlfz9AMUo38JsyLSBWSFcHR1Rri62LZc12vLr1gb3jl7iwQhgwpAbGQ== +signal-exit@^4.0.1, signal-exit@^4.1.0: + version "4.1.0" + resolved "https://registry.yarnpkg.com/signal-exit/-/signal-exit-4.1.0.tgz#952188c1cbd546070e2dd20d0f41c0ae0530cb04" + integrity sha512-bzyZ1e88w9O1iNJbKnOlvYTrWPDl46O1bG0D3XInv+9tkPrxrN8jUUTiFlDkkmKWgn1M6CfIA13SuGqOa9Korw== slash@^3.0.0: version "3.0.0" @@ -3983,15 +3787,10 @@ slash@^3.0.0: resolved "https://registry.yarnpkg.com/source-map-js/-/source-map-js-1.0.2.tgz#adbc361d9c62df380125e7f161f71c826f1e490c" integrity sha512-R0XvVJ9WusLiqTCEiGCmICCMplcCkIwwR11mOSD9CR5u+IXYdiseeEuXCVAjS54zqwkLcPNnmU4OeJ6tUrWhDw== -source-map@0.6.1, source-map@^0.6.1: - version "0.6.1" - resolved "https://registry.yarnpkg.com/source-map/-/source-map-0.6.1.tgz#74722af32e9614e9c287a8d0bbde48b5e2f1a263" - integrity sha512-UjgapumWlbMhkBgzT7Ykc5YXUT46F0iKu8SGXq0bcwP5dz/h0Plj6enJqjz1Zbq2l5WaqYnrVbwWOWMyF3F47g== - -sourcemap-codec@^1.4.8: - version "1.4.8" - resolved "https://registry.yarnpkg.com/sourcemap-codec/-/sourcemap-codec-1.4.8.tgz#ea804bd94857402e6992d05a38ef1ae35a9ab4c4" - integrity sha512-9NykojV5Uih4lgo5So5dtw+f0JgJX30KCNI8gwhz2J9A15wD0Ml6tjHKwf6fTSa6fAdVBdZeNOs9eJ71qCk8vA== +source-map-js@^1.2.0: + version "1.2.0" + resolved "https://registry.yarnpkg.com/source-map-js/-/source-map-js-1.2.0.tgz#16b809c162517b5b8c3e7dcd315a2a5c2612b2af" + integrity sha512-itJW8lvSA0TXEphiRoawsCksnlf8SyvmFzIhltqAHluXd88pkCd+cXJVHTDwdCr0IzwptSm035IHQktUu1QUMg== spdx-correct@^3.0.0: version "3.1.1" @@ -4024,27 +3823,20 @@ split-on-first@^3.0.0: resolved "https://registry.yarnpkg.com/split-on-first/-/split-on-first-3.0.0.tgz#f04959c9ea8101b9b0bbf35a61b9ebea784a23e7" integrity sha512-qxQJTx2ryR0Dw0ITYyekNQWpz6f8dGd7vffGNflQQ3Iqj9NJ6qiZ7ELpZsJ/QBhIVAiDfXdag3+Gp8RvWa62AA== -stack-utils@^2.0.3: - version "2.0.6" - resolved "https://registry.yarnpkg.com/stack-utils/-/stack-utils-2.0.6.tgz#aaf0748169c02fc33c8232abccf933f54a1cc34f" - integrity sha512-XlkWvfIm6RmsWtNJx+uqtKLS8eqFbxUg0ZzLXqY0caEy9l7hruX8IpiDnjsLavoBgqCCR71TqWO8MaXYheJ3RQ== - dependencies: - escape-string-regexp "^2.0.0" - stackback@0.0.2: version "0.0.2" resolved "https://registry.yarnpkg.com/stackback/-/stackback-0.0.2.tgz#1ac8a0d9483848d1695e418b6d031a3c3ce68e3b" integrity sha512-1XMJE5fQo1jGH6Y/7ebnwPOBEkIEnT4QF32d5R1+VXdXveM0IBMJt8zfaxX1P3QhVwrYe+576+jkANtSS2mBbw== -statuses@2.0.1: +statuses@2.0.1, statuses@^2.0.1: version "2.0.1" resolved "https://registry.yarnpkg.com/statuses/-/statuses-2.0.1.tgz#55cb000ccf1d48728bd23c685a063998cf1a1b63" integrity sha512-RwNA9Z/7PrK06rYLIzFMlaF+l73iwpzsqRIFgbMLbTcLD6cOao82TaWefPXQvB2fOC4AjuYSEndS7N/mTCbkdQ== -std-env@^3.3.2: - version "3.3.3" - resolved "https://registry.yarnpkg.com/std-env/-/std-env-3.3.3.tgz#a54f06eb245fdcfef53d56f3c0251f1d5c3d01fe" - integrity sha512-Rz6yejtVyWnVjC1RFvNmYL10kgjC49EOghxWn0RFqlCHGFpQx+Xe7yW3I4ceK1SGrWIGMjD5Kbue8W/udkbMJg== +std-env@^3.5.0: + version "3.7.0" + resolved "https://registry.yarnpkg.com/std-env/-/std-env-3.7.0.tgz#c9f7386ced6ecf13360b6c6c55b8aaa4ef7481d2" + integrity sha512-JPbdCEQLj1w5GilpiHAx3qJvFndqybBysA3qUOnznweH4QbNYUsW/ea8QzSrnh0vNsezMMw5bcVool8lM0gwzg== steno@^0.4.1: version "0.4.4" @@ -4060,17 +3852,19 @@ stop-iteration-iterator@^1.0.0: dependencies: internal-slot "^1.0.4" -strict-event-emitter@^0.2.4: - version "0.2.8" - resolved "https://registry.yarnpkg.com/strict-event-emitter/-/strict-event-emitter-0.2.8.tgz#b4e768927c67273c14c13d20e19d5e6c934b47ca" - integrity sha512-KDf/ujU8Zud3YaLtMCcTI4xkZlZVIYxTLr+XIULexP+77EEVWixeXroLUXQXiVtH4XH2W7jr/3PT1v3zBuvc3A== - dependencies: - events "^3.3.0" +strict-event-emitter@^0.5.1: + version "0.5.1" + resolved "https://registry.yarnpkg.com/strict-event-emitter/-/strict-event-emitter-0.5.1.tgz#1602ece81c51574ca39c6815e09f1a3e8550bd93" + integrity sha512-vMgjE/GGEPEFnhFub6pa4FmJBRBVOLpIII2hvCZ8Kzb7K0hlHo7mQv6xYrBvCL2LtAIBwFUK8wvuJgTVSQ5MFQ== -strict-event-emitter@^0.4.3: - version "0.4.6" - resolved "https://registry.yarnpkg.com/strict-event-emitter/-/strict-event-emitter-0.4.6.tgz#ff347c8162b3e931e3ff5f02cfce6772c3b07eb3" - integrity sha512-12KWeb+wixJohmnwNFerbyiBrAlq5qJLwIt38etRtKtmmHyDSoGlIqFE9wx+4IwG0aDjI7GV8tc8ZccjWZZtTg== +"string-width-cjs@npm:string-width@^4.2.0": + version "4.2.3" + resolved "https://registry.yarnpkg.com/string-width/-/string-width-4.2.3.tgz#269c7117d27b05ad2e536830a8ec895ef9c6d010" + integrity sha512-wKyQRQpjJ0sIp62ErSZdGsjMJWsap5oRNihHhu6G7JVO/9jIB6UyevL+tXuOqrng8j/cxKTWyWUwvSTriiZz/g== + dependencies: + emoji-regex "^8.0.0" + is-fullwidth-code-point "^3.0.0" + strip-ansi "^6.0.1" string-width@^4.1.0, string-width@^4.2.0, string-width@^4.2.3: version "4.2.3" @@ -4081,6 +3875,15 @@ string-width@^4.1.0, string-width@^4.2.0, string-width@^4.2.3: is-fullwidth-code-point "^3.0.0" strip-ansi "^6.0.1" +string-width@^5.0.1, string-width@^5.1.2: + version "5.1.2" + resolved "https://registry.yarnpkg.com/string-width/-/string-width-5.1.2.tgz#14f8daec6d81e7221d2a357e668cab73bdbca794" + integrity sha512-HnLOCR3vjcY8beoNLtcjZ5/nxn2afmME6lhrDrebokqMap+XbeW8n9TXpPDOqdGK5qcI3oT0GKTW6wC7EMiVqA== + dependencies: + eastasianwidth "^0.2.0" + emoji-regex "^9.2.2" + strip-ansi "^7.0.1" + string.prototype.padend@^3.0.0: version "3.1.4" resolved "https://registry.yarnpkg.com/string.prototype.padend/-/string.prototype.padend-3.1.4.tgz#2c43bb3a89eb54b6750de5942c123d6c98dd65b6" @@ -4108,12 +3911,12 @@ string.prototype.trimstart@^1.0.6: define-properties "^1.1.4" es-abstract "^1.20.4" -string_decoder@^1.1.1: - version "1.3.0" - resolved "https://registry.yarnpkg.com/string_decoder/-/string_decoder-1.3.0.tgz#42f114594a46cf1a8e30b0a84f56c78c3edac21e" - integrity sha512-hkRX8U1WjJFd8LsDJ2yQ/wWWxaopEsABU1XfkM8A+j0+85JAGppt16cr1Whg6KIbb4okU6Mql6BOj+uup/wKeA== +"strip-ansi-cjs@npm:strip-ansi@^6.0.1": + version "6.0.1" + resolved "https://registry.yarnpkg.com/strip-ansi/-/strip-ansi-6.0.1.tgz#9e26c63d30f53443e9489495b2105d37b67a85d9" + integrity sha512-Y38VPSHcqkFrCpFnQ9vuSXmquuv5oXOKpGeT6aGrr3o3Gc9AlVa6JBfUSOCnbxGGZF+/0ooI7KrPuUSztUdU5A== dependencies: - safe-buffer "~5.2.0" + ansi-regex "^5.0.1" strip-ansi@^6.0.0, strip-ansi@^6.0.1: version "6.0.1" @@ -4122,11 +3925,23 @@ strip-ansi@^6.0.0, strip-ansi@^6.0.1: dependencies: ansi-regex "^5.0.1" +strip-ansi@^7.0.1: + version "7.1.0" + resolved "https://registry.yarnpkg.com/strip-ansi/-/strip-ansi-7.1.0.tgz#d5b6568ca689d8561370b0707685d22434faff45" + integrity sha512-iq6eVVI64nQQTRYq2KtEg2d2uU7LElhTJwsH4YzIHZshxlgZms/wIc4VoDQTlG/IvVIrBKG06CrZnp0qv7hkcQ== + dependencies: + ansi-regex "^6.0.1" + strip-bom@^3.0.0: version "3.0.0" resolved "https://registry.yarnpkg.com/strip-bom/-/strip-bom-3.0.0.tgz#2334c18e9c759f7bdd56fdef7e9ae3d588e68ed3" integrity sha512-vavAMRXOgBVNF6nyEEmL3DBK19iRpDcoIwW+swQ+CbGiu7lju6t+JklA1MHweoWtadgt4ISVUsXLyDq34ddcwA== +strip-final-newline@^3.0.0: + version "3.0.0" + resolved "https://registry.yarnpkg.com/strip-final-newline/-/strip-final-newline-3.0.0.tgz#52894c313fbff318835280aed60ff71ebf12b8fd" + integrity sha512-dOESqjYr96iWYylGObzd39EuNTa5VJxyvVAEm5Jnh7KGo75V43Hk1odPQkNDyXNmUR6k+gEiDVXnjB8HJ3crXw== + strip-indent@^3.0.0: version "3.0.0" resolved "https://registry.yarnpkg.com/strip-indent/-/strip-indent-3.0.0.tgz#c32e1cee940b6b3432c771bc2c54bcce73cd3001" @@ -4134,17 +3949,17 @@ strip-indent@^3.0.0: dependencies: min-indent "^1.0.0" -strip-json-comments@^3.1.0, strip-json-comments@^3.1.1: +strip-json-comments@^3.1.1: version "3.1.1" resolved "https://registry.yarnpkg.com/strip-json-comments/-/strip-json-comments-3.1.1.tgz#31f1281b3832630434831c310c01cccda8cbe006" integrity sha512-6fPc+R4ihwqP6N/aIv2f1gMH8lOVtWQHoqC4yK6oSDVVocumAsfCqjkXnqiYMhmMwS/mEHLp7Vehlt3ql6lEig== -strip-literal@^1.0.1: - version "1.0.1" - resolved "https://registry.yarnpkg.com/strip-literal/-/strip-literal-1.0.1.tgz#0115a332710c849b4e46497891fb8d585e404bd2" - integrity sha512-QZTsipNpa2Ppr6v1AmJHESqJ3Uz247MUS0OjrnnZjFAvEoWqxuyFuXn2xLgMtRnijJShAa1HL0gtJyUs7u7n3Q== +strip-literal@^2.0.0: + version "2.1.0" + resolved "https://registry.yarnpkg.com/strip-literal/-/strip-literal-2.1.0.tgz#6d82ade5e2e74f5c7e8739b6c84692bd65f0bd2a" + integrity sha512-Op+UycaUt/8FbN/Z2TWPBLge3jWrP3xj10f3fnYxf052bKuS3EKs1ZQcVGjnEMdsNVAM+plXRdmjrZ/KgG3Skw== dependencies: - acorn "^8.8.2" + js-tokens "^9.0.0" supports-color@^5.3.0: version "5.5.0" @@ -4170,42 +3985,33 @@ symbol-tree@^3.2.4: resolved "https://registry.yarnpkg.com/symbol-tree/-/symbol-tree-3.2.4.tgz#430637d248ba77e078883951fb9aa0eed7c63fa2" integrity sha512-9QNk5KwDF+Bvz+PyObkmSYjI5ksVUYtjW7AU22r2NKcfLJcXp96hkDWU3+XndOsUb+AQ9QhfzfCT2O+CNWT5Tw== +synckit@^0.8.6: + version "0.8.8" + resolved "https://registry.yarnpkg.com/synckit/-/synckit-0.8.8.tgz#fe7fe446518e3d3d49f5e429f443cf08b6edfcd7" + integrity sha512-HwOKAP7Wc5aRGYdKH+dw0PRRpbO841v2DENBtjnR5HFWoiNByAl7vrx3p0G/rCyYXQsrxqtX48TImFtPcIHSpQ== + dependencies: + "@pkgr/core" "^0.1.0" + tslib "^2.6.2" + text-table@^0.2.0: version "0.2.0" resolved "https://registry.yarnpkg.com/text-table/-/text-table-0.2.0.tgz#7f5ee823ae805207c00af2df4a84ec3fcfa570b4" integrity sha512-N+8UisAXDGk8PFXP4HAzVR9nbfmVJ3zYLAWiTIoqC5v5isinhr+r5uaO8+7r3BMfuNIufIsA7RdpVgacC2cSpw== -through@^2.3.6: - version "2.3.8" - resolved "https://registry.yarnpkg.com/through/-/through-2.3.8.tgz#0dd4c9ffaabc357960b1b724115d7e0e86a2e1f5" - integrity sha512-w89qg7PI8wAdvX60bMDP+bFoD5Dvhm9oLheFp5O4a2QF0cSBGsBX4qZmadPMvVqlLJBBci+WqGGOAPvcDeNSVg== - -time-zone@^1.0.0: - version "1.0.0" - resolved "https://registry.yarnpkg.com/time-zone/-/time-zone-1.0.0.tgz#99c5bf55958966af6d06d83bdf3800dc82faec5d" - integrity sha512-TIsDdtKo6+XrPtiTm1ssmMngN1sAhyKnTO2kunQWqNPWIVvCm15Wmw4SWInwTVgJ5u/Tr04+8Ei9TNcw4x4ONA== - -tinybench@^2.5.0: - version "2.5.0" - resolved "https://registry.yarnpkg.com/tinybench/-/tinybench-2.5.0.tgz#4711c99bbf6f3e986f67eb722fed9cddb3a68ba5" - integrity sha512-kRwSG8Zx4tjF9ZiyH4bhaebu+EDz1BOx9hOigYHlUW4xxI/wKIUQUqo018UlU4ar6ATPBsaMrdbKZ+tmPdohFA== - -tinypool@^0.5.0: - version "0.5.0" - resolved "https://registry.yarnpkg.com/tinypool/-/tinypool-0.5.0.tgz#3861c3069bf71e4f1f5aa2d2e6b3aaacc278961e" - integrity sha512-paHQtnrlS1QZYKF/GnLoOM/DN9fqaGOFbCbxzAhwniySnzl9Ebk8w73/dd34DAhe/obUbPAOldTyYXQZxnPBPQ== +tinybench@^2.5.1: + version "2.7.0" + resolved "https://registry.yarnpkg.com/tinybench/-/tinybench-2.7.0.tgz#d56198a69bead7e240c8f9542484f3eb3c3f749d" + integrity sha512-Qgayeb106x2o4hNzNjsZEfFziw8IbKqtbXBjVh7VIZfBxfD5M4gWtpyx5+YTae2gJ6Y6Dz/KLepiv16RFeQWNA== -tinyspy@^2.1.0: - version "2.1.1" - resolved "https://registry.yarnpkg.com/tinyspy/-/tinyspy-2.1.1.tgz#9e6371b00c259e5c5b301917ca18c01d40ae558c" - integrity sha512-XPJL2uSzcOyBMky6OFrusqWlzfFrXtE0hPuMgW8A2HmaqrPo4ZQHRN/V0QXN3FSjKxpsbRrFc5LI7KOwBsT1/w== +tinypool@^0.8.3: + version "0.8.4" + resolved "https://registry.yarnpkg.com/tinypool/-/tinypool-0.8.4.tgz#e217fe1270d941b39e98c625dcecebb1408c9aa8" + integrity sha512-i11VH5gS6IFeLY3gMBQ00/MmLncVP7JLXOw1vlgkytLmJK7QnEr7NXf0LBdxfmNPAeyetukOk0bOYrJrFGjYJQ== -tmp@^0.0.33: - version "0.0.33" - resolved "https://registry.yarnpkg.com/tmp/-/tmp-0.0.33.tgz#6d34335889768d21b2bcda0aa277ced3b1bfadf9" - integrity sha512-jRCJlojKnZ3addtTOjdIqoRuPEKBvNXcGYqzO6zWZX8KfKEpnGY5jfggJQ3EjKuu8D4bJRr0y+cYJFmYbImXGw== - dependencies: - os-tmpdir "~1.0.2" +tinyspy@^2.2.0: + version "2.2.1" + resolved "https://registry.yarnpkg.com/tinyspy/-/tinyspy-2.2.1.tgz#117b2342f1f38a0dbdcc73a50a454883adf861d1" + integrity sha512-KYad6Vy5VDWV4GH3fjpseMQ/XU2BhIYP7Vzd0LG44qRWm/Yt2WCOTicFdvmgo6gWaqooMQCawTtILVQJupKu7A== to-regex-range@^5.0.1: version "5.0.1" @@ -4219,44 +4025,32 @@ toidentifier@1.0.1: resolved "https://registry.yarnpkg.com/toidentifier/-/toidentifier-1.0.1.tgz#3be34321a88a820ed1bd80dfaa33e479fbb8dd35" integrity sha512-o5sSPKEkg/DIQNmH43V0/uerLrpzVedkUh8tGNvaeXpfpuwjKenlSox/2O/BTlZUtEe+JG7s5YhEz608PlAHRA== -tough-cookie@^4.1.2: - version "4.1.2" - resolved "https://registry.yarnpkg.com/tough-cookie/-/tough-cookie-4.1.2.tgz#e53e84b85f24e0b65dd526f46628db6c85f6b874" - integrity sha512-G9fqXWoYFZgTc2z8Q5zaHy/vJMjm+WV0AkAeHxVCQiEB1b+dGvWzFW6QV07cY5jQ5gRkeid2qIkzkxUnmoQZUQ== +tough-cookie@^4.1.3: + version "4.1.3" + resolved "https://registry.yarnpkg.com/tough-cookie/-/tough-cookie-4.1.3.tgz#97b9adb0728b42280aa3d814b6b999b2ff0318bf" + integrity sha512-aX/y5pVRkfRnfmuX+OdbSdXvPe6ieKX/G2s7e98f4poJHnqH3281gDPm/metm6E/WRamfx7WC4HUqkWHfQHprw== dependencies: psl "^1.1.33" punycode "^2.1.1" universalify "^0.2.0" url-parse "^1.5.3" -tr46@^4.1.1: - version "4.1.1" - resolved "https://registry.yarnpkg.com/tr46/-/tr46-4.1.1.tgz#281a758dcc82aeb4fe38c7dfe4d11a395aac8469" - integrity sha512-2lv/66T7e5yNyhAAC4NaKe5nVavzuGJQVVtRYLyQ2OI8tsJ61PMLlelehb0wi2Hx6+hT/OJUWZcw8MjlSRnxvw== +tr46@^5.0.0: + version "5.0.0" + resolved "https://registry.yarnpkg.com/tr46/-/tr46-5.0.0.tgz#3b46d583613ec7283020d79019f1335723801cec" + integrity sha512-tk2G5R2KRwBd+ZN0zaEXpmzdKyOYksXwywulIX95MBODjSzMIuQnQ3m8JxgbhnL1LeVo7lqQKsYa1O3Htl7K5g== dependencies: - punycode "^2.3.0" - -tr46@~0.0.3: - version "0.0.3" - resolved "https://registry.yarnpkg.com/tr46/-/tr46-0.0.3.tgz#8184fd347dac9cdc185992f3a6622e14b9d9ab6a" - integrity sha512-N3WMsuqV66lT30CrXNbEjx4GEwlow3v6rr4mCcv6prnfwhS01rkgyFdjPNBYd9br7LpXV1+Emh01fHnq2Gdgrw== - -tslib@^1.8.1: - version "1.14.1" - resolved "https://registry.yarnpkg.com/tslib/-/tslib-1.14.1.tgz#cf2d38bdc34a134bcaf1091c41f6619e2f672d00" - integrity sha512-Xni35NKzjgMrwevysHTCArtLDpPvye8zV/0E4EyYn43P7/7qvQwPh9BGkHewbMulVntbigmcT7rdX3BNo9wRJg== + punycode "^2.3.1" -tslib@^2.1.0: - version "2.6.0" - resolved "https://registry.yarnpkg.com/tslib/-/tslib-2.6.0.tgz#b295854684dbda164e181d259a22cd779dcd7bc3" - integrity sha512-7At1WUettjcSRHXCyYtTselblcHl9PJFFVKiCAy/bY97+BPZXSQ2wbq0P9s8tK2G7dFQfNnlJnPAiArVBVBsfA== +ts-api-utils@^1.3.0: + version "1.3.0" + resolved "https://registry.yarnpkg.com/ts-api-utils/-/ts-api-utils-1.3.0.tgz#4b490e27129f1e8e686b45cc4ab63714dc60eea1" + integrity sha512-UQMIo7pb8WRomKR1/+MFVLTroIvDVtMX3K6OUir8ynLyzB8Jeriont2bTAtmNPa1ekAgN7YPDyf6V+ygrdU+eQ== -tsutils@^3.21.0: - version "3.21.0" - resolved "https://registry.yarnpkg.com/tsutils/-/tsutils-3.21.0.tgz#b48717d394cea6c1e096983eed58e9d61715b623" - integrity sha512-mHKK3iUXL+3UF6xL5k0PEhKRUBKPBCv/+RkEOpjRWxxx27KKRBmmA60A9pgOUvMi8GKhRMPEmjBRPzs2W7O1OA== - dependencies: - tslib "^1.8.1" +tslib@^2.6.2: + version "2.6.2" + resolved "https://registry.yarnpkg.com/tslib/-/tslib-2.6.2.tgz#703ac29425e7b37cd6fd456e92404d46d1f3e4ae" + integrity sha512-AEYxH93jGFPn/a2iVAwW87VuUIkR1FVUKB77NwMF7nBTDkDrrT/Hpt/IrCJ0QXhW27jTBDcf5ZY7w6RiqTMw2Q== type-check@^0.4.0, type-check@~0.4.0: version "0.4.0" @@ -4265,7 +4059,7 @@ type-check@^0.4.0, type-check@~0.4.0: dependencies: prelude-ls "^1.2.1" -type-detect@^4.0.0, type-detect@^4.0.5: +type-detect@^4.0.0, type-detect@^4.0.8: version "4.0.8" resolved "https://registry.yarnpkg.com/type-detect/-/type-detect-4.0.8.tgz#7646fb5f18871cfbb7749e69bd39a6388eb7450c" integrity sha512-0fr/mIH1dlO+x7TlcMy+bIDqKPsw/70tVyeHW787goQjhmqaZe10uwLujubK9q9Lg6Fiho1KUKDYz0Z7k7g5/g== @@ -4280,10 +4074,10 @@ type-fest@^0.21.3: resolved "https://registry.yarnpkg.com/type-fest/-/type-fest-0.21.3.tgz#d260a24b0198436e133fa26a524a6d65fa3b2e37" integrity sha512-t0rzBq87m3fVcduHDUFhKmyyX+9eo6WQjZvf51Ea/M0Q7+T374Jp1aUiyUl0GKxp8M/OETVHSDvmkyPgvX+X2w== -type-fest@^2.19.0: - version "2.19.0" - resolved "https://registry.yarnpkg.com/type-fest/-/type-fest-2.19.0.tgz#88068015bb33036a598b952e55e9311a60fd3a9b" - integrity sha512-RAH822pAdBgcNMAfWnCBU3CFZcfZ/i1eZjwFU/dsLKumyuuP3niueg2UAukXYF0E2AAoc82ZSSf9J0WQBinzHA== +type-fest@^4.9.0: + version "4.15.0" + resolved "https://registry.yarnpkg.com/type-fest/-/type-fest-4.15.0.tgz#21da206b89c15774cc718c4f2d693e13a1a14a43" + integrity sha512-tB9lu0pQpX5KJq54g+oHOLumOx+pMep4RaM6liXh2PKmVRFF+/vAtUP0ZaJ0kOySfVNjF6doBWPHhBhISKdlIA== type-is@~1.6.18: version "1.6.18" @@ -4312,6 +4106,11 @@ ufo@^1.1.2: resolved "https://registry.yarnpkg.com/ufo/-/ufo-1.1.2.tgz#d0d9e0fa09dece0c31ffd57bd363f030a35cfe76" integrity sha512-TrY6DsjTQQgyS3E3dBaOXf0TpPD8u9FVrVYmKVegJuFw51n/YB9XPt+U6ydzFG5ZIN7+DIjPbNmXoBj9esYhgQ== +ufo@^1.3.2: + version "1.5.3" + resolved "https://registry.yarnpkg.com/ufo/-/ufo-1.5.3.tgz#3325bd3c977b6c6cd3160bf4ff52989adc9d3344" + integrity sha512-Y7HYmWaFwPUmkoQCUIAYpKqkOf+SbVj/2fJJZ4RJMCfZp0rTGwRbzQD+HghfnhKOjL9E01okqz+ncJskGYfBNw== + unbox-primitive@^1.0.2: version "1.0.2" resolved "https://registry.yarnpkg.com/unbox-primitive/-/unbox-primitive-1.0.2.tgz#29032021057d5e6cdbd08c5129c226dff8ed6f9e" @@ -4322,6 +4121,11 @@ unbox-primitive@^1.0.2: has-symbols "^1.0.3" which-boxed-primitive "^1.0.2" +undici-types@~5.26.4: + version "5.26.5" + resolved "https://registry.yarnpkg.com/undici-types/-/undici-types-5.26.5.tgz#bcd539893d00b56e964fd2657a4866b221a65617" + integrity sha512-JlCMO+ehdEIKqlFxk6IfVoAUVmgz7cU7zD/h9XZ0qzeosSHmUJVOzSQvvYSYWXkFXC+IfLKSIffhv0sVZup6pA== + universalify@^0.2.0: version "0.2.0" resolved "https://registry.yarnpkg.com/universalify/-/universalify-0.2.0.tgz#6451760566fa857534745ab1dde952d1b1761be0" @@ -4352,31 +4156,20 @@ url-parse@^1.5.3: querystringify "^2.1.1" requires-port "^1.0.0" -util-deprecate@^1.0.1, util-deprecate@^1.0.2: +util-deprecate@^1.0.2: version "1.0.2" resolved "https://registry.yarnpkg.com/util-deprecate/-/util-deprecate-1.0.2.tgz#450d4dc9fa70de732762fbd2d4a28981419a0ccf" integrity sha512-EPD5q1uXyFxJpCrLnCc1nHnq3gOa6DZBocAIiI2TaSCA7VCJ1UJDMagCzIkXNsUYfD1daK//LTEQ8xiIbrHtcw== -util@^0.12.3: - version "0.12.5" - resolved "https://registry.yarnpkg.com/util/-/util-0.12.5.tgz#5f17a6059b73db61a875668781a1c2b136bd6fbc" - integrity sha512-kZf/K6hEIrWHI6XqOFUiiMa+79wE/D8Q+NCNAWclkyg3b4d2k7s0QGepNjiABc+aR3N1PAyHL7p6UcLY6LmrnA== - dependencies: - inherits "^2.0.3" - is-arguments "^1.0.4" - is-generator-function "^1.0.7" - is-typed-array "^1.1.3" - which-typed-array "^1.1.2" - utils-merge@1.0.1: version "1.0.1" resolved "https://registry.yarnpkg.com/utils-merge/-/utils-merge-1.0.1.tgz#9f95710f50a267947b2ccc124741c1028427e713" integrity sha512-pMZTvIkT1d+TFGvDOqodOclx0QWkkgi6Tdoa8gC8ffGAAqz9pzPTZWAybbsHHoED/ztMtkv/VoYTYyShUn81hA== -uuid@9.0.0: - version "9.0.0" - resolved "https://registry.yarnpkg.com/uuid/-/uuid-9.0.0.tgz#592f550650024a38ceb0c562f2f6aa435761efb5" - integrity sha512-MXcSTerfPa4uqyzStbRoTgt5XIe3x5+42+q1sDuy3R5MDk66URdLMOZe5aPX/SQd+kuYAh0FdP/pO28IkQyTeg== +uuid@9.0.1: + version "9.0.1" + resolved "https://registry.yarnpkg.com/uuid/-/uuid-9.0.1.tgz#e188d4c8853cc722220392c424cd637f32293f30" + integrity sha512-b+1eJOlsR9K8HJpow9Ok3fiWOWSIcIzXodvv0rQjVoOVNpWMpxf1wZNpt4y9h10odCNrqnYp1OBzRktckBe3sA== validate-npm-package-license@^3.0.1: version "3.0.4" @@ -4391,17 +4184,16 @@ vary@^1, vary@~1.1.2: resolved "https://registry.yarnpkg.com/vary/-/vary-1.1.2.tgz#2299f02c6ded30d4a5961b0b9f74524a18f634fc" integrity sha512-BNGbWLfd0eUPabhkXUVm0j8uuvREyTh5ovRa/dyow/BqAbZJyC+5fU+IzQOzmAKzYqYRAISoRhdQr3eIZ/PXqg== -vite-node@0.31.4: - version "0.31.4" - resolved "https://registry.yarnpkg.com/vite-node/-/vite-node-0.31.4.tgz#0437f76c35fa83f0a868d3fb5896ca9e164291f5" - integrity sha512-uzL377GjJtTbuc5KQxVbDu2xfU/x0wVjUtXQR2ihS21q/NK6ROr4oG0rsSkBBddZUVCwzfx22in76/0ZZHXgkQ== +vite-node@1.5.0: + version "1.5.0" + resolved "https://registry.yarnpkg.com/vite-node/-/vite-node-1.5.0.tgz#7f74dadfecb15bca016c5ce5ef85e5cc4b82abf2" + integrity sha512-tV8h6gMj6vPzVCa7l+VGq9lwoJjW8Y79vst8QZZGiuRAfijU+EEWuc0kFpmndQrWhMMhet1jdSF+40KSZUqIIw== dependencies: cac "^6.7.14" debug "^4.3.4" - mlly "^1.2.0" - pathe "^1.1.0" + pathe "^1.1.1" picocolors "^1.0.0" - vite "^3.0.0 || ^4.0.0" + vite "^5.0.0" vite-plugin-rewrite-all@1.0.1: version "1.0.1" @@ -4410,67 +4202,66 @@ vite-plugin-rewrite-all@1.0.1: dependencies: connect-history-api-fallback "^1.6.0" -vite-plugin-vuetify@1.0.2: - version "1.0.2" - resolved "https://registry.yarnpkg.com/vite-plugin-vuetify/-/vite-plugin-vuetify-1.0.2.tgz#d1777c63aa1b3a308756461b3d0299fd101ee8f4" - integrity sha512-MubIcKD33O8wtgQXlbEXE7ccTEpHZ8nPpe77y9Wy3my2MWw/PgehP9VqTp92BLqr0R1dSL970Lynvisx3UxBFw== +vite-plugin-vuetify@2.0.3: + version "2.0.3" + resolved "https://registry.yarnpkg.com/vite-plugin-vuetify/-/vite-plugin-vuetify-2.0.3.tgz#b65ee4e05cfc6bf2b478a32b6d58b42398519f1e" + integrity sha512-HbYajgGgb/noaVKNRhnnXIiQZrNXfNIeanUGAwXgOxL6h/KULS40Uf51Kyz8hNmdegF+DwjgXXI/8J1PNS83xw== dependencies: - "@vuetify/loader-shared" "^1.7.1" + "@vuetify/loader-shared" "^2.0.3" debug "^4.3.3" upath "^2.0.1" -vite@4.1.1, "vite@^3.0.0 || ^4.0.0": - version "4.1.1" - resolved "https://registry.yarnpkg.com/vite/-/vite-4.1.1.tgz#3b18b81a4e85ce3df5cbdbf4c687d93ebf402e6b" - integrity sha512-LM9WWea8vsxhr782r9ntg+bhSFS06FJgCvvB0+8hf8UWtvaiDagKYWXndjfX6kGl74keHJUcpzrQliDXZlF5yg== +vite@5.2.9, vite@^5.0.0: + version "5.2.9" + resolved "https://registry.yarnpkg.com/vite/-/vite-5.2.9.tgz#cd9a356c6ff5f7456c09c5ce74068ffa8df743d9" + integrity sha512-uOQWfuZBlc6Y3W/DTuQ1Sr+oIXWvqljLvS881SVmAj00d5RdgShLcuXWxseWPd4HXwiYBFW/vXHfKFeqj9uQnw== dependencies: - esbuild "^0.16.14" - postcss "^8.4.21" - resolve "^1.22.1" - rollup "^3.10.0" + esbuild "^0.20.1" + postcss "^8.4.38" + rollup "^4.13.0" optionalDependencies: - fsevents "~2.3.2" + fsevents "~2.3.3" -vitest@0.31.4: - version "0.31.4" - resolved "https://registry.yarnpkg.com/vitest/-/vitest-0.31.4.tgz#5abe02562675262949c10e40811f348a80f6b2a6" - integrity sha512-GoV0VQPmWrUFOZSg3RpQAPN+LPmHg2/gxlMNJlyxJihkz6qReHDV6b0pPDcqFLNEPya4tWJ1pgwUNP9MLmUfvQ== - dependencies: - "@types/chai" "^4.3.5" - "@types/chai-subset" "^1.3.3" - "@types/node" "*" - "@vitest/expect" "0.31.4" - "@vitest/runner" "0.31.4" - "@vitest/snapshot" "0.31.4" - "@vitest/spy" "0.31.4" - "@vitest/utils" "0.31.4" - acorn "^8.8.2" - acorn-walk "^8.2.0" - cac "^6.7.14" - chai "^4.3.7" - concordance "^5.0.4" +vitest@1.5.0: + version "1.5.0" + resolved "https://registry.yarnpkg.com/vitest/-/vitest-1.5.0.tgz#6ebb396bd358650011a9c96c18fa614b668365c1" + integrity sha512-d8UKgR0m2kjdxDWX6911uwxout6GHS0XaGH1cksSIVVG8kRlE7G7aBw7myKQCvDI5dT4j7ZMa+l706BIORMDLw== + dependencies: + "@vitest/expect" "1.5.0" + "@vitest/runner" "1.5.0" + "@vitest/snapshot" "1.5.0" + "@vitest/spy" "1.5.0" + "@vitest/utils" "1.5.0" + acorn-walk "^8.3.2" + chai "^4.3.10" debug "^4.3.4" - local-pkg "^0.4.3" - magic-string "^0.30.0" - pathe "^1.1.0" + execa "^8.0.1" + local-pkg "^0.5.0" + magic-string "^0.30.5" + pathe "^1.1.1" picocolors "^1.0.0" - std-env "^3.3.2" - strip-literal "^1.0.1" - tinybench "^2.5.0" - tinypool "^0.5.0" - vite "^3.0.0 || ^4.0.0" - vite-node "0.31.4" + std-env "^3.5.0" + strip-literal "^2.0.0" + tinybench "^2.5.1" + tinypool "^0.8.3" + vite "^5.0.0" + vite-node "1.5.0" why-is-node-running "^2.2.2" +vue-component-type-helpers@^2.0.0: + version "2.0.13" + resolved "https://registry.yarnpkg.com/vue-component-type-helpers/-/vue-component-type-helpers-2.0.13.tgz#4cd08fe6c42a071159664e6c30b7fe1b01e17b68" + integrity sha512-xNO5B7DstNWETnoYflLkVgh8dK8h2ZDgxY1M2O0zrqGeBNq5yAZ8a10yCS9+HnixouNGYNX+ggU9MQQq86HTpg== + vue-demi@>=0.14.5: version "0.14.5" resolved "https://registry.yarnpkg.com/vue-demi/-/vue-demi-0.14.5.tgz#676d0463d1a1266d5ab5cba932e043d8f5f2fbd9" integrity sha512-o9NUVpl/YlsGJ7t+xuqJKx8EBGf1quRhCiT6D/J0pfwmk9zUwYkC7yrF4SZCe6fETvSM3UNL2edcbYrSyc4QHA== -vue-eslint-parser@^9.0.0, vue-eslint-parser@^9.0.1: - version "9.1.0" - resolved "https://registry.yarnpkg.com/vue-eslint-parser/-/vue-eslint-parser-9.1.0.tgz#0e121d1bb29bd10763c83e3cc583ee03434a9dd5" - integrity sha512-NGn/iQy8/Wb7RrRa4aRkokyCZfOUWk19OP5HP6JEozQFX5AoS/t+Z0ZN7FY4LlmWc4FNI922V7cvX28zctN8dQ== +vue-eslint-parser@^9.3.1, vue-eslint-parser@^9.4.2: + version "9.4.2" + resolved "https://registry.yarnpkg.com/vue-eslint-parser/-/vue-eslint-parser-9.4.2.tgz#02ffcce82042b082292f2d1672514615f0d95b6d" + integrity sha512-Ry9oiGmCAK91HrKMtCrKFWmSFWvYkpGglCeFAIqDdr9zdXmMMpJOmUJS7WWsW7fX81h6mwHmUZCQQ1E0PkSwYQ== dependencies: debug "^4.3.4" eslint-scope "^7.1.1" @@ -4480,22 +4271,21 @@ vue-eslint-parser@^9.0.0, vue-eslint-parser@^9.0.1: lodash "^4.17.21" semver "^7.3.6" -vue-i18n@9.2.2: - version "9.2.2" - resolved "https://registry.yarnpkg.com/vue-i18n/-/vue-i18n-9.2.2.tgz#aeb49d9424923c77e0d6441e3f21dafcecd0e666" - integrity sha512-yswpwtj89rTBhegUAv9Mu37LNznyu3NpyLQmozF3i1hYOhwpG8RjcjIFIIfnu+2MDZJGSZPXaKWvnQA71Yv9TQ== +vue-i18n@9.13.0: + version "9.13.0" + resolved "https://registry.yarnpkg.com/vue-i18n/-/vue-i18n-9.13.0.tgz#bedc0b9c5ebdc299c56296db747d8616db5eebc7" + integrity sha512-NlZ+e8rhGSGNk/Vfh4IUvlPRjljPCRslbNYgQmYZY+sLXZgahw8fylQguZU3e8ntJDvitfe40f8p3udOiKMS0A== dependencies: - "@intlify/core-base" "9.2.2" - "@intlify/shared" "9.2.2" - "@intlify/vue-devtools" "9.2.2" - "@vue/devtools-api" "^6.2.1" + "@intlify/core-base" "9.13.0" + "@intlify/shared" "9.13.0" + "@vue/devtools-api" "^6.5.0" -vue-router@4.2.2: - version "4.2.2" - resolved "https://registry.yarnpkg.com/vue-router/-/vue-router-4.2.2.tgz#b0097b66d89ca81c0986be03da244c7b32a4fd81" - integrity sha512-cChBPPmAflgBGmy3tBsjeoe3f3VOSG6naKyY5pjtrqLGbNEXdzCigFUHgBvp9e3ysAtFtEx7OLqcSDh/1Cq2TQ== +vue-router@4.3.2: + version "4.3.2" + resolved "https://registry.yarnpkg.com/vue-router/-/vue-router-4.3.2.tgz#08096c7765dacc6832f58e35f7a081a8b34116a7" + integrity sha512-hKQJ1vDAZ5LVkKEnHhmm1f9pMiWIBNGF5AwU67PdH7TyXCj/a4hTccuUuYCAMgJK6rO/NVYtQIEN3yL8CECa7Q== dependencies: - "@vue/devtools-api" "^6.5.0" + "@vue/devtools-api" "^6.5.1" vue-template-compiler@^2.7.14: version "2.7.14" @@ -4505,103 +4295,70 @@ vue-template-compiler@^2.7.14: de-indent "^1.0.2" he "^1.2.0" -vue-tsc@1.0.24: - version "1.0.24" - resolved "https://registry.yarnpkg.com/vue-tsc/-/vue-tsc-1.0.24.tgz#c0b270a7c8422408d3b6694fee61b39a4b9e4740" - integrity sha512-mmU1s5SAqE1nByQAiQnao9oU4vX+mSdsgI8H57SfKH6UVzq/jP9+Dbi2GaV+0b4Cn361d2ln8m6xeU60ApiEXg== +vue-tsc@2.0.13: + version "2.0.13" + resolved "https://registry.yarnpkg.com/vue-tsc/-/vue-tsc-2.0.13.tgz#6ee557705456442e0f43ec0d1774ebf5ffec54f1" + integrity sha512-a3nL3FvguCWVJUQW/jFrUxdeUtiEkbZoQjidqvMeBK//tuE2w6NWQAbdrEpY2+6nSa4kZoKZp8TZUMtHpjt4mQ== dependencies: - "@volar/vue-language-core" "1.0.24" - "@volar/vue-typescript" "1.0.24" + "@volar/typescript" "2.2.0-alpha.8" + "@vue/language-core" "2.0.13" + semver "^7.5.4" -vue3-ace-editor@2.2.3: - version "2.2.3" - resolved "https://registry.yarnpkg.com/vue3-ace-editor/-/vue3-ace-editor-2.2.3.tgz#3ce7dad1e00ef2b7bd9f07f29487d04ed040a609" - integrity sha512-CkbiZSgIraZCBRa5egRqxDFUow5b949aHMcVgjce09O1n1TIJ8wVR++9mWg6F7lNWH47DkYSu0lwB6OocxUJ7g== +vue3-ace-editor@2.2.4: + version "2.2.4" + resolved "https://registry.yarnpkg.com/vue3-ace-editor/-/vue3-ace-editor-2.2.4.tgz#1f2a787f91cf7979f27fab29e0e0604bb3ee1c17" + integrity sha512-FZkEyfpbH068BwjhMyNROxfEI8135Sc+x8ouxkMdCNkuj/Tuw83VP/gStFQqZHqljyX9/VfMTCdTqtOnJZGN8g== dependencies: resize-observer-polyfill "^1.5.1" -vue@3.3.4: - version "3.3.4" - resolved "https://registry.yarnpkg.com/vue/-/vue-3.3.4.tgz#8ed945d3873667df1d0fcf3b2463ada028f88bd6" - integrity sha512-VTyEYn3yvIeY1Py0WaYGZsXnz3y5UnGi62GjVEqvEGPl6nxbOrCXbVOTQWBEJUqAyTUk2uJ5JLVnYJ6ZzGbrSw== - dependencies: - "@vue/compiler-dom" "3.3.4" - "@vue/compiler-sfc" "3.3.4" - "@vue/runtime-dom" "3.3.4" - "@vue/server-renderer" "3.3.4" - "@vue/shared" "3.3.4" - -vuetify@3.3.2: - version "3.3.2" - resolved "https://registry.yarnpkg.com/vuetify/-/vuetify-3.3.2.tgz#8682486eb36b5d9d64c8f1c2bdfe36119f6ffbc6" - integrity sha512-m/R42di8FlyMaoktUe6k8JbF+A0vbJMpjQrZK7nH1ptK8VinVVQcaw+9m94wlO74IMR+LubxLh5t9I2ZtVCvjw== - -w3c-xmlserializer@^4.0.0: - version "4.0.0" - resolved "https://registry.yarnpkg.com/w3c-xmlserializer/-/w3c-xmlserializer-4.0.0.tgz#aebdc84920d806222936e3cdce408e32488a3073" - integrity sha512-d+BFHzbiCx6zGfz0HyQ6Rg69w9k19nviJspaj4yNscGjrHu94sVP+aRm75yEbCh+r2/yR+7q6hux9LVtbuTGBw== +vue@3.4.23: + version "3.4.23" + resolved "https://registry.yarnpkg.com/vue/-/vue-3.4.23.tgz#9d5a990a71c5bd5446f80377828e6587cfc488d5" + integrity sha512-X1y6yyGJ28LMUBJ0k/qIeKHstGd+BlWQEOT40x3auJFTmpIhpbKLgN7EFsqalnJXq1Km5ybDEsp6BhuWKciUDg== dependencies: - xml-name-validator "^4.0.0" + "@vue/compiler-dom" "3.4.23" + "@vue/compiler-sfc" "3.4.23" + "@vue/runtime-dom" "3.4.23" + "@vue/server-renderer" "3.4.23" + "@vue/shared" "3.4.23" -wcwidth@^1.0.1: - version "1.0.1" - resolved "https://registry.yarnpkg.com/wcwidth/-/wcwidth-1.0.1.tgz#f0b0dcf915bc5ff1528afadb2c0e17b532da2fe8" - integrity sha512-XHPEwS0q6TaxcvG85+8EYkbiCux2XtWG2mkc47Ng2A77BQu9+DqIOJldST4HgPkuea7dvKSj5VgX3P1d4rW8Tg== - dependencies: - defaults "^1.0.3" +vuetify@3.5.16: + version "3.5.16" + resolved "https://registry.yarnpkg.com/vuetify/-/vuetify-3.5.16.tgz#5046aab39bfa536f0d99c5be4f9d91a7245c3246" + integrity sha512-jyApfATreFMkgjvK0bL7ntZnr+p9TU73+4E3kX6fIvUitdAP9fltG7yj+v3k14HLqZRSNhTL1GhQ95DFx631zw== -web-encoding@^1.1.5: - version "1.1.5" - resolved "https://registry.yarnpkg.com/web-encoding/-/web-encoding-1.1.5.tgz#fc810cf7667364a6335c939913f5051d3e0c4864" - integrity sha512-HYLeVCdJ0+lBYV2FvNZmv3HJ2Nt0QYXqZojk3d9FJOLkwnuhzM9tmamh8d7HPM8QqjKH8DeHkFTx+CFlWpZZDA== +w3c-xmlserializer@^5.0.0: + version "5.0.0" + resolved "https://registry.yarnpkg.com/w3c-xmlserializer/-/w3c-xmlserializer-5.0.0.tgz#f925ba26855158594d907313cedd1476c5967f6c" + integrity sha512-o8qghlI8NZHU1lLPrpi2+Uq7abh4GGPpYANlalzWxyWteJOCsr/P+oPBA49TOLu5FTZO4d3F9MnWJfiMo4BkmA== dependencies: - util "^0.12.3" - optionalDependencies: - "@zxing/text-encoding" "0.9.0" - -webidl-conversions@^3.0.0: - version "3.0.1" - resolved "https://registry.yarnpkg.com/webidl-conversions/-/webidl-conversions-3.0.1.tgz#24534275e2a7bc6be7bc86611cc16ae0a5654871" - integrity sha512-2JAn3z8AR6rjK8Sm8orRC0h/bcl/DqL7tRPdGZ4I1CjdF+EaMLmYxBHyXuKL849eucPFhvBoxMsflfOb8kxaeQ== + xml-name-validator "^5.0.0" webidl-conversions@^7.0.0: version "7.0.0" resolved "https://registry.yarnpkg.com/webidl-conversions/-/webidl-conversions-7.0.0.tgz#256b4e1882be7debbf01d05f0aa2039778ea080a" integrity sha512-VwddBukDzu71offAQR975unBIGqfKZpM+8ZX6ySk8nYhVoo5CYaZyzt3YBvYtRtO+aoGlqxPg/B87NGVZ/fu6g== -well-known-symbols@^2.0.0: - version "2.0.0" - resolved "https://registry.yarnpkg.com/well-known-symbols/-/well-known-symbols-2.0.0.tgz#e9c7c07dbd132b7b84212c8174391ec1f9871ba5" - integrity sha512-ZMjC3ho+KXo0BfJb7JgtQ5IBuvnShdlACNkKkdsqBmYw3bPAaJfPeYUo6tLUaT5tG/Gkh7xkpBhKRQ9e7pyg9Q== - -whatwg-encoding@^2.0.0: - version "2.0.0" - resolved "https://registry.yarnpkg.com/whatwg-encoding/-/whatwg-encoding-2.0.0.tgz#e7635f597fd87020858626805a2729fa7698ac53" - integrity sha512-p41ogyeMUrw3jWclHWTQg1k05DSVXPLcVxRTYsXUk+ZooOCZLcoYgPZ/HL/D/N+uQPOtcp1me1WhBEaX02mhWg== +whatwg-encoding@^3.1.1: + version "3.1.1" + resolved "https://registry.yarnpkg.com/whatwg-encoding/-/whatwg-encoding-3.1.1.tgz#d0f4ef769905d426e1688f3e34381a99b60b76e5" + integrity sha512-6qN4hJdMwfYBtE3YBTTHhoeuUrDBPZmbQaxWAqSALV/MeEnR5z1xd8UKud2RAkFoPkmB+hli1TZSnyi84xz1vQ== dependencies: iconv-lite "0.6.3" -whatwg-mimetype@^3.0.0: - version "3.0.0" - resolved "https://registry.yarnpkg.com/whatwg-mimetype/-/whatwg-mimetype-3.0.0.tgz#5fa1a7623867ff1af6ca3dc72ad6b8a4208beba7" - integrity sha512-nt+N2dzIutVRxARx1nghPKGv1xHikU7HKdfafKkLNLindmPU/ch3U31NOCGGA/dmPcmb1VlofO0vnKAcsm0o/Q== +whatwg-mimetype@^4.0.0: + version "4.0.0" + resolved "https://registry.yarnpkg.com/whatwg-mimetype/-/whatwg-mimetype-4.0.0.tgz#bc1bf94a985dc50388d54a9258ac405c3ca2fc0a" + integrity sha512-QaKxh0eNIi2mE9p2vEdzfagOKHCcj1pJ56EEHGQOVxp8r9/iszLUUV7v89x9O1p/T+NlTM5W7jW6+cz4Fq1YVg== -whatwg-url@^12.0.0, whatwg-url@^12.0.1: - version "12.0.1" - resolved "https://registry.yarnpkg.com/whatwg-url/-/whatwg-url-12.0.1.tgz#fd7bcc71192e7c3a2a97b9a8d6b094853ed8773c" - integrity sha512-Ed/LrqB8EPlGxjS+TrsXcpUond1mhccS3pchLhzSgPCnTimUCKj3IZE75pAs5m6heB2U2TMerKFUXheyHY+VDQ== +whatwg-url@^14.0.0: + version "14.0.0" + resolved "https://registry.yarnpkg.com/whatwg-url/-/whatwg-url-14.0.0.tgz#00baaa7fd198744910c4b1ef68378f2200e4ceb6" + integrity sha512-1lfMEm2IEr7RIV+f4lUNPOqfFL+pO+Xw3fJSqmjX9AbXcXcYOkCe1P6+9VBZB6n94af16NfZf+sSk0JCBZC9aw== dependencies: - tr46 "^4.1.1" + tr46 "^5.0.0" webidl-conversions "^7.0.0" -whatwg-url@^5.0.0: - version "5.0.0" - resolved "https://registry.yarnpkg.com/whatwg-url/-/whatwg-url-5.0.0.tgz#966454e8765462e37644d3626f6742ce8b70965d" - integrity sha512-saE57nupxk6v3HY35+jzBwYa0rKSy0XR8JSxZPwgLr7ys0IBzhGviA1/TUGJLmSVqs8pb9AnvICXEuOHLprYTw== - dependencies: - tr46 "~0.0.3" - webidl-conversions "^3.0.0" - which-boxed-primitive@^1.0.2: version "1.0.2" resolved "https://registry.yarnpkg.com/which-boxed-primitive/-/which-boxed-primitive-1.0.2.tgz#13757bc89b209b049fe5d86430e21cf40a89a8e6" @@ -4623,7 +4380,7 @@ which-collection@^1.0.1: is-weakmap "^2.0.1" is-weakset "^2.0.1" -which-typed-array@^1.1.2, which-typed-array@^1.1.9: +which-typed-array@^1.1.9: version "1.1.9" resolved "https://registry.yarnpkg.com/which-typed-array/-/which-typed-array-1.1.9.tgz#307cf898025848cf995e795e8423c7f337efbde6" integrity sha512-w9c4xkx6mPidwp7180ckYWfMmvxpjlZuIudNtDf4N/tTAUB8VJbX25qZoAsrtGuYNnGw3pa0AXgbGKRB8/EceA== @@ -4657,10 +4414,23 @@ why-is-node-running@^2.2.2: siginfo "^2.0.0" stackback "0.0.2" -word-wrap@^1.2.3: - version "1.2.3" - resolved "https://registry.yarnpkg.com/word-wrap/-/word-wrap-1.2.3.tgz#610636f6b1f703891bd34771ccb17fb93b47079c" - integrity sha512-Hz/mrNwitNRh/HUAtM/VT/5VH+ygD6DV7mYKZAtHOrbs8U7lvPS6xf7EJKMF0uW1KJCl0H701g3ZGus+muE5vQ== +"wrap-ansi-cjs@npm:wrap-ansi@^7.0.0": + version "7.0.0" + resolved "https://registry.yarnpkg.com/wrap-ansi/-/wrap-ansi-7.0.0.tgz#67e145cff510a6a6984bdf1152911d69d2eb9e43" + integrity sha512-YVGIj2kamLSTxw6NsZjoBxfSwsn0ycdesmc4p+Q21c5zPuZ1pl+NfxVdxPtdHvmNVOQ6XSYG4AUtyt/Fi7D16Q== + dependencies: + ansi-styles "^4.0.0" + string-width "^4.1.0" + strip-ansi "^6.0.0" + +wrap-ansi@^6.2.0: + version "6.2.0" + resolved "https://registry.yarnpkg.com/wrap-ansi/-/wrap-ansi-6.2.0.tgz#e9393ba07102e6c91a3b221478f0257cd2856e53" + integrity sha512-r6lPcBGxZXlIcymEu7InxDMhdW0KDxpLgoFLcguasxCaJ/SOIZwINatK9KY/tf+ZrlywOKU0UDj3ATXUBfxJXA== + dependencies: + ansi-styles "^4.0.0" + string-width "^4.1.0" + strip-ansi "^6.0.0" wrap-ansi@^7.0.0: version "7.0.0" @@ -4671,21 +4441,35 @@ wrap-ansi@^7.0.0: string-width "^4.1.0" strip-ansi "^6.0.0" +wrap-ansi@^8.1.0: + version "8.1.0" + resolved "https://registry.yarnpkg.com/wrap-ansi/-/wrap-ansi-8.1.0.tgz#56dc22368ee570face1b49819975d9b9a5ead214" + integrity sha512-si7QWI6zUMq56bESFvagtmzMdGOtoxfR+Sez11Mobfc7tm+VkUckk9bW2UeffTGVUbOksxmSw0AA2gs8g71NCQ== + dependencies: + ansi-styles "^6.1.0" + string-width "^5.0.1" + strip-ansi "^7.0.1" + wrappy@1: version "1.0.2" resolved "https://registry.yarnpkg.com/wrappy/-/wrappy-1.0.2.tgz#b5243d8f3ec1aa35f1364605bc0d1036e30ab69f" integrity sha512-l4Sp/DRseor9wL6EvV2+TuQn63dMkPjZ/sp9XkghTEbV9KlPS1xUsZ3u7/IQO4wxtcFB4bgpQPRcR3QCvezPcQ== -ws@^8.13.0: - version "8.13.0" - resolved "https://registry.yarnpkg.com/ws/-/ws-8.13.0.tgz#9a9fb92f93cf41512a0735c8f4dd09b8a1211cd0" - integrity sha512-x9vcZYTrFPC7aSIbj7sRCYo7L/Xb8Iy+pW0ng0wt2vCJv7M9HOMy0UoN3rr+IFC7hb7vXoqS+P9ktyLLLhO+LA== +ws@^8.16.0: + version "8.16.0" + resolved "https://registry.yarnpkg.com/ws/-/ws-8.16.0.tgz#d1cd774f36fbc07165066a60e40323eab6446fd4" + integrity sha512-HS0c//TP7Ina87TfiPUz1rQzMhHrl/SG2guqRcTOIUYD2q8uhUdNHZYJUaQ8aTGPzCh+c6oawMKW35nFl1dxyQ== xml-name-validator@^4.0.0: version "4.0.0" resolved "https://registry.yarnpkg.com/xml-name-validator/-/xml-name-validator-4.0.0.tgz#79a006e2e63149a8600f15430f0a4725d1524835" integrity sha512-ICP2e+jsHvAj2E2lIHxa5tjXRlKDJo4IdvPvCXbXQGdzSfmSpNVyIKMvoZHjDY9DP0zV17iI85o90vRFXNccRw== +xml-name-validator@^5.0.0: + version "5.0.0" + resolved "https://registry.yarnpkg.com/xml-name-validator/-/xml-name-validator-5.0.0.tgz#82be9b957f7afdacf961e5980f1bf227c0bf7673" + integrity sha512-EvGK8EJ3DhaHfbRlETOWAS5pO9MZITeauHKJyb8wyajUfQUenkIg2MvLDTZ4T/TgIcm3HU0TFBgWWboAZ30UHg== + xmlchars@^2.2.0: version "2.2.0" resolved "https://registry.yarnpkg.com/xmlchars/-/xmlchars-2.2.0.tgz#060fe1bcb7f9c76fe2a17db86a9bc3ab894210cb" @@ -4696,11 +4480,6 @@ y18n@^5.0.5: resolved "https://registry.yarnpkg.com/y18n/-/y18n-5.0.8.tgz#7f4934d0f7ca8c56f95314939ddcd2dd91ce1d55" integrity sha512-0pfFzegeDWJHJIAmTLRP2DwHjdF5s7jo9tuztdQxAhINCdvS+3nGINqPd00AphqJR/0LhANUS6/+7SCb98YOfA== -yallist@^2.1.2: - version "2.1.2" - resolved "https://registry.yarnpkg.com/yallist/-/yallist-2.1.2.tgz#1c11f9218f076089a47dd512f93c6699a6a81d52" - integrity sha512-ncTzHV7NvsQZkYe1DW7cbDLm0YpzHmZF5r/iyP3ZnQtMiJ+pjzisCiMNI+Sj+xQF5pXhSHxSB3uDbsBTzY/c2A== - yallist@^4.0.0: version "4.0.0" resolved "https://registry.yarnpkg.com/yallist/-/yallist-4.0.0.tgz#9bb92790d9c0effec63be73519e11a35019a3a72" @@ -4724,7 +4503,7 @@ yargs@^17.0.1: y18n "^5.0.5" yargs-parser "^21.1.1" -yargs@^17.3.1: +yargs@^17.7.2: version "17.7.2" resolved "https://registry.yarnpkg.com/yargs/-/yargs-17.7.2.tgz#991df39aca675a192b816e1e0363f9d75d2aa269" integrity sha512-7dSzzRQ++CKnNI/krKnYRV7JKKPUXMEh61soaHKg9mrWEhzFWhFnxPxGl+69cD1Ou63C13NUPCnmIcrvqCuM6w== From 6dd3829f56db182c9c4c2a11cb6839b22f887b29 Mon Sep 17 00:00:00 2001 From: Maciej Moscicki Date: Tue, 30 Apr 2024 11:24:33 +0200 Subject: [PATCH 48/87] Add kafka producer metric for total record count (#1852) --- .../hermes/common/metric/ProducerMetrics.java | 15 ++ .../FailFastKafkaProducerProperties.java | 176 +++--------------- .../FailFastLocalKafkaProducerProperties.java | 161 ++++++++++++++++ ...FailFastRemoteKafkaProducerProperties.java | 161 ++++++++++++++++ .../config/FrontendProducerConfiguration.java | 5 +- .../producer/kafka/KafkaMessageSender.java | 43 +++-- .../kafka/KafkaMessageSendersFactory.java | 17 +- .../PrometheusMetricsAssertion.java | 4 + .../KafkaProducerMetricsTest.java | 47 +++++ 9 files changed, 448 insertions(+), 181 deletions(-) create mode 100644 hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FailFastLocalKafkaProducerProperties.java create mode 100644 hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FailFastRemoteKafkaProducerProperties.java create mode 100644 integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/KafkaProducerMetricsTest.java diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/ProducerMetrics.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/ProducerMetrics.java index a89c181da1..42f95521e6 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/ProducerMetrics.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/ProducerMetrics.java @@ -78,6 +78,14 @@ public double getBufferAvailableBytes() { + meterRegistry.get(ACK_LEADER_BUFFER_AVAILABLE_BYTES).gauge().value(); } + public void registerAckLeaderRecordSendCounter(T stateObj, ToDoubleFunction f, String sender, String datacenter) { + registerCounter(ACK_LEADER_RECORD_SEND_TOTAL, tags(sender, datacenter), stateObj, f); + } + + public void registerAckAllRecordSendCounter(T stateObj, ToDoubleFunction f, String sender, String datacenter) { + registerCounter(ACK_ALL_RECORD_SEND_TOTAL, tags(sender, datacenter), stateObj, f); + } + public void registerProducerInflightRequestGauge(T stateObj, ToDoubleFunction f) { meterRegistry.gauge(INFLIGHT_REQUESTS, stateObj, f); hermesMetrics.registerProducerInflightRequest(() -> (int) f.applyAsDouble(stateObj)); @@ -98,6 +106,10 @@ private void registerTimeGauge(T stateObj, meterRegistry.more().timeGauge(prometheusName, tags, stateObj, timeUnit, f); } + private void registerCounter(String name, Tags tags, T stateObj, ToDoubleFunction f) { + meterRegistry.more().counter(name, tags, stateObj, f); + } + private static final String KAFKA_PRODUCER = "kafka-producer."; private static final String ACK_LEADER = "ack-leader."; private static final String ACK_ALL = "ack-all."; @@ -108,6 +120,7 @@ private void registerTimeGauge(T stateObj, private static final String ACK_ALL_RECORD_QUEUE_TIME_MAX = KAFKA_PRODUCER + ACK_ALL + "record-queue-time-max"; private static final String ACK_ALL_COMPRESSION_RATE = KAFKA_PRODUCER + ACK_ALL + "compression-rate-avg"; private static final String ACK_ALL_FAILED_BATCHES_TOTAL = KAFKA_PRODUCER + ACK_ALL + "failed-batches-total"; + private static final String ACK_ALL_RECORD_SEND_TOTAL = KAFKA_PRODUCER + ACK_ALL + "record-send"; private static final String ACK_LEADER_FAILED_BATCHES_TOTAL = KAFKA_PRODUCER + ACK_LEADER + "failed-batches-total"; private static final String ACK_LEADER_BUFFER_TOTAL_BYTES = KAFKA_PRODUCER + ACK_LEADER + "buffer-total-bytes"; @@ -115,4 +128,6 @@ private void registerTimeGauge(T stateObj, private static final String ACK_LEADER_RECORD_QUEUE_TIME_MAX = KAFKA_PRODUCER + ACK_LEADER + "record-queue-time-max"; private static final String ACK_LEADER_BUFFER_AVAILABLE_BYTES = KAFKA_PRODUCER + ACK_LEADER + "buffer-available-bytes"; private static final String ACK_LEADER_COMPRESSION_RATE = KAFKA_PRODUCER + ACK_LEADER + "compression-rate-avg"; + private static final String ACK_LEADER_RECORD_SEND_TOTAL = KAFKA_PRODUCER + ACK_LEADER + "record-send"; + } diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FailFastKafkaProducerProperties.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FailFastKafkaProducerProperties.java index 5dde59c5bc..e65eb59c12 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FailFastKafkaProducerProperties.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FailFastKafkaProducerProperties.java @@ -6,7 +6,11 @@ import java.time.Duration; @ConfigurationProperties(prefix = "frontend.kafka.fail-fast-producer") -public class FailFastKafkaProducerProperties implements KafkaProducerParameters { +public class FailFastKafkaProducerProperties { + + private KafkaProducerParameters local = new FailFastLocalKafkaProducerProperties(); + + private KafkaProducerParameters remote = new FailFastRemoteKafkaProducerProperties(); private Duration speculativeSendDelay = Duration.ofMillis(250); @@ -14,151 +18,6 @@ public class FailFastKafkaProducerProperties implements KafkaProducerParameters private ChaosSchedulerProperties chaosScheduler = new ChaosSchedulerProperties(); - private Duration maxBlock = Duration.ofMillis(500); - - private Duration metadataMaxAge = Duration.ofMinutes(5); - - private String compressionCodec = "none"; - - private int retries = Integer.MAX_VALUE; - - private Duration retryBackoff = Duration.ofMillis(50); - - private Duration requestTimeout = Duration.ofMillis(500); - - private Duration deliveryTimeout = Duration.ofMillis(500); - - private int batchSize = 16 * 1024; - - private int tcpSendBuffer = 128 * 1024; - - private int maxRequestSize = 1024 * 1024; - - private Duration linger = Duration.ofMillis(0); - - private Duration metricsSampleWindow = Duration.ofSeconds(30); - - private int maxInflightRequestsPerConnection = 5; - - private boolean reportNodeMetricsEnabled = false; - - @Override - public Duration getMaxBlock() { - return maxBlock; - } - - public void setMaxBlock(Duration maxBlock) { - this.maxBlock = maxBlock; - } - - @Override - public Duration getMetadataMaxAge() { - return metadataMaxAge; - } - - public void setMetadataMaxAge(Duration metadataMaxAge) { - this.metadataMaxAge = metadataMaxAge; - } - - @Override - public String getCompressionCodec() { - return compressionCodec; - } - - public void setCompressionCodec(String compressionCodec) { - this.compressionCodec = compressionCodec; - } - - @Override - public int getRetries() { - return retries; - } - - public void setRetries(int retries) { - this.retries = retries; - } - - @Override - public Duration getRetryBackoff() { - return retryBackoff; - } - - public void setRetryBackoff(Duration retryBackoff) { - this.retryBackoff = retryBackoff; - } - - @Override - public Duration getRequestTimeout() { - return requestTimeout; - } - - public void setRequestTimeout(Duration requestTimeout) { - this.requestTimeout = requestTimeout; - } - - @Override - public int getBatchSize() { - return batchSize; - } - - public void setBatchSize(int batchSize) { - this.batchSize = batchSize; - } - - @Override - public int getTcpSendBuffer() { - return tcpSendBuffer; - } - - public void setTcpSendBuffer(int tcpSendBuffer) { - this.tcpSendBuffer = tcpSendBuffer; - } - - @Override - public int getMaxRequestSize() { - return maxRequestSize; - } - - public void setMaxRequestSize(int maxRequestSize) { - this.maxRequestSize = maxRequestSize; - } - - @Override - public Duration getLinger() { - return linger; - } - - public void setLinger(Duration linger) { - this.linger = linger; - } - - @Override - public Duration getMetricsSampleWindow() { - return metricsSampleWindow; - } - - public void setMetricsSampleWindow(Duration metricsSampleWindow) { - this.metricsSampleWindow = metricsSampleWindow; - } - - @Override - public int getMaxInflightRequestsPerConnection() { - return maxInflightRequestsPerConnection; - } - - public void setMaxInflightRequestsPerConnection(int maxInflightRequestsPerConnection) { - this.maxInflightRequestsPerConnection = maxInflightRequestsPerConnection; - } - - @Override - public boolean isReportNodeMetricsEnabled() { - return reportNodeMetricsEnabled; - } - - public void setReportNodeMetricsEnabled(boolean reportNodeMetricsEnabled) { - this.reportNodeMetricsEnabled = reportNodeMetricsEnabled; - } - public Duration getSpeculativeSendDelay() { return speculativeSendDelay; } @@ -167,15 +26,6 @@ public void setSpeculativeSendDelay(Duration speculativeSendDelay) { this.speculativeSendDelay = speculativeSendDelay; } - @Override - public Duration getDeliveryTimeout() { - return deliveryTimeout; - } - - public void setDeliveryTimeout(Duration deliveryTimeout) { - this.deliveryTimeout = deliveryTimeout; - } - public FallbackSchedulerProperties getFallbackScheduler() { return fallbackScheduler; } @@ -184,6 +34,22 @@ public void setFallbackScheduler(FallbackSchedulerProperties fallbackScheduler) this.fallbackScheduler = fallbackScheduler; } + public KafkaProducerParameters getLocal() { + return local; + } + + public void setLocal(KafkaProducerParameters local) { + this.local = local; + } + + public KafkaProducerParameters getRemote() { + return remote; + } + + public void setRemote(KafkaProducerParameters remote) { + this.remote = remote; + } + public ChaosSchedulerProperties getChaosScheduler() { return chaosScheduler; } diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FailFastLocalKafkaProducerProperties.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FailFastLocalKafkaProducerProperties.java new file mode 100644 index 0000000000..d54456e0d7 --- /dev/null +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FailFastLocalKafkaProducerProperties.java @@ -0,0 +1,161 @@ +package pl.allegro.tech.hermes.frontend.config; + +import pl.allegro.tech.hermes.frontend.producer.kafka.KafkaProducerParameters; + +import java.time.Duration; + +public class FailFastLocalKafkaProducerProperties implements KafkaProducerParameters { + private Duration maxBlock = Duration.ofMillis(500); + + private Duration metadataMaxAge = Duration.ofMinutes(5); + + private String compressionCodec = "none"; + + private int retries = Integer.MAX_VALUE; + + private Duration retryBackoff = Duration.ofMillis(50); + + private Duration requestTimeout = Duration.ofMillis(500); + + private Duration deliveryTimeout = Duration.ofMillis(500); + + private int batchSize = 16 * 1024; + + private int tcpSendBuffer = 128 * 1024; + + private int maxRequestSize = 1024 * 1024; + + private Duration linger = Duration.ofMillis(0); + + private Duration metricsSampleWindow = Duration.ofSeconds(30); + + private int maxInflightRequestsPerConnection = 5; + + private boolean reportNodeMetricsEnabled = false; + + @Override + public Duration getMaxBlock() { + return maxBlock; + } + + public void setMaxBlock(Duration maxBlock) { + this.maxBlock = maxBlock; + } + + @Override + public Duration getMetadataMaxAge() { + return metadataMaxAge; + } + + public void setMetadataMaxAge(Duration metadataMaxAge) { + this.metadataMaxAge = metadataMaxAge; + } + + @Override + public String getCompressionCodec() { + return compressionCodec; + } + + public void setCompressionCodec(String compressionCodec) { + this.compressionCodec = compressionCodec; + } + + @Override + public int getRetries() { + return retries; + } + + public void setRetries(int retries) { + this.retries = retries; + } + + @Override + public Duration getRetryBackoff() { + return retryBackoff; + } + + public void setRetryBackoff(Duration retryBackoff) { + this.retryBackoff = retryBackoff; + } + + @Override + public Duration getRequestTimeout() { + return requestTimeout; + } + + public void setRequestTimeout(Duration requestTimeout) { + this.requestTimeout = requestTimeout; + } + + @Override + public int getBatchSize() { + return batchSize; + } + + public void setBatchSize(int batchSize) { + this.batchSize = batchSize; + } + + @Override + public int getTcpSendBuffer() { + return tcpSendBuffer; + } + + public void setTcpSendBuffer(int tcpSendBuffer) { + this.tcpSendBuffer = tcpSendBuffer; + } + + @Override + public int getMaxRequestSize() { + return maxRequestSize; + } + + public void setMaxRequestSize(int maxRequestSize) { + this.maxRequestSize = maxRequestSize; + } + + @Override + public Duration getLinger() { + return linger; + } + + public void setLinger(Duration linger) { + this.linger = linger; + } + + @Override + public Duration getMetricsSampleWindow() { + return metricsSampleWindow; + } + + public void setMetricsSampleWindow(Duration metricsSampleWindow) { + this.metricsSampleWindow = metricsSampleWindow; + } + + @Override + public int getMaxInflightRequestsPerConnection() { + return maxInflightRequestsPerConnection; + } + + public void setMaxInflightRequestsPerConnection(int maxInflightRequestsPerConnection) { + this.maxInflightRequestsPerConnection = maxInflightRequestsPerConnection; + } + + @Override + public boolean isReportNodeMetricsEnabled() { + return reportNodeMetricsEnabled; + } + + public void setReportNodeMetricsEnabled(boolean reportNodeMetricsEnabled) { + this.reportNodeMetricsEnabled = reportNodeMetricsEnabled; + } + + @Override + public Duration getDeliveryTimeout() { + return deliveryTimeout; + } + + public void setDeliveryTimeout(Duration deliveryTimeout) { + this.deliveryTimeout = deliveryTimeout; + } +} diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FailFastRemoteKafkaProducerProperties.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FailFastRemoteKafkaProducerProperties.java new file mode 100644 index 0000000000..a35722f5d4 --- /dev/null +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FailFastRemoteKafkaProducerProperties.java @@ -0,0 +1,161 @@ +package pl.allegro.tech.hermes.frontend.config; + +import pl.allegro.tech.hermes.frontend.producer.kafka.KafkaProducerParameters; + +import java.time.Duration; + +public class FailFastRemoteKafkaProducerProperties implements KafkaProducerParameters { + private Duration maxBlock = Duration.ofMillis(250); + + private Duration metadataMaxAge = Duration.ofMinutes(5); + + private String compressionCodec = "none"; + + private int retries = Integer.MAX_VALUE; + + private Duration retryBackoff = Duration.ofMillis(50); + + private Duration requestTimeout = Duration.ofMillis(250); + + private Duration deliveryTimeout = Duration.ofMillis(250); + + private int batchSize = 16 * 1024; + + private int tcpSendBuffer = 128 * 1024; + + private int maxRequestSize = 1024 * 1024; + + private Duration linger = Duration.ofMillis(0); + + private Duration metricsSampleWindow = Duration.ofSeconds(30); + + private int maxInflightRequestsPerConnection = 5; + + private boolean reportNodeMetricsEnabled = false; + + @Override + public Duration getMaxBlock() { + return maxBlock; + } + + public void setMaxBlock(Duration maxBlock) { + this.maxBlock = maxBlock; + } + + @Override + public Duration getMetadataMaxAge() { + return metadataMaxAge; + } + + public void setMetadataMaxAge(Duration metadataMaxAge) { + this.metadataMaxAge = metadataMaxAge; + } + + @Override + public String getCompressionCodec() { + return compressionCodec; + } + + public void setCompressionCodec(String compressionCodec) { + this.compressionCodec = compressionCodec; + } + + @Override + public int getRetries() { + return retries; + } + + public void setRetries(int retries) { + this.retries = retries; + } + + @Override + public Duration getRetryBackoff() { + return retryBackoff; + } + + public void setRetryBackoff(Duration retryBackoff) { + this.retryBackoff = retryBackoff; + } + + @Override + public Duration getRequestTimeout() { + return requestTimeout; + } + + public void setRequestTimeout(Duration requestTimeout) { + this.requestTimeout = requestTimeout; + } + + @Override + public int getBatchSize() { + return batchSize; + } + + public void setBatchSize(int batchSize) { + this.batchSize = batchSize; + } + + @Override + public int getTcpSendBuffer() { + return tcpSendBuffer; + } + + public void setTcpSendBuffer(int tcpSendBuffer) { + this.tcpSendBuffer = tcpSendBuffer; + } + + @Override + public int getMaxRequestSize() { + return maxRequestSize; + } + + public void setMaxRequestSize(int maxRequestSize) { + this.maxRequestSize = maxRequestSize; + } + + @Override + public Duration getLinger() { + return linger; + } + + public void setLinger(Duration linger) { + this.linger = linger; + } + + @Override + public Duration getMetricsSampleWindow() { + return metricsSampleWindow; + } + + public void setMetricsSampleWindow(Duration metricsSampleWindow) { + this.metricsSampleWindow = metricsSampleWindow; + } + + @Override + public int getMaxInflightRequestsPerConnection() { + return maxInflightRequestsPerConnection; + } + + public void setMaxInflightRequestsPerConnection(int maxInflightRequestsPerConnection) { + this.maxInflightRequestsPerConnection = maxInflightRequestsPerConnection; + } + + @Override + public boolean isReportNodeMetricsEnabled() { + return reportNodeMetricsEnabled; + } + + public void setReportNodeMetricsEnabled(boolean reportNodeMetricsEnabled) { + this.reportNodeMetricsEnabled = reportNodeMetricsEnabled; + } + + @Override + public Duration getDeliveryTimeout() { + return deliveryTimeout; + } + + public void setDeliveryTimeout(Duration deliveryTimeout) { + this.deliveryTimeout = deliveryTimeout; + } +} diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FrontendProducerConfiguration.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FrontendProducerConfiguration.java index 44924c138f..f1265cd44e 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FrontendProducerConfiguration.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FrontendProducerConfiguration.java @@ -105,7 +105,10 @@ public KafkaMessageSenders kafkaMessageSenders(KafkaProducerProperties kafkaProd @Bean(destroyMethod = "close") public KafkaMessageSenders failFastKafkaMessageSenders(FailFastKafkaProducerProperties kafkaProducerProperties, KafkaMessageSendersFactory kafkaMessageSendersFactory) { - return kafkaMessageSendersFactory.provide(kafkaProducerProperties, "failFast"); + return kafkaMessageSendersFactory.provide( + kafkaProducerProperties.getLocal(), + kafkaProducerProperties.getRemote(), + "failFast"); } @Bean(destroyMethod = "close") diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSender.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSender.java index c88bd5d94d..934b7ef66d 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSender.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSender.java @@ -111,27 +111,30 @@ public void onCompletion(RecordMetadata metadata, Exception exception) { } public void registerGauges(Topic.Ack ack, String sender) { - MetricName bufferTotalBytes = producerMetric("buffer-total-bytes", "producer-metrics", "buffer total bytes"); - MetricName bufferAvailableBytes = producerMetric("buffer-available-bytes", "producer-metrics", "buffer available bytes"); - MetricName compressionRate = producerMetric("compression-rate-avg", "producer-metrics", "average compression rate"); - MetricName failedBatches = producerMetric("record-error-total", "producer-metrics", "failed publishing batches"); - MetricName metadataAge = producerMetric("metadata-age", "producer-metrics", "age [s] of metadata"); - MetricName queueTimeMax = producerMetric("record-queue-time-max", "producer-metrics", "maximum time [ms] that batch spent in the send buffer"); + MetricName bufferTotalBytes = producerMetricMame("buffer-total-bytes", "producer-metrics", "buffer total bytes"); + MetricName bufferAvailableBytes = producerMetricMame("buffer-available-bytes", "producer-metrics", "buffer available bytes"); + MetricName compressionRate = producerMetricMame("compression-rate-avg", "producer-metrics", "average compression rate"); + MetricName failedBatches = producerMetricMame("record-error-total", "producer-metrics", "failed publishing batches"); + MetricName metadataAge = producerMetricMame("metadata-age", "producer-metrics", "age [s] of metadata"); + MetricName queueTimeMax = producerMetricMame("record-queue-time-max", "producer-metrics", "maximum time [ms] that batch spent in the send buffer"); + MetricName recordSendTotal = producerMetricMame("record-send-total", "producer-metrics", "total number of records sent - including retries"); if (ack == Topic.Ack.ALL) { - metricsFacade.producer().registerAckAllTotalBytesGauge(producer, producerGauge(bufferTotalBytes), sender, datacenter); - metricsFacade.producer().registerAckAllAvailableBytesGauge(producer, producerGauge(bufferAvailableBytes), sender, datacenter); - metricsFacade.producer().registerAckAllCompressionRateGauge(producer, producerGauge(compressionRate), sender, datacenter); - metricsFacade.producer().registerAckAllFailedBatchesGauge(producer, producerGauge(failedBatches), sender, datacenter); - metricsFacade.producer().registerAckAllMetadataAgeGauge(producer, producerGauge(metadataAge), sender, datacenter); - metricsFacade.producer().registerAckAllRecordQueueTimeMaxGauge(producer, producerGauge(queueTimeMax), sender, datacenter); + metricsFacade.producer().registerAckAllTotalBytesGauge(producer, producerMetric(bufferTotalBytes), sender, datacenter); + metricsFacade.producer().registerAckAllAvailableBytesGauge(producer, producerMetric(bufferAvailableBytes), sender, datacenter); + metricsFacade.producer().registerAckAllCompressionRateGauge(producer, producerMetric(compressionRate), sender, datacenter); + metricsFacade.producer().registerAckAllFailedBatchesGauge(producer, producerMetric(failedBatches), sender, datacenter); + metricsFacade.producer().registerAckAllMetadataAgeGauge(producer, producerMetric(metadataAge), sender, datacenter); + metricsFacade.producer().registerAckAllRecordQueueTimeMaxGauge(producer, producerMetric(queueTimeMax), sender, datacenter); + metricsFacade.producer().registerAckAllRecordSendCounter(producer, producerMetric(recordSendTotal), sender, datacenter); } else if (ack == Topic.Ack.LEADER) { - metricsFacade.producer().registerAckLeaderTotalBytesGauge(producer, producerGauge(bufferTotalBytes), sender, datacenter); - metricsFacade.producer().registerAckLeaderAvailableBytesGauge(producer, producerGauge(bufferAvailableBytes), sender, datacenter); - metricsFacade.producer().registerAckLeaderCompressionRateGauge(producer, producerGauge(compressionRate), sender, datacenter); - metricsFacade.producer().registerAckLeaderFailedBatchesGauge(producer, producerGauge(failedBatches), sender, datacenter); - metricsFacade.producer().registerAckLeaderMetadataAgeGauge(producer, producerGauge(metadataAge), sender, datacenter); - metricsFacade.producer().registerAckLeaderRecordQueueTimeMaxGauge(producer, producerGauge(queueTimeMax), sender, datacenter); + metricsFacade.producer().registerAckLeaderTotalBytesGauge(producer, producerMetric(bufferTotalBytes), sender, datacenter); + metricsFacade.producer().registerAckLeaderAvailableBytesGauge(producer, producerMetric(bufferAvailableBytes), sender, datacenter); + metricsFacade.producer().registerAckLeaderCompressionRateGauge(producer, producerMetric(compressionRate), sender, datacenter); + metricsFacade.producer().registerAckLeaderFailedBatchesGauge(producer, producerMetric(failedBatches), sender, datacenter); + metricsFacade.producer().registerAckLeaderMetadataAgeGauge(producer, producerMetric(metadataAge), sender, datacenter); + metricsFacade.producer().registerAckLeaderRecordQueueTimeMaxGauge(producer, producerMetric(queueTimeMax), sender, datacenter); + metricsFacade.producer().registerAckLeaderRecordSendCounter(producer, producerMetric(recordSendTotal), sender, datacenter); } } @@ -143,13 +146,13 @@ private double findProducerMetric(Producer producer, return value < 0 ? 0.0 : value; } - private ToDoubleFunction> producerGauge(MetricName producerMetricName) { + private ToDoubleFunction> producerMetric(MetricName producerMetricName) { Predicate> predicate = entry -> entry.getKey().group().equals(producerMetricName.group()) && entry.getKey().name().equals(producerMetricName.name()); return producer -> findProducerMetric(producer, predicate); } - private static MetricName producerMetric(String name, String group, String description) { + private static MetricName producerMetricMame(String name, String group, String description) { return new MetricName(name, group, description, Collections.emptyMap()); } } diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSendersFactory.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSendersFactory.java index d4d8583439..7636d4ecb5 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSendersFactory.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSendersFactory.java @@ -66,16 +66,23 @@ public KafkaMessageSendersFactory(KafkaParameters kafkaParameters, this.brokerLatencyReporter = brokerLatencyReporter; } + public KafkaMessageSenders provide(KafkaProducerParameters kafkaProducerParameters, String senderName) { - KafkaMessageSenders.Tuple localProducers = new KafkaMessageSenders.Tuple( - sender(kafkaParameters, kafkaProducerParameters, ACK_LEADER), - sender(kafkaParameters, kafkaProducerParameters, ACK_ALL) + return provide(kafkaProducerParameters, kafkaProducerParameters, senderName); + } + + public KafkaMessageSenders provide(KafkaProducerParameters localKafkaProducerParameters, + KafkaProducerParameters remoteKafkaProducerParameters, + String senderName) { + KafkaMessageSenders.Tuple localProducers = new KafkaMessageSenders.Tuple( + sender(kafkaParameters, localKafkaProducerParameters, ACK_LEADER), + sender(kafkaParameters, localKafkaProducerParameters, ACK_ALL) ); List remoteProducers = remoteKafkaParameters.stream().map( kafkaProperties -> new KafkaMessageSenders.Tuple( - sender(kafkaProperties, kafkaProducerParameters, ACK_LEADER), - sender(kafkaProperties, kafkaProducerParameters, ACK_ALL))).toList(); + sender(kafkaProperties, remoteKafkaProducerParameters, ACK_LEADER), + sender(kafkaProperties, remoteKafkaProducerParameters, ACK_ALL))).toList(); KafkaMessageSenders senders = new KafkaMessageSenders( topicMetadataLoadingExecutor, localMinInSyncReplicasLoader, diff --git a/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/assertions/PrometheusMetricsAssertion.java b/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/assertions/PrometheusMetricsAssertion.java index 8bfc48bd49..5e705533dc 100644 --- a/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/assertions/PrometheusMetricsAssertion.java +++ b/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/assertions/PrometheusMetricsAssertion.java @@ -96,6 +96,10 @@ public void withValue(double expectedValue) { assertThat(actualValue).isEqualTo(expectedValue); } + public double withInitialValue() { + return extractValue(); + } + public void withValueGreaterThan(double expectedValue) { double actualValue = extractValue(); assertThat(actualValue).isGreaterThan(expectedValue); diff --git a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/KafkaProducerMetricsTest.java b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/KafkaProducerMetricsTest.java new file mode 100644 index 0000000000..5f3a98a155 --- /dev/null +++ b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/KafkaProducerMetricsTest.java @@ -0,0 +1,47 @@ +package pl.allegro.tech.hermes.integrationtests; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import pl.allegro.tech.hermes.api.Topic; +import pl.allegro.tech.hermes.integrationtests.assertions.PrometheusMetricsAssertion; +import pl.allegro.tech.hermes.integrationtests.setup.HermesExtension; + +import java.util.concurrent.TimeUnit; + +import static com.jayway.awaitility.Awaitility.await; +import static pl.allegro.tech.hermes.integrationtests.assertions.HermesAssertions.assertThatMetrics; +import static pl.allegro.tech.hermes.test.helper.builder.TopicBuilder.topicWithRandomName; + +public class KafkaProducerMetricsTest { + @RegisterExtension + public static final HermesExtension hermes = new HermesExtension(); + + @Test + public void shouldRegisterSendMetrics() { + // given + Topic topic = hermes.initHelper().createTopic(topicWithRandomName().build()); + + double initialMetricValue = assertMetricsContainTotalSendMetric().withInitialValue(); + + // when + hermes.api().publish(topic.getQualifiedName(), "hello world"); + hermes.api().publish(topic.getQualifiedName(), "hello world"); + + // then + await().atMost(10, TimeUnit.SECONDS).until(() -> + assertMetricsContainTotalSendMetric().withValue(initialMetricValue + 2.0)); + } + + PrometheusMetricsAssertion.PrometheusMetricAssertion assertMetricsContainTotalSendMetric() { + return assertThatMetrics(hermes.api() + .getFrontendMetrics().expectStatus().isOk() + .expectBody(String.class).returnResult().getResponseBody()) + .contains("hermes_frontend_kafka_producer_ack_leader_record_send_total") + .withLabels( + "storageDc", "dc", + "sender", "default" + ); + } + + +} From 67255cc43d6665edf496f308cf1f69823de3592a Mon Sep 17 00:00:00 2001 From: Maciej Moscicki Date: Tue, 30 Apr 2024 14:39:30 +0200 Subject: [PATCH 49/87] provide more visibility into offline retransmission task creation (#1855) --- .../management/api/OfflineRetransmissionEndpoint.java | 8 ++++---- .../domain/retransmit/OfflineRetransmissionService.java | 7 ++++--- 2 files changed, 8 insertions(+), 7 deletions(-) diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/api/OfflineRetransmissionEndpoint.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/api/OfflineRetransmissionEndpoint.java index 2349656f48..23662d469d 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/api/OfflineRetransmissionEndpoint.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/api/OfflineRetransmissionEndpoint.java @@ -49,8 +49,8 @@ public OfflineRetransmissionEndpoint(OfflineRetransmissionService retransmission public Response createRetransmissionTask(@Valid OfflineRetransmissionRequest request, @Context ContainerRequestContext requestContext) { retransmissionService.validateRequest(request); permissions.ensurePermissionsToBothTopics(request, requestContext); - retransmissionService.createTask(request); - auditor.auditRetransmissionCreation(request, requestContext); + OfflineRetransmissionTask task = retransmissionService.createTask(request); + auditor.auditRetransmissionCreation(request, requestContext, task); return Response.status(Response.Status.CREATED).build(); } @@ -91,9 +91,9 @@ private void ensurePermissionsToBothTopics(OfflineRetransmissionRequest request, private static class OfflineRetransmissionAuditor { private static final Logger logger = LoggerFactory.getLogger(OfflineRetransmissionAuditor.class); - public void auditRetransmissionCreation(OfflineRetransmissionRequest request, ContainerRequestContext requestContext) { + public void auditRetransmissionCreation(OfflineRetransmissionRequest request, ContainerRequestContext requestContext, OfflineRetransmissionTask task) { String username = extractUsername(requestContext); - logger.info("User {} created retransmission task: {}", username, request); + logger.info("User {} created offline retransmission task: {}, taskId: {}", username, request, task.getTaskId()); } private String extractUsername(ContainerRequestContext requestContext) { diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/retransmit/OfflineRetransmissionService.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/retransmit/OfflineRetransmissionService.java index b8c49e281a..6d5633e25e 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/retransmit/OfflineRetransmissionService.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/retransmit/OfflineRetransmissionService.java @@ -29,8 +29,8 @@ public void validateRequest(OfflineRetransmissionRequest request) { ensureTopicIsNotStoredOffline(targetTopicName); } - public void createTask(OfflineRetransmissionRequest request) { - saveTask(request); + public OfflineRetransmissionTask createTask(OfflineRetransmissionRequest request) { + return saveTask(request); } public List getAllTasks() { @@ -73,9 +73,10 @@ private void ensureTopicIsNotStoredOffline(TopicName targetTopicName) { } } - private void saveTask(OfflineRetransmissionRequest request) { + private OfflineRetransmissionTask saveTask(OfflineRetransmissionRequest request) { OfflineRetransmissionTask task = new OfflineRetransmissionTask(UUID.randomUUID().toString(), request, Instant.now()); offlineRetransmissionRepository.saveTask(task); + return task; } } From 0a6cfd7a39ba7a5be9063f53fdf02daf8b149824 Mon Sep 17 00:00:00 2001 From: Maciej Moscicki Date: Mon, 6 May 2024 12:06:39 +0200 Subject: [PATCH 50/87] bump kafka clients to version 3.6.2 (#1853) * bump kafka clients to 3.x * remove test that depends on partitioner internals * fix test that depends on producer internals --- build.gradle | 2 +- .../FailFastLocalKafkaProducerProperties.java | 10 +++++ ...FailFastRemoteKafkaProducerProperties.java | 11 +++++ .../config/KafkaProducerProperties.java | 10 +++++ .../producer/kafka/KafkaMessageSender.java | 8 +++- .../kafka/KafkaMessageSendersFactory.java | 2 + .../kafka/KafkaProducerParameters.java | 2 + ...enterMessageProducerIntegrationTest.groovy | 42 ++----------------- .../test/helper/containers/ImageTags.java | 2 +- .../KafkaRetransmissionServiceTest.java | 8 ++-- 10 files changed, 52 insertions(+), 45 deletions(-) diff --git a/build.gradle b/build.gradle index 55d670209e..0eec952889 100644 --- a/build.gradle +++ b/build.gradle @@ -48,7 +48,7 @@ allprojects { targetCompatibility = JavaVersion.VERSION_17 project.ext.versions = [ - kafka : '2.8.2', + kafka : '3.6.2', guava : '23.0', jackson : '2.15.2', jersey : '3.1.2', diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FailFastLocalKafkaProducerProperties.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FailFastLocalKafkaProducerProperties.java index d54456e0d7..7d5b24ed02 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FailFastLocalKafkaProducerProperties.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FailFastLocalKafkaProducerProperties.java @@ -33,6 +33,8 @@ public class FailFastLocalKafkaProducerProperties implements KafkaProducerParame private boolean reportNodeMetricsEnabled = false; + private boolean idempotenceEnabled = false; + @Override public Duration getMaxBlock() { return maxBlock; @@ -158,4 +160,12 @@ public Duration getDeliveryTimeout() { public void setDeliveryTimeout(Duration deliveryTimeout) { this.deliveryTimeout = deliveryTimeout; } + + public boolean isIdempotenceEnabled() { + return idempotenceEnabled; + } + + public void setIdempotenceEnabled(boolean idempotenceEnabled) { + this.idempotenceEnabled = idempotenceEnabled; + } } diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FailFastRemoteKafkaProducerProperties.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FailFastRemoteKafkaProducerProperties.java index a35722f5d4..1265f343bb 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FailFastRemoteKafkaProducerProperties.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FailFastRemoteKafkaProducerProperties.java @@ -33,6 +33,8 @@ public class FailFastRemoteKafkaProducerProperties implements KafkaProducerParam private boolean reportNodeMetricsEnabled = false; + private boolean idempotenceEnabled = false; + @Override public Duration getMaxBlock() { return maxBlock; @@ -158,4 +160,13 @@ public Duration getDeliveryTimeout() { public void setDeliveryTimeout(Duration deliveryTimeout) { this.deliveryTimeout = deliveryTimeout; } + + @Override + public boolean isIdempotenceEnabled() { + return idempotenceEnabled; + } + + public void setIdempotenceEnabled(boolean idempotenceEnabled) { + this.idempotenceEnabled = idempotenceEnabled; + } } diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/KafkaProducerProperties.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/KafkaProducerProperties.java index 8bdb084e47..34e97143e3 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/KafkaProducerProperties.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/KafkaProducerProperties.java @@ -36,6 +36,8 @@ public class KafkaProducerProperties implements KafkaProducerParameters { private boolean reportNodeMetricsEnabled = false; + private boolean idempotenceEnabled = false; + @Override public Duration getMaxBlock() { return maxBlock; @@ -161,4 +163,12 @@ public Duration getDeliveryTimeout() { public void setDeliveryTimeout(Duration deliveryTimeout) { this.deliveryTimeout = deliveryTimeout; } + + public boolean isIdempotenceEnabled() { + return idempotenceEnabled; + } + + public void setIdempotenceEnabled(boolean idempotenceEnabled) { + this.idempotenceEnabled = idempotenceEnabled; + } } diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSender.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSender.java index 934b7ef66d..b9c53dc048 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSender.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSender.java @@ -142,8 +142,12 @@ private double findProducerMetric(Producer producer, Predicate> predicate) { Optional> first = producer.metrics().entrySet().stream().filter(predicate).findFirst(); - double value = first.map(metricNameEntry -> metricNameEntry.getValue().value()).orElse(0.0); - return value < 0 ? 0.0 : value; + Object value = first.map(metricNameEntry -> metricNameEntry.getValue().metricValue()).orElse(0.0d); + if (value instanceof Number) { + return ((Number) value).doubleValue(); + } else { + return 0.0; + } } private ToDoubleFunction> producerMetric(MetricName producerMetricName) { diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSendersFactory.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSendersFactory.java index 7636d4ecb5..b03b8bc387 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSendersFactory.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSendersFactory.java @@ -18,6 +18,7 @@ import static org.apache.kafka.clients.producer.ProducerConfig.BUFFER_MEMORY_CONFIG; import static org.apache.kafka.clients.producer.ProducerConfig.COMPRESSION_TYPE_CONFIG; import static org.apache.kafka.clients.producer.ProducerConfig.DELIVERY_TIMEOUT_MS_CONFIG; +import static org.apache.kafka.clients.producer.ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG; import static org.apache.kafka.clients.producer.ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG; import static org.apache.kafka.clients.producer.ProducerConfig.LINGER_MS_CONFIG; import static org.apache.kafka.clients.producer.ProducerConfig.MAX_BLOCK_MS_CONFIG; @@ -114,6 +115,7 @@ private KafkaMessageSender sender(KafkaParameters kafkaParameter props.put(LINGER_MS_CONFIG, (int) kafkaProducerParameters.getLinger().toMillis()); props.put(METRICS_SAMPLE_WINDOW_MS_CONFIG, (int) kafkaProducerParameters.getMetricsSampleWindow().toMillis()); props.put(MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, kafkaProducerParameters.getMaxInflightRequestsPerConnection()); + props.put(ENABLE_IDEMPOTENCE_CONFIG, kafkaProducerParameters.isIdempotenceEnabled()); props.put(ACKS_CONFIG, acks); if (kafkaParameters.isAuthenticationEnabled()) { diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaProducerParameters.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaProducerParameters.java index 79454b522f..c0caa3913c 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaProducerParameters.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaProducerParameters.java @@ -31,4 +31,6 @@ public interface KafkaProducerParameters { int getMaxInflightRequestsPerConnection(); boolean isReportNodeMetricsEnabled(); + + boolean isIdempotenceEnabled(); } diff --git a/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducerIntegrationTest.groovy b/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducerIntegrationTest.groovy index f7b9d91c1b..edbab24478 100644 --- a/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducerIntegrationTest.groovy +++ b/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducerIntegrationTest.groovy @@ -13,11 +13,7 @@ import org.apache.kafka.common.TopicPartition import org.testcontainers.containers.KafkaContainer import org.testcontainers.containers.wait.strategy.Wait import org.testcontainers.spock.Testcontainers -import pl.allegro.tech.hermes.api.ContentType -import pl.allegro.tech.hermes.api.DeliveryType -import pl.allegro.tech.hermes.api.Subscription -import pl.allegro.tech.hermes.api.SubscriptionMode -import pl.allegro.tech.hermes.api.Topic +import pl.allegro.tech.hermes.api.* import pl.allegro.tech.hermes.common.kafka.ConsumerGroupId import pl.allegro.tech.hermes.common.kafka.JsonToAvroMigrationKafkaNamesMapper import pl.allegro.tech.hermes.common.kafka.KafkaNamesMapper @@ -33,6 +29,7 @@ import pl.allegro.tech.hermes.frontend.server.CachedTopicsTestHelper import pl.allegro.tech.hermes.metrics.PathsCompiler import pl.allegro.tech.hermes.test.helper.avro.AvroUser import pl.allegro.tech.hermes.test.helper.builder.TopicBuilder +import pl.allegro.tech.hermes.test.helper.containers.ImageTags import spock.lang.Shared import spock.lang.Specification @@ -42,10 +39,7 @@ import java.util.stream.Collectors import static java.util.Collections.emptyList import static java.util.Collections.emptyMap import static java.util.concurrent.TimeUnit.MILLISECONDS -import static org.apache.kafka.clients.consumer.ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG -import static org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_ID_CONFIG -import static org.apache.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG -import static org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG +import static org.apache.kafka.clients.consumer.ConsumerConfig.* import static org.apache.kafka.clients.producer.ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG import static org.apache.kafka.clients.producer.ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG @@ -58,7 +52,7 @@ class LocalDatacenterMessageProducerIntegrationTest extends Specification { BrokerLatencyReporter brokerLatencyReporter = new BrokerLatencyReporter(false, null, null, null) @Shared - KafkaContainer kafkaContainer = new KafkaContainer() + KafkaContainer kafkaContainer = new KafkaContainer(ImageTags.confluentImagesTag()) @Shared KafkaProducer leaderConfirms @@ -161,34 +155,6 @@ class LocalDatacenterMessageProducerIntegrationTest extends Specification { partitionsWithMessagesData.get(0).offset() == 10 } - def "should publish messages with random distribiution when pratition-key is not present"() { - Topic topic = createAvroTopic("pl.allegro.test.randomFoo") - Subscription subscription = createTestSubscription(topic, "test-subscription") - String kafkaTopicName = topic.getName().toString() - ConsumerGroupId consumerGroupId = kafkaNamesMapper.toConsumerGroupId(subscription.qualifiedName) - createTopicInKafka(kafkaTopicName, NUMBER_OF_PARTITION) - CachedTopic cachedTopic = CachedTopicsTestHelper.cachedTopic(topic) - KafkaConsumer consumer = createConsumer(consumerGroupId, kafkaTopicName) - - when: - 1.upto(10) { - brokerMessageProducer.send(generateAvroMessage(null), cachedTopic, null) - waitForRecordPublishing(consumer) - } - - then: - consumer.close() - - List partitionsWithMessagesData = adminClient - .listConsumerGroupOffsets(consumerGroupId.asString()) - .partitionsToOffsetAndMetadata() - .get().values().stream() - .filter { metadata -> metadata.offset() != 0 } - .collect(Collectors.toList()) - - partitionsWithMessagesData.size() == NUMBER_OF_PARTITION - } - private static AvroMessage generateAvroMessage(String partitionKey) { def avroUser = new AvroUser() return new AvroMessage(UUID.randomUUID().toString(), avroUser.asBytes(), 0L, avroUser.compiledSchema, diff --git a/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/containers/ImageTags.java b/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/containers/ImageTags.java index 82d59230a5..d140544320 100644 --- a/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/containers/ImageTags.java +++ b/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/containers/ImageTags.java @@ -1,7 +1,7 @@ package pl.allegro.tech.hermes.test.helper.containers; public class ImageTags { - static String confluentImagesTag() { + public static String confluentImagesTag() { return System.getProperty("confluentImagesTag", "6.1.0"); } } diff --git a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/KafkaRetransmissionServiceTest.java b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/KafkaRetransmissionServiceTest.java index f5ffb06549..883e408a87 100644 --- a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/KafkaRetransmissionServiceTest.java +++ b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/KafkaRetransmissionServiceTest.java @@ -25,6 +25,7 @@ import java.util.ArrayList; import java.util.List; import java.util.function.Predicate; +import java.util.stream.Collectors; import static com.jayway.awaitility.Awaitility.waitAtMost; import static java.util.stream.IntStream.range; @@ -84,7 +85,7 @@ public void shouldMoveOffsetInDryRunMode() throws InterruptedException { TestSubscriber subscriber = subscribers.createSubscriber(); Topic topic = hermes.initHelper().createTopic(topicWithRandomName().build()); final Subscription subscription = hermes.initHelper().createSubscription(subscriptionWithRandomName(topic.getName(), subscriber.getEndpoint()).build()); - // we have 2 partitions, thus 4 messages to get 2 per partition + // 4 messages publishAndConsumeMessages(messages, topic, subscriber); Thread.sleep(2000); final OffsetRetransmissionDate retransmissionDate = new OffsetRetransmissionDate(OffsetDateTime.now()); @@ -100,8 +101,9 @@ public void shouldMoveOffsetInDryRunMode() throws InterruptedException { MultiDCOffsetChangeSummary summary = response.expectBody(MultiDCOffsetChangeSummary.class).returnResult().getResponseBody(); assert summary != null; - assertThat(summary.getPartitionOffsetListPerBrokerName().get(PRIMARY_KAFKA_CLUSTER_NAME).get(0).getOffset()) - .isEqualTo(2); + Long offsetSum = summary.getPartitionOffsetListPerBrokerName().get(PRIMARY_KAFKA_CLUSTER_NAME).stream() + .collect(Collectors.summarizingLong(PartitionOffset::getOffset)).getSum(); + assertThat(offsetSum).isEqualTo(4); subscriber.noMessagesReceived(); } From 69df8b0b3ab9a1630c3767a484d22f40a433c48c Mon Sep 17 00:00:00 2001 From: Maciej Moscicki Date: Fri, 10 May 2024 11:59:19 +0200 Subject: [PATCH 51/87] Fix producer with incorrect ACK level being returned for remote sender (#1859) * fix incorrect producer being returned for remote * use remote dc name constant --- .../producer/kafka/KafkaMessageSenders.java | 2 +- .../RemoteDatacenterProduceFallbackTest.java | 47 +++++++++++++------ 2 files changed, 33 insertions(+), 16 deletions(-) diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSenders.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSenders.java index 109c381122..ddd707657e 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSenders.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSenders.java @@ -54,7 +54,7 @@ KafkaMessageSender get(Topic topic) { } List> getRemote(Topic topic) { - return topic.isReplicationConfirmRequired() ? remoteAckLeader : remoteAckAll; + return topic.isReplicationConfirmRequired() ? remoteAckAll : remoteAckLeader; } List getDatacenters() { diff --git a/integration-tests/src/slowIntegrationTest/java/pl/allegro/tech/hermes/integrationtests/RemoteDatacenterProduceFallbackTest.java b/integration-tests/src/slowIntegrationTest/java/pl/allegro/tech/hermes/integrationtests/RemoteDatacenterProduceFallbackTest.java index 3cf7b5eca3..360b6ecc13 100644 --- a/integration-tests/src/slowIntegrationTest/java/pl/allegro/tech/hermes/integrationtests/RemoteDatacenterProduceFallbackTest.java +++ b/integration-tests/src/slowIntegrationTest/java/pl/allegro/tech/hermes/integrationtests/RemoteDatacenterProduceFallbackTest.java @@ -5,12 +5,12 @@ import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.RegisterExtension; -import org.springframework.test.web.reactive.server.WebTestClient; import org.testcontainers.lifecycle.Startable; import pl.allegro.tech.hermes.api.PublishingChaosPolicy; import pl.allegro.tech.hermes.api.PublishingChaosPolicy.ChaosMode; import pl.allegro.tech.hermes.api.PublishingChaosPolicy.ChaosPolicy; import pl.allegro.tech.hermes.api.Topic; +import pl.allegro.tech.hermes.integrationtests.assertions.PrometheusMetricsAssertion; import pl.allegro.tech.hermes.integrationtests.setup.HermesConsumersTestApp; import pl.allegro.tech.hermes.integrationtests.setup.HermesFrontendTestApp; import pl.allegro.tech.hermes.integrationtests.setup.HermesManagementTestApp; @@ -47,6 +47,7 @@ public class RemoteDatacenterProduceFallbackTest { private static HermesConsumersTestApp consumerDC2; private static HermesTestClient DC1; + private static final String REMOTE_DC_NAME = "dc2"; @BeforeAll public static void setup() { @@ -55,13 +56,13 @@ public static void setup() { .forEach(HermesDatacenter::startKafkaAndZookeeper); schemaRegistry.start(); management = new HermesManagementTestApp( - Map.of(DEFAULT_DC_NAME, dc1.hermesZookeeper, "dc2", dc2.hermesZookeeper), - Map.of(DEFAULT_DC_NAME, dc1.kafka, "dc2", dc2.kafka), + Map.of(DEFAULT_DC_NAME, dc1.hermesZookeeper, REMOTE_DC_NAME, dc2.hermesZookeeper), + Map.of(DEFAULT_DC_NAME, dc1.kafka, REMOTE_DC_NAME, dc2.kafka), schemaRegistry ); management.start(); frontendDC1 = new HermesFrontendTestApp(dc1.hermesZookeeper, - Map.of("dc", dc1.kafka, "dc2", dc2.kafka), + Map.of("dc", dc1.kafka, REMOTE_DC_NAME, dc2.kafka), schemaRegistry ); frontendDC1.start(); @@ -92,7 +93,7 @@ public static void clean() { public void afterEach() { Stream.of(dc1, dc2).forEach(dc -> dc.kafka.restoreConnectionsBetweenBrokersAndClients()); DC1.setReadiness(DEFAULT_DC_NAME, true); - DC1.setReadiness("dc2", true); + DC1.setReadiness(REMOTE_DC_NAME, true); } @Test @@ -104,6 +105,8 @@ public void shouldPublishAndConsumeViaRemoteDCWhenLocalKafkaIsUnavailable() { subscription(topic.getQualifiedName(), "subscription", subscriber.getEndpoint()).build() ); + double remoteDCInitialSendTotal = assertRemoteDCSendTotalMetric().withInitialValue(); + // when dc1 is not available dc1.kafka.cutOffConnectionsBetweenBrokersAndClients(); @@ -119,14 +122,17 @@ public void shouldPublishAndConsumeViaRemoteDCWhenLocalKafkaIsUnavailable() { .expectStatus() .isOk() .expectBody(String.class) - .value((body) -> assertThatMetrics(body) - .contains("hermes_frontend_topic_published_total") - .withLabels( - "group", topic.getName().getGroupName(), - "topic", topic.getName().getName(), - "storageDc", "dc2" - ) - .withValue(1.0) + .value((body) -> { + assertThatMetrics(body) + .contains("hermes_frontend_topic_published_total") + .withLabels( + "group", topic.getName().getGroupName(), + "topic", topic.getName().getName(), + "storageDc", REMOTE_DC_NAME + ) + .withValue(1.0); + assertRemoteDCSendTotalMetric().withValueGreaterThan(remoteDCInitialSendTotal); + } ); } @@ -162,7 +168,7 @@ public void shouldNotFallBackToNotReadyDatacenter() { // when local datacenter is not available and remote is not ready dc1.kafka.cutOffConnectionsBetweenBrokersAndClients(); - DC1.setReadiness("dc2", false); + DC1.setReadiness(REMOTE_DC_NAME, false); // and message is published TestMessage message = TestMessage.of("key1", "value1"); @@ -204,7 +210,7 @@ public void shouldPublishAndConsumeViaRemoteDCWhenChaosExperimentIsEnabledForLoc .withLabels( "group", topic.getName().getGroupName(), "topic", topic.getName().getName(), - "storageDc", "dc2" + "storageDc", REMOTE_DC_NAME ) .withValue(1.0) ); @@ -279,4 +285,15 @@ void stop() { .forEach(Startable::stop); } } + + PrometheusMetricsAssertion.PrometheusMetricAssertion assertRemoteDCSendTotalMetric() { + return assertThatMetrics(DC1 + .getFrontendMetrics().expectStatus().isOk() + .expectBody(String.class).returnResult().getResponseBody()) + .contains("hermes_frontend_kafka_producer_ack_leader_record_send_total") + .withLabels( + "storageDc", REMOTE_DC_NAME, + "sender", "failFast" + ); + } } From c1fa629ae5ebc97717ee00d28eae0aa6ead6bb86 Mon Sep 17 00:00:00 2001 From: Mateusz <76775507+szczygiel-m@users.noreply.github.com> Date: Thu, 23 May 2024 14:07:38 +0200 Subject: [PATCH 52/87] Confluent image tags dependent on os arch (#1863) * Updated confluent image tags from 6.1.0 to 7.6.1 * Upgraded testcontainers * Image tags dependent on the system architecture * Fixed test --- build.gradle | 2 +- .../KafkaConsumerGroupManagerSpec.groovy | 3 ++- .../test/helper/containers/ImageTags.java | 3 +++ .../hermes/integrationtests/MetricsTest.java | 26 ++++++++++--------- 4 files changed, 20 insertions(+), 14 deletions(-) diff --git a/build.gradle b/build.gradle index 0eec952889..5f99480281 100644 --- a/build.gradle +++ b/build.gradle @@ -66,7 +66,7 @@ allprojects { spring_web : '6.1.2', failsafe : '2.3.1', junit_jupiter : '5.9.1', - testcontainers : '1.18.1', + testcontainers : '1.19.8', spring : '3.2.1', assertj : '3.24.2' ] diff --git a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/kafka/service/KafkaConsumerGroupManagerSpec.groovy b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/kafka/service/KafkaConsumerGroupManagerSpec.groovy index 4a0283d5a5..11b514ed95 100644 --- a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/kafka/service/KafkaConsumerGroupManagerSpec.groovy +++ b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/kafka/service/KafkaConsumerGroupManagerSpec.groovy @@ -22,6 +22,7 @@ import pl.allegro.tech.hermes.common.kafka.KafkaNamesMapper import pl.allegro.tech.hermes.management.config.kafka.KafkaProperties import pl.allegro.tech.hermes.management.domain.subscription.ConsumerGroupManager import pl.allegro.tech.hermes.test.helper.builder.TopicBuilder +import pl.allegro.tech.hermes.test.helper.containers.ImageTags import spock.lang.Shared import spock.lang.Specification @@ -38,7 +39,7 @@ class KafkaConsumerGroupManagerSpec extends Specification { OutputCaptureRule output = new OutputCaptureRule() @Shared - KafkaContainer kafkaContainer = new KafkaContainer() + KafkaContainer kafkaContainer = new KafkaContainer(ImageTags.confluentImagesTag()) @Shared String containerId diff --git a/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/containers/ImageTags.java b/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/containers/ImageTags.java index d140544320..3c2751bc42 100644 --- a/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/containers/ImageTags.java +++ b/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/containers/ImageTags.java @@ -2,6 +2,9 @@ public class ImageTags { public static String confluentImagesTag() { + if (System.getProperty("os.arch").equals("aarch64")) { + return System.getProperty("confluentImagesTag", "7.6.1"); + } return System.getProperty("confluentImagesTag", "6.1.0"); } } diff --git a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/MetricsTest.java b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/MetricsTest.java index caf42ae534..359c7cb239 100644 --- a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/MetricsTest.java +++ b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/MetricsTest.java @@ -240,19 +240,21 @@ public void shouldReportMetricForFilteredSubscription() { // then subscriber.waitUntilReceived(unfiltered.body()); - hermes.api().getConsumersMetrics() - .expectStatus() - .isOk() - .expectBody(String.class) - .value((body) -> assertThatMetrics(body) - .contains("hermes_consumers_subscription_filtered_out_total") - .withLabels( - "group", topic.getName().getGroupName(), - "subscription", subscription.getName(), - "topic", topic.getName().getName() + waitAtMost(Duration.TEN_SECONDS).until(() -> + hermes.api().getConsumersMetrics() + .expectStatus() + .isOk() + .expectBody(String.class) + .value((body) -> assertThatMetrics(body) + .contains("hermes_consumers_subscription_filtered_out_total") + .withLabels( + "group", topic.getName().getGroupName(), + "subscription", subscription.getName(), + "topic", topic.getName().getName() + ) + .withValue(1.0) ) - .withValue(1.0) - ); + ); } @Test From c0b31d37880aba0ce7b887fe20b8916c32036743 Mon Sep 17 00:00:00 2001 From: Mateusz <76775507+szczygiel-m@users.noreply.github.com> Date: Fri, 24 May 2024 15:28:10 +0200 Subject: [PATCH 53/87] Consumer profiling (#1857) * Added consumer profiler * Added consumer profiler * Added consumer profiler * CR fixes * Fixed tests * CR fixes * CR fixes * Fixed checkstyle --- docs/docs/configuration/consumers-tuning.md | 26 +- .../allegro/tech/hermes/api/Subscription.java | 28 +- .../consumer/ConsumerMessageSender.java | 53 ++-- .../consumers/consumer/SerialConsumer.java | 21 +- .../consumer/profiling/ConsumerProfiler.java | 27 ++ .../consumer/profiling/ConsumerRun.java | 5 + .../profiling/DefaultConsumerProfiler.java | 77 ++++++ .../consumer/profiling/Measurement.java | 14 + .../profiling/NoOpConsumerProfiler.java | 34 +++ .../consumer/ConsumerMessageSenderTest.java | 34 +-- ...enterMessageProducerIntegrationTest.groovy | 4 +- .../KafkaConsumerGroupManagerSpec.groovy | 2 +- .../helper/builder/SubscriptionBuilder.java | 18 +- .../integration/FrontendTestClient.java | 3 + .../ConsumerProfilingTest.java | 244 ++++++++++++++++++ 15 files changed, 533 insertions(+), 57 deletions(-) create mode 100644 hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/profiling/ConsumerProfiler.java create mode 100644 hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/profiling/ConsumerRun.java create mode 100644 hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/profiling/DefaultConsumerProfiler.java create mode 100644 hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/profiling/Measurement.java create mode 100644 hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/profiling/NoOpConsumerProfiler.java create mode 100644 integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/ConsumerProfilingTest.java diff --git a/docs/docs/configuration/consumers-tuning.md b/docs/docs/configuration/consumers-tuning.md index f4433b819f..904df5eede 100644 --- a/docs/docs/configuration/consumers-tuning.md +++ b/docs/docs/configuration/consumers-tuning.md @@ -2,18 +2,18 @@ ## HTTP Sender -Option | Description | Default value ----------------------------------------------------- | ----------------------------------------------------------- | ------------- -consumer.http-client.serial.http1.threadPoolSize | size of thread pool for sender threads (global) | 30 -consumer.http-client.serial.http1.maxConnectionsPerDestination | max connections per remote host | 100 +| Option | Description | Default value | +|----------------------------------------------------------------|-------------------------------------------------|---------------| +| consumer.http-client.serial.http1.threadPoolSize | size of thread pool for sender threads (global) | 30 | +| consumer.http-client.serial.http1.maxConnectionsPerDestination | max connections per remote host | 100 | ## Consumers core -Option | Description | Default value ------------------------------ | ------------------------------------------------------------------------ | ------------- -consumer.commit.offset.period | interval between committing offsets to Kafka | 60s -consumer.threadPoolSize | thread pool for threads involved in consuming, 1 thread per subscription | 500 -consumer.serialConsumer.inflightSize | how many messages can be kept in send queue, per subscription | 100 +| Option | Description | Default value | +|--------------------------------------|--------------------------------------------------------------------------|---------------| +| consumer.commit.offset.period | interval between committing offsets to Kafka | 60s | +| consumer.threadPoolSize | thread pool for threads involved in consuming, 1 thread per subscription | 500 | +| consumer.serialConsumer.inflightSize | how many messages can be kept in send queue, per subscription | 100 | ## Workload constraints management @@ -26,10 +26,10 @@ subscriptions assigned to itself. These numbers can be configured: -Option | Description | Default value ---------------------------------------------------- | ----------------------------------------- | --------------------- -consumer.workload.consumersPerSubscription | Number of consumers to which the subscription will be assigned. If this value is greater than the number of available consumers, Hermes will assign the subscription to all available consumers. | 2 -consumer.workload.maxSubscriptionsPerConsumer | The maximum number of subscriptions assigned to a single consumer. If all consumers have the maximum number of subscriptions assigned, a new subscription will not be activated until a new consumer is added or another subscription is unassigned. | 200 +| Option | Description | Default value | +|-----------------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|---------------| +| consumer.workload.consumersPerSubscription | Number of consumers to which the subscription will be assigned. If this value is greater than the number of available consumers, Hermes will assign the subscription to all available consumers. | 2 | +| consumer.workload.maxSubscriptionsPerConsumer | The maximum number of subscriptions assigned to a single consumer. If all consumers have the maximum number of subscriptions assigned, a new subscription will not be activated until a new consumer is added or another subscription is unassigned. | 200 | Additionally, Hermes allows to configure the property `consumer.workload.consumersPerSubscription` for specific topics or subscriptions in the runtime via REST API. diff --git a/hermes-api/src/main/java/pl/allegro/tech/hermes/api/Subscription.java b/hermes-api/src/main/java/pl/allegro/tech/hermes/api/Subscription.java index 3a8c673590..14849437df 100644 --- a/hermes-api/src/main/java/pl/allegro/tech/hermes/api/Subscription.java +++ b/hermes-api/src/main/java/pl/allegro/tech/hermes/api/Subscription.java @@ -52,6 +52,8 @@ public class Subscription implements Anonymizable { private boolean trackingEnabled = false; private TrackingMode trackingMode = TrackingMode.TRACKING_OFF; private boolean http2Enabled = false; + private boolean profilingEnabled = false; + private long profilingThresholdMs = 0; @Valid @NotNull private OwnerId owner; @@ -94,6 +96,8 @@ private Subscription(TopicName topicName, EndpointAddressResolverMetadata endpointAddressResolverMetadata, SubscriptionOAuthPolicy oAuthPolicy, boolean http2Enabled, + boolean profilingEnabled, + long profilingThresholdMs, boolean subscriptionIdentityHeadersEnabled, boolean autoDeleteWithTopicEnabled) { this.topicName = topicName; @@ -112,6 +116,8 @@ private Subscription(TopicName topicName, this.filters = filters; this.mode = mode; this.http2Enabled = http2Enabled; + this.profilingEnabled = profilingEnabled; + this.profilingThresholdMs = profilingThresholdMs; this.subscriptionName = new SubscriptionName(name, topicName); this.headers = headers; this.endpointAddressResolverMetadata = endpointAddressResolverMetadata; @@ -137,11 +143,13 @@ public static Subscription createSerialSubscription(TopicName topicName, EndpointAddressResolverMetadata endpointAddressResolverMetadata, SubscriptionOAuthPolicy oAuthPolicy, boolean http2Enabled, + boolean profilingEnabled, + long profilingThresholdMs, boolean subscriptionIdentityHeadersEnabled, boolean autoDeleteWithTopicEnabled) { return new Subscription(topicName, name, endpoint, state, description, subscriptionPolicy, trackingEnabled, trackingMode, owner, monitoringDetails, contentType, DeliveryType.SERIAL, filters, mode, headers, - endpointAddressResolverMetadata, oAuthPolicy, http2Enabled, subscriptionIdentityHeadersEnabled, autoDeleteWithTopicEnabled); + endpointAddressResolverMetadata, oAuthPolicy, http2Enabled, profilingEnabled, profilingThresholdMs, subscriptionIdentityHeadersEnabled, autoDeleteWithTopicEnabled); } public static Subscription createBatchSubscription(TopicName topicName, @@ -164,7 +172,7 @@ public static Subscription createBatchSubscription(TopicName topicName, boolean autoDeleteWithTopicEnabled) { return new Subscription(topicName, name, endpoint, state, description, subscriptionPolicy, trackingEnabled, trackingMode, owner, monitoringDetails, contentType, DeliveryType.BATCH, filters, SubscriptionMode.ANYCAST, headers, - endpointAddressResolverMetadata, oAuthPolicy, http2Enabled, subscriptionIdentityHeadersEnabled, autoDeleteWithTopicEnabled); + endpointAddressResolverMetadata, oAuthPolicy, http2Enabled, false, 0, subscriptionIdentityHeadersEnabled, autoDeleteWithTopicEnabled); } @JsonCreator @@ -187,6 +195,8 @@ public static Subscription create( @JsonProperty("endpointAddressResolverMetadata") EndpointAddressResolverMetadata endpointAddressResolverMetadata, @JsonProperty("oAuthPolicy") SubscriptionOAuthPolicy oAuthPolicy, @JsonProperty("http2Enabled") boolean http2Enabled, + @JsonProperty("profilingEnabled") boolean profilingEnabled, + @JsonProperty("profilingThresholdMs") long profilingThresholdMs, @JsonProperty("subscriptionIdentityHeadersEnabled") boolean subscriptionIdentityHeadersEnabled, @JsonProperty("autoDeleteWithTopicEnabled") boolean autoDeleteWithTopicEnabled) { @@ -219,6 +229,8 @@ public static Subscription create( endpointAddressResolverMetadata == null ? EndpointAddressResolverMetadata.empty() : endpointAddressResolverMetadata, oAuthPolicy, http2Enabled, + profilingEnabled, + profilingThresholdMs, subscriptionIdentityHeadersEnabled, autoDeleteWithTopicEnabled ); @@ -257,6 +269,8 @@ public boolean equals(Object obj) { && Objects.equals(this.headers, other.headers) && Objects.equals(this.endpointAddressResolverMetadata, other.endpointAddressResolverMetadata) && Objects.equals(this.http2Enabled, other.http2Enabled) + && Objects.equals(this.profilingEnabled, other.profilingEnabled) + && Objects.equals(this.profilingThresholdMs, other.profilingThresholdMs) && Objects.equals(this.oAuthPolicy, other.oAuthPolicy) && Objects.equals(this.subscriptionIdentityHeadersEnabled, other.subscriptionIdentityHeadersEnabled) && Objects.equals(this.autoDeleteWithTopicEnabled, other.autoDeleteWithTopicEnabled); @@ -391,6 +405,14 @@ public boolean isHttp2Enabled() { return http2Enabled; } + public boolean isProfilingEnabled() { + return profilingEnabled; + } + + public long getProfilingThresholdMs() { + return profilingThresholdMs; + } + public boolean isSubscriptionIdentityHeadersEnabled() { return subscriptionIdentityHeadersEnabled; } @@ -437,6 +459,8 @@ public Subscription anonymize() { endpointAddressResolverMetadata, oAuthPolicy != null ? oAuthPolicy.anonymize() : null, http2Enabled, + profilingEnabled, + profilingThresholdMs, subscriptionIdentityHeadersEnabled, autoDeleteWithTopicEnabled ); diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSender.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSender.java index 4f46845e58..eb37f61a85 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSender.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSender.java @@ -7,6 +7,11 @@ import pl.allegro.tech.hermes.api.Subscription; import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.consumers.consumer.load.SubscriptionLoadRecorder; +import pl.allegro.tech.hermes.consumers.consumer.profiling.ConsumerProfiler; +import pl.allegro.tech.hermes.consumers.consumer.profiling.ConsumerRun; +import pl.allegro.tech.hermes.consumers.consumer.profiling.DefaultConsumerProfiler; +import pl.allegro.tech.hermes.consumers.consumer.profiling.Measurement; +import pl.allegro.tech.hermes.consumers.consumer.profiling.NoOpConsumerProfiler; import pl.allegro.tech.hermes.consumers.consumer.rate.InflightsPool; import pl.allegro.tech.hermes.consumers.consumer.rate.SerialConsumerRateLimiter; import pl.allegro.tech.hermes.consumers.consumer.result.ErrorHandler; @@ -107,13 +112,13 @@ public void shutdown() { } } - public void sendAsync(Message message) { + public void sendAsync(Message message, ConsumerProfiler profiler) { inflightCount.increment(); - sendAsync(message, calculateMessageDelay(message.getPublishingTimestamp())); + sendAsync(message, calculateMessageDelay(message.getPublishingTimestamp()), profiler); } - private void sendAsync(Message message, int delayMillis) { - retrySingleThreadExecutor.schedule(() -> sendMessage(message), delayMillis, TimeUnit.MILLISECONDS); + private void sendAsync(Message message, int delayMillis, ConsumerProfiler profiler) { + retrySingleThreadExecutor.schedule(() -> sendMessage(message, profiler), delayMillis, TimeUnit.MILLISECONDS); } private int calculateMessageDelay(long publishingMessageTimestamp) { @@ -134,13 +139,15 @@ private int calculateMessageDelay(long publishingMessageTimestamp) { * Method is calling MessageSender and is registering listeners to handle response. * Main responsibility of this method is that no message will be fully processed or rejected without release on semaphore. */ - private void sendMessage(final Message message) { + private void sendMessage(final Message message, ConsumerProfiler profiler) { loadRecorder.recordSingleOperation(); + profiler.measure(Measurement.ACQUIRE_RATE_LIMITER); acquireRateLimiterWithTimer(); HermesTimerContext timer = consumerLatencyTimer.time(); + profiler.measure(Measurement.MESSAGE_SENDER_SEND); CompletableFuture response = messageSender.send(message); - response.thenAcceptAsync(new ResponseHandlingListener(message, timer), deliveryReportingExecutor) + response.thenAcceptAsync(new ResponseHandlingListener(message, timer, profiler), deliveryReportingExecutor) .exceptionally(e -> { logger.error( "An error occurred while handling message sending response of subscription {} [partition={}, offset={}, id={}]", @@ -201,21 +208,26 @@ private boolean willExceedTtl(Message message, long delay) { return message.isTtlExceeded(remainingTtl); } - private void handleFailedSending(Message message, MessageSendingResult result) { - retrySending(message, result); + private void handleFailedSending(Message message, MessageSendingResult result, ConsumerProfiler profiler) { errorHandlers.forEach(h -> h.handleFailed(message, subscription, result)); + retrySendingOrDiscard(message, result, profiler); } - private void retrySending(Message message, MessageSendingResult result) { + private void retrySendingOrDiscard(Message message, MessageSendingResult result, ConsumerProfiler profiler) { List succeededUris = result.getSucceededUris(ConsumerMessageSender.this::messageSentSucceeded); message.incrementRetryCounter(succeededUris); long retryDelay = extractRetryDelay(message, result); if (shouldAttemptResending(message, result, retryDelay)) { retries.increment(); - retrySingleThreadExecutor.schedule(() -> resend(message, result), retryDelay, TimeUnit.MILLISECONDS); + profiler.flushMeasurements(ConsumerRun.RETRIED); + ConsumerProfiler resendProfiler = subscription.isProfilingEnabled() + ? new DefaultConsumerProfiler(subscription.getQualifiedName(), subscription.getProfilingThresholdMs()) : new NoOpConsumerProfiler(); + resendProfiler.startMeasurements(Measurement.SCHEDULE_RESEND); + resendProfiler.saveRetryDelay(retryDelay); + retrySingleThreadExecutor.schedule(() -> resend(message, result, resendProfiler), retryDelay, TimeUnit.MILLISECONDS); } else { - handleMessageDiscarding(message, result); + handleMessageDiscarding(message, result, profiler); } } @@ -229,11 +241,11 @@ private long extractRetryDelay(Message message, MessageSendingResult result) { return result.getRetryAfterMillis().map(delay -> Math.min(delay, ttl)).orElse(defaultBackoff); } - private void resend(Message message, MessageSendingResult result) { + private void resend(Message message, MessageSendingResult result, ConsumerProfiler profiler) { if (result.isLoggable()) { result.getLogInfo().forEach(logInfo -> logResultInfo(message, logInfo)); } - sendMessage(message); + sendMessage(message, profiler); } private void logResultInfo(Message message, MessageSendingResultLogInfo logInfo) { @@ -244,16 +256,18 @@ private void logResultInfo(Message message, MessageSendingResultLogInfo logInfo) logInfo.getFailure()); } - private void handleMessageDiscarding(Message message, MessageSendingResult result) { + private void handleMessageDiscarding(Message message, MessageSendingResult result, ConsumerProfiler profiler) { inflight.release(); inflightCount.decrement(); errorHandlers.forEach(h -> h.handleDiscarded(message, subscription, result)); + profiler.flushMeasurements(ConsumerRun.DISCARDED); } - private void handleMessageSendingSuccess(Message message, MessageSendingResult result) { + private void handleMessageSendingSuccess(Message message, MessageSendingResult result, ConsumerProfiler profiler) { inflight.release(); inflightCount.decrement(); successHandlers.forEach(h -> h.handleSuccess(message, subscription, result)); + profiler.flushMeasurements(ConsumerRun.DELIVERED); } private boolean messageSentSucceeded(MessageSendingResult result) { @@ -277,21 +291,24 @@ class ResponseHandlingListener implements java.util.function.Consumer maybeMessage = messageReceiver.next(); + profiler.measure(Measurement.MESSAGE_CONVERSION); if (maybeMessage.isPresent()) { Message message = maybeMessage.get(); @@ -114,9 +124,10 @@ public void consume(Runnable signalsInterrupt) { } Message convertedMessage = messageConverterResolver.converterFor(message, subscription).convert(message, topic); - sendMessage(convertedMessage); + sendMessage(convertedMessage, profiler); } else { inflightSemaphore.release(); + profiler.flushMeasurements(ConsumerRun.EMPTY); } } catch (InterruptedException e) { logger.info("Restoring interrupted status {}", subscription.getQualifiedName(), e); @@ -126,16 +137,18 @@ public void consume(Runnable signalsInterrupt) { } } - private void sendMessage(Message message) { + private void sendMessage(Message message, ConsumerProfiler profiler) { + profiler.measure(Measurement.OFFER_INFLIGHT_OFFSET); offsetQueue.offerInflightOffset( subscriptionPartitionOffset(subscription.getQualifiedName(), message.getPartitionOffset(), message.getPartitionAssignmentTerm()) ); + profiler.measure(Measurement.TRACKERS_LOG_INFLIGHT); trackers.get(subscription).logInflight(toMessageMetadata(message, subscription)); - sender.sendAsync(message); + sender.sendAsync(message, profiler); } @Override diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/profiling/ConsumerProfiler.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/profiling/ConsumerProfiler.java new file mode 100644 index 0000000000..5978ab90aa --- /dev/null +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/profiling/ConsumerProfiler.java @@ -0,0 +1,27 @@ +package pl.allegro.tech.hermes.consumers.consumer.profiling; + +public interface ConsumerProfiler { + + void startMeasurements(String measurement); + + /** + * Measures the execution time of a specific piece of code. + * The measurement starts with a call to this method, + * and is terminated by another call to the same method with a different parameter (to keep the measurement continuity), + * or by calling the flushMeasurements method. + */ + void measure(String measurement); + + /** + * Measures the same piece of code several times, for example, a method call in the middle of a loop. + * Default implementation stores individual measurements, as well as their sum. + * stopPartialMeasurements should be called before measuring again. + */ + void startPartialMeasurement(String measurement); + + void stopPartialMeasurement(); + + void saveRetryDelay(long retryDelay); + + void flushMeasurements(ConsumerRun consumerRun); +} diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/profiling/ConsumerRun.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/profiling/ConsumerRun.java new file mode 100644 index 0000000000..ea4e3dbfe1 --- /dev/null +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/profiling/ConsumerRun.java @@ -0,0 +1,5 @@ +package pl.allegro.tech.hermes.consumers.consumer.profiling; + +public enum ConsumerRun { + EMPTY, DELIVERED, DISCARDED, RETRIED +} diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/profiling/DefaultConsumerProfiler.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/profiling/DefaultConsumerProfiler.java new file mode 100644 index 0000000000..347c2cc379 --- /dev/null +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/profiling/DefaultConsumerProfiler.java @@ -0,0 +1,77 @@ +package pl.allegro.tech.hermes.consumers.consumer.profiling; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.util.StopWatch; +import pl.allegro.tech.hermes.api.SubscriptionName; + +import java.util.concurrent.TimeUnit; + +/** + * This class is designed to be fully thread safe. + */ +public class DefaultConsumerProfiler implements ConsumerProfiler { + + private static final Logger logger = LoggerFactory.getLogger(DefaultConsumerProfiler.class); + + private final SubscriptionName subscriptionName; + private final long profilingThresholdMs; + private StopWatch stopWatch; + private StopWatch partialMeasurements; + private long retryDelayMillis = 0; + + public DefaultConsumerProfiler(SubscriptionName subscriptionName, long profilingThresholdMs) { + this.subscriptionName = subscriptionName; + this.profilingThresholdMs = profilingThresholdMs; + } + + @Override + public synchronized void startMeasurements(String measurement) { + this.stopWatch = new StopWatch(); + this.stopWatch.start(measurement); + } + + @Override + public synchronized void measure(String measurement) { + this.stopWatch.stop(); + this.stopWatch.start(measurement); + } + + @Override + public synchronized void startPartialMeasurement(String measurement) { + if (partialMeasurements == null) { + partialMeasurements = new StopWatch(); + } + partialMeasurements.start(measurement); + } + + @Override + public synchronized void stopPartialMeasurement() { + partialMeasurements.stop(); + } + + @Override + public synchronized void saveRetryDelay(long retryDelay) { + this.retryDelayMillis = retryDelay; + } + + @Override + public synchronized void flushMeasurements(ConsumerRun consumerRun) { + this.stopWatch.stop(); + if (stopWatch.getTotalTimeMillis() > profilingThresholdMs) { + logMeasurements(consumerRun); + } + } + + private void logMeasurements(ConsumerRun consumerRun) { + if (partialMeasurements != null) { + logger.info("Consumer profiler measurements for subscription {} and {} run: \n {} partialMeasurements: {} retryDelayMillis {}", + subscriptionName, consumerRun, stopWatch.prettyPrint(TimeUnit.MILLISECONDS), + partialMeasurements.prettyPrint(TimeUnit.MILLISECONDS), retryDelayMillis); + } else { + logger.info("Consumer profiler measurements for subscription {} and {} run: \n {} partialMeasurements: {}, retryDelayMillis {}", + subscriptionName, consumerRun, stopWatch.prettyPrint(TimeUnit.MILLISECONDS), + null, retryDelayMillis); + } + } +} diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/profiling/Measurement.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/profiling/Measurement.java new file mode 100644 index 0000000000..2de07271fb --- /dev/null +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/profiling/Measurement.java @@ -0,0 +1,14 @@ +package pl.allegro.tech.hermes.consumers.consumer.profiling; + +public class Measurement { + public static final String SIGNALS_AND_SEMAPHORE_ACQUIRE = "signalsAndSemaphoreAcquire"; + public static final String SIGNALS_INTERRUPT_RUN = "signalsInterrupt.run"; + public static final String SCHEDULE_RESEND = "schedule.resend"; + public static final String MESSAGE_RECEIVER_NEXT = "messageReceiver.next"; + public static final String MESSAGE_CONVERSION = "messageConverter.convert"; + public static final String OFFER_INFLIGHT_OFFSET = "offsetQueue.offerInflightOffset"; + public static final String TRACKERS_LOG_INFLIGHT = "trackers.logInflight"; + public static final String ACQUIRE_RATE_LIMITER = "acquireRateLimiter"; + public static final String MESSAGE_SENDER_SEND = "messageSender.send"; + public static final String HANDLERS = "handlers"; +} diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/profiling/NoOpConsumerProfiler.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/profiling/NoOpConsumerProfiler.java new file mode 100644 index 0000000000..9b3355f4a1 --- /dev/null +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/profiling/NoOpConsumerProfiler.java @@ -0,0 +1,34 @@ +package pl.allegro.tech.hermes.consumers.consumer.profiling; + +public class NoOpConsumerProfiler implements ConsumerProfiler { + + @Override + public void startMeasurements(String measurement) { + + } + + @Override + public void measure(String measurement) { + + } + + @Override + public void startPartialMeasurement(String measurement) { + + } + + @Override + public void stopPartialMeasurement() { + + } + + @Override + public void saveRetryDelay(long retryDelay) { + + } + + @Override + public void flushMeasurements(ConsumerRun consumerRun) { + + } +} diff --git a/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSenderTest.java b/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSenderTest.java index a630aafcc8..d5ca9aac60 100644 --- a/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSenderTest.java +++ b/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSenderTest.java @@ -8,6 +8,8 @@ import pl.allegro.tech.hermes.api.Subscription; import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.common.metric.SubscriptionMetrics; +import pl.allegro.tech.hermes.consumers.consumer.profiling.ConsumerProfiler; +import pl.allegro.tech.hermes.consumers.consumer.profiling.NoOpConsumerProfiler; import pl.allegro.tech.hermes.consumers.consumer.rate.AdjustableSemaphore; import pl.allegro.tech.hermes.consumers.consumer.rate.SerialConsumerRateLimiter; import pl.allegro.tech.hermes.consumers.consumer.result.ErrorHandler; @@ -89,6 +91,8 @@ public class ConsumerMessageSenderTest { private AdjustableSemaphore inflightSemaphore; + private final ConsumerProfiler profiler = new NoOpConsumerProfiler(); + private ConsumerMessageSender sender; @Mock @@ -123,7 +127,7 @@ public void shouldHandleSuccessfulSending() { when(messageSender.send(message)).thenReturn(success()); // when - sender.sendAsync(message); + sender.sendAsync(message, profiler); verify(successHandler, timeout(1000)).handleSuccess(eq(message), eq(subscription), any(MessageSendingResult.class)); // then @@ -143,7 +147,7 @@ public void shouldKeepTryingToSendMessageFailedSending() { doReturn(failure()).doReturn(failure()).doReturn(success()).when(messageSender).send(message); // when - sender.sendAsync(message); + sender.sendAsync(message, profiler); verify(successHandler, timeout(1000)).handleSuccess(eq(message), eq(subscription), any(MessageSendingResult.class)); // then @@ -162,7 +166,7 @@ public void shouldDiscardMessageWhenTTLIsExceeded() { doReturn(failure()).when(messageSender).send(message); // when - sender.sendAsync(message); + sender.sendAsync(message, profiler); // then verify(errorHandler, timeout(1000)).handleDiscarded(eq(message), eq(subscription), any(MessageSendingResult.class)); @@ -181,7 +185,7 @@ public void shouldNotKeepTryingToSendMessageFailedWithStatusCode4xx() { doReturn(failure(403)).doReturn(success()).when(messageSender).send(message); // when - sender.sendAsync(message); + sender.sendAsync(message, profiler); // then verify(errorHandler, timeout(1000)).handleDiscarded(eq(message), eq(subscription), any(MessageSendingResult.class)); @@ -202,7 +206,7 @@ public void shouldKeepTryingToSendMessageFailedWithStatusCode4xxForSubscriptionW doReturn(failure(403)).doReturn(failure(403)).doReturn(failure(403)).doReturn(success()).when(messageSender).send(message); // when - sender.sendAsync(message); + sender.sendAsync(message, profiler); verify(successHandler, timeout(1000)).handleSuccess(eq(message), eq(subscriptionWith4xxRetry), any(MessageSendingResult.class)); // then @@ -227,7 +231,7 @@ public void shouldRetryOn401UnauthorizedForOAuthSecuredSubscription() { doReturn(failure(401)).doReturn(failure(401)).doReturn(success()).when(messageSender).send(message); // when - sender.sendAsync(message); + sender.sendAsync(message, profiler); // then verifyErrorHandlerHandleFailed(message, subscription, expectedNumbersOfFailures); @@ -250,7 +254,7 @@ public void shouldBackoffRetriesWhenEndpointFails() throws InterruptedException doReturn(failure(500)).when(messageSender).send(message); //when - sender.sendAsync(message); + sender.sendAsync(message, profiler); //then Thread.sleep(executionTime); @@ -267,7 +271,7 @@ public void shouldNotRetryOnRetryAfterAboveTtl() { doReturn(backoff(retrySeconds + 1)).doReturn(success()).when(messageSender).send(message); // when - sender.sendAsync(message); + sender.sendAsync(message, profiler); // then verify(errorHandler, timeout(1000)).handleDiscarded(eq(message), eq(subscription), any(MessageSendingResult.class)); @@ -292,7 +296,7 @@ public void shouldDeliverToModifiedEndpoint() { // when sender.updateSubscription(subscriptionWithModfiedEndpoint); - sender.sendAsync(message); + sender.sendAsync(message, profiler); // then verify(otherMessageSender, timeout(1000)).send(message); @@ -313,7 +317,7 @@ public void shouldDeliverToNewSenderAfterModifiedTimeout() { // when sender.updateSubscription(subscriptionWithModifiedTimeout); - sender.sendAsync(message); + sender.sendAsync(message, profiler); // then verify(otherMessageSender, timeout(1000)).send(message); @@ -337,7 +341,7 @@ public void shouldDelaySendingMessageForHalfSecond() { // when long sendingStartTime = System.currentTimeMillis(); - sender.sendAsync(message); + sender.sendAsync(message, profiler); verify(successHandler, timeout(1000)).handleSuccess(eq(message), eq(subscription), any(MessageSendingResult.class)); // then @@ -363,7 +367,7 @@ public void shouldCalculateSendingDelayBasingOnPublishingTimestamp() { // when long sendingStartTime = System.currentTimeMillis(); - sender.sendAsync(message); + sender.sendAsync(message, profiler); verify(successHandler, timeout(500)).handleSuccess(eq(message), eq(subscription), any(MessageSendingResult.class)); // then @@ -386,7 +390,7 @@ public void shouldIncreaseRetryBackoffExponentially() throws InterruptedExceptio ConsumerMessageSender sender = consumerMessageSender(subscription); // when - sender.sendAsync(message); + sender.sendAsync(message, profiler); Thread.sleep(backoff + (long) multiplier * backoff - 100); // then @@ -409,7 +413,7 @@ public void shouldIgnoreExponentialRetryBackoffWithRetryAfter() { ConsumerMessageSender sender = consumerMessageSender(subscription); // when - sender.sendAsync(message); + sender.sendAsync(message, profiler); //then verify(successHandler, timeout(retrySeconds * 1000 * 2 + 500)) @@ -430,7 +434,7 @@ public void shouldIgnoreExponentialRetryBackoffAfterExceededTtl() throws Interru ConsumerMessageSender sender = consumerMessageSender(subscription); // when - sender.sendAsync(message); + sender.sendAsync(message, profiler); Thread.sleep(backoff + (long) multiplier * backoff + 1000); //then diff --git a/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducerIntegrationTest.groovy b/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducerIntegrationTest.groovy index edbab24478..401112312c 100644 --- a/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducerIntegrationTest.groovy +++ b/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducerIntegrationTest.groovy @@ -82,7 +82,7 @@ class LocalDatacenterMessageProducerIntegrationTest extends Specification { KafkaHeaderNameProperties kafkaHeaderNameProperties = new KafkaHeaderNameProperties() @Shared - String datacenter = "dc"; + String datacenter = "dc" @Shared MetricsFacade metricsFacade = new MetricsFacade( @@ -163,7 +163,7 @@ class LocalDatacenterMessageProducerIntegrationTest extends Specification { private static def createTestSubscription(Topic topic, String subscriptionName) { Subscription.create(topic.getQualifiedName(), subscriptionName, null, Subscription.State.PENDING, "test", [:], false, null, null, - null, ContentType.JSON, DeliveryType.SERIAL, [], SubscriptionMode.ANYCAST, [], null, null, false, false, false + null, ContentType.JSON, DeliveryType.SERIAL, [], SubscriptionMode.ANYCAST, [], null, null, false, false, 0, false, false ) } diff --git a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/kafka/service/KafkaConsumerGroupManagerSpec.groovy b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/kafka/service/KafkaConsumerGroupManagerSpec.groovy index 11b514ed95..b43834d8ad 100644 --- a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/kafka/service/KafkaConsumerGroupManagerSpec.groovy +++ b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/kafka/service/KafkaConsumerGroupManagerSpec.groovy @@ -168,7 +168,7 @@ class KafkaConsumerGroupManagerSpec extends Specification { private static def createTestSubscription(Topic topic, String subscriptionName) { Subscription.create(topic.getQualifiedName(), subscriptionName, null, Subscription.State.PENDING, "test", [:], false, null, null, - null, ContentType.JSON, DeliveryType.SERIAL, [], SubscriptionMode.ANYCAST, [], null, null, false, false, false + null, ContentType.JSON, DeliveryType.SERIAL, [], SubscriptionMode.ANYCAST, [], null, null, false, false, 0, false, false ) } diff --git a/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/builder/SubscriptionBuilder.java b/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/builder/SubscriptionBuilder.java index 81b8012f30..63a4ecedff 100644 --- a/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/builder/SubscriptionBuilder.java +++ b/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/builder/SubscriptionBuilder.java @@ -50,6 +50,10 @@ public class SubscriptionBuilder { private boolean http2Enabled = false; + private boolean profilingEnabled = false; + + private long profilingThresholdMs = 0; + private OwnerId owner = new OwnerId("Plaintext", "some team"); private MonitoringDetails monitoringDetails = MonitoringDetails.EMPTY; @@ -126,8 +130,8 @@ public Subscription build() { topicName, name, endpoint, state, description, serialSubscriptionPolicy, trackingEnabled, trackingMode, owner, monitoringDetails, contentType, - filters, mode, headers, metadata, oAuthPolicy, http2Enabled, - attachingIdentityHeadersEnabled, autoDeleteWithTopicEnabled + filters, mode, headers, metadata, oAuthPolicy, http2Enabled, profilingEnabled, + profilingThresholdMs, attachingIdentityHeadersEnabled, autoDeleteWithTopicEnabled ); } else { return Subscription.createBatchSubscription( @@ -196,6 +200,16 @@ public SubscriptionBuilder withHttp2Enabled(boolean http2Enabled) { return this; } + public SubscriptionBuilder withProfilingEnabled(boolean profilingEnabled) { + this.profilingEnabled = profilingEnabled; + return this; + } + + public SubscriptionBuilder withProfilingThresholdMs(long profilingThresholdMs) { + this.profilingThresholdMs = profilingThresholdMs; + return this; + } + public SubscriptionBuilder withOwner(OwnerId owner) { this.owner = owner; return this; diff --git a/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/client/integration/FrontendTestClient.java b/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/client/integration/FrontendTestClient.java index 86964105e9..bb29c99df1 100644 --- a/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/client/integration/FrontendTestClient.java +++ b/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/client/integration/FrontendTestClient.java @@ -40,6 +40,9 @@ public FrontendTestClient(int frontendPort) { this.webTestClient = WebTestClient .bindToServer(new JdkClientHttpConnector()) .baseUrl(frontendContainerUrl) + .codecs(configurer -> configurer + .defaultCodecs() + .maxInMemorySize(16 * 1024 * 1024)) .build(); this.slowTestClient = new FrontendSlowClient("localhost", frontendPort); this.chunkedClient = newClient(new ClientConfig().property(REQUEST_ENTITY_PROCESSING, CHUNKED)); diff --git a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/ConsumerProfilingTest.java b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/ConsumerProfilingTest.java new file mode 100644 index 0000000000..f7c6eb1a42 --- /dev/null +++ b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/ConsumerProfilingTest.java @@ -0,0 +1,244 @@ +package pl.allegro.tech.hermes.integrationtests; + +import ch.qos.logback.classic.Logger; +import ch.qos.logback.classic.spi.ILoggingEvent; +import ch.qos.logback.core.read.ListAppender; +import com.jayway.awaitility.Duration; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.slf4j.LoggerFactory; +import pl.allegro.tech.hermes.api.Subscription; +import pl.allegro.tech.hermes.api.Topic; +import pl.allegro.tech.hermes.consumers.consumer.profiling.ConsumerRun; +import pl.allegro.tech.hermes.consumers.consumer.profiling.DefaultConsumerProfiler; +import pl.allegro.tech.hermes.consumers.consumer.profiling.Measurement; +import pl.allegro.tech.hermes.integrationtests.setup.HermesExtension; +import pl.allegro.tech.hermes.integrationtests.subscriber.TestSubscriber; +import pl.allegro.tech.hermes.integrationtests.subscriber.TestSubscribersExtension; +import pl.allegro.tech.hermes.test.helper.message.TestMessage; + +import java.util.List; + +import static com.jayway.awaitility.Awaitility.waitAtMost; +import static org.assertj.core.api.Assertions.assertThat; +import static pl.allegro.tech.hermes.test.helper.builder.SubscriptionBuilder.subscriptionWithRandomName; +import static pl.allegro.tech.hermes.test.helper.builder.TopicBuilder.topicWithRandomName; + +public class ConsumerProfilingTest { + + @RegisterExtension + public static final HermesExtension hermes = new HermesExtension(); + + @RegisterExtension + public static final TestSubscribersExtension subscribers = new TestSubscribersExtension(); + + private ListAppender listAppender; + + @BeforeEach + void createLogAppender() { + Logger logWatcher = (Logger) LoggerFactory.getLogger(DefaultConsumerProfiler.class); + + listAppender = new ListAppender<>(); + listAppender.start(); + + logWatcher.addAppender(listAppender); + } + + @AfterEach + void teardown() { + ((Logger) LoggerFactory.getLogger(DefaultConsumerProfiler.class)).detachAndStopAllAppenders(); + hermes.clearManagementData(); + } + + @AfterAll + static void teardownClass() { + hermes.clearManagementData(); + } + + @Test + public void shouldNotProfileWhenProfilingIsDisabled() { + // given + TestSubscriber subscriber = subscribers.createSubscriber(); + Topic topic = hermes.initHelper().createTopic(topicWithRandomName().build()); + Subscription subscription = hermes.initHelper().createSubscription(subscriptionWithRandomName(topic.getName(), subscriber.getEndpoint()) + .withProfilingEnabled(false).build()); + TestMessage message = TestMessage.random(); + hermes.api().publishUntilSuccess(topic.getQualifiedName(), message.body()); + + // when + subscriber.waitUntilReceived(message.body()); + + // then + List logsList = listAppender.list.stream() + .filter(log -> log.getFormattedMessage().contains(subscription.getQualifiedName().toString())).toList(); + assertThat(logsList).hasSize(0); + } + + @Test + public void shouldProfileEmptyRun() { + // given + Topic topic = hermes.initHelper().createTopic(topicWithRandomName().build()); + + // when + Subscription subscription = hermes.initHelper().createSubscription(subscriptionWithRandomName(topic.getName()) + .withProfilingEnabled(true).build()); + + // then + waitAtMost(Duration.TEN_SECONDS).until(() -> { + List logsList = listAppender.list.stream() + .filter(log -> log.getFormattedMessage().contains(subscription.getQualifiedName().toString())).toList(); + assertThat(logsList).hasSizeGreaterThan(0); + assertThat(logsList.get(0).getFormattedMessage()).contains( + String.format("Consumer profiler measurements for subscription %s and %s run:", subscription.getQualifiedName(), ConsumerRun.EMPTY), + Measurement.SIGNALS_AND_SEMAPHORE_ACQUIRE, + Measurement.MESSAGE_RECEIVER_NEXT, + Measurement.MESSAGE_CONVERSION, + "partialMeasurements", + Measurement.SIGNALS_INTERRUPT_RUN + ); + }); + } + + @Test + public void shouldNotProfileRunsBelowThreshold() { + // given + TestSubscriber subscriber = subscribers.createSubscriber(); + Topic topic = hermes.initHelper().createTopic(topicWithRandomName().build()); + Subscription subscription = hermes.initHelper().createSubscription(subscriptionWithRandomName(topic.getName(), subscriber.getEndpoint()) + .withProfilingEnabled(true) + .withProfilingThresholdMs(100_000) + .build()); + TestMessage message = TestMessage.random(); + hermes.api().publishUntilSuccess(topic.getQualifiedName(), message.body()); + + // when + subscriber.waitUntilReceived(message.body()); + + // then + List logsList = listAppender.list.stream() + .filter(log -> log.getFormattedMessage().contains(subscription.getQualifiedName().toString())).toList(); + assertThat(logsList).hasSize(0); + } + + @Test + public void shouldProfileSuccessfulMessageProcessing() { + // given + TestSubscriber subscriber = subscribers.createSubscriber(); + Topic topic = hermes.initHelper().createTopic(topicWithRandomName().build()); + Subscription subscription = hermes.initHelper().createSubscription(subscriptionWithRandomName(topic.getName(), subscriber.getEndpoint()) + .withProfilingEnabled(true).build()); + TestMessage message = TestMessage.random(); + hermes.api().publishUntilSuccess(topic.getQualifiedName(), message.body()); + + // when + subscriber.waitUntilReceived(message.body()); + + // then + waitAtMost(Duration.TEN_SECONDS).until(() -> { + List logsList = listAppender.list.stream() + .filter(log -> log.getFormattedMessage().contains(ConsumerRun.DELIVERED.name())).toList(); + assertThat(logsList).hasSizeGreaterThan(0); + assertThat(logsList.get(0).getFormattedMessage()).contains( + String.format("Consumer profiler measurements for subscription %s and %s run:", subscription.getQualifiedName(), ConsumerRun.DELIVERED), + Measurement.SIGNALS_AND_SEMAPHORE_ACQUIRE, + Measurement.MESSAGE_RECEIVER_NEXT, + Measurement.MESSAGE_CONVERSION, + Measurement.OFFER_INFLIGHT_OFFSET, + Measurement.TRACKERS_LOG_INFLIGHT, + Measurement.ACQUIRE_RATE_LIMITER, + Measurement.MESSAGE_SENDER_SEND, + Measurement.HANDLERS, + "partialMeasurements", + Measurement.SIGNALS_INTERRUPT_RUN + ); + }); + } + + @Test + public void shouldProfileDiscardedMessageProcessing() { + // given + Topic topic = hermes.initHelper().createTopic(topicWithRandomName().build()); + TestSubscriber subscriber = subscribers.createSubscriber(400); + Subscription subscription = hermes.initHelper().createSubscription(subscriptionWithRandomName(topic.getName(), subscriber.getEndpoint()) + .withProfilingEnabled(true) + .build()); + TestMessage message = TestMessage.random(); + hermes.api().publishUntilSuccess(topic.getQualifiedName(), message.body()); + + // when + subscriber.waitUntilReceived(message.body()); + + // then + waitAtMost(Duration.TEN_SECONDS).until(() -> { + List logsList = listAppender.list.stream() + .filter(log -> log.getFormattedMessage().contains(ConsumerRun.DISCARDED.name())).toList(); + assertThat(logsList).hasSizeGreaterThan(0); + assertThat(logsList.get(0).getFormattedMessage()).contains( + String.format("Consumer profiler measurements for subscription %s and %s run:", subscription.getQualifiedName(), ConsumerRun.DISCARDED), + Measurement.SIGNALS_AND_SEMAPHORE_ACQUIRE, + Measurement.MESSAGE_RECEIVER_NEXT, + Measurement.MESSAGE_CONVERSION, + Measurement.OFFER_INFLIGHT_OFFSET, + Measurement.TRACKERS_LOG_INFLIGHT, + Measurement.ACQUIRE_RATE_LIMITER, + Measurement.MESSAGE_SENDER_SEND, + Measurement.HANDLERS, + "partialMeasurements", + Measurement.SIGNALS_INTERRUPT_RUN + ); + }); + } + + @Test + public void shouldProfileRetriedMessageProcessing() { + // given + Topic topic = hermes.initHelper().createTopic(topicWithRandomName().build()); + TestMessage message = TestMessage.random(); + TestSubscriber subscriber = subscribers.createSubscriberWithRetry(message.body(), 1); + Subscription subscription = hermes.initHelper().createSubscription(subscriptionWithRandomName(topic.getName(), subscriber.getEndpoint()) + .withProfilingEnabled(true) + .build()); + hermes.api().publishUntilSuccess(topic.getQualifiedName(), message.body()); + + // when + subscriber.waitUntilReceived(Duration.FIVE_SECONDS, 2); + + // then + waitAtMost(Duration.TEN_SECONDS).until(() -> { + List retriedLogsList = listAppender.list.stream() + .filter(log -> log.getFormattedMessage().contains(ConsumerRun.RETRIED.name())).toList(); + assertThat(retriedLogsList).hasSizeGreaterThan(0); + assertThat(retriedLogsList.get(0).getFormattedMessage()).contains( + String.format("Consumer profiler measurements for subscription %s and %s run:", subscription.getQualifiedName(), ConsumerRun.RETRIED), + Measurement.SIGNALS_AND_SEMAPHORE_ACQUIRE, + Measurement.MESSAGE_RECEIVER_NEXT, + Measurement.MESSAGE_CONVERSION, + Measurement.OFFER_INFLIGHT_OFFSET, + Measurement.TRACKERS_LOG_INFLIGHT, + Measurement.ACQUIRE_RATE_LIMITER, + Measurement.MESSAGE_SENDER_SEND, + Measurement.HANDLERS, + "partialMeasurements", + Measurement.SIGNALS_INTERRUPT_RUN + ); + }); + + + // and + waitAtMost(Duration.TEN_SECONDS).until(() -> { + List processedLogsList = listAppender.list.stream() + .filter(log -> log.getFormattedMessage().contains(ConsumerRun.DELIVERED.name())).toList(); + assertThat(processedLogsList).hasSizeGreaterThan(0); + assertThat(processedLogsList.get(0).getFormattedMessage()).contains( + String.format("Consumer profiler measurements for subscription %s and %s run:", subscription.getQualifiedName(), ConsumerRun.DELIVERED), + Measurement.SCHEDULE_RESEND, + Measurement.MESSAGE_SENDER_SEND, + Measurement.HANDLERS, + "retryDelayMillis 1000" + ); + }); + } +} From 9f8b00418b9363f3f5410ffd6fd6a27951195912 Mon Sep 17 00:00:00 2001 From: Mateusz <76775507+szczygiel-m@users.noreply.github.com> Date: Fri, 24 May 2024 17:55:17 +0200 Subject: [PATCH 54/87] Fixed dead link (#1864) --- docs/docs/user/java-client.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/docs/user/java-client.md b/docs/docs/user/java-client.md index 57e78b766b..942c9675a7 100644 --- a/docs/docs/user/java-client.md +++ b/docs/docs/user/java-client.md @@ -19,7 +19,7 @@ At the moment there are four implementations of `HermesSender`: for asynchronous transmission * **WebClientHermesSender** - for services using [Spring WebFlux](https://docs.spring.io/spring-framework/docs/current/reference/html/web-reactive.html); uses [WebClient](https://docs.spring.io/spring-framework/docs/current/javadoc-api/org/springframework/web/reactive/function/client/WebClient.html) -* **JerseyHermesSender** - recommended for services using [Jersey]() +* **JerseyHermesSender** - recommended for services using [Jersey]() * **OkHttpHermesSender** - supports both HTTP/1.1 and HTTP/2 protocols, uses [OkHttp3 client](http://square.github.io/okhttp/) From 1ebd3e4fd9077ff9d42ee6683c86fbca87e79935 Mon Sep 17 00:00:00 2001 From: Maciej Moscicki Date: Fri, 24 May 2024 18:13:03 +0200 Subject: [PATCH 55/87] add support for debugging filters with all unicode characters (#1856) --- .../useSubscriptionFiltersDebug.ts | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/hermes-console/src/composables/subscription/use-subscription-filters-debug/useSubscriptionFiltersDebug.ts b/hermes-console/src/composables/subscription/use-subscription-filters-debug/useSubscriptionFiltersDebug.ts index 43bc147515..35f7045596 100644 --- a/hermes-console/src/composables/subscription/use-subscription-filters-debug/useSubscriptionFiltersDebug.ts +++ b/hermes-console/src/composables/subscription/use-subscription-filters-debug/useSubscriptionFiltersDebug.ts @@ -35,6 +35,15 @@ const toFiltersJSON = ( }; }; +// https://stackoverflow.com/a/30106551 +function b64EncodeUnicode(str: string): string { + return btoa( + encodeURIComponent(str).replace(/%([0-9A-F]{2})/g, function (match, p1) { + return String.fromCharCode(parseInt(p1, 16)); + }), + ); +} + export function useSubscriptionFiltersDebug(): UseSubscriptionFiltersDebug { const notificationStore = useNotificationsStore(); const status: Ref = ref(); @@ -52,7 +61,7 @@ export function useSubscriptionFiltersDebug(): UseSubscriptionFiltersDebug { ); const response = ( await verifyFilters(topicName, { - message: btoa(message), + message: b64EncodeUnicode(message), filters: filtersJSON, }) ).data; From 7c2e3ffdbc065af5156c011969cd02263c32c9e6 Mon Sep 17 00:00:00 2001 From: Maciej Moscicki Date: Fri, 24 May 2024 20:20:33 +0200 Subject: [PATCH 56/87] Remove graphite and dropwizard support (#1847) * remove graphite max rate metrics * remove graphite offset commit metrics * remove workload and consumer sender graphite metrics * remove deserialization graphite metrics * remove consumer graphite metrics * remove undelivered messages graphite metrics * remove schema client graphite metrics * remove executor graphite metrics * remove producer graphite metrics * remove persistent buffer graphite metrics * remove tracker elasticsearch graphite metrics * remove subscription graphite metrics * remove topic graphite metrics * remove unused graphite metric * fix dropwizard based tests * remove consumers and frontend graphite config * remove graphite client * remove graphite dependencies * remove graphite tests * cleanup paths compiler * remove metric registry * remove dropwizard from hermes-client * lint * lint * use shared throughput meters * fix executor metrics test * lint * remove graphite from docs and configs --------- Co-authored-by: Mateusz <76775507+szczygiel-m@users.noreply.github.com> --- docker/docker-compose.yml | 19 -- docker/latest/consumers/consumers.yaml | 5 - docker/latest/frontend/frontend.yaml | 5 - docker/latest/management/management.yaml | 5 - docs/docs/overview/architecture.md | 2 +- docs/docs/quickstart.md | 3 +- .../environment/HermesPublisher.java | 7 +- .../environment/HermesServerEnvironment.java | 9 +- .../environment/HermesServerFactory.java | 6 +- .../environment/InMemoryTopicsCache.java | 6 +- hermes-client/build.gradle | 2 - .../metrics/DropwizardMetricsProvider.java | 40 --- .../client/HermesClientMetricsTest.groovy | 207 ------------- ...esClientMicrometerTaggedMetricsTest.groovy | 2 +- .../ReactiveHermesClientMetricsTest.groovy | 205 ------------- hermes-common/build.gradle | 4 +- .../di/factories/GraphiteParameters.java | 10 - .../di/factories/MetricRegistryFactory.java | 107 ------- .../factories/MetricRegistryParameters.java | 16 - .../hermes/common/metric/ConsumerMetrics.java | 32 +- .../common/metric/ConsumerSenderMetrics.java | 10 +- .../tech/hermes/common/metric/Counters.java | 17 -- .../common/metric/DeserializationMetrics.java | 36 +-- .../hermes/common/metric/GaugeRegistrar.java | 24 +- .../tech/hermes/common/metric/Gauges.java | 20 -- .../hermes/common/metric/HermesMetrics.java | 274 ------------------ .../tech/hermes/common/metric/Histograms.java | 6 - .../hermes/common/metric/MaxRateMetrics.java | 20 +- .../tech/hermes/common/metric/Meters.java | 33 --- ...tricRegistryWithHdrHistogramReservoir.java | 19 -- .../hermes/common/metric/MetricsFacade.java | 71 +---- .../common/metric/OffsetCommitsMetrics.java | 19 +- .../metric/PersistentBufferMetrics.java | 5 +- .../hermes/common/metric/ProducerMetrics.java | 22 +- .../common/metric/SchemaClientMetrics.java | 13 +- .../metric/SubscriptionHermesCounter.java | 20 +- .../common/metric/SubscriptionMetrics.java | 80 ++--- .../tech/hermes/common/metric/Timers.java | 33 --- .../hermes/common/metric/TopicMetrics.java | 65 ++--- .../metric/TrackerElasticSearchMetrics.java | 41 +-- .../metric/UndeliveredMessagesMetrics.java | 13 +- .../hermes/common/metric/WorkloadMetrics.java | 50 +--- .../zookeeper/ZookeeperCounterReporter.java | 4 +- .../common/schema/RawSchemaClientFactory.java | 1 - ...edExecutorServiceFactoryMetricsTest.groovy | 6 +- ...dMetricsTrackingRawSchemaClientTest.groovy | 15 +- .../ZookeeperUndeliveredMessageLogTest.java | 11 +- .../wrapper/MessageContentWrapperTest.java | 26 +- .../ZookeeperCounterReporterTest.java | 6 +- .../consumers/config/CommonConfiguration.java | 23 +- .../consumers/config/GraphiteProperties.java | 41 --- .../consumers/config/MetricsProperties.java | 58 +--- .../resources/application-integration.yaml | 6 - .../src/main/resources/application-local.yaml | 5 - .../HttpClientConnectionMonitoringTest.groovy | 11 +- .../ConsumerProcessSupervisorTest.groovy | 17 +- .../WeightedWorkBalancingListenerTest.groovy | 12 +- .../topic/NotificationBasedTopicsCache.java | 8 +- .../frontend/config/CommonConfiguration.java | 24 +- .../config/FrontendConfiguration.java | 4 +- .../FrontendPublishingConfiguration.java | 11 +- .../frontend/config/GraphiteProperties.java | 41 --- .../config/MetricRegistryProperties.java | 58 +--- .../hermes/frontend/metric/CachedTopic.java | 25 +- .../frontend/metric/ThroughputMeter.java | 36 +++ .../frontend/metric/ThroughputRegistry.java | 32 ++ .../handlers/ThroughputLimiterFactory.java | 10 +- .../resources/application-integration.yaml | 6 - .../src/main/resources/application-local.yaml | 5 - ...enterMessageProducerIntegrationTest.groovy | 8 +- .../metric/ThroughputRegistryTest.groovy | 65 +++++ .../server/CachedTopicsTestHelper.groovy | 13 +- .../buffer/BackupMessagesLoaderTest.java | 3 +- .../LocalDatacenterMessageProducerTest.java | 15 +- .../ExternalMonitoringConfiguration.java | 21 +- .../GraphiteMonitoringMetricsProperties.java | 14 - .../config/ManagementConfiguration.java | 18 +- ...nitoringClientPropertiesConfiguration.java | 6 - .../graphite/CachingGraphiteClient.java | 62 ---- .../graphite/GraphiteClient.java | 8 - .../graphite/GraphiteResponse.java | 29 -- .../graphite/RestTemplateGraphiteClient.java | 95 ------ .../graphite/CachingGraphiteClientTest.groovy | 54 ---- .../RestTemplateGraphiteClientTest.groovy | 101 ------- hermes-metrics/build.gradle | 2 +- .../metrics/DefaultHermesHistogram.java | 15 +- .../tech/hermes/metrics/HermesTimer.java | 16 +- .../hermes/metrics/HermesTimerContext.java | 9 +- .../tech/hermes/metrics/PathContext.java | 108 +------ .../tech/hermes/metrics/PathsCompiler.java | 16 - .../counters/DefaultHermesCounter.java | 6 +- .../metrics/counters/HermesCounters.java | 10 +- .../counters/MeterBackedHermesCounter.java | 26 -- .../PathsCompilerTest.groovy | 6 +- .../metrics/TestMetricsFacadeFactory.java | 8 +- .../ElasticsearchQueueCommitter.java | 1 - ...nsumersElasticsearchLogRepositoryTest.java | 6 +- ...rontendElasticsearchLogRepositoryTest.java | 6 +- .../ElasticsearchLogRepositoryTest.java | 6 +- .../MultiElasticsearchLogRepositoryTest.java | 6 +- .../tech/hermes/tracker/QueueCommitter.java | 1 - .../setup/HermesConsumersTestApp.java | 1 - 102 files changed, 382 insertions(+), 2464 deletions(-) delete mode 100644 hermes-client/src/main/java/pl/allegro/tech/hermes/client/metrics/DropwizardMetricsProvider.java delete mode 100644 hermes-client/src/test/groovy/pl/allegro/tech/hermes/client/HermesClientMetricsTest.groovy delete mode 100644 hermes-client/src/test/groovy/pl/allegro/tech/hermes/client/ReactiveHermesClientMetricsTest.groovy delete mode 100644 hermes-common/src/main/java/pl/allegro/tech/hermes/common/di/factories/GraphiteParameters.java delete mode 100644 hermes-common/src/main/java/pl/allegro/tech/hermes/common/di/factories/MetricRegistryFactory.java delete mode 100644 hermes-common/src/main/java/pl/allegro/tech/hermes/common/di/factories/MetricRegistryParameters.java delete mode 100644 hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/Counters.java delete mode 100644 hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/HermesMetrics.java delete mode 100644 hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/MetricRegistryWithHdrHistogramReservoir.java delete mode 100644 hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/Timers.java delete mode 100644 hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/GraphiteProperties.java delete mode 100644 hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/GraphiteProperties.java create mode 100644 hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/metric/ThroughputMeter.java create mode 100644 hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/metric/ThroughputRegistry.java create mode 100644 hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/publishing/metric/ThroughputRegistryTest.groovy delete mode 100644 hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/GraphiteMonitoringMetricsProperties.java delete mode 100644 hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/graphite/CachingGraphiteClient.java delete mode 100644 hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/graphite/GraphiteClient.java delete mode 100644 hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/graphite/GraphiteResponse.java delete mode 100644 hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/graphite/RestTemplateGraphiteClient.java delete mode 100644 hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/graphite/CachingGraphiteClientTest.groovy delete mode 100644 hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/graphite/RestTemplateGraphiteClientTest.groovy delete mode 100644 hermes-metrics/src/main/java/pl/allegro/tech/hermes/metrics/counters/MeterBackedHermesCounter.java diff --git a/docker/docker-compose.yml b/docker/docker-compose.yml index dc2d859510..9f07d83fc8 100644 --- a/docker/docker-compose.yml +++ b/docker/docker-compose.yml @@ -30,19 +30,6 @@ services: - kafka_data:/var/lib/kafka/data - kafka_secrets:/etc/kafka/secrets - graphite: - image: graphiteapp/graphite-statsd:1.1.3 - ports: - - '2003-2004:2003-2004' - - '2023-2024:2023-2024' - - '8125:8125/udp' - - '8126:8126' - - '8082:80' - volumes: - - graphite_conf:/opt/graphite/conf - - graphite_data:/opt/graphite/storage - - statsd_data:/opt/statsd - frontend: build: context: ../ @@ -52,7 +39,6 @@ services: depends_on: - zk - kafka - - graphite consumers: build: @@ -61,7 +47,6 @@ services: depends_on: - zk - kafka - - graphite management: build: @@ -72,7 +57,6 @@ services: depends_on: - zk - kafka - - graphite schema-registry: image: "confluentinc/cp-schema-registry:${CONFLUENT_IMAGES_TAG}" @@ -87,9 +71,6 @@ services: - "8081:8081" volumes: - graphite_conf: - graphite_data: - statsd_data: zk_secrets: zk_data: zk_log: diff --git a/docker/latest/consumers/consumers.yaml b/docker/latest/consumers/consumers.yaml index 76af83d9b5..f64008630f 100644 --- a/docker/latest/consumers/consumers.yaml +++ b/docker/latest/consumers/consumers.yaml @@ -8,11 +8,6 @@ consumer: clusters: - datacenter: "dc" brokerList: "kafka:29092" - graphite: - host: "graphite" - metrics: - metric-registry: - graphiteReporterEnabled: true workload: consumerPerSubscription: 1 schema: diff --git a/docker/latest/frontend/frontend.yaml b/docker/latest/frontend/frontend.yaml index 2ccd0f3432..0ba33ab769 100644 --- a/docker/latest/frontend/frontend.yaml +++ b/docker/latest/frontend/frontend.yaml @@ -8,11 +8,6 @@ frontend: clusters: - datacenter: "dc" brokerList: "kafka:29092" - graphite: - host: "graphite" - metrics: - metric-registry: - graphiteReporterEnabled: true schema: cache: refreshAfterWrite: 1m diff --git a/docker/latest/management/management.yaml b/docker/latest/management/management.yaml index 0953e3660d..a678f65d33 100644 --- a/docker/latest/management/management.yaml +++ b/docker/latest/management/management.yaml @@ -17,11 +17,6 @@ kafka: connectionTimeout: 3000 bootstrapKafkaServer: kafka:29092 -graphite: - client: - enabled: true - externalMonitoringUrl: graphite:8082 - server: port: 8090 diff --git a/docs/docs/overview/architecture.md b/docs/docs/overview/architecture.md index 7726745068..6d647e64d2 100644 --- a/docs/docs/overview/architecture.md +++ b/docs/docs/overview/architecture.md @@ -16,7 +16,7 @@ Hermes integrates with multiple systems, each having different role. * **Message Store** - stores and routes messages, current implementation: Kafka * **Metadata Store** - shared metadata storage for all Hermes modules, current implementation: Zookeeper -* **Metrics Store** *[optional]* - stores metrics gathered by Hermes, current implementation: Graphite +* **Metrics Store** *[optional]* - stores metrics gathered by Hermes, currently Hermes exposes metrics in Prometheus format * **Tracking Store** *[optional]* - stores tracking (message trace) information, current implementation: ElasticSearch ## Message flow diff --git a/docs/docs/quickstart.md b/docs/docs/quickstart.md index c31db5a01a..3ad00ea47b 100644 --- a/docs/docs/quickstart.md +++ b/docs/docs/quickstart.md @@ -56,7 +56,7 @@ image: allegro/hermes-management:hermes-[specific version tag] ## Development The default `docker-compose` setup will start all hermes modules (consumers, frontend, management), together -with its dependencies (Kafka, ZooKeeper, Graphite, Schema Registry). To run a specific module with gradle/IntelliJ, +with its dependencies (Kafka, ZooKeeper, Schema Registry). To run a specific module with gradle/IntelliJ, just comment out the module in `services` section of the `docker-compose.yml` file, and start the java process locally: `./gradlew -p hermes-frontend run` @@ -175,7 +175,6 @@ management: depends_on: - zk - kafka - - graphite [...] ``` diff --git a/hermes-benchmark/src/jmh/java/pl/allegro/tech/hermes/benchmark/environment/HermesPublisher.java b/hermes-benchmark/src/jmh/java/pl/allegro/tech/hermes/benchmark/environment/HermesPublisher.java index 610eafea81..65d758873d 100644 --- a/hermes-benchmark/src/jmh/java/pl/allegro/tech/hermes/benchmark/environment/HermesPublisher.java +++ b/hermes-benchmark/src/jmh/java/pl/allegro/tech/hermes/benchmark/environment/HermesPublisher.java @@ -1,6 +1,5 @@ package pl.allegro.tech.hermes.benchmark.environment; -import com.codahale.metrics.MetricRegistry; import org.apache.http.HttpEntity; import org.apache.http.HttpResponse; import org.apache.http.client.config.CookieSpecs; @@ -28,14 +27,12 @@ public class HermesPublisher { private static final Logger logger = LoggerFactory.getLogger(HermesPublisher.class); private final CloseableHttpAsyncClient httpClient; - private final MetricRegistry metricRegistry; private final URI targetUrl; private final HttpEntity body; - public HermesPublisher(int maxConnectionsPerRoute, String targetUrl, String body, MetricRegistry metricRegistry) + public HermesPublisher(int maxConnectionsPerRoute, String targetUrl, String body) throws IOReactorException, UnsupportedEncodingException { this.targetUrl = URI.create(targetUrl); - this.metricRegistry = metricRegistry; RequestConfig requestConfig = RequestConfig.custom() .setCookieSpec(CookieSpecs.IGNORE_COOKIES) @@ -71,9 +68,7 @@ public int publish() { try { Future future = httpClient.execute(httpPost, null); response = future.get().getStatusLine().getStatusCode(); - metricRegistry.counter("response." + response).inc(); } catch (RuntimeException | InterruptedException | ExecutionException exception) { - metricRegistry.counter("client.exceptions").inc(); logger.error("Client exception", exception); } return response; diff --git a/hermes-benchmark/src/jmh/java/pl/allegro/tech/hermes/benchmark/environment/HermesServerEnvironment.java b/hermes-benchmark/src/jmh/java/pl/allegro/tech/hermes/benchmark/environment/HermesServerEnvironment.java index 52aece4222..2f987a610f 100644 --- a/hermes-benchmark/src/jmh/java/pl/allegro/tech/hermes/benchmark/environment/HermesServerEnvironment.java +++ b/hermes-benchmark/src/jmh/java/pl/allegro/tech/hermes/benchmark/environment/HermesServerEnvironment.java @@ -1,6 +1,5 @@ package pl.allegro.tech.hermes.benchmark.environment; -import com.codahale.metrics.MetricRegistry; import org.apache.commons.io.IOUtils; import org.openjdk.jmh.annotations.Level; import org.openjdk.jmh.annotations.Scope; @@ -23,7 +22,6 @@ public class HermesServerEnvironment { public static final String BENCHMARK_TOPIC = "bench.topic"; private HermesPublisher publisher; - private MetricRegistry metricRegistry; private HermesServer hermesServer; @@ -39,10 +37,9 @@ public void setupEnvironment() throws Exception { @Setup(Level.Trial) public void setupPublisher() throws Exception { - metricRegistry = new MetricRegistry(); String messageBody = loadMessageResource("completeMessage"); - publisher = new HermesPublisher(MAX_CONNECTIONS_PER_ROUTE, "http://localhost:8080/topics/" + BENCHMARK_TOPIC, messageBody, metricRegistry); + publisher = new HermesPublisher(MAX_CONNECTIONS_PER_ROUTE, "http://localhost:8080/topics/" + BENCHMARK_TOPIC, messageBody); } @TearDown(Level.Trial) @@ -52,7 +49,6 @@ public void shutdownServers() throws Exception { @TearDown(Level.Trial) public void shutdownPublisherAndReportMetrics() throws Exception { - reportMetrics(); publisher.stop(); } @@ -65,7 +61,4 @@ public static String loadMessageResource(String name) throws IOException { .getResourceAsStream(String.format("/message/%s.json", name)))); } - private void reportMetrics() { - metricRegistry.getCounters().forEach((key, value) -> logger.info(key + ": " + value.getCount())); - } } diff --git a/hermes-benchmark/src/jmh/java/pl/allegro/tech/hermes/benchmark/environment/HermesServerFactory.java b/hermes-benchmark/src/jmh/java/pl/allegro/tech/hermes/benchmark/environment/HermesServerFactory.java index c47b324f7b..47c5dbca6f 100644 --- a/hermes-benchmark/src/jmh/java/pl/allegro/tech/hermes/benchmark/environment/HermesServerFactory.java +++ b/hermes-benchmark/src/jmh/java/pl/allegro/tech/hermes/benchmark/environment/HermesServerFactory.java @@ -1,12 +1,10 @@ package pl.allegro.tech.hermes.benchmark.environment; -import com.codahale.metrics.MetricRegistry; import com.fasterxml.jackson.databind.ObjectMapper; import io.micrometer.core.instrument.simple.SimpleMeterRegistry; import io.undertow.server.HttpHandler; import pl.allegro.tech.hermes.api.Topic; import pl.allegro.tech.hermes.common.message.wrapper.AvroMessageContentWrapper; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.frontend.cache.topic.TopicsCache; import pl.allegro.tech.hermes.frontend.config.HTTPHeadersProperties; @@ -28,7 +26,6 @@ import pl.allegro.tech.hermes.frontend.readiness.HealthCheckService; import pl.allegro.tech.hermes.frontend.server.HermesServer; import pl.allegro.tech.hermes.frontend.validator.MessageValidators; -import pl.allegro.tech.hermes.metrics.PathsCompiler; import pl.allegro.tech.hermes.schema.DirectCompiledSchemaRepository; import pl.allegro.tech.hermes.schema.DirectSchemaVersionsRepository; import pl.allegro.tech.hermes.schema.RawSchemaClient; @@ -52,8 +49,7 @@ class HermesServerFactory { static HermesServer provideHermesServer() throws IOException { ThroughputLimiter throughputLimiter = (exampleTopic, throughput) -> quotaConfirmed(); - HermesMetrics hermesMetrics = new HermesMetrics(new MetricRegistry(), new PathsCompiler("")); - MetricsFacade metricsFacade = new MetricsFacade(new SimpleMeterRegistry(), hermesMetrics); + MetricsFacade metricsFacade = new MetricsFacade(new SimpleMeterRegistry()); TopicsCache topicsCache = new InMemoryTopicsCache(metricsFacade, topic); BrokerMessageProducer brokerMessageProducer = new InMemoryBrokerMessageProducer(); RawSchemaClient rawSchemaClient = new InMemorySchemaClient(topic.getName(), loadMessageResource("schema"), 1, 1); diff --git a/hermes-benchmark/src/jmh/java/pl/allegro/tech/hermes/benchmark/environment/InMemoryTopicsCache.java b/hermes-benchmark/src/jmh/java/pl/allegro/tech/hermes/benchmark/environment/InMemoryTopicsCache.java index d673c51187..d7425b67d5 100644 --- a/hermes-benchmark/src/jmh/java/pl/allegro/tech/hermes/benchmark/environment/InMemoryTopicsCache.java +++ b/hermes-benchmark/src/jmh/java/pl/allegro/tech/hermes/benchmark/environment/InMemoryTopicsCache.java @@ -1,13 +1,14 @@ package pl.allegro.tech.hermes.benchmark.environment; +import com.codahale.metrics.MetricRegistry; import pl.allegro.tech.hermes.api.Topic; import pl.allegro.tech.hermes.common.kafka.KafkaTopic; import pl.allegro.tech.hermes.common.kafka.KafkaTopicName; import pl.allegro.tech.hermes.common.kafka.KafkaTopics; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.frontend.cache.topic.TopicsCache; import pl.allegro.tech.hermes.frontend.metric.CachedTopic; +import pl.allegro.tech.hermes.frontend.metric.ThroughputRegistry; import java.util.List; import java.util.Optional; @@ -17,12 +18,14 @@ class InMemoryTopicsCache implements TopicsCache { private final MetricsFacade metricsFacade; private final KafkaTopics kafkaTopics; private final Topic topic; + private final ThroughputRegistry throughputRegistry; InMemoryTopicsCache(MetricsFacade metricsFacade, Topic topic) { this.metricsFacade = metricsFacade; this.topic = topic; this.kafkaTopics = new KafkaTopics(new KafkaTopic(KafkaTopicName.valueOf(topic.getQualifiedName()), topic.getContentType())); + this.throughputRegistry = new ThroughputRegistry(metricsFacade, new MetricRegistry()); } @Override @@ -32,6 +35,7 @@ public Optional getTopic(String qualifiedTopicName) { new CachedTopic( topic, metricsFacade, + throughputRegistry, kafkaTopics ) ); diff --git a/hermes-client/build.gradle b/hermes-client/build.gradle index c8dd30dfe6..77f46f6722 100644 --- a/hermes-client/build.gradle +++ b/hermes-client/build.gradle @@ -3,7 +3,6 @@ plugins { } dependencies { - compileOnly group: 'io.dropwizard.metrics', name: 'metrics-core', version: versions.dropwizard_metrics compileOnly group: 'io.micrometer', name: 'micrometer-core', version: versions.micrometer_metrics compileOnly group: 'org.glassfish.jersey.core', name: 'jersey-client', version: versions.jersey compileOnly group: 'org.glassfish.jersey.inject', name: 'jersey-hk2', version: versions.jersey @@ -20,7 +19,6 @@ dependencies { testImplementation group: 'javax.servlet', name: 'javax.servlet-api', version: '3.1.0' testImplementation group: 'com.jayway.jsonpath', name: 'json-path', version: '2.5.0' - testImplementation group: 'io.dropwizard.metrics', name: 'metrics-core', version: versions.dropwizard_metrics testImplementation group: 'io.micrometer', name: 'micrometer-core', version: versions.micrometer_metrics testImplementation group: 'org.glassfish.jersey.core', name: 'jersey-client', version: versions.jersey testImplementation group: 'org.glassfish.jersey.inject', name: 'jersey-hk2', version: versions.jersey diff --git a/hermes-client/src/main/java/pl/allegro/tech/hermes/client/metrics/DropwizardMetricsProvider.java b/hermes-client/src/main/java/pl/allegro/tech/hermes/client/metrics/DropwizardMetricsProvider.java deleted file mode 100644 index ded184d90e..0000000000 --- a/hermes-client/src/main/java/pl/allegro/tech/hermes/client/metrics/DropwizardMetricsProvider.java +++ /dev/null @@ -1,40 +0,0 @@ -package pl.allegro.tech.hermes.client.metrics; - -import com.codahale.metrics.MetricRegistry; - -import java.util.Map; -import java.util.concurrent.TimeUnit; - -public class DropwizardMetricsProvider implements MetricsProvider { - - private final MetricRegistry metrics; - - - public DropwizardMetricsProvider(MetricRegistry metrics) { - this.metrics = metrics; - } - - @Override - public void counterIncrement(String topic, String key) { - metrics.counter(prefix + topic + "." + key).inc(); - } - - @Override - public void counterIncrement(String topic, String key, Map tags) { - counterIncrement(topic, buildCounterName(key, tags)); - } - - @Override - public void timerRecord(String topic, String key, long duration, TimeUnit unit) { - metrics.timer(prefix + topic + "." + key).update(duration, unit); - } - - @Override - public void histogramUpdate(String topic, String key, int value) { - metrics.histogram(prefix + topic + "." + key).update(value); - } - - private String buildCounterName(String key, Map tags) { - return key + "." + String.join(".", tags.values()); - } -} diff --git a/hermes-client/src/test/groovy/pl/allegro/tech/hermes/client/HermesClientMetricsTest.groovy b/hermes-client/src/test/groovy/pl/allegro/tech/hermes/client/HermesClientMetricsTest.groovy deleted file mode 100644 index c608c1c5cc..0000000000 --- a/hermes-client/src/test/groovy/pl/allegro/tech/hermes/client/HermesClientMetricsTest.groovy +++ /dev/null @@ -1,207 +0,0 @@ -package pl.allegro.tech.hermes.client - -import com.codahale.metrics.MetricRegistry -import pl.allegro.tech.hermes.client.metrics.DropwizardMetricsProvider -import pl.allegro.tech.hermes.client.metrics.MetricsProvider -import spock.lang.Specification - -import java.time.Duration -import java.time.temporal.ChronoUnit -import java.util.concurrent.CompletableFuture - -import static java.util.concurrent.CompletableFuture.completedFuture -import static pl.allegro.tech.hermes.client.HermesClientBuilder.hermesClient - -class HermesClientMetricsTest extends Specification { - - private MetricRegistry metrics = new MetricRegistry() - private MetricsProvider metricsProvider = new DropwizardMetricsProvider(metrics) - - def "should measure publish latency"() { - given: - HermesClient client = hermesClient(delayedHermesSender(Duration.ofMillis(100))) - .withRetrySleep(0) - .withMetrics(metricsProvider).build() - - when: - client.publish("com.group.topic", "123").join() - - then: - metrics.counter("hermes-client.com_group.topic.status.201").count == 1 - metrics.timer("hermes-client.com_group.topic.latency").getSnapshot().getMax() >= Duration.ofMillis(100).get(ChronoUnit.NANOS) - } - - def "should close timer on exceptional completion and log failure metric"() { - given: - HermesClient client = hermesClient({uri, msg -> failingFuture(new RuntimeException())}) - .withRetrySleep(0) - .withRetries(3) - .withMetrics(metricsProvider).build() - - when: - silence({ client.publish("com.group.topic", "123").join() }) - - then: - metrics.counter("hermes-client.com_group.topic.failure").count == 4 - metrics.timers.containsKey("hermes-client.com_group.topic.latency") - - metrics.counter("hermes-client.com_group.topic.publish.failure").count == 4 - metrics.counter("hermes-client.com_group.topic.publish.finally.failure").count == 1 - metrics.counter("hermes-client.com_group.topic.publish.retry.failure").count == 3 - } - - def "should update max retries exceeded metric"() { - given: - HermesClient client = hermesClient({uri, msg -> failingFuture(new RuntimeException())}) - .withRetrySleep(0) - .withRetries(3) - .withMetrics(metricsProvider).build() - - when: - silence({ client.publish("com.group.topic", "123").join() }) - - then: - metrics.counter("hermes-client.com_group.topic.failure").count == 4 - metrics.counter("hermes-client.com_group.topic.retries.count").count == 3 - metrics.counter("hermes-client.com_group.topic.retries.exhausted").count == 1 - metrics.counter("hermes-client.com_group.topic.retries.success").count == 0 - metrics.histogram("hermes-client.com_group.topic.retries.attempts").getSnapshot().size() == 0 - metrics.timers.containsKey("hermes-client.com_group.topic.latency") - - metrics.counter("hermes-client.com_group.topic.publish.finally.success").count == 0 - metrics.counter("hermes-client.com_group.topic.publish.finally.failure").count == 1 - metrics.counter("hermes-client.com_group.topic.publish.failure").count == 4 - metrics.counter("hermes-client.com_group.topic.publish.attempt").count == 1 - metrics.counter("hermes-client.com_group.topic.publish.retry.success").count == 0 - metrics.counter("hermes-client.com_group.topic.publish.retry.failure").count == 3 - metrics.counter("hermes-client.com_group.topic.publish.retry.attempt").count == 1 - } - - def "should update retries metrics"() { - given: - HermesClient client1 = hermesClient(failingHermesSender(2)) - .withRetrySleep(0) - .withRetries(4) - .withMetrics(metricsProvider).build() - - HermesClient client2 = hermesClient(failingHermesSender(5)) - .withRetrySleep(0) - .withRetries(6) - .withMetrics(metricsProvider).build() - - HermesClient client3 = hermesClient(failingHermesSender(3)) - .withRetrySleep(0) - .withRetries(2) - .withMetrics(metricsProvider).build() - - when: - silence({ client1.publish("com.group.topic", "123").join() }) - silence({ client2.publish("com.group.topic", "456").join() }) - silence({ client3.publish("com.group.topic", "789").join() }) - - then: - metrics.counter("hermes-client.com_group.topic.failure").count == 10 - metrics.counter("hermes-client.com_group.topic.retries.exhausted").count == 1 - metrics.counter("hermes-client.com_group.topic.retries.success").count == 2 - metrics.counter("hermes-client.com_group.topic.retries.count").count == 9 - metrics.histogram("hermes-client.com_group.topic.retries.attempts").getSnapshot().getMin() == 2 - metrics.histogram("hermes-client.com_group.topic.retries.attempts").getSnapshot().getMax() == 5 - metrics.timers.containsKey("hermes-client.com_group.topic.latency") - - metrics.counter("hermes-client.com_group.topic.publish.finally.success").count == 2 - metrics.counter("hermes-client.com_group.topic.publish.finally.failure").count == 1 - metrics.counter("hermes-client.com_group.topic.publish.failure").count == 10 - metrics.counter("hermes-client.com_group.topic.publish.attempt").count == 3 - metrics.counter("hermes-client.com_group.topic.publish.retry.success").count == 2 - metrics.counter("hermes-client.com_group.topic.publish.retry.failure").count == 9 - metrics.counter("hermes-client.com_group.topic.publish.retry.attempt").count == 3 - } - - def "should update failure metrics when there is an application-level error"() { - given: - HermesClient client1 = hermesClient(badRequestHermesSender()) - .withRetrySleep(0) - .withRetries(4) - .withMetrics(metricsProvider).build() - - HermesClient client2 = hermesClient(badRequestHermesSender()) - .withRetrySleep(0) - .withRetries(6) - .withMetrics(metricsProvider).build() - - HermesClient client3 = hermesClient(badRequestHermesSender()) - .withRetrySleep(0) - .withRetries(2) - .withMetrics(metricsProvider).build() - - when: - silence({ client1.publish("com.group.topic", "123").join() }) - silence({ client2.publish("com.group.topic", "456").join() }) - silence({ client3.publish("com.group.topic", "789").join() }) - - then: - metrics.counter("hermes-client.com_group.topic.publish.finally.success").count == 0 - metrics.counter("hermes-client.com_group.topic.publish.finally.failure").count == 3 - metrics.counter("hermes-client.com_group.topic.publish.failure").count == 3 - metrics.counter("hermes-client.com_group.topic.publish.attempt").count == 3 - metrics.counter("hermes-client.com_group.topic.publish.retry.success").count == 0 - metrics.counter("hermes-client.com_group.topic.publish.retry.failure").count == 0 - metrics.counter("hermes-client.com_group.topic.publish.retry.attempt").count == 0 - } - - private CompletableFuture successFuture(HermesMessage message) { - return completedFuture(HermesResponseBuilder.hermesResponse(message).withHttpStatus(201).build()) - } - - private CompletableFuture badRequestFuture(HermesMessage message) { - return completedFuture(HermesResponseBuilder.hermesResponse(message).withHttpStatus(400).build()) - } - - private CompletableFuture failingFuture(Throwable throwable) { - CompletableFuture future = new CompletableFuture<>() - future.completeExceptionally(throwable) - return future - } - - private HermesSender failingHermesSender(int errorNo) { - new HermesSender() { - int i = 0 - @Override - CompletableFuture send(URI uri, HermesMessage message) { - i++ - if (i <= errorNo) { - return failingFuture(new RuntimeException()) - } - return successFuture(message) - } - } - } - - private HermesSender delayedHermesSender(Duration sendLatencyMs) { - new HermesSender() { - @Override - CompletableFuture send(URI uri, HermesMessage message) { - Thread.sleep(sendLatencyMs.toMillis()) - return successFuture(message) - } - } - } - - private HermesSender badRequestHermesSender() { - new HermesSender() { - @Override - CompletableFuture send(URI uri, HermesMessage message) { - return badRequestFuture(message) - } - } - } - - private void silence(Runnable runnable) { - try { - runnable.run() - } catch (Exception ex) { - // do nothing - } - } - -} diff --git a/hermes-client/src/test/groovy/pl/allegro/tech/hermes/client/HermesClientMicrometerTaggedMetricsTest.groovy b/hermes-client/src/test/groovy/pl/allegro/tech/hermes/client/HermesClientMicrometerTaggedMetricsTest.groovy index d7175066ec..117c63054a 100644 --- a/hermes-client/src/test/groovy/pl/allegro/tech/hermes/client/HermesClientMicrometerTaggedMetricsTest.groovy +++ b/hermes-client/src/test/groovy/pl/allegro/tech/hermes/client/HermesClientMicrometerTaggedMetricsTest.groovy @@ -32,7 +32,7 @@ class HermesClientMicrometerTaggedMetricsTest extends Specification { metrics.counter("hermes-client.status", "code", String.valueOf(201), "topic", "com_group.topic").count() == 1 def timer = metrics.timer("hermes-client.latency", "topic", "com_group.topic") timer.totalTime(TimeUnit.NANOSECONDS) >= Duration.ofMillis(100).get(ChronoUnit.NANOS) - timer.totalTime(TimeUnit.NANOSECONDS) < Duration.ofMillis(300).get(ChronoUnit.NANOS) + timer.totalTime(TimeUnit.NANOSECONDS) < Duration.ofMillis(500).get(ChronoUnit.NANOS) } def "should close timer on exceptional completion and log failure metric"() { diff --git a/hermes-client/src/test/groovy/pl/allegro/tech/hermes/client/ReactiveHermesClientMetricsTest.groovy b/hermes-client/src/test/groovy/pl/allegro/tech/hermes/client/ReactiveHermesClientMetricsTest.groovy deleted file mode 100644 index c1993857dd..0000000000 --- a/hermes-client/src/test/groovy/pl/allegro/tech/hermes/client/ReactiveHermesClientMetricsTest.groovy +++ /dev/null @@ -1,205 +0,0 @@ -package pl.allegro.tech.hermes.client - -import com.codahale.metrics.MetricRegistry -import pl.allegro.tech.hermes.client.metrics.DropwizardMetricsProvider -import pl.allegro.tech.hermes.client.metrics.MetricsProvider -import reactor.core.publisher.Mono -import spock.lang.Specification - -import java.time.Duration -import java.time.temporal.ChronoUnit - -import static pl.allegro.tech.hermes.client.ReactiveHermesClientBuilder.hermesClient - -class ReactiveHermesClientMetricsTest extends Specification { - - private MetricRegistry metrics = new MetricRegistry() - private MetricsProvider metricsProvider = new DropwizardMetricsProvider(metrics) - - def "should measure publish latency"() { - given: - ReactiveHermesClient client = hermesClient(delayedHermesSender(Duration.ofMillis(100))) - .withRetrySleep(0) - .withMetrics(metricsProvider).build() - - when: - client.publish("com.group.topic", "123").block() - - then: - metrics.counter("hermes-client.com_group.topic.status.201").count == 1 - metrics.timer("hermes-client.com_group.topic.latency").getSnapshot().getMax() >= Duration.ofMillis(100).get(ChronoUnit.NANOS) - metrics.timer("hermes-client.com_group.topic.latency").getSnapshot().getMax() < Duration.ofMillis(300).get(ChronoUnit.NANOS) - } - - def "should close timer on exceptional completion and log failure metric"() { - given: - ReactiveHermesClient client = hermesClient({uri, msg -> failingMono(new RuntimeException())}) - .withRetrySleep(0) - .withRetries(3) - .withMetrics(metricsProvider).build() - - when: - silence({ client.publish("com.group.topic", "123").block() }) - - then: - metrics.counter("hermes-client.com_group.topic.failure").count == 4 - metrics.timers.containsKey("hermes-client.com_group.topic.latency") - - metrics.counter("hermes-client.com_group.topic.publish.failure").count == 4 - metrics.counter("hermes-client.com_group.topic.publish.finally.failure").count == 1 - metrics.counter("hermes-client.com_group.topic.publish.retry.failure").count == 3 - } - - def "should update max retries exceeded metric"() { - given: - ReactiveHermesClient client = hermesClient({uri, msg -> failingMono(new RuntimeException())}) - .withRetrySleep(0) - .withRetries(3) - .withMetrics(metricsProvider).build() - - when: - silence({ client.publish("com.group.topic", "123").block() }) - - then: - metrics.counter("hermes-client.com_group.topic.failure").count == 4 - metrics.counter("hermes-client.com_group.topic.retries.count").count == 3 - metrics.counter("hermes-client.com_group.topic.retries.exhausted").count == 1 - metrics.counter("hermes-client.com_group.topic.retries.success").count == 0 - metrics.histogram("hermes-client.com_group.topic.retries.attempts").getSnapshot().size() == 0 - metrics.timers.containsKey("hermes-client.com_group.topic.latency") - - metrics.counter("hermes-client.com_group.topic.publish.finally.success").count == 0 - metrics.counter("hermes-client.com_group.topic.publish.finally.failure").count == 1 - metrics.counter("hermes-client.com_group.topic.publish.failure").count == 4 - metrics.counter("hermes-client.com_group.topic.publish.attempt").count == 1 - metrics.counter("hermes-client.com_group.topic.publish.retry.success").count == 0 - metrics.counter("hermes-client.com_group.topic.publish.retry.failure").count == 3 - metrics.counter("hermes-client.com_group.topic.publish.retry.attempt").count == 1 - } - - def "should update retries metrics"() { - given: - ReactiveHermesClient client1 = hermesClient(failingHermesSender(2)) - .withRetrySleep(0) - .withRetries(4) - .withMetrics(metricsProvider).build() - - ReactiveHermesClient client2 = hermesClient(failingHermesSender(5)) - .withRetrySleep(0) - .withRetries(6) - .withMetrics(metricsProvider).build() - - ReactiveHermesClient client3 = hermesClient(failingHermesSender(3)) - .withRetrySleep(0) - .withRetries(2) - .withMetrics(metricsProvider).build() - - when: - silence({ client1.publish("com.group.topic", "123").block() }) - silence({ client2.publish("com.group.topic", "456").block() }) - silence({ client3.publish("com.group.topic", "789").block() }) - - then: - metrics.counter("hermes-client.com_group.topic.failure").count == 10 - metrics.counter("hermes-client.com_group.topic.retries.exhausted").count == 1 - metrics.counter("hermes-client.com_group.topic.retries.success").count == 2 - metrics.counter("hermes-client.com_group.topic.retries.count").count == 9 - metrics.histogram("hermes-client.com_group.topic.retries.attempts").getSnapshot().getMin() == 2 - metrics.histogram("hermes-client.com_group.topic.retries.attempts").getSnapshot().getMax() == 5 - metrics.timers.containsKey("hermes-client.com_group.topic.latency") - - metrics.counter("hermes-client.com_group.topic.publish.finally.success").count == 2 - metrics.counter("hermes-client.com_group.topic.publish.finally.failure").count == 1 - metrics.counter("hermes-client.com_group.topic.publish.failure").count == 10 - metrics.counter("hermes-client.com_group.topic.publish.attempt").count == 3 - metrics.counter("hermes-client.com_group.topic.publish.retry.success").count == 2 - metrics.counter("hermes-client.com_group.topic.publish.retry.failure").count == 9 - metrics.counter("hermes-client.com_group.topic.publish.retry.attempt").count == 3 - } - - def "should update failure metrics when there is an application-level error"() { - given: - ReactiveHermesClient client1 = hermesClient(badRequestHermesSender()) - .withRetrySleep(0) - .withRetries(4) - .withMetrics(metricsProvider).build() - - ReactiveHermesClient client2 = hermesClient(badRequestHermesSender()) - .withRetrySleep(0) - .withRetries(6) - .withMetrics(metricsProvider).build() - - ReactiveHermesClient client3 = hermesClient(badRequestHermesSender()) - .withRetrySleep(0) - .withRetries(2) - .withMetrics(metricsProvider).build() - - when: - silence({ client1.publish("com.group.topic", "123").block() }) - silence({ client2.publish("com.group.topic", "456").block() }) - silence({ client3.publish("com.group.topic", "789").block() }) - - then: - metrics.counter("hermes-client.com_group.topic.publish.finally.success").count == 0 - metrics.counter("hermes-client.com_group.topic.publish.finally.failure").count == 3 - metrics.counter("hermes-client.com_group.topic.publish.failure").count == 3 - metrics.counter("hermes-client.com_group.topic.publish.attempt").count == 3 - metrics.counter("hermes-client.com_group.topic.publish.retry.success").count == 0 - metrics.counter("hermes-client.com_group.topic.publish.retry.failure").count == 0 - metrics.counter("hermes-client.com_group.topic.publish.retry.attempt").count == 0 - } - - private Mono successMono(HermesMessage message) { - return Mono.just(HermesResponseBuilder.hermesResponse(message).withHttpStatus(201).build()) - } - - private Mono badRequestMono(HermesMessage message) { - return Mono.just(HermesResponseBuilder.hermesResponse(message).withHttpStatus(400).build()) - } - - private Mono failingMono(Throwable throwable) { - return Mono.error(throwable) - } - - private ReactiveHermesSender failingHermesSender(int errorNo) { - new ReactiveHermesSender() { - int i = 0 - @Override - Mono sendReactively(URI uri, HermesMessage message) { - i++ - if (i <= errorNo) { - return failingMono(new RuntimeException()) - } - return successMono(message) - } - } - } - - private ReactiveHermesSender delayedHermesSender(Duration sendLatencyMs) { - new ReactiveHermesSender() { - @Override - Mono sendReactively(URI uri, HermesMessage message) { - Thread.sleep(sendLatencyMs.toMillis()) - return successMono(message) - } - } - } - - private ReactiveHermesSender badRequestHermesSender() { - new ReactiveHermesSender() { - @Override - Mono sendReactively(URI uri, HermesMessage message) { - return badRequestMono(message) - } - } - } - - private void silence(Runnable runnable) { - try { - runnable.run() - } catch (Exception ex) { - // do nothing - } - } - -} diff --git a/hermes-common/build.gradle b/hermes-common/build.gradle index ccf959276b..85b4ff6ee3 100644 --- a/hermes-common/build.gradle +++ b/hermes-common/build.gradle @@ -32,9 +32,7 @@ dependencies { api group: 'org.apache.avro', name: 'avro', version: versions.avro api group: 'com.jayway.jsonpath', name: 'json-path', version: '2.5.0' - implementation group: 'io.dropwizard.metrics', name: 'metrics-graphite', version: versions.dropwizard_metrics - implementation group: 'io.dropwizard.metrics', name: 'metrics-jvm', version: versions.dropwizard_metrics - implementation group: 'org.mpierce.metrics.reservoir', name: 'hdrhistogram-metrics-reservoir', version: '1.1.0' + implementation group: 'io.dropwizard.metrics', name: 'metrics-core', version: versions.dropwizard_metrics implementation group: 'com.google.code.findbugs', name: 'annotations', version: '3.0.0' api group: 'io.micrometer', name: 'micrometer-core', version: versions.micrometer_metrics diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/di/factories/GraphiteParameters.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/di/factories/GraphiteParameters.java deleted file mode 100644 index 84314096c4..0000000000 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/di/factories/GraphiteParameters.java +++ /dev/null @@ -1,10 +0,0 @@ -package pl.allegro.tech.hermes.common.di.factories; - -public interface GraphiteParameters { - - String getPrefix(); - - String getHost(); - - int getPort(); -} diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/di/factories/MetricRegistryFactory.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/di/factories/MetricRegistryFactory.java deleted file mode 100644 index 1d1de61579..0000000000 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/di/factories/MetricRegistryFactory.java +++ /dev/null @@ -1,107 +0,0 @@ -package pl.allegro.tech.hermes.common.di.factories; - -import com.codahale.metrics.ConsoleReporter; -import com.codahale.metrics.Metric; -import com.codahale.metrics.MetricAttribute; -import com.codahale.metrics.MetricRegistry; -import com.codahale.metrics.MetricSet; -import com.codahale.metrics.graphite.Graphite; -import com.codahale.metrics.graphite.GraphiteReporter; -import com.codahale.metrics.jvm.FileDescriptorRatioGauge; -import com.codahale.metrics.jvm.GarbageCollectorMetricSet; -import com.codahale.metrics.jvm.MemoryUsageGaugeSet; -import com.google.common.base.Joiner; -import com.google.common.collect.Sets; -import jakarta.inject.Named; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; -import pl.allegro.tech.hermes.common.metric.MetricRegistryWithHdrHistogramReservoir; -import pl.allegro.tech.hermes.common.util.InstanceIdResolver; - -import java.net.InetSocketAddress; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.TimeUnit; - -public class MetricRegistryFactory { - - private static final Logger logger = LoggerFactory.getLogger(MetricRegistryFactory.class); - private final MetricRegistryParameters metricRegistryParameters; - private final GraphiteParameters graphiteParameters; - private final InstanceIdResolver instanceIdResolver; - private final String moduleName; - - public MetricRegistryFactory(MetricRegistryParameters metricRegistryParameters, - GraphiteParameters graphiteParameters, - InstanceIdResolver instanceIdResolver, - @Named("moduleName") String moduleName) { - this.metricRegistryParameters = metricRegistryParameters; - this.graphiteParameters = graphiteParameters; - this.instanceIdResolver = instanceIdResolver; - this.moduleName = moduleName; - } - - public MetricRegistry provide() { - MetricRegistry registry = new MetricRegistryWithHdrHistogramReservoir(); - - if (metricRegistryParameters.isGraphiteReporterEnabled()) { - String prefix = Joiner.on(".").join( - graphiteParameters.getPrefix(), - moduleName, - instanceIdResolver.resolve().replaceAll("\\.", HermesMetrics.REPLACEMENT_CHAR)); - - GraphiteReporter - .forRegistry(registry) - .prefixedWith(prefix) - .disabledMetricAttributes(getDisabledAttributesFromConfig()) - .build(new Graphite(new InetSocketAddress( - graphiteParameters.getHost(), - graphiteParameters.getPort() - ))) - .start(metricRegistryParameters.getReportPeriod().toSeconds(), TimeUnit.SECONDS); - } - if (metricRegistryParameters.isConsoleReporterEnabled()) { - ConsoleReporter.forRegistry(registry).build().start( - metricRegistryParameters.getReportPeriod().toSeconds(), TimeUnit.SECONDS - ); - } - registerJvmMetrics(registry); - - return registry; - } - - private void registerJvmMetrics(MetricRegistry metricRegistry) { - registerAll("jvm.gc", new GarbageCollectorMetricSet(), metricRegistry); - registerAll("jvm.memory", new MemoryUsageGaugeSet(), metricRegistry); - metricRegistry.register("jvm.descriptors", new FileDescriptorRatioGauge()); - } - - private void registerAll(String prefix, MetricSet metricSet, MetricRegistry registry) { - for (Map.Entry entry : metricSet.getMetrics().entrySet()) { - if (entry.getValue() instanceof MetricSet) { - registerAll(prefix + "." + entry.getKey(), (MetricSet) entry.getValue(), registry); - } else { - registry.register(prefix + "." + entry.getKey(), entry.getValue()); - } - } - } - - private Set getDisabledAttributesFromConfig() { - Set disabledAttributes = Sets.newHashSet(); - String disabledAttributesFromConfig = metricRegistryParameters.getDisabledAttributes(); - List disabledAttributesList = Arrays.asList(disabledAttributesFromConfig.split("\\s*,\\s*")); - - disabledAttributesList.forEach(singleAttribute -> { - try { - disabledAttributes.add(MetricAttribute.valueOf(singleAttribute)); - } catch (IllegalArgumentException e) { - logger.warn("Failed to add disabled attribute from config: {}", e.getMessage()); - } - }); - - return disabledAttributes; - } -} diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/di/factories/MetricRegistryParameters.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/di/factories/MetricRegistryParameters.java deleted file mode 100644 index d112cea526..0000000000 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/di/factories/MetricRegistryParameters.java +++ /dev/null @@ -1,16 +0,0 @@ -package pl.allegro.tech.hermes.common.di.factories; - -import java.time.Duration; - -public interface MetricRegistryParameters { - - boolean isZookeeperReporterEnabled(); - - boolean isGraphiteReporterEnabled(); - - boolean isConsoleReporterEnabled(); - - String getDisabledAttributes(); - - Duration getReportPeriod(); -} diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/ConsumerMetrics.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/ConsumerMetrics.java index 8ef5a5d086..c6b3bb2af0 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/ConsumerMetrics.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/ConsumerMetrics.java @@ -9,20 +9,15 @@ import java.util.function.ToDoubleFunction; -import static pl.allegro.tech.hermes.common.metric.Gauges.BATCH_BUFFER_AVAILABLE_BYTES; -import static pl.allegro.tech.hermes.common.metric.Gauges.BATCH_BUFFER_TOTAL_BYTES; -import static pl.allegro.tech.hermes.common.metric.Gauges.THREADS; import static pl.allegro.tech.hermes.common.metric.SubscriptionTagsFactory.subscriptionTags; public class ConsumerMetrics { - private final HermesMetrics hermesMetrics; private final MeterRegistry meterRegistry; private final GaugeRegistrar gaugeRegistrar; - public ConsumerMetrics(HermesMetrics hermesMetrics, MeterRegistry meterRegistry) { - this.hermesMetrics = hermesMetrics; + public ConsumerMetrics(MeterRegistry meterRegistry) { this.meterRegistry = meterRegistry; - this.gaugeRegistrar = new GaugeRegistrar(meterRegistry, hermesMetrics); + this.gaugeRegistrar = new GaugeRegistrar(meterRegistry); } public void registerQueueUtilizationGauge(T obj, String queueName, ToDoubleFunction f) { @@ -31,31 +26,28 @@ public void registerQueueUtilizationGauge(T obj, String queueName, ToDouble public HermesCounter queueFailuresCounter(String name) { return HermesCounters.from( - meterRegistry.counter("queue." + name + ".failures"), - hermesMetrics.counter("queue." + name + ".failures") + meterRegistry.counter("queue." + name + ".failures") ); } public void registerConsumerProcessesThreadsGauge(T obj, ToDoubleFunction f) { - gaugeRegistrar.registerGauge(THREADS, "consumer-processes.threads", obj, f); + gaugeRegistrar.registerGauge("consumer-processes.threads", obj, f); } public void registerRunningConsumerProcessesGauge(T obj, ToDoubleFunction f) { - hermesMetrics.registerRunningConsumerProcessesCountGauge(() -> (int) f.applyAsDouble(obj)); meterRegistry.gauge("consumer-processes.running", obj, f); } public void registerDyingConsumerProcessesGauge(T obj, ToDoubleFunction f) { - hermesMetrics.registerDyingConsumerProcessesCountGauge(() -> (int) f.applyAsDouble(obj)); meterRegistry.gauge("consumer-processes.dying", obj, f); } public void registerBatchBufferTotalBytesGauge(T obj, ToDoubleFunction f) { - gaugeRegistrar.registerGauge(BATCH_BUFFER_TOTAL_BYTES, "batch-buffer.total-bytes", obj, f); + gaugeRegistrar.registerGauge("batch-buffer.total-bytes", obj, f); } public void registerBatchBufferAvailableBytesGauge(T obj, ToDoubleFunction f) { - gaugeRegistrar.registerGauge(BATCH_BUFFER_AVAILABLE_BYTES, "batch-buffer.available-bytes", obj, f); + gaugeRegistrar.registerGauge("batch-buffer.available-bytes", obj, f); } public HermesCounter oAuthSubscriptionTokenRequestCounter(Subscription subscription, String providerName) { @@ -63,29 +55,25 @@ public HermesCounter oAuthSubscriptionTokenRequestCounter(Subscription subscript meterRegistry.counter("oauth.token-requests", Tags.concat( subscriptionTags(subscription.getQualifiedName()), "provider", providerName - )), - hermesMetrics.oAuthSubscriptionTokenRequestMeter(subscription, providerName) + )) ); } public HermesTimer oAuthProviderLatencyTimer(String providerName) { return HermesTimer.from( - meterRegistry.timer("oauth.token-request-latency", Tags.of("provider", providerName)), - hermesMetrics.oAuthProviderLatencyTimer(providerName) + meterRegistry.timer("oauth.token-request-latency", Tags.of("provider", providerName)) ); } public HermesCounter processedSignalsCounter(String name) { return HermesCounters.from( - meterRegistry.counter("signals.processed", Tags.of("signal", name)), - hermesMetrics.counter("supervisor.signal." + name) + meterRegistry.counter("signals.processed", Tags.of("signal", name)) ); } public HermesCounter droppedSignalsCounter(String name) { return HermesCounters.from( - meterRegistry.counter("signals.dropped", Tags.of("signal", name)), - hermesMetrics.counter("supervisor.signal.dropped." + name) + meterRegistry.counter("signals.dropped", Tags.of("signal", name)) ); } } diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/ConsumerSenderMetrics.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/ConsumerSenderMetrics.java index 76bba6dde7..d3d8a3c80e 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/ConsumerSenderMetrics.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/ConsumerSenderMetrics.java @@ -13,33 +13,27 @@ public class ConsumerSenderMetrics { - private final HermesMetrics hermesMetrics; private final MeterRegistry meterRegistry; private final GaugeRegistrar gaugeRegistrar; - ConsumerSenderMetrics(HermesMetrics hermesMetrics, MeterRegistry meterRegistry) { - this.hermesMetrics = hermesMetrics; + ConsumerSenderMetrics(MeterRegistry meterRegistry) { this.meterRegistry = meterRegistry; - this.gaugeRegistrar = new GaugeRegistrar(meterRegistry, hermesMetrics); + this.gaugeRegistrar = new GaugeRegistrar(meterRegistry); } public void registerRequestQueueSizeGauge(T obj, ToDoubleFunction f) { - hermesMetrics.registerConsumerSenderRequestQueueSize(() -> (int) f.applyAsDouble(obj)); meterRegistry.gauge("http-clients.request-queue-size", obj, f); } public void registerHttp1SerialClientRequestQueueSizeGauge(T obj, ToDoubleFunction f) { - hermesMetrics.registerConsumerSenderHttp1SerialClientRequestQueueSize(() -> (int) f.applyAsDouble(obj)); meterRegistry.gauge("http-clients.serial.http1.request-queue-size", obj, f); } public void registerHttp1BatchClientRequestQueueSizeGauge(T obj, ToDoubleFunction f) { - hermesMetrics.registerConsumerSenderHttp1BatchClientRequestQueueSize(() -> (int) f.applyAsDouble(obj)); meterRegistry.gauge("http-clients.batch.http1.request-queue-size", obj, f); } public void registerHttp2RequestQueueSizeGauge(T obj, ToDoubleFunction f) { - hermesMetrics.registerConsumerSenderHttp2RequestQueueSize(() -> (int) f.applyAsDouble(obj)); meterRegistry.gauge("http-clients.serial.http2.request-queue-size", obj, f); } diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/Counters.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/Counters.java deleted file mode 100644 index 486c053e38..0000000000 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/Counters.java +++ /dev/null @@ -1,17 +0,0 @@ -package pl.allegro.tech.hermes.common.metric; - -import static pl.allegro.tech.hermes.metrics.PathsCompiler.GROUP; -import static pl.allegro.tech.hermes.metrics.PathsCompiler.SUBSCRIPTION; -import static pl.allegro.tech.hermes.metrics.PathsCompiler.TOPIC; - -public class Counters { - - public static final String PUBLISHED = "published." + GROUP + "." + TOPIC; - public static final String DELIVERED = "delivered." + GROUP + "." + TOPIC + "." + SUBSCRIPTION; - public static final String DISCARDED = "discarded." + GROUP + "." + TOPIC + "." + SUBSCRIPTION; - public static final String RETRIES = "retries." + GROUP + "." + TOPIC + "." + SUBSCRIPTION; - public static final String MAXRATE_RATE_HISTORY_FAILURES = - "consumers-rate.max-rate.node." + GROUP + "." + TOPIC + "." + SUBSCRIPTION + ".history.failures"; - public static final String MAXRATE_FETCH_FAILURES = - "consumers-rate.max-rate.node." + GROUP + "." + TOPIC + "." + SUBSCRIPTION + ".fetch.failures"; -} diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/DeserializationMetrics.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/DeserializationMetrics.java index e8628afbf8..3efe8fac71 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/DeserializationMetrics.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/DeserializationMetrics.java @@ -1,50 +1,43 @@ package pl.allegro.tech.hermes.common.metric; +import io.micrometer.core.instrument.Counter; import io.micrometer.core.instrument.MeterRegistry; import io.micrometer.core.instrument.Tags; import pl.allegro.tech.hermes.metrics.HermesCounter; import pl.allegro.tech.hermes.metrics.counters.HermesCounters; -import static com.codahale.metrics.MetricRegistry.name; - public class DeserializationMetrics { - private final HermesMetrics hermesMetrics; private final MeterRegistry meterRegistry; private static final String BASE_PATH = "content.avro.deserialization"; private static final String ERRORS_PATH = BASE_PATH + ".errors"; - public DeserializationMetrics(HermesMetrics hermesMetrics, MeterRegistry meterRegistry) { - this.hermesMetrics = hermesMetrics; + public DeserializationMetrics(MeterRegistry meterRegistry) { this.meterRegistry = meterRegistry; } public HermesCounter errorsForHeaderSchemaVersion() { return HermesCounters.from( - deserializationErrorCounter("headerSchemaVersion"), - hermesMetrics.counter(name(ERRORS_PATH, "headerSchemaVersion")) + deserializationErrorCounter("headerSchemaVersion") ); } public HermesCounter errorsForHeaderSchemaId() { return HermesCounters.from( - deserializationErrorCounter("headerSchemaId"), - hermesMetrics.counter(name(ERRORS_PATH, "headerSchemaId")) + deserializationErrorCounter("headerSchemaId") ); } public HermesCounter errorsForSchemaIdAwarePayload() { return HermesCounters.from( - deserializationErrorCounter("payloadWithSchemaId"), - hermesMetrics.counter(name(ERRORS_PATH, "payloadWithSchemaId")) + deserializationErrorCounter("payloadWithSchemaId") ); } public HermesCounter errorsForSchemaVersionTruncation() { return HermesCounters.from( - deserializationErrorCounter("schemaVersionTruncation"), - hermesMetrics.counter(name(ERRORS_PATH, "schemaVersionTruncation")) + deserializationErrorCounter("schemaVersionTruncation") ); } @@ -54,40 +47,35 @@ private io.micrometer.core.instrument.Counter deserializationErrorCounter(String public HermesCounter missingSchemaIdInPayload() { return HermesCounters.from( - meterRegistry.counter(name(BASE_PATH, "missing_schemaIdInPayload")), - hermesMetrics.counter(name(BASE_PATH, "missed", "schemaIdInPayload")) + meterRegistry.counter(BASE_PATH + ".missing_schemaIdInPayload") ); } public HermesCounter usingHeaderSchemaVersion() { return HermesCounters.from( - deserializationAttemptCounter("headerSchemaVersion"), - hermesMetrics.counter(name(BASE_PATH, "using", "headerSchemaVersion")) + deserializationAttemptCounter("headerSchemaVersion") ); } public HermesCounter usingHeaderSchemaId() { return HermesCounters.from( - deserializationAttemptCounter("headerSchemaId"), - hermesMetrics.counter(name(BASE_PATH, "using", "headerSchemaId")) + deserializationAttemptCounter("headerSchemaId") ); } public HermesCounter usingSchemaIdAware() { return HermesCounters.from( - deserializationAttemptCounter("payloadWithSchemaId"), - hermesMetrics.counter(name(BASE_PATH, "using", "schemaIdAware")) + deserializationAttemptCounter("payloadWithSchemaId") ); } public HermesCounter usingSchemaVersionTruncation() { return HermesCounters.from( - deserializationAttemptCounter("schemaVersionTruncation"), - hermesMetrics.counter(name(BASE_PATH, "using", "schemaVersionTruncation")) + deserializationAttemptCounter("schemaVersionTruncation") ); } - private io.micrometer.core.instrument.Counter deserializationAttemptCounter(String deserializationType) { + private Counter deserializationAttemptCounter(String deserializationType) { return meterRegistry.counter(BASE_PATH, Tags.of("deserialization_type", deserializationType)); } } diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/GaugeRegistrar.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/GaugeRegistrar.java index 2d2a7b4c4d..cbcb86ba84 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/GaugeRegistrar.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/GaugeRegistrar.java @@ -8,40 +8,22 @@ public class GaugeRegistrar { private final MeterRegistry meterRegistry; - private final HermesMetrics hermesMetrics; - public GaugeRegistrar(MeterRegistry meterRegistry, HermesMetrics hermesMetrics) { + public GaugeRegistrar(MeterRegistry meterRegistry) { this.meterRegistry = meterRegistry; - this.hermesMetrics = hermesMetrics; - } - - public void registerGauge(String graphiteName, - String prometheusName, - T stateObj, - ToDoubleFunction f, - Iterable tags) { - meterRegistry.gauge(prometheusName, tags, stateObj, f); - hermesMetrics.registerGauge(graphiteName, () -> f.applyAsDouble(stateObj)); - } - - public void registerGauge(String graphiteName, - String prometheusName, - T stateObj, - ToDoubleFunction f) { - registerGauge(graphiteName, prometheusName, stateObj, f, Tags.empty()); } public void registerGauge(String name, T stateObj, ToDoubleFunction f) { - registerGauge(name, name, stateObj, f); + registerGauge(name, stateObj, f, Tags.empty()); } public void registerGauge(String name, T stateObj, ToDoubleFunction f, Iterable tags) { - registerGauge(name, name, stateObj, f, tags); + meterRegistry.gauge(name, tags, stateObj, f); } } diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/Gauges.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/Gauges.java index c75cbfcdea..5202b7355d 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/Gauges.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/Gauges.java @@ -1,37 +1,17 @@ package pl.allegro.tech.hermes.common.metric; -import static pl.allegro.tech.hermes.metrics.PathsCompiler.GROUP; -import static pl.allegro.tech.hermes.metrics.PathsCompiler.SUBSCRIPTION; -import static pl.allegro.tech.hermes.metrics.PathsCompiler.TOPIC; - public class Gauges { - public static final String BATCH_BUFFER_TOTAL_BYTES = "batch-buffer-total-bytes"; - public static final String BATCH_BUFFER_AVAILABLE_BYTES = "batch-buffer-available-bytes"; - public static final String THREADS = "threads"; public static final String INFLIGHT_REQUESTS = "inflight-requests"; - public static final String OUTPUT_RATE = "output-rate." + GROUP + "." + TOPIC + "." + SUBSCRIPTION; public static final String BACKUP_STORAGE_SIZE = "backup-storage.size"; - public static final String MAX_RATE_CALCULATION_DURATION = "consumers-rate.max-rate.coordinator.duration"; - public static final String MAX_RATE_VALUE = - "consumers-rate.max-rate.node." + GROUP + "." + TOPIC + "." + SUBSCRIPTION + ".max-rate"; - public static final String MAX_RATE_ACTUAL_RATE_VALUE = - "consumers-rate.max-rate.node." + GROUP + "." + TOPIC + "." + SUBSCRIPTION + ".rate"; - public static final String RUNNING_CONSUMER_PROCESSES_COUNT = "consumer-processes.running-consumer-processes.count"; - public static final String DYING_CONSUMER_PROCESSES_COUNT = "consumer-processes.dying-consumer-processes.count"; - public static final String CONSUMER_SENDER_REQUEST_QUEUE_SIZE = "http-clients.request-queue-size"; - public static final String CONSUMER_SENDER_HTTP_1_SERIAL_CLIENT_REQUEST_QUEUE_SIZE = "http-clients.serial.http1.request-queue-size"; public static final String CONSUMER_SENDER_HTTP_1_SERIAL_CLIENT_ACTIVE_CONNECTIONS = "http-clients.serial.http1.active-connections"; public static final String CONSUMER_SENDER_HTTP_1_SERIAL_CLIENT_IDLE_CONNECTIONS = "http-clients.serial.http1.idle-connections"; - public static final String CONSUMER_SENDER_HTTP_1_BATCH_CLIENT_REQUEST_QUEUE_SIZE = "http-clients.batch.http1.request-queue-size"; public static final String CONSUMER_SENDER_HTTP_1_BATCH_CLIENT_ACTIVE_CONNECTIONS = "http-clients.batch.http1.active-connections"; public static final String CONSUMER_SENDER_HTTP_1_BATCH_CLIENT_IDLE_CONNECTIONS = "http-clients.batch.http1.idle-connections"; - public static final String CONSUMER_SENDER_HTTP_2_SERIAL_CLIENT_REQUEST_QUEUE_SIZE = "http-clients.serial.http2.request-queue-size"; public static final String CONSUMER_SENDER_HTTP_2_SERIAL_CLIENT_CONNECTIONS = "http-clients.serial.http2.connections"; public static final String CONSUMER_SENDER_HTTP_2_SERIAL_CLIENT_PENDING_CONNECTIONS = "http-clients.serial.http2.pending-connections"; - public static final String INFLIGHT = "inflight." + GROUP + "." + TOPIC + "." + SUBSCRIPTION + ".count"; } diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/HermesMetrics.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/HermesMetrics.java deleted file mode 100644 index 23d477ac6e..0000000000 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/HermesMetrics.java +++ /dev/null @@ -1,274 +0,0 @@ -package pl.allegro.tech.hermes.common.metric; - -import com.codahale.metrics.Counter; -import com.codahale.metrics.Gauge; -import com.codahale.metrics.Histogram; -import com.codahale.metrics.Meter; -import com.codahale.metrics.MetricFilter; -import com.codahale.metrics.MetricRegistry; -import com.codahale.metrics.Timer; -import pl.allegro.tech.hermes.api.Subscription; -import pl.allegro.tech.hermes.api.SubscriptionName; -import pl.allegro.tech.hermes.api.TopicName; -import pl.allegro.tech.hermes.metrics.PathContext; -import pl.allegro.tech.hermes.metrics.PathsCompiler; - -import static pl.allegro.tech.hermes.common.metric.Histograms.INFLIGHT_TIME; -import static pl.allegro.tech.hermes.common.metric.Meters.ERRORS_HTTP_BY_CODE; -import static pl.allegro.tech.hermes.common.metric.Meters.ERRORS_HTTP_BY_FAMILY; -import static pl.allegro.tech.hermes.common.metric.Meters.ERRORS_OTHER; -import static pl.allegro.tech.hermes.common.metric.Meters.ERRORS_TIMEOUTS; -import static pl.allegro.tech.hermes.common.metric.Meters.SUBSCRIPTION_STATUS; -import static pl.allegro.tech.hermes.metrics.PathContext.pathContext; - -public class HermesMetrics { - - public static final String REPLACEMENT_CHAR = "_"; - - private final MetricRegistry metricRegistry; - private final PathsCompiler pathCompiler; - - public HermesMetrics( - MetricRegistry metricRegistry, - PathsCompiler pathCompiler) { - this.metricRegistry = metricRegistry; - this.pathCompiler = pathCompiler; - } - - public static String escapeDots(String value) { - return value.replaceAll("\\.", REPLACEMENT_CHAR); - } - - public Timer timer(String metric) { - return metricRegistry.timer(metricRegistryName(metric)); - } - - public Timer timer(String metric, TopicName topicName) { - return metricRegistry.timer(metricRegistryName(metric, topicName)); - } - - public Timer timer(String metric, TopicName topicName, String name) { - return metricRegistry.timer(metricRegistryName(metric, topicName, name)); - } - - public Meter meter(String metric) { - return metricRegistry.meter(metricRegistryName(metric)); - } - - public Meter meter(String metric, TopicName topicName, String name) { - return metricRegistry.meter(metricRegistryName(metric, topicName, name)); - } - - public Meter meter(String metric, TopicName topicName) { - return metricRegistry.meter(metricRegistryName(metric, topicName)); - } - - public Meter httpStatusCodeMeter(int statusCode) { - return metricRegistry.meter(pathCompiler.compile(Meters.STATUS_CODES, pathContext().withHttpCode(statusCode).build())); - } - - public Meter httpStatusCodeMeter(int statusCode, TopicName topicName) { - return metricRegistry.meter(pathCompiler.compile(Meters.TOPIC_STATUS_CODES, - pathContext().withHttpCode(statusCode).withGroup(topicName.getGroupName()).withTopic(topicName.getName()).build())); - } - - public Histogram histogram(String metric) { - return metricRegistry.histogram(metricRegistryName(metric)); - } - - public Counter counter(String metric) { - return metricRegistry.counter(metricRegistryName(metric)); - } - - public Counter counter(String metric, TopicName topicName) { - return metricRegistry.counter(metricRegistryName(metric, topicName)); - } - - public Counter counter(String metric, TopicName topicName, String name) { - return metricRegistry.counter(metricRegistryName(metric, topicName, name)); - } - - public void registerProducerInflightRequest(Gauge gauge) { - metricRegistry.register(metricRegistryName(Gauges.INFLIGHT_REQUESTS), gauge); - } - - public void registerMessageRepositorySizeGauge(Gauge gauge) { - metricRegistry.register(metricRegistryName(Gauges.BACKUP_STORAGE_SIZE), gauge); - } - - public void registerConsumerSenderRequestQueueSize(Gauge gauge) { - metricRegistry.register(metricRegistryName(Gauges.CONSUMER_SENDER_REQUEST_QUEUE_SIZE), gauge); - } - - public void registerConsumerSenderHttp1SerialClientRequestQueueSize(Gauge gauge) { - metricRegistry.register(metricRegistryName(Gauges.CONSUMER_SENDER_HTTP_1_SERIAL_CLIENT_REQUEST_QUEUE_SIZE), gauge); - } - - public void registerConsumerSenderHttp1BatchClientRequestQueueSize(Gauge gauge) { - metricRegistry.register(metricRegistryName(Gauges.CONSUMER_SENDER_HTTP_1_BATCH_CLIENT_REQUEST_QUEUE_SIZE), gauge); - } - - public void registerConsumerSenderHttp2RequestQueueSize(Gauge gauge) { - metricRegistry.register(metricRegistryName(Gauges.CONSUMER_SENDER_HTTP_2_SERIAL_CLIENT_REQUEST_QUEUE_SIZE), gauge); - } - - public void registerInflightGauge(SubscriptionName subscription, Gauge gauge) { - registerGauge(metricRegistryName(Gauges.INFLIGHT, subscription.getTopicName(), subscription.getName()), gauge); - } - - public void unregisterInflightGauge(SubscriptionName subscription) { - unregister(Gauges.INFLIGHT, subscription); - } - - public static void close(Timer.Context... timers) { - for (Timer.Context timer : timers) { - if (timer != null) { - timer.close(); - } - } - } - - public void registerGauge(String name, Gauge gauge) { - String path = pathCompiler.compile(name); - if (!metricRegistry.getGauges().containsKey(name)) { - metricRegistry.register(path, gauge); - } - } - - public void registerGauge(String name, SubscriptionName subscription, Gauge gauge) { - if (!metricRegistry.getGauges().containsKey(name)) { - metricRegistry.register(metricRegistryName(name, subscription.getTopicName(), subscription.getName()), gauge); - } - } - - public void unregister(String metric, SubscriptionName subscription) { - metricRegistry.remove(metricRegistryName(metric, subscription.getTopicName(), subscription.getName())); - } - - public void unregister(String name) { - String path = pathCompiler.compile(name); - metricRegistry.remove(path); - } - - private String metricRegistryName(String metricDisplayName, TopicName topicName, String subscription) { - PathContext pathContext = PathContext.pathContext() - .withGroup(escapeDots(topicName.getGroupName())) - .withTopic(escapeDots(topicName.getName())) - .withSubscription(escapeDots(subscription)) - .build(); - - return pathCompiler.compile(metricDisplayName, pathContext); - } - - private String metricRegistryName(String metricDisplayName, TopicName topicName) { - PathContext pathContext = PathContext.pathContext() - .withGroup(escapeDots(topicName.getGroupName())) - .withTopic(escapeDots(topicName.getName())).build(); - - return pathCompiler.compile(metricDisplayName, pathContext); - } - - private String metricRegistryName(String metricDisplayName) { - return pathCompiler.compile(metricDisplayName); - } - - public Timer schemaTimer(String schemaMetric) { - return metricRegistry.timer(pathCompiler.compile(schemaMetric, pathContext().withSchemaRepoType("schema-registry").build())); - } - - private Gauge registerExecutorGauge(String path, String executorName, Gauge gauge) { - return metricRegistry.register(pathCompiler.compile(path, pathContext().withExecutorName(executorName).build()), gauge); - } - - private Counter executorCounter(String path, String executorName) { - return metricRegistry.counter(pathCompiler.compile(path, pathContext().withExecutorName(executorName).build())); - } - - - public Histogram messageContentSizeHistogram() { - return metricRegistry.histogram(pathCompiler.compile(Histograms.GLOBAL_MESSAGE_SIZE)); - } - - public Histogram messageContentSizeHistogram(TopicName topic) { - return metricRegistry.histogram(pathCompiler.compile(Histograms.MESSAGE_SIZE, pathContext() - .withGroup(escapeDots(topic.getGroupName())) - .withTopic(escapeDots(topic.getName())) - .build())); - } - - public Histogram inflightTimeHistogram(SubscriptionName subscription) { - return metricRegistry.histogram(metricRegistryName(INFLIGHT_TIME, subscription.getTopicName(), subscription.getName())); - } - - public void unregisterInflightTimeHistogram(SubscriptionName subscription) { - unregister(INFLIGHT_TIME, subscription); - } - - public void registerConsumerHttpAnswer(SubscriptionName subscription, int statusCode, long count) { - PathContext pathContext = pathContext() - .withGroup(escapeDots(subscription.getTopicName().getGroupName())) - .withTopic(escapeDots(subscription.getTopicName().getName())) - .withSubscription(escapeDots(subscription.getName())) - .withHttpCode(statusCode) - .withHttpCodeFamily(httpStatusFamily(statusCode)) - .build(); - metricRegistry.meter(pathCompiler.compile(ERRORS_HTTP_BY_FAMILY, pathContext)).mark(count); - metricRegistry.meter(pathCompiler.compile(ERRORS_HTTP_BY_CODE, pathContext)).mark(count); - } - - public void unregisterStatusMeters(SubscriptionName subscription) { - String prefix = metricRegistryName(SUBSCRIPTION_STATUS, subscription.getTopicName(), subscription.getName()); - metricRegistry.removeMatching(MetricFilter.startsWith(prefix)); - } - - private String httpStatusFamily(int statusCode) { - return String.format("%dxx", statusCode / 100); - } - - public Meter consumerErrorsTimeoutMeter(SubscriptionName subscription) { - return metricRegistry.meter(metricRegistryName(ERRORS_TIMEOUTS, subscription.getTopicName(), subscription.getName())); - } - - public void unregisterConsumerErrorsTimeoutMeter(SubscriptionName subscription) { - unregister(ERRORS_TIMEOUTS, subscription); - } - - public Meter consumerErrorsOtherMeter(SubscriptionName subscription) { - return metricRegistry.meter(metricRegistryName(ERRORS_OTHER, subscription.getTopicName(), subscription.getName())); - } - - public void unregisterConsumerErrorsOtherMeter(SubscriptionName subscription) { - unregister(ERRORS_OTHER, subscription); - } - - public Timer consumersWorkloadRebalanceDurationTimer(String kafkaCluster) { - PathContext pathContext = pathContext().withKafkaCluster(kafkaCluster).build(); - return metricRegistry.timer(pathCompiler.compile(Timers.CONSUMER_WORKLOAD_REBALANCE_DURATION, pathContext)); - } - - public Timer oAuthProviderLatencyTimer(String oAuthProviderName) { - PathContext pathContext = pathContext() - .withOAuthProvider(escapeDots(oAuthProviderName)) - .build(); - return metricRegistry.timer(pathCompiler.compile(Timers.OAUTH_PROVIDER_TOKEN_REQUEST_LATENCY, pathContext)); - } - - public Meter oAuthSubscriptionTokenRequestMeter(Subscription subscription, String oAuthProviderName) { - PathContext pathContext = pathContext() - .withGroup(escapeDots(subscription.getTopicName().getGroupName())) - .withTopic(escapeDots(subscription.getTopicName().getName())) - .withSubscription(escapeDots(subscription.getName())) - .withOAuthProvider(escapeDots(oAuthProviderName)) - .build(); - return metricRegistry.meter(pathCompiler.compile(Meters.OAUTH_SUBSCRIPTION_TOKEN_REQUEST, pathContext)); - } - - public void registerRunningConsumerProcessesCountGauge(Gauge gauge) { - metricRegistry.register(metricRegistryName(Gauges.RUNNING_CONSUMER_PROCESSES_COUNT), gauge); - } - - public void registerDyingConsumerProcessesCountGauge(Gauge gauge) { - metricRegistry.register(metricRegistryName(Gauges.DYING_CONSUMER_PROCESSES_COUNT), gauge); - } -} - diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/Histograms.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/Histograms.java index a12b6f58b5..04238a427c 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/Histograms.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/Histograms.java @@ -1,12 +1,6 @@ package pl.allegro.tech.hermes.common.metric; -import static pl.allegro.tech.hermes.metrics.PathsCompiler.GROUP; -import static pl.allegro.tech.hermes.metrics.PathsCompiler.SUBSCRIPTION; -import static pl.allegro.tech.hermes.metrics.PathsCompiler.TOPIC; public class Histograms { - public static final String MESSAGE_SIZE = "message-size." + GROUP + "." + TOPIC; - public static final String GLOBAL_MESSAGE_SIZE = "message-size"; - public static final String INFLIGHT_TIME = "inflight." + GROUP + "." + TOPIC + "." + SUBSCRIPTION + ".time"; public static final String PERSISTED_UNDELIVERED_MESSAGE_SIZE = "undelivered-messages.persisted.message-size"; } diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/MaxRateMetrics.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/MaxRateMetrics.java index f1a759308d..18a458e52b 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/MaxRateMetrics.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/MaxRateMetrics.java @@ -9,55 +9,41 @@ import java.util.function.ToDoubleFunction; import static java.util.concurrent.TimeUnit.MILLISECONDS; -import static pl.allegro.tech.hermes.common.metric.Counters.MAXRATE_FETCH_FAILURES; -import static pl.allegro.tech.hermes.common.metric.Counters.MAXRATE_RATE_HISTORY_FAILURES; -import static pl.allegro.tech.hermes.common.metric.Gauges.MAX_RATE_ACTUAL_RATE_VALUE; -import static pl.allegro.tech.hermes.common.metric.Gauges.MAX_RATE_CALCULATION_DURATION; -import static pl.allegro.tech.hermes.common.metric.Gauges.MAX_RATE_VALUE; -import static pl.allegro.tech.hermes.common.metric.Gauges.OUTPUT_RATE; import static pl.allegro.tech.hermes.common.metric.SubscriptionTagsFactory.subscriptionTags; public class MaxRateMetrics { - private final HermesMetrics hermesMetrics; private final MeterRegistry meterRegistry; - MaxRateMetrics(HermesMetrics hermesMetrics, MeterRegistry meterRegistry) { - this.hermesMetrics = hermesMetrics; + MaxRateMetrics(MeterRegistry meterRegistry) { this.meterRegistry = meterRegistry; } public void registerCalculationDurationInMillisGauge(T obj, ToDoubleFunction f) { - hermesMetrics.registerGauge(MAX_RATE_CALCULATION_DURATION, () -> (int) f.applyAsDouble(obj)); meterRegistry.more().timeGauge("max-rate.calculation.duration", List.of(), obj, MILLISECONDS, f); } public HermesCounter historyUpdateFailuresCounter(SubscriptionName subscription) { return HermesCounters.from( - meterRegistry.counter("max-rate.history-update.failures", subscriptionTags(subscription)), - hermesMetrics.counter(MAXRATE_RATE_HISTORY_FAILURES, subscription.getTopicName(), subscription.getName()) + meterRegistry.counter("max-rate.history-update.failures", subscriptionTags(subscription)) ); } public HermesCounter fetchFailuresCounter(SubscriptionName subscription) { return HermesCounters.from( - meterRegistry.counter("max-rate.fetch.failures", subscriptionTags(subscription)), - hermesMetrics.counter(MAXRATE_FETCH_FAILURES, subscription.getTopicName(), subscription.getName()) + meterRegistry.counter("max-rate.fetch.failures", subscriptionTags(subscription)) ); } public void registerCalculatedRateGauge(SubscriptionName subscription, T obj, ToDoubleFunction f) { - hermesMetrics.registerGauge(MAX_RATE_VALUE, subscription, () -> (int) f.applyAsDouble(obj)); meterRegistry.gauge("max-rate.calculated-rate", subscriptionTags(subscription), obj, f); } public void registerActualRateGauge(SubscriptionName subscription, T obj, ToDoubleFunction f) { - hermesMetrics.registerGauge(MAX_RATE_ACTUAL_RATE_VALUE, subscription, () -> (int) f.applyAsDouble(obj)); meterRegistry.gauge("max-rate.actual-rate", subscriptionTags(subscription), obj, f); } public void registerOutputRateGauge(SubscriptionName subscription, T obj, ToDoubleFunction f) { - hermesMetrics.registerGauge(OUTPUT_RATE, subscription, () -> (int) f.applyAsDouble(obj)); meterRegistry.gauge("max-rate.output-rate", subscriptionTags(subscription), obj, f); } } diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/Meters.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/Meters.java index 9fd5a360f3..4438260971 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/Meters.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/Meters.java @@ -1,45 +1,12 @@ package pl.allegro.tech.hermes.common.metric; import static pl.allegro.tech.hermes.metrics.PathsCompiler.GROUP; -import static pl.allegro.tech.hermes.metrics.PathsCompiler.HTTP_CODE; -import static pl.allegro.tech.hermes.metrics.PathsCompiler.HTTP_CODE_FAMILY; -import static pl.allegro.tech.hermes.metrics.PathsCompiler.OAUTH_PROVIDER_NAME; -import static pl.allegro.tech.hermes.metrics.PathsCompiler.SUBSCRIPTION; import static pl.allegro.tech.hermes.metrics.PathsCompiler.TOPIC; public class Meters { - public static final String METER = "meter"; - public static final String TOPIC_METER = METER + "." + GROUP + "." + TOPIC; - public static final String SUBSCRIPTION_METER = TOPIC_METER + "." + SUBSCRIPTION; - public static final String FILTERED_METER = SUBSCRIPTION_METER + ".filtered"; - public static final String SUBSCRIPTION_BATCH_METER = TOPIC_METER + "." + SUBSCRIPTION + ".batch"; - public static final String FAILED_METER = "failed-meter"; - public static final String FAILED_TOPIC_METER = FAILED_METER + "." + GROUP + "." + TOPIC; - public static final String FAILED_METER_SUBSCRIPTION = FAILED_TOPIC_METER + "." + SUBSCRIPTION; public static final String THROUGHPUT_BYTES = "throughput"; public static final String TOPIC_THROUGHPUT_BYTES = THROUGHPUT_BYTES + "." + GROUP + "." + TOPIC; - public static final String SUBSCRIPTION_THROUGHPUT_BYTES = TOPIC_THROUGHPUT_BYTES + "." + SUBSCRIPTION; - public static final String STATUS_CODES = "http-status-codes.code" + HTTP_CODE; - public static final String TOPIC_STATUS_CODES = "http-status-codes." + GROUP + "." + TOPIC + ".code" + HTTP_CODE; - public static final String SUBSCRIPTION_STATUS = "status." + GROUP + "." + TOPIC + "." + SUBSCRIPTION; - public static final String ERRORS_TIMEOUTS = SUBSCRIPTION_STATUS + ".errors.timeout"; - public static final String ERRORS_OTHER = SUBSCRIPTION_STATUS + ".errors.other"; - public static final String ERRORS_HTTP_BY_FAMILY = SUBSCRIPTION_STATUS + "." + HTTP_CODE_FAMILY; - public static final String ERRORS_HTTP_BY_CODE = ERRORS_HTTP_BY_FAMILY + "." + HTTP_CODE; - public static final String DISCARDED_METER = "discarded-meter"; - public static final String DISCARDED_TOPIC_METER = DISCARDED_METER + "." + GROUP + "." + TOPIC; - public static final String DISCARDED_SUBSCRIPTION_METER = DISCARDED_TOPIC_METER + "." + SUBSCRIPTION; - public static final String RETRIES_METER = "retries-meter"; - public static final String RETRIES_TOPIC_METER = RETRIES_METER + "." + GROUP + "." + TOPIC; - public static final String RETRIES_SUBSCRIPTION_METER = RETRIES_TOPIC_METER + "." + SUBSCRIPTION; - public static final String DELAYED_PROCESSING = "delayed-processing"; - public static final String TOPIC_DELAYED_PROCESSING = DELAYED_PROCESSING + "." + GROUP + "." + TOPIC; - - public static final String TOPIC_DUPLICATED_MESSAGE = "duplicated-message" + "." + GROUP + "." + TOPIC; - - public static final String OAUTH_SUBSCRIPTION_TOKEN_REQUEST = "oauth.subscription." + GROUP + "." + TOPIC + "." + SUBSCRIPTION - + ".token-request." + OAUTH_PROVIDER_NAME; public static final String PERSISTED_UNDELIVERED_MESSAGES_METER = "undelivered-messages.persisted"; } diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/MetricRegistryWithHdrHistogramReservoir.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/MetricRegistryWithHdrHistogramReservoir.java deleted file mode 100644 index a8c931742d..0000000000 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/MetricRegistryWithHdrHistogramReservoir.java +++ /dev/null @@ -1,19 +0,0 @@ -package pl.allegro.tech.hermes.common.metric; - -import com.codahale.metrics.Histogram; -import com.codahale.metrics.MetricRegistry; -import com.codahale.metrics.Timer; -import org.mpierce.metrics.reservoir.hdrhistogram.HdrHistogramResetOnSnapshotReservoir; - -public class MetricRegistryWithHdrHistogramReservoir extends MetricRegistry { - - @Override - public Histogram histogram(String name) { - return histogram(name, () -> new Histogram(new HdrHistogramResetOnSnapshotReservoir())); - } - - @Override - public Timer timer(String name) { - return timer(name, () -> new Timer(new HdrHistogramResetOnSnapshotReservoir())); - } -} diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/MetricsFacade.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/MetricsFacade.java index ab57f56456..23f489aaf8 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/MetricsFacade.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/MetricsFacade.java @@ -7,30 +7,11 @@ import java.util.Collection; -import static pl.allegro.tech.hermes.common.metric.Counters.DELIVERED; -import static pl.allegro.tech.hermes.common.metric.Counters.DISCARDED; -import static pl.allegro.tech.hermes.common.metric.Counters.MAXRATE_FETCH_FAILURES; -import static pl.allegro.tech.hermes.common.metric.Counters.MAXRATE_RATE_HISTORY_FAILURES; -import static pl.allegro.tech.hermes.common.metric.Counters.RETRIES; -import static pl.allegro.tech.hermes.common.metric.Gauges.MAX_RATE_ACTUAL_RATE_VALUE; -import static pl.allegro.tech.hermes.common.metric.Gauges.MAX_RATE_VALUE; -import static pl.allegro.tech.hermes.common.metric.Gauges.OUTPUT_RATE; -import static pl.allegro.tech.hermes.common.metric.Meters.DISCARDED_SUBSCRIPTION_METER; -import static pl.allegro.tech.hermes.common.metric.Meters.FAILED_METER_SUBSCRIPTION; -import static pl.allegro.tech.hermes.common.metric.Meters.FILTERED_METER; -import static pl.allegro.tech.hermes.common.metric.Meters.RETRIES_SUBSCRIPTION_METER; -import static pl.allegro.tech.hermes.common.metric.Meters.SUBSCRIPTION_BATCH_METER; -import static pl.allegro.tech.hermes.common.metric.Meters.SUBSCRIPTION_METER; -import static pl.allegro.tech.hermes.common.metric.Meters.SUBSCRIPTION_THROUGHPUT_BYTES; import static pl.allegro.tech.hermes.common.metric.SubscriptionTagsFactory.subscriptionTags; -import static pl.allegro.tech.hermes.common.metric.Timers.CONSUMER_IDLE_TIME; -import static pl.allegro.tech.hermes.common.metric.Timers.RATE_LIMITER_ACQUIRE; -import static pl.allegro.tech.hermes.common.metric.Timers.SUBSCRIPTION_LATENCY; public class MetricsFacade { private final MeterRegistry meterRegistry; - private final HermesMetrics hermesMetrics; private final TopicMetrics topicMetrics; private final SubscriptionMetrics subscriptionMetrics; private final ConsumerMetrics consumerMetrics; @@ -47,23 +28,22 @@ public class MetricsFacade { private final MaxRateMetrics maxRateMetrics; private final BrokerMetrics brokerMetrics; - public MetricsFacade(MeterRegistry meterRegistry, HermesMetrics hermesMetrics) { + public MetricsFacade(MeterRegistry meterRegistry) { this.meterRegistry = meterRegistry; - this.hermesMetrics = hermesMetrics; - this.topicMetrics = new TopicMetrics(hermesMetrics, meterRegistry); - this.subscriptionMetrics = new SubscriptionMetrics(hermesMetrics, meterRegistry); - this.consumerMetrics = new ConsumerMetrics(hermesMetrics, meterRegistry); - this.trackerElasticSearchMetrics = new TrackerElasticSearchMetrics(hermesMetrics, meterRegistry); - this.persistentBufferMetrics = new PersistentBufferMetrics(hermesMetrics, meterRegistry); - this.producerMetrics = new ProducerMetrics(hermesMetrics, meterRegistry); + this.topicMetrics = new TopicMetrics(meterRegistry); + this.subscriptionMetrics = new SubscriptionMetrics(meterRegistry); + this.consumerMetrics = new ConsumerMetrics(meterRegistry); + this.trackerElasticSearchMetrics = new TrackerElasticSearchMetrics(meterRegistry); + this.persistentBufferMetrics = new PersistentBufferMetrics(meterRegistry); + this.producerMetrics = new ProducerMetrics(meterRegistry); this.executorMetrics = new ExecutorMetrics(meterRegistry); - this.schemaClientMetrics = new SchemaClientMetrics(hermesMetrics, meterRegistry); - this.undeliveredMessagesMetrics = new UndeliveredMessagesMetrics(hermesMetrics, meterRegistry); - this.deserializationMetrics = new DeserializationMetrics(hermesMetrics, meterRegistry); - this.workloadMetrics = new WorkloadMetrics(hermesMetrics, meterRegistry); - this.consumerSenderMetrics = new ConsumerSenderMetrics(hermesMetrics, meterRegistry); - this.offsetCommitsMetrics = new OffsetCommitsMetrics(hermesMetrics, meterRegistry); - this.maxRateMetrics = new MaxRateMetrics(hermesMetrics, meterRegistry); + this.schemaClientMetrics = new SchemaClientMetrics(meterRegistry); + this.undeliveredMessagesMetrics = new UndeliveredMessagesMetrics(meterRegistry); + this.deserializationMetrics = new DeserializationMetrics(meterRegistry); + this.workloadMetrics = new WorkloadMetrics(meterRegistry); + this.consumerSenderMetrics = new ConsumerSenderMetrics(meterRegistry); + this.offsetCommitsMetrics = new OffsetCommitsMetrics(meterRegistry); + this.maxRateMetrics = new MaxRateMetrics(meterRegistry); this.brokerMetrics = new BrokerMetrics(meterRegistry); } @@ -134,28 +114,5 @@ public void unregisterAllMetricsRelatedTo(SubscriptionName subscription) { for (Meter meter : meters) { meterRegistry.remove(meter); } - hermesMetrics.unregister(DISCARDED_SUBSCRIPTION_METER, subscription); - hermesMetrics.unregister(RETRIES_SUBSCRIPTION_METER, subscription); - hermesMetrics.unregister(FAILED_METER_SUBSCRIPTION, subscription); - hermesMetrics.unregister(SUBSCRIPTION_BATCH_METER, subscription); - hermesMetrics.unregister(SUBSCRIPTION_METER, subscription); - hermesMetrics.unregister(DELIVERED, subscription); - hermesMetrics.unregister(DISCARDED, subscription); - hermesMetrics.unregister(RETRIES, subscription); - hermesMetrics.unregisterInflightGauge(subscription); - hermesMetrics.unregisterInflightTimeHistogram(subscription); - hermesMetrics.unregisterConsumerErrorsTimeoutMeter(subscription); - hermesMetrics.unregisterConsumerErrorsOtherMeter(subscription); - hermesMetrics.unregisterStatusMeters(subscription); - hermesMetrics.unregister(OUTPUT_RATE, subscription); - hermesMetrics.unregister(MAX_RATE_ACTUAL_RATE_VALUE, subscription); - hermesMetrics.unregister(MAX_RATE_VALUE, subscription); - hermesMetrics.unregister(MAXRATE_FETCH_FAILURES, subscription); - hermesMetrics.unregister(MAXRATE_RATE_HISTORY_FAILURES, subscription); - hermesMetrics.unregister(CONSUMER_IDLE_TIME, subscription); - hermesMetrics.unregister(FILTERED_METER, subscription); - hermesMetrics.unregister(SUBSCRIPTION_LATENCY, subscription); - hermesMetrics.unregister(RATE_LIMITER_ACQUIRE, subscription); - hermesMetrics.unregister(SUBSCRIPTION_THROUGHPUT_BYTES, subscription); } } diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/OffsetCommitsMetrics.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/OffsetCommitsMetrics.java index 84527282a4..b3b1376edb 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/OffsetCommitsMetrics.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/OffsetCommitsMetrics.java @@ -7,46 +7,39 @@ public class OffsetCommitsMetrics { - private final HermesMetrics hermesMetrics; private final MeterRegistry meterRegistry; - OffsetCommitsMetrics(HermesMetrics hermesMetrics, MeterRegistry meterRegistry) { - this.hermesMetrics = hermesMetrics; + OffsetCommitsMetrics(MeterRegistry meterRegistry) { this.meterRegistry = meterRegistry; } public HermesCounter skippedCounter() { return HermesCounters.from( - meterRegistry.counter("offset-commits.skipped"), - hermesMetrics.counter("offset-committer.skipped") + meterRegistry.counter("offset-commits.skipped") ); } public HermesCounter obsoleteCounter() { return HermesCounters.from( - meterRegistry.counter("offset-commits.obsolete"), - hermesMetrics.counter("offset-committer.obsolete") + meterRegistry.counter("offset-commits.obsolete") ); } public HermesCounter committedCounter() { return HermesCounters.from( - meterRegistry.counter("offset-commits.committed"), - hermesMetrics.counter("offset-committer.committed") + meterRegistry.counter("offset-commits.committed") ); } public HermesTimer duration() { return HermesTimer.from( - meterRegistry.timer("offset-commits.duration"), - hermesMetrics.timer("offset-committer.duration") + meterRegistry.timer("offset-commits.duration") ); } public HermesCounter failuresCounter() { return HermesCounters.from( - meterRegistry.counter("offset-commits.failures"), - hermesMetrics.counter("offset-committer.failed") + meterRegistry.counter("offset-commits.failures") ); } } diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/PersistentBufferMetrics.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/PersistentBufferMetrics.java index b67c522807..798e3f57fd 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/PersistentBufferMetrics.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/PersistentBufferMetrics.java @@ -8,15 +8,12 @@ public class PersistentBufferMetrics { private final MeterRegistry meterRegistry; - private final HermesMetrics hermesMetrics; - public PersistentBufferMetrics(HermesMetrics hermesMetrics, MeterRegistry meterRegistry) { + public PersistentBufferMetrics(MeterRegistry meterRegistry) { this.meterRegistry = meterRegistry; - this.hermesMetrics = hermesMetrics; } public void registerBackupStorageSizeGauge(T obj, ToDoubleFunction f) { - hermesMetrics.registerMessageRepositorySizeGauge(() -> (int) f.applyAsDouble(obj)); meterRegistry.gauge(BACKUP_STORAGE_SIZE, obj, f); } } diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/ProducerMetrics.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/ProducerMetrics.java index 42f95521e6..0c1b65b913 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/ProducerMetrics.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/ProducerMetrics.java @@ -9,14 +9,12 @@ import static pl.allegro.tech.hermes.common.metric.Gauges.INFLIGHT_REQUESTS; public class ProducerMetrics { - private final HermesMetrics hermesMetrics; private final MeterRegistry meterRegistry; private final GaugeRegistrar gaugeRegistrar; - public ProducerMetrics(HermesMetrics hermesMetrics, MeterRegistry meterRegistry) { - this.hermesMetrics = hermesMetrics; + public ProducerMetrics(MeterRegistry meterRegistry) { this.meterRegistry = meterRegistry; - this.gaugeRegistrar = new GaugeRegistrar(meterRegistry, hermesMetrics); + this.gaugeRegistrar = new GaugeRegistrar(meterRegistry); } public void registerAckAllTotalBytesGauge(T stateObj, ToDoubleFunction f, String sender, String datacenter) { @@ -52,20 +50,19 @@ public void registerAckLeaderFailedBatchesGauge(T stateObj, ToDoubleFunction } public void registerAckAllMetadataAgeGauge(T stateObj, ToDoubleFunction f, String sender, String datacenter) { - registerTimeGauge(stateObj, f, ACK_ALL_METADATA_AGE, ACK_ALL_METADATA_AGE, tags(sender, datacenter), TimeUnit.SECONDS); + registerTimeGauge(stateObj, f, ACK_ALL_METADATA_AGE, tags(sender, datacenter), TimeUnit.SECONDS); } public void registerAckLeaderMetadataAgeGauge(T stateObj, ToDoubleFunction f, String sender, String datacenter) { - registerTimeGauge(stateObj, f, ACK_LEADER_METADATA_AGE, ACK_LEADER_METADATA_AGE, tags(sender, datacenter), TimeUnit.SECONDS); + registerTimeGauge(stateObj, f, ACK_LEADER_METADATA_AGE, tags(sender, datacenter), TimeUnit.SECONDS); } public void registerAckAllRecordQueueTimeMaxGauge(T stateObj, ToDoubleFunction f, String sender, String datacenter) { - registerTimeGauge(stateObj, f, ACK_ALL_RECORD_QUEUE_TIME_MAX, ACK_ALL_RECORD_QUEUE_TIME_MAX, tags(sender, datacenter), TimeUnit.MILLISECONDS); + registerTimeGauge(stateObj, f, ACK_ALL_RECORD_QUEUE_TIME_MAX, tags(sender, datacenter), TimeUnit.MILLISECONDS); } public void registerAckLeaderRecordQueueTimeMaxGauge(T stateObj, ToDoubleFunction f, String sender, String datacenter) { - registerTimeGauge(stateObj, f, ACK_LEADER_RECORD_QUEUE_TIME_MAX, - ACK_LEADER_RECORD_QUEUE_TIME_MAX, tags(sender, datacenter), TimeUnit.MILLISECONDS); + registerTimeGauge(stateObj, f, ACK_LEADER_RECORD_QUEUE_TIME_MAX, tags(sender, datacenter), TimeUnit.MILLISECONDS); } public double getBufferTotalBytes() { @@ -88,7 +85,6 @@ public void registerAckAllRecordSendCounter(T stateObj, ToDoubleFunction public void registerProducerInflightRequestGauge(T stateObj, ToDoubleFunction f) { meterRegistry.gauge(INFLIGHT_REQUESTS, stateObj, f); - hermesMetrics.registerProducerInflightRequest(() -> (int) f.applyAsDouble(stateObj)); } private static Tags tags(String sender, String datacenter) { @@ -98,12 +94,10 @@ private static Tags tags(String sender, String datacenter) { private void registerTimeGauge(T stateObj, ToDoubleFunction f, - String graphiteName, - String prometheusName, + String name, Tags tags, TimeUnit timeUnit) { - hermesMetrics.registerGauge(graphiteName, () -> f.applyAsDouble(stateObj)); - meterRegistry.more().timeGauge(prometheusName, tags, stateObj, timeUnit, f); + meterRegistry.more().timeGauge(name, tags, stateObj, timeUnit, f); } private void registerCounter(String name, Tags tags, T stateObj, ToDoubleFunction f) { diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/SchemaClientMetrics.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/SchemaClientMetrics.java index a32f1d6910..020bffcc83 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/SchemaClientMetrics.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/SchemaClientMetrics.java @@ -5,29 +5,22 @@ import io.micrometer.core.instrument.Timer; import pl.allegro.tech.hermes.metrics.HermesTimer; -import static pl.allegro.tech.hermes.common.metric.Timers.GET_SCHEMA_LATENCY; -import static pl.allegro.tech.hermes.common.metric.Timers.GET_SCHEMA_VERSIONS_LATENCY; - public class SchemaClientMetrics { - private final HermesMetrics hermesMetrics; private final MeterRegistry meterRegistry; - public SchemaClientMetrics(HermesMetrics hermesMetrics, MeterRegistry meterRegistry) { - this.hermesMetrics = hermesMetrics; + public SchemaClientMetrics(MeterRegistry meterRegistry) { this.meterRegistry = meterRegistry; } public HermesTimer schemaTimer() { return HermesTimer.from( - timer("schema.get-schema"), - hermesMetrics.schemaTimer(GET_SCHEMA_LATENCY) + timer("schema.get-schema") ); } public HermesTimer versionsTimer() { return HermesTimer.from( - timer("schema.get-versions"), - hermesMetrics.schemaTimer(GET_SCHEMA_VERSIONS_LATENCY) + timer("schema.get-versions") ); } diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/SubscriptionHermesCounter.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/SubscriptionHermesCounter.java index 73da4f18f1..2d9d237b5a 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/SubscriptionHermesCounter.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/SubscriptionHermesCounter.java @@ -1,31 +1,23 @@ package pl.allegro.tech.hermes.common.metric; -import com.codahale.metrics.Meter; import io.micrometer.core.instrument.Counter; import pl.allegro.tech.hermes.api.SubscriptionName; -import pl.allegro.tech.hermes.metrics.counters.MeterBackedHermesCounter; +import pl.allegro.tech.hermes.metrics.counters.DefaultHermesCounter; -public class SubscriptionHermesCounter extends MeterBackedHermesCounter { +public class SubscriptionHermesCounter extends DefaultHermesCounter { - private final String graphiteName; private final SubscriptionName subscription; private SubscriptionHermesCounter(Counter micrometerCounter, - Meter graphiteMeter, - String graphiteName, SubscriptionName subscription) { - super(micrometerCounter, graphiteMeter); - this.graphiteName = graphiteName; + SubscriptionName subscription) { + super(micrometerCounter); this.subscription = subscription; } - public static SubscriptionHermesCounter from(Counter micrometerCounter, Meter graphiteMeter, - String graphiteName, SubscriptionName subscription) { - return new SubscriptionHermesCounter(micrometerCounter, graphiteMeter, graphiteName, subscription); + public static SubscriptionHermesCounter from(Counter micrometerCounter, SubscriptionName subscription) { + return new SubscriptionHermesCounter(micrometerCounter, subscription); } - String getGraphiteName() { - return graphiteName; - } SubscriptionName getSubscription() { return subscription; diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/SubscriptionMetrics.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/SubscriptionMetrics.java index 4f4916bc05..95a1b85369 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/SubscriptionMetrics.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/SubscriptionMetrics.java @@ -15,130 +15,94 @@ import static pl.allegro.tech.hermes.common.metric.SubscriptionTagsFactory.subscriptionTags; public class SubscriptionMetrics { - private final HermesMetrics hermesMetrics; private final MeterRegistry meterRegistry; - public SubscriptionMetrics(HermesMetrics hermesMetrics, MeterRegistry meterRegistry) { - this.hermesMetrics = hermesMetrics; + public SubscriptionMetrics(MeterRegistry meterRegistry) { this.meterRegistry = meterRegistry; } public SubscriptionHermesCounter throughputInBytes(SubscriptionName subscription) { return SubscriptionHermesCounter.from( - micrometerCounter(SubscriptionMetricsNames.SUBSCRIPTION_THROUGHPUT, subscription), - hermesMetrics.meter(Meters.SUBSCRIPTION_THROUGHPUT_BYTES, subscription.getTopicName(), subscription.getName()), - Meters.SUBSCRIPTION_THROUGHPUT_BYTES, subscription); + micrometerCounter(SubscriptionMetricsNames.SUBSCRIPTION_THROUGHPUT, subscription), subscription); } public HermesCounter successes(SubscriptionName subscription) { - return size -> { - hermesMetrics.meter(Meters.METER).mark(size); - hermesMetrics.meter(Meters.TOPIC_METER, subscription.getTopicName()).mark(size); - hermesMetrics.meter(Meters.SUBSCRIPTION_METER, subscription.getTopicName(), subscription.getName()).mark(size); - hermesMetrics.counter(Counters.DELIVERED, subscription.getTopicName(), subscription.getName()).inc(size); - micrometerCounter(SubscriptionMetricsNames.SUBSCRIPTION_DELIVERED, subscription).increment(size); - }; + return size -> micrometerCounter(SubscriptionMetricsNames.SUBSCRIPTION_DELIVERED, subscription).increment(size); } public HermesCounter batchSuccesses(SubscriptionName subscription) { return HermesCounters.from( - micrometerCounter(SubscriptionMetricsNames.SUBSCRIPTION_BATCHES, subscription), - hermesMetrics.meter(Meters.SUBSCRIPTION_BATCH_METER, subscription.getTopicName(), subscription.getName()) + micrometerCounter(SubscriptionMetricsNames.SUBSCRIPTION_BATCHES, subscription) ); } public HermesCounter discarded(SubscriptionName subscription) { - return size -> { - hermesMetrics.meter(Meters.DISCARDED_METER).mark(size); - hermesMetrics.meter(Meters.DISCARDED_TOPIC_METER, subscription.getTopicName()).mark(size); - hermesMetrics.meter(Meters.DISCARDED_SUBSCRIPTION_METER, subscription.getTopicName(), subscription.getName()).mark(size); - hermesMetrics.counter(Counters.DISCARDED, subscription.getTopicName(), subscription.getName()).inc(size); - micrometerCounter(SubscriptionMetricsNames.SUBSCRIPTION_DISCARDED, subscription).increment(size); - }; + return size -> micrometerCounter(SubscriptionMetricsNames.SUBSCRIPTION_DISCARDED, subscription).increment(size); + } public HermesCounter retries(SubscriptionName subscription) { - return size -> { - hermesMetrics.meter(Meters.RETRIES_METER).mark(size); - hermesMetrics.meter(Meters.RETRIES_TOPIC_METER, subscription.getTopicName()).mark(size); - hermesMetrics.meter(Meters.RETRIES_SUBSCRIPTION_METER, subscription.getTopicName(), subscription.getName()).mark(size); - hermesMetrics.counter(Counters.RETRIES, subscription.getTopicName(), subscription.getName()).inc(size); - micrometerCounter(SubscriptionMetricsNames.SUBSCRIPTION_RETRIES, subscription).increment(size); - }; + return size -> micrometerCounter(SubscriptionMetricsNames.SUBSCRIPTION_RETRIES, subscription).increment(size); } public HermesTimer latency(SubscriptionName subscription) { return HermesTimer.from( - meterRegistry.timer(SubscriptionMetricsNames.SUBSCRIPTION_LATENCY, subscriptionTags(subscription)), - hermesMetrics.timer(Timers.SUBSCRIPTION_LATENCY, subscription.getTopicName(), subscription.getName()) + meterRegistry.timer(SubscriptionMetricsNames.SUBSCRIPTION_LATENCY, subscriptionTags(subscription)) ); } public HermesTimer rateLimiterAcquire(SubscriptionName subscription) { return HermesTimer.from( - meterRegistry.timer(SubscriptionMetricsNames.SUBSCRIPTION_RATE_LIMITER_ACQUIRE, subscriptionTags(subscription)), - hermesMetrics.timer(Timers.RATE_LIMITER_ACQUIRE, subscription.getTopicName(), subscription.getName()) + meterRegistry.timer(SubscriptionMetricsNames.SUBSCRIPTION_RATE_LIMITER_ACQUIRE, subscriptionTags(subscription)) ); } public void registerInflightGauge(SubscriptionName subscription, T obj, ToDoubleFunction f) { - hermesMetrics.registerInflightGauge(subscription, () -> (int) f.applyAsDouble(obj)); meterRegistry.gauge(SubscriptionMetricsNames.SUBSCRIPTION_INFLIGHT, subscriptionTags(subscription), obj, f); } public HermesTimer consumerIdleTimer(SubscriptionName subscription) { return HermesTimer.from( - meterRegistry.timer(SubscriptionMetricsNames.SUBSCRIPTION_IDLE_DURATION, subscriptionTags(subscription)), - hermesMetrics.timer(Timers.CONSUMER_IDLE_TIME, subscription.getTopicName(), subscription.getName()) + meterRegistry.timer(SubscriptionMetricsNames.SUBSCRIPTION_IDLE_DURATION, subscriptionTags(subscription)) ); } public HermesCounter filteredOutCounter(SubscriptionName subscription) { return HermesCounters.from( - micrometerCounter(SubscriptionMetricsNames.SUBSCRIPTION_FILTERED_OUT, subscription), - hermesMetrics.meter(Meters.FILTERED_METER, subscription.getTopicName(), subscription.getName()) + micrometerCounter(SubscriptionMetricsNames.SUBSCRIPTION_FILTERED_OUT, subscription) ); } public HermesCounter httpAnswerCounter(SubscriptionName subscription, int statusCode) { - return size -> { - meterRegistry.counter( - SubscriptionMetricsNames.SUBSCRIPTION_HTTP_STATUS_CODES, - Tags.concat(subscriptionTags(subscription), "status_code", String.valueOf(statusCode)) - ).increment(size); - hermesMetrics.registerConsumerHttpAnswer(subscription, statusCode, size); - }; + return size -> meterRegistry.counter( + SubscriptionMetricsNames.SUBSCRIPTION_HTTP_STATUS_CODES, + Tags.concat(subscriptionTags(subscription), "status_code", String.valueOf(statusCode)) + ).increment(size); } public HermesCounter timeoutsCounter(SubscriptionName subscription) { return HermesCounters.from( - micrometerCounter(SubscriptionMetricsNames.SUBSCRIPTION_TIMEOUTS, subscription), - hermesMetrics.consumerErrorsTimeoutMeter(subscription) + micrometerCounter(SubscriptionMetricsNames.SUBSCRIPTION_TIMEOUTS, subscription) ); } public HermesCounter otherErrorsCounter(SubscriptionName subscription) { return HermesCounters.from( - micrometerCounter(SubscriptionMetricsNames.SUBSCRIPTION_OTHER_ERRORS, subscription), - hermesMetrics.consumerErrorsOtherMeter(subscription) + micrometerCounter(SubscriptionMetricsNames.SUBSCRIPTION_OTHER_ERRORS, subscription) ); } public HermesCounter failuresCounter(SubscriptionName subscription) { return HermesCounters.from( - micrometerCounter(SubscriptionMetricsNames.SUBSCRIPTION_FAILURES, subscription), - hermesMetrics.meter(Meters.FAILED_METER_SUBSCRIPTION, subscription.getTopicName(), subscription.getName()) + micrometerCounter(SubscriptionMetricsNames.SUBSCRIPTION_FAILURES, subscription) ); } public HermesHistogram inflightTimeInMillisHistogram(SubscriptionName subscriptionName) { - return value -> { - DistributionSummary.builder(SubscriptionMetricsNames.SUBSCRIPTION_INFLIGHT_TIME) - .tags(subscriptionTags(subscriptionName)) - .register(meterRegistry) - .record(value / 1000d); - hermesMetrics.inflightTimeHistogram(subscriptionName).update(value); - }; + return value -> DistributionSummary.builder(SubscriptionMetricsNames.SUBSCRIPTION_INFLIGHT_TIME) + .tags(subscriptionTags(subscriptionName)) + .register(meterRegistry) + .record(value / 1000d); } private Counter micrometerCounter(String metricName, SubscriptionName subscription) { diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/Timers.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/Timers.java deleted file mode 100644 index f130b21410..0000000000 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/Timers.java +++ /dev/null @@ -1,33 +0,0 @@ -package pl.allegro.tech.hermes.common.metric; - -import static pl.allegro.tech.hermes.metrics.PathsCompiler.GROUP; -import static pl.allegro.tech.hermes.metrics.PathsCompiler.KAFKA_CLUSTER; -import static pl.allegro.tech.hermes.metrics.PathsCompiler.OAUTH_PROVIDER_NAME; -import static pl.allegro.tech.hermes.metrics.PathsCompiler.SCHEMA_REPO_TYPE; -import static pl.allegro.tech.hermes.metrics.PathsCompiler.SUBSCRIPTION; -import static pl.allegro.tech.hermes.metrics.PathsCompiler.TOPIC; - -public class Timers { - - public static final String ACK_ALL_BROKER_LATENCY = "ack-all.broker-latency"; - public static final String ACK_LEADER_BROKER_LATENCY = "ack-leader.broker-latency"; - - public static final String ACK_ALL_LATENCY = "ack-all.latency"; - public static final String ACK_ALL_TOPIC_LATENCY = ACK_ALL_LATENCY + "." + GROUP + "." + TOPIC; - - public static final String ACK_LEADER_LATENCY = "ack-leader.latency"; - public static final String ACK_LEADER_TOPIC_LATENCY = ACK_LEADER_LATENCY + "." + GROUP + "." + TOPIC; - - public static final String LATENCY = "latency"; - public static final String SUBSCRIPTION_LATENCY = LATENCY + "." + GROUP + "." + TOPIC + "." + SUBSCRIPTION; - public static final String RATE_LIMITER_ACQUIRE = "rate-limiter-acquire" + "." + GROUP + "." + TOPIC + "." + SUBSCRIPTION; - - public static final String SCHEMA = "schema." + SCHEMA_REPO_TYPE; - public static final String GET_SCHEMA_LATENCY = SCHEMA + ".get-schema"; - public static final String GET_SCHEMA_VERSIONS_LATENCY = SCHEMA + ".get-schema-versions"; - - public static final String CONSUMER_WORKLOAD_REBALANCE_DURATION = "consumers-workload." + KAFKA_CLUSTER + ".rebalance-duration"; - public static final String CONSUMER_IDLE_TIME = "idle-time." + GROUP + "." + TOPIC + "." + SUBSCRIPTION; - - public static final String OAUTH_PROVIDER_TOKEN_REQUEST_LATENCY = "oauth.provider." + OAUTH_PROVIDER_NAME + ".token-request-latency"; -} diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/TopicMetrics.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/TopicMetrics.java index bab7b92e01..e7ae1dcf89 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/TopicMetrics.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/TopicMetrics.java @@ -12,132 +12,110 @@ import pl.allegro.tech.hermes.metrics.HermesHistogram; import pl.allegro.tech.hermes.metrics.HermesTimer; import pl.allegro.tech.hermes.metrics.counters.HermesCounters; -import pl.allegro.tech.hermes.metrics.counters.MeterBackedHermesCounter; - -import static pl.allegro.tech.hermes.common.metric.Meters.DELAYED_PROCESSING; public class TopicMetrics { - private final HermesMetrics hermesMetrics; private final MeterRegistry meterRegistry; - public TopicMetrics(HermesMetrics hermesMetrics, MeterRegistry meterRegistry) { - this.hermesMetrics = hermesMetrics; + public TopicMetrics(MeterRegistry meterRegistry) { this.meterRegistry = meterRegistry; } public HermesTimer ackAllGlobalLatency() { return HermesTimer.from( - meterRegistry.timer(TopicMetricsNames.TOPIC_ACK_ALL_GLOBAL_LATENCY), - hermesMetrics.timer(Timers.ACK_ALL_LATENCY) + meterRegistry.timer(TopicMetricsNames.TOPIC_ACK_ALL_GLOBAL_LATENCY) ); } public HermesTimer ackAllTopicLatency(TopicName topic) { return HermesTimer.from( - micrometerTimer(TopicMetricsNames.TOPIC_ACK_ALL_LATENCY, topic), - hermesMetrics.timer(Timers.ACK_ALL_TOPIC_LATENCY, topic)); + micrometerTimer(TopicMetricsNames.TOPIC_ACK_ALL_LATENCY, topic)); } public HermesTimer ackAllBrokerLatency() { return HermesTimer.from( - meterRegistry.timer(TopicMetricsNames.TOPIC_ACK_ALL_BROKER_LATENCY), - hermesMetrics.timer(Timers.ACK_ALL_BROKER_LATENCY)); + meterRegistry.timer(TopicMetricsNames.TOPIC_ACK_ALL_BROKER_LATENCY)); } public HermesTimer ackLeaderGlobalLatency() { return HermesTimer.from( - meterRegistry.timer(TopicMetricsNames.TOPIC_ACK_LEADER_GLOBAL_LATENCY), - hermesMetrics.timer(Timers.ACK_LEADER_LATENCY)); + meterRegistry.timer(TopicMetricsNames.TOPIC_ACK_LEADER_GLOBAL_LATENCY)); } public HermesTimer ackLeaderTopicLatency(TopicName topic) { return HermesTimer.from( - micrometerTimer(TopicMetricsNames.TOPIC_ACK_LEADER_LATENCY, topic), - hermesMetrics.timer(Timers.ACK_LEADER_TOPIC_LATENCY, topic)); + micrometerTimer(TopicMetricsNames.TOPIC_ACK_LEADER_LATENCY, topic)); } public HermesTimer ackLeaderBrokerLatency() { return HermesTimer.from( - meterRegistry.timer(TopicMetricsNames.TOPIC_ACK_LEADER_BROKER_LATENCY), - hermesMetrics.timer(Timers.ACK_LEADER_BROKER_LATENCY)); + meterRegistry.timer(TopicMetricsNames.TOPIC_ACK_LEADER_BROKER_LATENCY)); } - public MeterBackedHermesCounter topicThroughputBytes(TopicName topicName) { + public HermesCounter topicThroughputBytes(TopicName topicName) { return HermesCounters.from( - micrometerCounter(TopicMetricsNames.TOPIC_THROUGHPUT, topicName), - hermesMetrics.meter(Meters.TOPIC_THROUGHPUT_BYTES, topicName) + micrometerCounter(TopicMetricsNames.TOPIC_THROUGHPUT, topicName) ); } - public MeterBackedHermesCounter topicGlobalThroughputBytes() { + public HermesCounter topicGlobalThroughputBytes() { return HermesCounters.from( - meterRegistry.counter(TopicMetricsNames.TOPIC_GLOBAL_THROUGHPUT), - hermesMetrics.meter(Meters.THROUGHPUT_BYTES) + meterRegistry.counter(TopicMetricsNames.TOPIC_GLOBAL_THROUGHPUT) ); } public HermesCounter topicPublished(TopicName topicName, String datacenter) { return HermesCounters.from( - micrometerCounter(TopicMetricsNames.TOPIC_PUBLISHED, topicName, Tag.of("storageDc", datacenter)), - hermesMetrics.counter(Counters.PUBLISHED, topicName) + micrometerCounter(TopicMetricsNames.TOPIC_PUBLISHED, topicName, Tag.of("storageDc", datacenter)) ); } public HermesCounter topicGlobalRequestCounter() { return HermesCounters.from( - meterRegistry.counter(TopicMetricsNames.TOPIC_GLOBAL_REQUESTS), - hermesMetrics.meter(Meters.METER) + meterRegistry.counter(TopicMetricsNames.TOPIC_GLOBAL_REQUESTS) ); } public HermesCounter topicRequestCounter(TopicName topicName) { return HermesCounters.from( - micrometerCounter(TopicMetricsNames.TOPIC_REQUESTS, topicName), - hermesMetrics.meter(Meters.TOPIC_METER, topicName) + micrometerCounter(TopicMetricsNames.TOPIC_REQUESTS, topicName) ); } public HermesCounter topicGlobalDelayedProcessingCounter() { return HermesCounters.from( - meterRegistry.counter(TopicMetricsNames.TOPIC_GLOBAL_DELAYED_PROCESSING), - hermesMetrics.meter(DELAYED_PROCESSING) + meterRegistry.counter(TopicMetricsNames.TOPIC_GLOBAL_DELAYED_PROCESSING) ); } public HermesCounter topicDelayedProcessingCounter(TopicName topicName) { return HermesCounters.from( - micrometerCounter(TopicMetricsNames.TOPIC_DELAYED_PROCESSING, topicName), - hermesMetrics.meter(Meters.TOPIC_DELAYED_PROCESSING, topicName) + micrometerCounter(TopicMetricsNames.TOPIC_DELAYED_PROCESSING, topicName) ); } public HermesCounter topicGlobalHttpStatusCodeCounter(int statusCode) { return HermesCounters.from( - meterRegistry.counter(TopicMetricsNames.TOPIC_GLOBAL_HTTP_STATUS_CODES, Tags.of("status_code", String.valueOf(statusCode))), - hermesMetrics.httpStatusCodeMeter(statusCode) + meterRegistry.counter(TopicMetricsNames.TOPIC_GLOBAL_HTTP_STATUS_CODES, Tags.of("status_code", String.valueOf(statusCode))) ); } public HermesCounter topicHttpStatusCodeCounter(TopicName topicName, int statusCode) { return HermesCounters.from( meterRegistry.counter(TopicMetricsNames.TOPIC_HTTP_STATUS_CODES, topicTags(topicName) - .and("status_code", String.valueOf(statusCode))), - hermesMetrics.httpStatusCodeMeter(statusCode, topicName) + .and("status_code", String.valueOf(statusCode))) ); } public HermesCounter topicDuplicatedMessageCounter(TopicName topicName) { return HermesCounters.from( - micrometerCounter(TopicMetricsNames.TOPIC_DUPLICATED_MESSAGE, topicName), - hermesMetrics.meter(Meters.TOPIC_DUPLICATED_MESSAGE, topicName) + micrometerCounter(TopicMetricsNames.TOPIC_DUPLICATED_MESSAGE, topicName) ); } public HermesHistogram topicGlobalMessageContentSizeHistogram() { return DefaultHermesHistogram.of( DistributionSummary.builder(TopicMetricsNames.TOPIC_GLOBAL_MESSAGE_SIZE_BYTES) - .register(meterRegistry), - hermesMetrics.messageContentSizeHistogram() + .register(meterRegistry) ); } @@ -145,8 +123,7 @@ public HermesHistogram topicMessageContentSizeHistogram(TopicName topicName) { return DefaultHermesHistogram.of( DistributionSummary.builder(TopicMetricsNames.TOPIC_MESSAGE_SIZE_BYTES) .tags(topicTags(topicName)) - .register(meterRegistry), - hermesMetrics.messageContentSizeHistogram(topicName) + .register(meterRegistry) ); } diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/TrackerElasticSearchMetrics.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/TrackerElasticSearchMetrics.java index b9643f5a30..657fb68c1e 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/TrackerElasticSearchMetrics.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/TrackerElasticSearchMetrics.java @@ -5,75 +5,52 @@ import java.util.function.ToDoubleFunction; -import static pl.allegro.tech.hermes.metrics.PathsCompiler.HOSTNAME; - public class TrackerElasticSearchMetrics { private final MeterRegistry meterRegistry; - private final HermesMetrics hermesMetrics; private final GaugeRegistrar gaugeRegistrar; - public TrackerElasticSearchMetrics(HermesMetrics hermesMetrics, MeterRegistry meterRegistry) { + public TrackerElasticSearchMetrics(MeterRegistry meterRegistry) { this.meterRegistry = meterRegistry; - this.hermesMetrics = hermesMetrics; - this.gaugeRegistrar = new GaugeRegistrar(meterRegistry, hermesMetrics); + this.gaugeRegistrar = new GaugeRegistrar(meterRegistry); } public void registerProducerTrackerElasticSearchQueueSizeGauge(T stateObj, ToDoubleFunction f) { gaugeRegistrar.registerGauge( - Gauges.Graphite.PRODUCER_TRACKER_ELASTICSEARCH_QUEUE_SIZE, - Gauges.Prometheus.TRACKER_ELASTICSEARCH_QUEUE_SIZE, + Gauges.TRACKER_ELASTICSEARCH_QUEUE_SIZE, stateObj, f ); } public void registerProducerTrackerElasticSearchRemainingCapacity(T stateObj, ToDoubleFunction f) { gaugeRegistrar.registerGauge( - Gauges.Graphite.PRODUCER_TRACKER_ELASTICSEARCH_REMAINING_CAPACITY, - Gauges.Prometheus.TRACKER_ELASTICSEARCH_REMAINING_CAPACITY, + Gauges.TRACKER_ELASTICSEARCH_REMAINING_CAPACITY, stateObj, f ); } public void registerConsumerTrackerElasticSearchQueueSizeGauge(T stateObj, ToDoubleFunction f) { gaugeRegistrar.registerGauge( - Gauges.Graphite.CONSUMER_TRACKER_ELASTICSEARCH_QUEUE_SIZE, - Gauges.Prometheus.TRACKER_ELASTICSEARCH_QUEUE_SIZE, + Gauges.TRACKER_ELASTICSEARCH_QUEUE_SIZE, stateObj, f ); } public void registerConsumerTrackerElasticSearchRemainingCapacity(T stateObj, ToDoubleFunction f) { gaugeRegistrar.registerGauge( - Gauges.Graphite.CONSUMER_TRACKER_ELASTICSEARCH_REMAINING_CAPACITY, - Gauges.Prometheus.TRACKER_ELASTICSEARCH_REMAINING_CAPACITY, + Gauges.TRACKER_ELASTICSEARCH_REMAINING_CAPACITY, stateObj, f ); } public HermesTimer trackerElasticSearchCommitLatencyTimer() { return HermesTimer.from( - meterRegistry.timer(Timers.ELASTICSEARCH_COMMIT_LATENCY), - hermesMetrics.timer(Timers.ELASTICSEARCH_COMMIT_LATENCY) + meterRegistry.timer(Timers.ELASTICSEARCH_COMMIT_LATENCY) ); } private static class Gauges { - private static class Graphite { - public static final String PRODUCER_TRACKER_ELASTICSEARCH_QUEUE_SIZE = - "producer." + HOSTNAME + ".tracker.elasticsearch.queue-size"; - public static final String PRODUCER_TRACKER_ELASTICSEARCH_REMAINING_CAPACITY = - "producer." + HOSTNAME + ".tracker.elasticsearch.remaining-capacity"; - - public static final String CONSUMER_TRACKER_ELASTICSEARCH_QUEUE_SIZE = - "consumer." + HOSTNAME + ".tracker.elasticsearch.queue-size"; - public static final String CONSUMER_TRACKER_ELASTICSEARCH_REMAINING_CAPACITY = - "consumer." + HOSTNAME + ".tracker.elasticsearch.remaining-capacity"; - } - - private static class Prometheus { - public static final String TRACKER_ELASTICSEARCH_QUEUE_SIZE = "tracker.elasticsearch.queue-size"; - public static final String TRACKER_ELASTICSEARCH_REMAINING_CAPACITY = "tracker.elasticsearch.remaining-capacity"; - } + public static final String TRACKER_ELASTICSEARCH_QUEUE_SIZE = "tracker.elasticsearch.queue-size"; + public static final String TRACKER_ELASTICSEARCH_REMAINING_CAPACITY = "tracker.elasticsearch.remaining-capacity"; } private static class Timers { diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/UndeliveredMessagesMetrics.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/UndeliveredMessagesMetrics.java index ca4edaa24e..3a286a0bf3 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/UndeliveredMessagesMetrics.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/UndeliveredMessagesMetrics.java @@ -7,30 +7,23 @@ import pl.allegro.tech.hermes.metrics.HermesHistogram; import pl.allegro.tech.hermes.metrics.counters.HermesCounters; -import static pl.allegro.tech.hermes.common.metric.Histograms.PERSISTED_UNDELIVERED_MESSAGE_SIZE; -import static pl.allegro.tech.hermes.common.metric.Meters.PERSISTED_UNDELIVERED_MESSAGES_METER; - public class UndeliveredMessagesMetrics { - private final HermesMetrics hermesMetrics; private final MeterRegistry meterRegistry; - public UndeliveredMessagesMetrics(HermesMetrics hermesMetrics, MeterRegistry meterRegistry) { - this.hermesMetrics = hermesMetrics; + public UndeliveredMessagesMetrics(MeterRegistry meterRegistry) { this.meterRegistry = meterRegistry; } public HermesCounter undeliveredMessagesCounter() { return HermesCounters.from( - meterRegistry.counter("undelivered-messages.persisted"), - hermesMetrics.meter(PERSISTED_UNDELIVERED_MESSAGES_METER) + meterRegistry.counter("undelivered-messages.persisted") ); } public HermesHistogram undeliveredMessagesSizeHistogram() { return DefaultHermesHistogram.of( DistributionSummary.builder("undelivered-messages.persisted.message-size.bytes") - .register(meterRegistry), - hermesMetrics.histogram(PERSISTED_UNDELIVERED_MESSAGE_SIZE) + .register(meterRegistry) ); } } diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/WorkloadMetrics.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/WorkloadMetrics.java index 7c2e840d0f..1faa3486c3 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/WorkloadMetrics.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/WorkloadMetrics.java @@ -14,27 +14,17 @@ public class WorkloadMetrics { private static final String CONSUMER_ID_TAG = "consumer-id"; private static final String KAFKA_CLUSTER_TAG = "kafka-cluster"; - private static final String METRICS_PREFIX = "consumer-workload.weighted."; - private static final String CONSUMER_ID_PLACEHOLDER = "$consumerId"; - private static final String CURRENT_SCORE = METRICS_PREFIX + CONSUMER_ID_PLACEHOLDER + ".current-score"; - private static final String PROPOSED_SCORE = METRICS_PREFIX + CONSUMER_ID_PLACEHOLDER + ".proposed-score"; - private static final String SCORING_ERROR = METRICS_PREFIX + CONSUMER_ID_PLACEHOLDER + ".error"; - private static final String CURRENT_WEIGHT_OPS = METRICS_PREFIX + CONSUMER_ID_PLACEHOLDER + ".current-weight.ops"; - private static final String PROPOSED_WEIGHT_OPS = METRICS_PREFIX + CONSUMER_ID_PLACEHOLDER + ".proposed-weight.ops"; - - private final HermesMetrics hermesMetrics; + private final MeterRegistry meterRegistry; private final GaugeRegistrar gaugeRegistrar; - WorkloadMetrics(HermesMetrics hermesMetrics, MeterRegistry meterRegistry) { - this.hermesMetrics = hermesMetrics; + WorkloadMetrics(MeterRegistry meterRegistry) { this.meterRegistry = meterRegistry; - this.gaugeRegistrar = new GaugeRegistrar(meterRegistry, hermesMetrics); + this.gaugeRegistrar = new GaugeRegistrar(meterRegistry); } public void registerAllAssignmentsGauge(T obj, String kafkaCluster, ToDoubleFunction f) { gaugeRegistrar.registerGauge( - "consumers-workload." + kafkaCluster + ".all-assignments", "workload.all-assignments", obj, f, @@ -44,7 +34,6 @@ public void registerAllAssignmentsGauge(T obj, String kafkaCluster, ToDouble public void registerMissingResourcesGauge(T obj, String kafkaCluster, ToDoubleFunction f) { gaugeRegistrar.registerGauge( - "consumers-workload." + kafkaCluster + ".missing-resources", "workload.missing-resources", obj, f, @@ -54,7 +43,6 @@ public void registerMissingResourcesGauge(T obj, String kafkaCluster, ToDoub public void registerDeletedAssignmentsGauge(T obj, String kafkaCluster, ToDoubleFunction f) { gaugeRegistrar.registerGauge( - "consumers-workload." + kafkaCluster + ".deleted-assignments", "workload.deleted-assignments", obj, f, @@ -64,7 +52,6 @@ public void registerDeletedAssignmentsGauge(T obj, String kafkaCluster, ToDo public void registerCreatedAssignmentsGauge(T obj, String kafkaCluster, ToDoubleFunction f) { gaugeRegistrar.registerGauge( - "consumers-workload." + kafkaCluster + ".created-assignments", "workload.created-assignments", obj, f, @@ -74,38 +61,36 @@ public void registerCreatedAssignmentsGauge(T obj, String kafkaCluster, ToDo public HermesTimer rebalanceDurationTimer(String kafkaCluster) { return HermesTimer.from( - meterRegistry.timer("workload.rebalance-duration", Tags.of(KAFKA_CLUSTER_TAG, kafkaCluster)), - hermesMetrics.consumersWorkloadRebalanceDurationTimer(kafkaCluster) + meterRegistry.timer("workload.rebalance-duration", Tags.of(KAFKA_CLUSTER_TAG, kafkaCluster)) ); } public void registerRunningSubscriptionsGauge(T obj, ToDoubleFunction f) { - gaugeRegistrar.registerGauge("consumers-workload.monitor.running", "workload.subscriptions.running", obj, f); + gaugeRegistrar.registerGauge("workload.subscriptions.running", obj, f); } public void registerAssignedSubscriptionsGauge(T obj, ToDoubleFunction f) { - gaugeRegistrar.registerGauge("consumers-workload.monitor.assigned", "workload.subscriptions.assigned", obj, f); + gaugeRegistrar.registerGauge("workload.subscriptions.assigned", obj, f); } public void registerMissingSubscriptionsGauge(T obj, ToDoubleFunction f) { - gaugeRegistrar.registerGauge("consumers-workload.monitor.missing", "workload.subscriptions.missing", obj, f); + gaugeRegistrar.registerGauge("workload.subscriptions.missing", obj, f); } public void registerOversubscribedGauge(T obj, ToDoubleFunction f) { - gaugeRegistrar.registerGauge("consumers-workload.monitor.oversubscribed", "workload.subscriptions.oversubscribed", obj, f); + gaugeRegistrar.registerGauge("workload.subscriptions.oversubscribed", obj, f); } public void registerOperationsPerSecondGauge(T obj, ToDoubleFunction f) { - gaugeRegistrar.registerGauge("consumer-workload.weighted.load.ops", "workload.weighted.ops", obj, f); + gaugeRegistrar.registerGauge("workload.weighted.ops", obj, f); } public void registerCpuUtilizationGauge(T obj, ToDoubleFunction f) { - gaugeRegistrar.registerGauge("consumer-workload.weighted.load.cpu-utilization", "workload.weighted.cpu-utilization", obj, f); + gaugeRegistrar.registerGauge("workload.weighted.cpu-utilization", obj, f); } public void registerCurrentScoreGauge(String consumerId, T obj, ToDoubleFunction f) { gaugeRegistrar.registerGauge( - buildFullGraphiteMetricPath(CURRENT_SCORE, consumerId), "workload.weighted.current-score", obj, f, @@ -115,7 +100,6 @@ public void registerCurrentScoreGauge(String consumerId, T obj, ToDoubleFunc public void registerProposedErrorGauge(String consumerId, T obj, ToDoubleFunction f) { gaugeRegistrar.registerGauge( - buildFullGraphiteMetricPath(PROPOSED_SCORE, consumerId), "workload.weighted.proposed-error", obj, f, @@ -125,7 +109,6 @@ public void registerProposedErrorGauge(String consumerId, T obj, ToDoubleFun public void registerScoringErrorGauge(String consumerId, T obj, ToDoubleFunction f) { gaugeRegistrar.registerGauge( - buildFullGraphiteMetricPath(SCORING_ERROR, consumerId), "workload.weighted.scoring-error", obj, f, @@ -135,7 +118,6 @@ public void registerScoringErrorGauge(String consumerId, T obj, ToDoubleFunc public void registerCurrentWeightGauge(String consumerId, T obj, ToDoubleFunction f) { gaugeRegistrar.registerGauge( - buildFullGraphiteMetricPath(CURRENT_WEIGHT_OPS, consumerId), "workload.weighted.current-weight.ops", obj, f, @@ -145,7 +127,6 @@ public void registerCurrentWeightGauge(String consumerId, T obj, ToDoubleFun public void registerProposedWeightGauge(String consumerId, T obj, ToDoubleFunction f) { gaugeRegistrar.registerGauge( - buildFullGraphiteMetricPath(PROPOSED_WEIGHT_OPS, consumerId), "workload.weighted.proposed-weight.ops", obj, f, @@ -161,16 +142,5 @@ public void unregisterAllWorkloadWeightedGaugesForConsumerIds(Set consum for (Gauge gauge : gauges) { meterRegistry.remove(gauge); } - for (String consumerId : consumerIds) { - hermesMetrics.unregister(buildFullGraphiteMetricPath(CURRENT_SCORE, consumerId)); - hermesMetrics.unregister(buildFullGraphiteMetricPath(PROPOSED_SCORE, consumerId)); - hermesMetrics.unregister(buildFullGraphiteMetricPath(SCORING_ERROR, consumerId)); - hermesMetrics.unregister(buildFullGraphiteMetricPath(CURRENT_WEIGHT_OPS, consumerId)); - hermesMetrics.unregister(buildFullGraphiteMetricPath(PROPOSED_WEIGHT_OPS, consumerId)); - } - } - - private String buildFullGraphiteMetricPath(String metric, String consumerId) { - return metric.replace(CONSUMER_ID_PLACEHOLDER, HermesMetrics.escapeDots(consumerId)); } } diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/counter/zookeeper/ZookeeperCounterReporter.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/counter/zookeeper/ZookeeperCounterReporter.java index c66efdc640..3293bea92b 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/counter/zookeeper/ZookeeperCounterReporter.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/counter/zookeeper/ZookeeperCounterReporter.java @@ -7,8 +7,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import pl.allegro.tech.hermes.api.TopicName; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; import pl.allegro.tech.hermes.common.metric.counter.CounterStorage; +import pl.allegro.tech.hermes.metrics.PathsCompiler; import java.util.Collection; import java.util.concurrent.Executors; @@ -101,6 +101,6 @@ private static TopicName escapedTopicName(TopicName topicName) { } private static String escapeMetricsReplacementChar(String value) { - return value.replaceAll(HermesMetrics.REPLACEMENT_CHAR, "\\."); + return value.replaceAll(PathsCompiler.REPLACEMENT_CHAR, "\\."); } } diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/schema/RawSchemaClientFactory.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/schema/RawSchemaClientFactory.java index e4bf8933cc..028d649399 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/schema/RawSchemaClientFactory.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/schema/RawSchemaClientFactory.java @@ -1,7 +1,6 @@ package pl.allegro.tech.hermes.common.schema; import com.fasterxml.jackson.databind.ObjectMapper; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.schema.RawSchemaClient; import pl.allegro.tech.hermes.schema.SubjectNamingStrategy; diff --git a/hermes-common/src/test/groovy/pl/allegro/tech/hermes/common/metric/executor/InstrumentedExecutorServiceFactoryMetricsTest.groovy b/hermes-common/src/test/groovy/pl/allegro/tech/hermes/common/metric/executor/InstrumentedExecutorServiceFactoryMetricsTest.groovy index 91d5ad69ac..5794650177 100644 --- a/hermes-common/src/test/groovy/pl/allegro/tech/hermes/common/metric/executor/InstrumentedExecutorServiceFactoryMetricsTest.groovy +++ b/hermes-common/src/test/groovy/pl/allegro/tech/hermes/common/metric/executor/InstrumentedExecutorServiceFactoryMetricsTest.groovy @@ -1,12 +1,9 @@ package pl.allegro.tech.hermes.common.metric.executor -import com.codahale.metrics.MetricRegistry import io.micrometer.core.instrument.MeterRegistry import io.micrometer.core.instrument.search.Search import io.micrometer.core.instrument.simple.SimpleMeterRegistry -import pl.allegro.tech.hermes.common.metric.HermesMetrics import pl.allegro.tech.hermes.common.metric.MetricsFacade -import pl.allegro.tech.hermes.metrics.PathsCompiler import spock.lang.Specification import spock.lang.Subject @@ -25,8 +22,7 @@ class InstrumentedExecutorServiceFactoryMetricsTest extends Specification { private final InstrumentedExecutorServiceFactory factory = new InstrumentedExecutorServiceFactory( new MetricsFacade( - meterRegistry, - new HermesMetrics(new MetricRegistry(), new PathsCompiler("host")) + meterRegistry ) ) diff --git a/hermes-common/src/test/groovy/pl/allegro/tech/hermes/common/schema/ReadMetricsTrackingRawSchemaClientTest.groovy b/hermes-common/src/test/groovy/pl/allegro/tech/hermes/common/schema/ReadMetricsTrackingRawSchemaClientTest.groovy index 8e8d3d4e8a..f6e128fc8a 100644 --- a/hermes-common/src/test/groovy/pl/allegro/tech/hermes/common/schema/ReadMetricsTrackingRawSchemaClientTest.groovy +++ b/hermes-common/src/test/groovy/pl/allegro/tech/hermes/common/schema/ReadMetricsTrackingRawSchemaClientTest.groovy @@ -1,16 +1,12 @@ package pl.allegro.tech.hermes.common.schema -import com.codahale.metrics.MetricRegistry import io.micrometer.core.instrument.MeterRegistry import io.micrometer.core.instrument.Timer import io.micrometer.core.instrument.search.Search import io.micrometer.core.instrument.simple.SimpleMeterRegistry import pl.allegro.tech.hermes.api.RawSchema import pl.allegro.tech.hermes.api.TopicName -import pl.allegro.tech.hermes.common.metric.HermesMetrics import pl.allegro.tech.hermes.common.metric.MetricsFacade -import pl.allegro.tech.hermes.common.metric.Timers -import pl.allegro.tech.hermes.metrics.PathsCompiler import pl.allegro.tech.hermes.schema.RawSchemaClient import pl.allegro.tech.hermes.schema.SchemaVersion import pl.allegro.tech.hermes.test.helper.metrics.MicrometerUtils @@ -29,9 +25,8 @@ class ReadMetricsTrackingRawSchemaClientTest extends Specification { RawSchema schema = RawSchema.valueOf("some_schema") MeterRegistry meterRegistry = new SimpleMeterRegistry() - HermesMetrics hermesMetrics = new HermesMetrics(new MetricRegistry(), new PathsCompiler("")) - MetricsFacade metricsFacade = new MetricsFacade(meterRegistry, hermesMetrics) + MetricsFacade metricsFacade = new MetricsFacade(meterRegistry) RawSchemaClient rawSchemaClient = Mock() @@ -91,17 +86,15 @@ class ReadMetricsTrackingRawSchemaClientTest extends Specification { } private long getSchemaCounterValue() { - return getCounterValue("schema.get-schema", Timers.GET_SCHEMA_LATENCY) + return getCounterValue("schema.get-schema") } private long getVersionsCounterValue() { - return getCounterValue("schema.get-versions", Timers.GET_SCHEMA_VERSIONS_LATENCY) + return getCounterValue("schema.get-versions") } - private long getCounterValue(String meterRegistryName, String hermesMetricsName) { + private long getCounterValue(String meterRegistryName) { def meterRegistryCount = MicrometerUtils.metricValue(meterRegistry, meterRegistryName, Search.&timer, Timer.&count).orElse(0L); - def hermesMetricsCount = hermesMetrics.schemaTimer(hermesMetricsName).count - assert meterRegistryCount == hermesMetricsCount return meterRegistryCount } } diff --git a/hermes-common/src/test/java/pl/allegro/tech/hermes/common/message/undelivered/ZookeeperUndeliveredMessageLogTest.java b/hermes-common/src/test/java/pl/allegro/tech/hermes/common/message/undelivered/ZookeeperUndeliveredMessageLogTest.java index 0de45d0638..648f61134e 100644 --- a/hermes-common/src/test/java/pl/allegro/tech/hermes/common/message/undelivered/ZookeeperUndeliveredMessageLogTest.java +++ b/hermes-common/src/test/java/pl/allegro/tech/hermes/common/message/undelivered/ZookeeperUndeliveredMessageLogTest.java @@ -1,6 +1,5 @@ package pl.allegro.tech.hermes.common.message.undelivered; -import com.codahale.metrics.MetricRegistry; import com.fasterxml.jackson.databind.ObjectMapper; import io.micrometer.core.instrument.Counter; import io.micrometer.core.instrument.DistributionSummary; @@ -12,10 +11,8 @@ import org.junit.Test; import pl.allegro.tech.hermes.api.SentMessageTrace; import pl.allegro.tech.hermes.api.TopicName; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.infrastructure.zookeeper.ZookeeperPaths; -import pl.allegro.tech.hermes.metrics.PathsCompiler; import pl.allegro.tech.hermes.test.helper.zookeeper.ZookeeperBaseTest; import java.util.Optional; @@ -34,10 +31,8 @@ public class ZookeeperUndeliveredMessageLogTest extends ZookeeperBaseTest { private final ZookeeperPaths paths = new ZookeeperPaths("/hermes"); - private final HermesMetrics hermesMetrics = new HermesMetrics( - new MetricRegistry(), new PathsCompiler("host")); private final MeterRegistry meterRegistry = new SimpleMeterRegistry(); - private final MetricsFacade metricsFacade = new MetricsFacade(meterRegistry, hermesMetrics); + private final MetricsFacade metricsFacade = new MetricsFacade(meterRegistry); private final ZookeeperUndeliveredMessageLog log = new ZookeeperUndeliveredMessageLog( zookeeperClient, @@ -60,8 +55,6 @@ public void setUp() throws Exception { @After public void cleanUp() throws Exception { deleteData(paths.basePath()); - hermesMetrics.unregister(PERSISTED_UNDELIVERED_MESSAGES_METER); - hermesMetrics.unregister(PERSISTED_UNDELIVERED_MESSAGE_SIZE); } @Test @@ -129,8 +122,6 @@ private SentMessageTrace createUndeliveredMessage(String subscription, String me } private void assertThatMetricsHaveBeenReported(int persistedMessageCount) { - assertThat(hermesMetrics.meter(PERSISTED_UNDELIVERED_MESSAGES_METER).getCount()).isEqualTo(persistedMessageCount); - assertThat(hermesMetrics.histogram(PERSISTED_UNDELIVERED_MESSAGE_SIZE).getCount()).isEqualTo(persistedMessageCount); assertThat(metricValue(meterRegistry, PERSISTED_UNDELIVERED_MESSAGES_METER, Search::counter, Counter::count).orElse(0.0d)) .isEqualTo(persistedMessageCount); assertThat(metricValue(meterRegistry, PERSISTED_UNDELIVERED_MESSAGE_SIZE + ".bytes", Search::summary, DistributionSummary::count) diff --git a/hermes-common/src/test/java/pl/allegro/tech/hermes/common/message/wrapper/MessageContentWrapperTest.java b/hermes-common/src/test/java/pl/allegro/tech/hermes/common/message/wrapper/MessageContentWrapperTest.java index 3a38d65ca8..af5b937bed 100644 --- a/hermes-common/src/test/java/pl/allegro/tech/hermes/common/message/wrapper/MessageContentWrapperTest.java +++ b/hermes-common/src/test/java/pl/allegro/tech/hermes/common/message/wrapper/MessageContentWrapperTest.java @@ -1,6 +1,5 @@ package pl.allegro.tech.hermes.common.message.wrapper; -import com.codahale.metrics.MetricRegistry; import com.fasterxml.jackson.databind.ObjectMapper; import io.micrometer.core.instrument.Counter; import io.micrometer.core.instrument.MeterRegistry; @@ -9,12 +8,9 @@ import io.micrometer.core.instrument.simple.SimpleMeterRegistry; import org.apache.avro.Schema; import org.apache.commons.collections4.map.HashedMap; -import org.junit.Before; import org.junit.Test; import pl.allegro.tech.hermes.api.Topic; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; import pl.allegro.tech.hermes.common.metric.MetricsFacade; -import pl.allegro.tech.hermes.metrics.PathsCompiler; import pl.allegro.tech.hermes.schema.CompiledSchema; import pl.allegro.tech.hermes.schema.CompiledSchemaRepository; import pl.allegro.tech.hermes.schema.SchemaId; @@ -48,10 +44,8 @@ public class MessageContentWrapperTest { private static final int ID_THREE = 3; private static final int ID_FIVE = 5; - private final MetricRegistry metricRegistry = new MetricRegistry(); private final MeterRegistry meterRegistry = new SimpleMeterRegistry(); - private final HermesMetrics hermesMetrics = new HermesMetrics(metricRegistry, new PathsCompiler("")); - private final MetricsFacade metricsFacade = new MetricsFacade(meterRegistry, hermesMetrics); + private final MetricsFacade metricsFacade = new MetricsFacade(meterRegistry); private final JsonMessageContentWrapper jsonWrapper = new JsonMessageContentWrapper("message", "metadata", new ObjectMapper()); private final AvroMessageContentWrapper avroWrapper = new AvroMessageContentWrapper(Clock.systemDefaultZone()); @@ -135,11 +129,6 @@ public CompiledSchema getSchema(Topic topic, SchemaId id) { static SchemaRepository schemaRepository = new SchemaRepository(schemaVersionsRepository, compiledSchemaRepository); - @Before - public void clean() { - metricRegistry.getCounters().forEach((s, counter) -> counter.dec(counter.getCount())); - } - @Test public void shouldUnwrapMessageUsingSchemaIdFromPayload() { // given @@ -327,46 +316,33 @@ private void assertMetrics(int missedSchemaIdInPayload, int usingHeaderSchemaId, int usingSchemaVersionTruncation) { final String basePath = "content.avro.deserialization"; - assertThat(metricRegistryCounterValue(basePath + ".missed.schemaIdInPayload")).isEqualTo(missedSchemaIdInPayload); assertThat(meterRegistryCounterValue(basePath + ".missing_schemaIdInPayload", Tags.empty())) .isEqualTo(missedSchemaIdInPayload); - assertThat(metricRegistryCounterValue(basePath + ".errors.payloadWithSchemaId")).isEqualTo(errorsForPayloadWithSchemaId); assertThat(meterRegistryCounterValue(basePath + ".errors", Tags.of("deserialization_type", "payloadWithSchemaId"))) .isEqualTo(errorsForPayloadWithSchemaId); - assertThat(metricRegistryCounterValue(basePath + ".errors.headerSchemaVersion")).isEqualTo(errorsForHeaderSchemaVersion); assertThat(meterRegistryCounterValue(basePath + ".errors", Tags.of("deserialization_type", "headerSchemaVersion"))) .isEqualTo(errorsForHeaderSchemaVersion); - assertThat(metricRegistryCounterValue(basePath + ".errors.headerSchemaId")).isEqualTo(errorsForHeaderSchemaId); assertThat(meterRegistryCounterValue(basePath + ".errors", Tags.of("deserialization_type", "headerSchemaId"))) .isEqualTo(errorsForHeaderSchemaId); - assertThat(metricRegistryCounterValue(basePath + ".errors.schemaVersionTruncation")).isEqualTo(errorsWithSchemaVersionTruncation); assertThat(meterRegistryCounterValue(basePath + ".errors", Tags.of("deserialization_type", "schemaVersionTruncation"))) .isEqualTo(errorsWithSchemaVersionTruncation); - assertThat(metricRegistryCounterValue(basePath + ".using.schemaIdAware")).isEqualTo(usingSchemaIdAware); assertThat(meterRegistryCounterValue(basePath, Tags.of("deserialization_type", "payloadWithSchemaId"))) .isEqualTo(usingSchemaIdAware); - assertThat(metricRegistryCounterValue(basePath + ".using.headerSchemaVersion")).isEqualTo(usingHeaderSchemaVersion); assertThat(meterRegistryCounterValue(basePath, Tags.of("deserialization_type", "headerSchemaVersion"))) .isEqualTo(usingHeaderSchemaVersion); - assertThat(metricRegistryCounterValue(basePath + ".using.headerSchemaId")).isEqualTo(usingHeaderSchemaId); assertThat(meterRegistryCounterValue(basePath, Tags.of("deserialization_type", "headerSchemaId"))).isEqualTo(usingHeaderSchemaId); - assertThat(metricRegistryCounterValue(basePath + ".using.schemaVersionTruncation")).isEqualTo(usingSchemaVersionTruncation); assertThat(meterRegistryCounterValue(basePath, Tags.of("deserialization_type", "schemaVersionTruncation"))) .isEqualTo(usingSchemaVersionTruncation); } - private long metricRegistryCounterValue(String metricName) { - return metricRegistry.counter(metricName).getCount(); - } - private int meterRegistryCounterValue(String metricName, Tags tags) { return MicrometerUtils.metricValue(meterRegistry, metricName, tags, Search::counter, Counter::count).orElse(0.0d).intValue(); } diff --git a/hermes-common/src/test/java/pl/allegro/tech/hermes/common/metric/counter/zookeeper/ZookeeperCounterReporterTest.java b/hermes-common/src/test/java/pl/allegro/tech/hermes/common/metric/counter/zookeeper/ZookeeperCounterReporterTest.java index 5ded453054..1e9fd3d017 100644 --- a/hermes-common/src/test/java/pl/allegro/tech/hermes/common/metric/counter/zookeeper/ZookeeperCounterReporterTest.java +++ b/hermes-common/src/test/java/pl/allegro/tech/hermes/common/metric/counter/zookeeper/ZookeeperCounterReporterTest.java @@ -1,6 +1,5 @@ package pl.allegro.tech.hermes.common.metric.counter.zookeeper; -import com.codahale.metrics.MetricRegistry; import io.micrometer.core.instrument.MeterRegistry; import io.micrometer.core.instrument.simple.SimpleMeterRegistry; import org.junit.Before; @@ -10,11 +9,9 @@ import org.mockito.runners.MockitoJUnitRunner; import pl.allegro.tech.hermes.api.SubscriptionName; import pl.allegro.tech.hermes.api.TopicName; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.common.metric.counter.CounterStorage; import pl.allegro.tech.hermes.common.util.InstanceIdResolver; -import pl.allegro.tech.hermes.metrics.PathsCompiler; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -34,8 +31,7 @@ public class ZookeeperCounterReporterTest { private final MeterRegistry meterRegistry = new SimpleMeterRegistry(); - private final MetricsFacade metricsFacade = new MetricsFacade( - meterRegistry, new HermesMetrics(new MetricRegistry(), new PathsCompiler("localhost"))); + private final MetricsFacade metricsFacade = new MetricsFacade(meterRegistry); @Mock private InstanceIdResolver instanceIdResolver; diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/CommonConfiguration.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/CommonConfiguration.java index 218ccec73f..2ea5eab12e 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/CommonConfiguration.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/CommonConfiguration.java @@ -1,6 +1,5 @@ package pl.allegro.tech.hermes.consumers.config; -import com.codahale.metrics.MetricRegistry; import com.fasterxml.jackson.databind.ObjectMapper; import io.micrometer.core.instrument.MeterRegistry; import io.micrometer.core.instrument.composite.CompositeMeterRegistry; @@ -18,7 +17,6 @@ import pl.allegro.tech.hermes.common.concurrent.ExecutorServiceFactory; import pl.allegro.tech.hermes.common.di.factories.CuratorClientFactory; import pl.allegro.tech.hermes.common.di.factories.HermesCuratorClientFactory; -import pl.allegro.tech.hermes.common.di.factories.MetricRegistryFactory; import pl.allegro.tech.hermes.common.di.factories.MicrometerRegistryParameters; import pl.allegro.tech.hermes.common.di.factories.ModelAwareZookeeperNotifyingCacheFactory; import pl.allegro.tech.hermes.common.di.factories.ObjectMapperFactory; @@ -35,7 +33,6 @@ import pl.allegro.tech.hermes.common.message.wrapper.AvroMessageSchemaVersionTruncationContentWrapper; import pl.allegro.tech.hermes.common.message.wrapper.CompositeMessageContentWrapper; import pl.allegro.tech.hermes.common.message.wrapper.JsonMessageContentWrapper; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.common.metric.counter.CounterStorage; import pl.allegro.tech.hermes.common.metric.counter.zookeeper.ZookeeperCounterStorage; @@ -82,7 +79,6 @@ @EnableConfigurationProperties({ MetricsProperties.class, MicrometerRegistryProperties.class, - GraphiteProperties.class, PrometheusProperties.class, SchemaProperties.class, ZookeeperClustersProperties.class, @@ -232,23 +228,8 @@ public WorkloadConstraintsRepository workloadConstraintsRepository(CuratorFramew } @Bean - public HermesMetrics hermesMetrics(MetricRegistry metricRegistry, - PathsCompiler pathsCompiler) { - return new HermesMetrics(metricRegistry, pathsCompiler); - } - - @Bean - public MetricsFacade metricsFacade(MeterRegistry meterRegistry, HermesMetrics hermesMetrics) { - return new MetricsFacade(meterRegistry, hermesMetrics); - } - - @Bean - public MetricRegistry metricRegistry(MetricsProperties metricsProperties, - GraphiteProperties graphiteProperties, - InstanceIdResolver instanceIdResolver, - @Named("moduleName") String moduleName) { - return new MetricRegistryFactory(metricsProperties, graphiteProperties, instanceIdResolver, moduleName) - .provide(); + public MetricsFacade metricsFacade(MeterRegistry meterRegistry) { + return new MetricsFacade(meterRegistry); } @Bean diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/GraphiteProperties.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/GraphiteProperties.java deleted file mode 100644 index 56682552b6..0000000000 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/GraphiteProperties.java +++ /dev/null @@ -1,41 +0,0 @@ -package pl.allegro.tech.hermes.consumers.config; - -import org.springframework.boot.context.properties.ConfigurationProperties; -import pl.allegro.tech.hermes.common.di.factories.GraphiteParameters; - -@ConfigurationProperties(prefix = "consumer.graphite") -public class GraphiteProperties implements GraphiteParameters { - - private String prefix = "stats.tech.hermes"; - - private String host = "localhost"; - - private int port = 2003; - - @Override - public String getPrefix() { - return prefix; - } - - public void setPrefix(String prefix) { - this.prefix = prefix; - } - - @Override - public String getHost() { - return host; - } - - public void setHost(String host) { - this.host = host; - } - - @Override - public int getPort() { - return port; - } - - public void setPort(int port) { - this.port = port; - } -} diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/MetricsProperties.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/MetricsProperties.java index b03087193b..835ccb19bd 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/MetricsProperties.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/MetricsProperties.java @@ -1,52 +1,14 @@ package pl.allegro.tech.hermes.consumers.config; import org.springframework.boot.context.properties.ConfigurationProperties; -import pl.allegro.tech.hermes.common.di.factories.MetricRegistryParameters; import java.time.Duration; @ConfigurationProperties(prefix = "consumer.metrics.metric-registry") -public class MetricsProperties implements MetricRegistryParameters { - - private boolean zookeeperReporterEnabled = true; - - private boolean graphiteReporterEnabled = false; - - private boolean consoleReporterEnabled = false; +public class MetricsProperties { private Duration counterExpireAfterAccess = Duration.ofHours(72); - private String disabledAttributes = "M15_RATE, M5_RATE, MEAN, MEAN_RATE, MIN, STDDEV"; - - private Duration reportPeriod = Duration.ofSeconds(20); - - @Override - public boolean isZookeeperReporterEnabled() { - return zookeeperReporterEnabled; - } - - public void setZookeeperReporterEnabled(boolean zookeeperReporterEnabled) { - this.zookeeperReporterEnabled = zookeeperReporterEnabled; - } - - @Override - public boolean isGraphiteReporterEnabled() { - return graphiteReporterEnabled; - } - - public void setGraphiteReporterEnabled(boolean graphiteReporterEnabled) { - this.graphiteReporterEnabled = graphiteReporterEnabled; - } - - @Override - public boolean isConsoleReporterEnabled() { - return consoleReporterEnabled; - } - - public void setConsoleReporterEnabled(boolean consoleReporterEnabled) { - this.consoleReporterEnabled = consoleReporterEnabled; - } - public Duration getCounterExpireAfterAccess() { return counterExpireAfterAccess; } @@ -54,22 +16,4 @@ public Duration getCounterExpireAfterAccess() { public void setCounterExpireAfterAccess(Duration counterExpireAfterAccess) { this.counterExpireAfterAccess = counterExpireAfterAccess; } - - @Override - public String getDisabledAttributes() { - return disabledAttributes; - } - - public void setDisabledAttributes(String disabledAttributes) { - this.disabledAttributes = disabledAttributes; - } - - @Override - public Duration getReportPeriod() { - return reportPeriod; - } - - public void setReportPeriod(Duration reportPeriod) { - this.reportPeriod = reportPeriod; - } } diff --git a/hermes-consumers/src/main/resources/application-integration.yaml b/hermes-consumers/src/main/resources/application-integration.yaml index 11e0d59de2..a79777f93c 100644 --- a/hermes-consumers/src/main/resources/application-integration.yaml +++ b/hermes-consumers/src/main/resources/application-integration.yaml @@ -73,12 +73,6 @@ consumer: workload: rebalanceInterval: 1s consumerPerSubscription: 1 - metrics: - metric-registry: - graphiteReporterEnabled: true - reportPeriod: 1s - graphite: - port: 18023 schema: cache: refreshAfterWrite: 1m diff --git a/hermes-consumers/src/main/resources/application-local.yaml b/hermes-consumers/src/main/resources/application-local.yaml index 08db5b9583..8777e70205 100644 --- a/hermes-consumers/src/main/resources/application-local.yaml +++ b/hermes-consumers/src/main/resources/application-local.yaml @@ -9,11 +9,6 @@ consumer: clusters: - datacenter: "dc" brokerList: "localhost:9092" - graphite: - host: "localhost" - metrics: - metrics-registry: - graphiteReporterEnabled: true workload: consumerPerSubscription: 1 schema: diff --git a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/sender/http/HttpClientConnectionMonitoringTest.groovy b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/sender/http/HttpClientConnectionMonitoringTest.groovy index fed0b23ef0..8c686cc39c 100644 --- a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/sender/http/HttpClientConnectionMonitoringTest.groovy +++ b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/sender/http/HttpClientConnectionMonitoringTest.groovy @@ -1,18 +1,15 @@ package pl.allegro.tech.hermes.consumers.consumer.sender.http -import com.codahale.metrics.MetricRegistry import com.github.tomakehurst.wiremock.WireMockServer import io.micrometer.core.instrument.MeterRegistry import io.micrometer.core.instrument.search.Search import io.micrometer.core.instrument.simple.SimpleMeterRegistry import org.eclipse.jetty.client.HttpClient -import pl.allegro.tech.hermes.common.metric.HermesMetrics import pl.allegro.tech.hermes.common.metric.MetricsFacade import pl.allegro.tech.hermes.common.metric.executor.InstrumentedExecutorServiceFactory import pl.allegro.tech.hermes.consumers.config.ConsumerSenderConfiguration import pl.allegro.tech.hermes.consumers.config.Http1ClientProperties import pl.allegro.tech.hermes.consumers.config.SslContextProperties -import pl.allegro.tech.hermes.metrics.PathsCompiler import pl.allegro.tech.hermes.test.helper.util.Ports import spock.lang.Shared import spock.lang.Specification @@ -27,10 +24,8 @@ class HttpClientConnectionMonitoringTest extends Specification { HttpClient client HttpClient batchClient - MetricRegistry metricRegistry = new MetricRegistry() - HermesMetrics hermesMetrics = new HermesMetrics(metricRegistry, new PathsCompiler("localhost")) MeterRegistry meterRegistry = new SimpleMeterRegistry() - MetricsFacade metrics = new MetricsFacade(meterRegistry, hermesMetrics) + MetricsFacade metrics = new MetricsFacade(meterRegistry) def setupSpec() { port = Ports.nextAvailable() @@ -59,13 +54,10 @@ class HttpClientConnectionMonitoringTest extends Specification { client.POST("http://localhost:${port}/hello").send() and: - def idleDropwizard = metricRegistry.gauges['http-clients.serial.http1.idle-connections'].value - def activeDropwizard = metricRegistry.gauges['http-clients.serial.http1.active-connections'].value def idleMicrometer = Search.in(meterRegistry).name("http-clients.serial.http1.idle-connections").gauge().value() def activeMicrometer = Search.in(meterRegistry).name("http-clients.serial.http1.active-connections").gauge().value() then: - idleDropwizard + activeDropwizard > 0 idleMicrometer + activeMicrometer > 0 } @@ -77,7 +69,6 @@ class HttpClientConnectionMonitoringTest extends Specification { reporter.start() then: - metricRegistry.gauges.size() == 0 Search.in(meterRegistry).gauges().size() == 0 } } diff --git a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/supervisor/process/ConsumerProcessSupervisorTest.groovy b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/supervisor/process/ConsumerProcessSupervisorTest.groovy index 30df929486..98a087a9d3 100644 --- a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/supervisor/process/ConsumerProcessSupervisorTest.groovy +++ b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/supervisor/process/ConsumerProcessSupervisorTest.groovy @@ -1,6 +1,5 @@ package pl.allegro.tech.hermes.consumers.supervisor.process -import com.codahale.metrics.MetricRegistry import com.jayway.awaitility.Awaitility import com.jayway.awaitility.core.ConditionFactory import io.micrometer.core.instrument.MeterRegistry @@ -10,11 +9,9 @@ import pl.allegro.tech.hermes.api.DeliveryType import pl.allegro.tech.hermes.api.Subscription import pl.allegro.tech.hermes.api.SubscriptionName import pl.allegro.tech.hermes.api.Topic -import pl.allegro.tech.hermes.common.metric.HermesMetrics import pl.allegro.tech.hermes.common.metric.MetricsFacade import pl.allegro.tech.hermes.consumers.config.CommonConsumerProperties import pl.allegro.tech.hermes.consumers.supervisor.ConsumersExecutorService -import pl.allegro.tech.hermes.metrics.PathsCompiler import pl.allegro.tech.hermes.test.helper.builder.TopicBuilder import spock.lang.Specification import spock.lang.Unroll @@ -26,12 +23,7 @@ import java.time.ZoneId import java.util.function.Consumer import static java.util.concurrent.TimeUnit.MILLISECONDS -import static pl.allegro.tech.hermes.consumers.supervisor.process.Signal.SignalType.COMMIT -import static pl.allegro.tech.hermes.consumers.supervisor.process.Signal.SignalType.RETRANSMIT -import static pl.allegro.tech.hermes.consumers.supervisor.process.Signal.SignalType.START -import static pl.allegro.tech.hermes.consumers.supervisor.process.Signal.SignalType.STOP -import static pl.allegro.tech.hermes.consumers.supervisor.process.Signal.SignalType.UPDATE_SUBSCRIPTION -import static pl.allegro.tech.hermes.consumers.supervisor.process.Signal.SignalType.UPDATE_TOPIC +import static pl.allegro.tech.hermes.consumers.supervisor.process.Signal.SignalType.* import static pl.allegro.tech.hermes.test.helper.builder.SubscriptionBuilder.subscription import static pl.allegro.tech.hermes.test.helper.endpoint.TimeoutAdjuster.adjust @@ -52,7 +44,6 @@ class ConsumerProcessSupervisorTest extends Specification { ConsumerProcessSupervisor supervisor MeterRegistry meterRegistry = new SimpleMeterRegistry() - MetricRegistry metricRegistry = new MetricRegistry() MetricsFacade metrics ConsumerStub consumer @@ -70,10 +61,7 @@ class ConsumerProcessSupervisorTest extends Specification { return new ConsumerProcess(startSignal, consumer, Stub(Retransmitter), clock, unhealthyAfter, onConsumerStopped) } - metrics = new MetricsFacade( - meterRegistry, - new HermesMetrics(metricRegistry, new PathsCompiler("localhost")) - ) + metrics = new MetricsFacade(meterRegistry) supervisor = new ConsumerProcessSupervisor( new ConsumersExecutorService(new CommonConsumerProperties().getThreadPoolSize(), metrics), @@ -176,7 +164,6 @@ class ConsumerProcessSupervisorTest extends Specification { then: signalsToDrop.forEach { String signal = it.type.name() - assert metricRegistry.counter("supervisor.signal.dropped." + signal).getCount() == 1 assert Search.in(meterRegistry) .name {it.startsWith("signals.dropped")} .tag("signal", signal) diff --git a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/supervisor/workload/weighted/WeightedWorkBalancingListenerTest.groovy b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/supervisor/workload/weighted/WeightedWorkBalancingListenerTest.groovy index ced5eaa583..5e1f84aec3 100644 --- a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/supervisor/workload/weighted/WeightedWorkBalancingListenerTest.groovy +++ b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/supervisor/workload/weighted/WeightedWorkBalancingListenerTest.groovy @@ -1,14 +1,11 @@ package pl.allegro.tech.hermes.consumers.supervisor.workload.weighted -import com.codahale.metrics.MetricFilter -import com.codahale.metrics.MetricRegistry + import io.micrometer.core.instrument.search.Search import io.micrometer.core.instrument.simple.SimpleMeterRegistry import pl.allegro.tech.hermes.api.SubscriptionName -import pl.allegro.tech.hermes.common.metric.HermesMetrics import pl.allegro.tech.hermes.common.metric.MetricsFacade import pl.allegro.tech.hermes.consumers.supervisor.workload.WorkDistributionChanges -import pl.allegro.tech.hermes.metrics.PathsCompiler import pl.allegro.tech.hermes.test.helper.time.ModifiableClock import spock.lang.Specification import spock.lang.Subject @@ -25,12 +22,10 @@ class WeightedWorkBalancingListenerTest extends Specification { def subscriptionProfileRegistry = new MockSubscriptionProfileRegistry() def weightWindowSize = Duration.ofMinutes(1) def currentLoadProvider = new CurrentLoadProvider() - def metricsRegistry = new MetricRegistry() def meterRegistry = new SimpleMeterRegistry() def metrics = new WeightedWorkloadMetricsReporter( new MetricsFacade( - meterRegistry, - new HermesMetrics(metricsRegistry, new PathsCompiler("host")) + meterRegistry ) ) @@ -180,8 +175,6 @@ class WeightedWorkBalancingListenerTest extends Specification { listener.onBeforeBalancing(["c2"]) then: - metricsRegistry.getGauges(MetricFilter.contains(".c2.")).size() == 1 - metricsRegistry.getGauges(MetricFilter.contains(".c1.")).size() == 0 Search.in(meterRegistry).tag("consumer-id", "c2").gauges().size() == 1 Search.in(meterRegistry).tag("consumer-id", "c1").gauges().size() == 0 } @@ -195,7 +188,6 @@ class WeightedWorkBalancingListenerTest extends Specification { listener.onBalancingSkipped() then: - metricsRegistry.getGauges().size() == 0 Search.in(meterRegistry).gauges().size() == 0 } diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/cache/topic/NotificationBasedTopicsCache.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/cache/topic/NotificationBasedTopicsCache.java index bf2e7a9d92..5ac0b5d232 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/cache/topic/NotificationBasedTopicsCache.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/cache/topic/NotificationBasedTopicsCache.java @@ -14,6 +14,7 @@ import pl.allegro.tech.hermes.frontend.blacklist.BlacklistZookeeperNotifyingCache; import pl.allegro.tech.hermes.frontend.blacklist.TopicBlacklistCallback; import pl.allegro.tech.hermes.frontend.metric.CachedTopic; +import pl.allegro.tech.hermes.frontend.metric.ThroughputRegistry; import java.util.List; import java.util.Optional; @@ -30,17 +31,20 @@ public class NotificationBasedTopicsCache implements TopicCallback, TopicsCache, private final TopicRepository topicRepository; private final MetricsFacade metricsFacade; private final KafkaNamesMapper kafkaNamesMapper; + private final ThroughputRegistry throughputRegistry; public NotificationBasedTopicsCache(InternalNotificationsBus notificationsBus, BlacklistZookeeperNotifyingCache blacklistZookeeperNotifyingCache, GroupRepository groupRepository, TopicRepository topicRepository, MetricsFacade metricsFacade, + ThroughputRegistry throughputRegistry, KafkaNamesMapper kafkaNamesMapper) { this.groupRepository = groupRepository; this.topicRepository = topicRepository; this.metricsFacade = metricsFacade; this.kafkaNamesMapper = kafkaNamesMapper; + this.throughputRegistry = throughputRegistry; notificationsBus.registerTopicCallback(this); blacklistZookeeperNotifyingCache.addCallback(this); } @@ -108,10 +112,10 @@ public void start() { } private CachedTopic cachedTopic(Topic topic) { - return new CachedTopic(topic, metricsFacade, kafkaNamesMapper.toKafkaTopics(topic)); + return new CachedTopic(topic, metricsFacade, throughputRegistry, kafkaNamesMapper.toKafkaTopics(topic)); } private CachedTopic bannedTopic(Topic topic) { - return new CachedTopic(topic, metricsFacade, kafkaNamesMapper.toKafkaTopics(topic), true); + return new CachedTopic(topic, metricsFacade, throughputRegistry, kafkaNamesMapper.toKafkaTopics(topic), true); } } diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/CommonConfiguration.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/CommonConfiguration.java index 92b0b446bd..d5e95e5c9f 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/CommonConfiguration.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/CommonConfiguration.java @@ -1,12 +1,10 @@ package pl.allegro.tech.hermes.frontend.config; -import com.codahale.metrics.MetricRegistry; import com.fasterxml.jackson.databind.ObjectMapper; import io.micrometer.core.instrument.MeterRegistry; import io.micrometer.core.instrument.composite.CompositeMeterRegistry; import io.micrometer.prometheus.PrometheusConfig; import io.micrometer.prometheus.PrometheusMeterRegistry; -import jakarta.inject.Named; import org.apache.curator.framework.CuratorFramework; import org.springframework.boot.context.properties.EnableConfigurationProperties; import org.springframework.context.annotation.Bean; @@ -16,7 +14,6 @@ import pl.allegro.tech.hermes.common.clock.ClockFactory; import pl.allegro.tech.hermes.common.di.factories.CuratorClientFactory; import pl.allegro.tech.hermes.common.di.factories.HermesCuratorClientFactory; -import pl.allegro.tech.hermes.common.di.factories.MetricRegistryFactory; import pl.allegro.tech.hermes.common.di.factories.MicrometerRegistryParameters; import pl.allegro.tech.hermes.common.di.factories.ModelAwareZookeeperNotifyingCacheFactory; import pl.allegro.tech.hermes.common.di.factories.ObjectMapperFactory; @@ -33,7 +30,6 @@ import pl.allegro.tech.hermes.common.message.wrapper.AvroMessageSchemaVersionTruncationContentWrapper; import pl.allegro.tech.hermes.common.message.wrapper.CompositeMessageContentWrapper; import pl.allegro.tech.hermes.common.message.wrapper.JsonMessageContentWrapper; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.common.metric.counter.CounterStorage; import pl.allegro.tech.hermes.common.metric.counter.zookeeper.ZookeeperCounterStorage; @@ -82,7 +78,6 @@ @EnableConfigurationProperties({ MetricRegistryProperties.class, MicrometerRegistryProperties.class, - GraphiteProperties.class, PrometheusProperties.class, SchemaProperties.class, ZookeeperClustersProperties.class, @@ -283,23 +278,8 @@ public WorkloadConstraintsRepository workloadConstraintsRepository(CuratorFramew } @Bean - public HermesMetrics hermesMetrics(MetricRegistry metricRegistry, - PathsCompiler pathsCompiler) { - return new HermesMetrics(metricRegistry, pathsCompiler); - } - - @Bean - public MetricsFacade micrometerHermesMetrics(MeterRegistry meterRegistry, HermesMetrics hermesMetrics) { - return new MetricsFacade(meterRegistry, hermesMetrics); - } - - @Bean - public MetricRegistry metricRegistry(MetricRegistryProperties metricRegistryProperties, - GraphiteProperties graphiteProperties, - InstanceIdResolver instanceIdResolver, - @Named("moduleName") String moduleName) { - return new MetricRegistryFactory(metricRegistryProperties, graphiteProperties, - instanceIdResolver, moduleName).provide(); + public MetricsFacade micrometerHermesMetrics(MeterRegistry meterRegistry) { + return new MetricsFacade(meterRegistry); } @Bean diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FrontendConfiguration.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FrontendConfiguration.java index c219608443..a11e42a9a0 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FrontendConfiguration.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FrontendConfiguration.java @@ -16,6 +16,7 @@ import pl.allegro.tech.hermes.frontend.cache.topic.NotificationBasedTopicsCache; import pl.allegro.tech.hermes.frontend.cache.topic.TopicsCache; import pl.allegro.tech.hermes.frontend.listeners.BrokerListeners; +import pl.allegro.tech.hermes.frontend.metric.ThroughputRegistry; import pl.allegro.tech.hermes.frontend.producer.BrokerMessageProducer; import pl.allegro.tech.hermes.frontend.validator.MessageValidators; import pl.allegro.tech.hermes.frontend.validator.TopicMessageValidator; @@ -41,11 +42,12 @@ public TopicsCache notificationBasedTopicsCache(InternalNotificationsBus interna GroupRepository groupRepository, TopicRepository topicRepository, MetricsFacade metricsFacade, + ThroughputRegistry throughputRegistry, KafkaNamesMapper kafkaNamesMapper, BlacklistZookeeperNotifyingCache blacklistZookeeperNotifyingCache) { return new NotificationBasedTopicsCache(internalNotificationsBus, blacklistZookeeperNotifyingCache, - groupRepository, topicRepository, metricsFacade, kafkaNamesMapper); + groupRepository, topicRepository, metricsFacade, throughputRegistry, kafkaNamesMapper); } @Bean diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FrontendPublishingConfiguration.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FrontendPublishingConfiguration.java index 69f57beaae..cbb9920840 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FrontendPublishingConfiguration.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FrontendPublishingConfiguration.java @@ -1,5 +1,6 @@ package pl.allegro.tech.hermes.frontend.config; +import com.codahale.metrics.MetricRegistry; import com.fasterxml.jackson.databind.ObjectMapper; import io.undertow.server.HttpHandler; import jakarta.inject.Named; @@ -11,6 +12,7 @@ import pl.allegro.tech.hermes.domain.topic.preview.MessagePreviewRepository; import pl.allegro.tech.hermes.frontend.cache.topic.TopicsCache; import pl.allegro.tech.hermes.frontend.listeners.BrokerListeners; +import pl.allegro.tech.hermes.frontend.metric.ThroughputRegistry; import pl.allegro.tech.hermes.frontend.producer.BrokerMessageProducer; import pl.allegro.tech.hermes.frontend.publishing.handlers.HandlersChainFactory; import pl.allegro.tech.hermes.frontend.publishing.handlers.ThroughputLimiter; @@ -56,8 +58,13 @@ public HttpHandler httpHandler(TopicsCache topicsCache, MessageErrorProcessor me } @Bean - public ThroughputLimiter throughputLimiter(ThroughputProperties throughputProperties, MetricsFacade metricsFacade) { - return new ThroughputLimiterFactory(throughputProperties, metricsFacade).provide(); + public ThroughputRegistry throughputRegistry(MetricsFacade metricsFacade) { + return new ThroughputRegistry(metricsFacade, new MetricRegistry()); + } + + @Bean + public ThroughputLimiter throughputLimiter(ThroughputProperties throughputProperties, ThroughputRegistry throughputRegistry) { + return new ThroughputLimiterFactory(throughputProperties, throughputRegistry).provide(); } @Bean diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/GraphiteProperties.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/GraphiteProperties.java deleted file mode 100644 index 60e3a7fab2..0000000000 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/GraphiteProperties.java +++ /dev/null @@ -1,41 +0,0 @@ -package pl.allegro.tech.hermes.frontend.config; - -import org.springframework.boot.context.properties.ConfigurationProperties; -import pl.allegro.tech.hermes.common.di.factories.GraphiteParameters; - -@ConfigurationProperties(prefix = "frontend.graphite") -public class GraphiteProperties implements GraphiteParameters { - - private String prefix = "stats.tech.hermes"; - - private String host = "localhost"; - - private int port = 2003; - - @Override - public String getPrefix() { - return prefix; - } - - public void setPrefix(String prefix) { - this.prefix = prefix; - } - - @Override - public String getHost() { - return host; - } - - public void setHost(String host) { - this.host = host; - } - - @Override - public int getPort() { - return port; - } - - public void setPort(int port) { - this.port = port; - } -} diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/MetricRegistryProperties.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/MetricRegistryProperties.java index eceaadc4f8..3e5e71d80f 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/MetricRegistryProperties.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/MetricRegistryProperties.java @@ -1,52 +1,14 @@ package pl.allegro.tech.hermes.frontend.config; import org.springframework.boot.context.properties.ConfigurationProperties; -import pl.allegro.tech.hermes.common.di.factories.MetricRegistryParameters; import java.time.Duration; @ConfigurationProperties(prefix = "frontend.metrics.metric-registry") -public class MetricRegistryProperties implements MetricRegistryParameters { - - private boolean zookeeperReporterEnabled = true; - - private boolean graphiteReporterEnabled = false; - - private boolean consoleReporterEnabled = false; +public class MetricRegistryProperties { private Duration counterExpireAfterAccess = Duration.ofHours(72); - private String disabledAttributes = "M15_RATE, M5_RATE, MEAN, MEAN_RATE, MIN, STDDEV"; - - private Duration reportPeriod = Duration.ofSeconds(20); - - @Override - public boolean isZookeeperReporterEnabled() { - return zookeeperReporterEnabled; - } - - public void setZookeeperReporterEnabled(boolean zookeeperReporterEnabled) { - this.zookeeperReporterEnabled = zookeeperReporterEnabled; - } - - @Override - public boolean isGraphiteReporterEnabled() { - return graphiteReporterEnabled; - } - - public void setGraphiteReporterEnabled(boolean graphiteReporterEnabled) { - this.graphiteReporterEnabled = graphiteReporterEnabled; - } - - @Override - public boolean isConsoleReporterEnabled() { - return consoleReporterEnabled; - } - - public void setConsoleReporterEnabled(boolean consoleReporterEnabled) { - this.consoleReporterEnabled = consoleReporterEnabled; - } - public Duration getCounterExpireAfterAccess() { return counterExpireAfterAccess; } @@ -54,22 +16,4 @@ public Duration getCounterExpireAfterAccess() { public void setCounterExpireAfterAccess(Duration counterExpireAfterAccess) { this.counterExpireAfterAccess = counterExpireAfterAccess; } - - @Override - public String getDisabledAttributes() { - return disabledAttributes; - } - - public void setDisabledAttributes(String disabledAttributes) { - this.disabledAttributes = disabledAttributes; - } - - @Override - public Duration getReportPeriod() { - return reportPeriod; - } - - public void setReportPeriod(Duration reportPeriod) { - this.reportPeriod = reportPeriod; - } } diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/metric/CachedTopic.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/metric/CachedTopic.java index 033351b232..9ffb726071 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/metric/CachedTopic.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/metric/CachedTopic.java @@ -10,7 +10,6 @@ import pl.allegro.tech.hermes.metrics.HermesRateMeter; import pl.allegro.tech.hermes.metrics.HermesTimer; import pl.allegro.tech.hermes.metrics.HermesTimerContext; -import pl.allegro.tech.hermes.metrics.counters.MeterBackedHermesCounter; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; @@ -36,8 +35,7 @@ public class CachedTopic { private final HermesHistogram topicMessageContentSize; private final HermesHistogram globalMessageContentSize; - private final MeterBackedHermesCounter topicThroughputMeter; - private final MeterBackedHermesCounter globalThroughputMeter; + private final ThroughputMeter throughputMeter; private final HermesCounter topicDuplicatedMessageCounter; @@ -45,13 +43,18 @@ public class CachedTopic { private final Map httpStatusCodesMeters = new ConcurrentHashMap<>(); - public CachedTopic(Topic topic, MetricsFacade metricsFacade, + public CachedTopic(Topic topic, + MetricsFacade metricsFacade, + ThroughputRegistry throughputRegistry, KafkaTopics kafkaTopics) { - this(topic, metricsFacade, kafkaTopics, false); + this(topic, metricsFacade, throughputRegistry, kafkaTopics, false); } - public CachedTopic(Topic topic, MetricsFacade metricsFacade, - KafkaTopics kafkaTopics, boolean blacklisted) { + public CachedTopic(Topic topic, + MetricsFacade metricsFacade, + ThroughputRegistry throughputRegistry, + KafkaTopics kafkaTopics, + boolean blacklisted) { this.topic = topic; this.kafkaTopics = kafkaTopics; this.metricsFacade = metricsFacade; @@ -66,8 +69,7 @@ public CachedTopic(Topic topic, MetricsFacade metricsFacade, globalMessageContentSize = metricsFacade.topics().topicGlobalMessageContentSizeHistogram(); topicMessageContentSize = metricsFacade.topics().topicMessageContentSizeHistogram(topic.getName()); - globalThroughputMeter = metricsFacade.topics().topicGlobalThroughputBytes(); - topicThroughputMeter = metricsFacade.topics().topicThroughputBytes(topic.getName()); + throughputMeter = throughputRegistry.forTopic(topic.getName()); if (Topic.Ack.ALL.equals(topic.getAck())) { globalProducerLatencyTimer = metricsFacade.topics().ackAllGlobalLatency(); @@ -134,8 +136,7 @@ public void incrementPublished(String datacenter) { public void reportMessageContentSize(int size) { topicMessageContentSize.record(size); globalMessageContentSize.record(size); - topicThroughputMeter.increment(size); - globalThroughputMeter.increment(size); + throughputMeter.increment(size); } public void markDelayedProcessing() { @@ -144,7 +145,7 @@ public void markDelayedProcessing() { } public HermesRateMeter getThroughput() { - return topicThroughputMeter; + return throughputMeter; } public void markMessageDuplicated() { diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/metric/ThroughputMeter.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/metric/ThroughputMeter.java new file mode 100644 index 0000000000..3f225b87e0 --- /dev/null +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/metric/ThroughputMeter.java @@ -0,0 +1,36 @@ +package pl.allegro.tech.hermes.frontend.metric; + +import com.codahale.metrics.Meter; +import pl.allegro.tech.hermes.metrics.HermesCounter; +import pl.allegro.tech.hermes.metrics.HermesRateMeter; + +public class ThroughputMeter implements HermesRateMeter { + + private final HermesCounter topicThroughputMetric; + private final HermesCounter globalThroughputMetric; + private final Meter topicMeter; + private final Meter globalMeter; + + public ThroughputMeter( + HermesCounter topicThroughputMetric, + HermesCounter globalThroughputMetric, + Meter topicMeter, + Meter globalMeter) { + this.topicThroughputMetric = topicThroughputMetric; + this.globalThroughputMetric = globalThroughputMetric; + this.topicMeter = topicMeter; + this.globalMeter = globalMeter; + } + + @Override + public double getOneMinuteRate() { + return topicMeter.getOneMinuteRate(); + } + + public void increment(long size) { + this.topicMeter.mark(size); + this.globalMeter.mark(size); + this.topicThroughputMetric.increment(size); + this.globalThroughputMetric.increment(size); + } +} diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/metric/ThroughputRegistry.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/metric/ThroughputRegistry.java new file mode 100644 index 0000000000..ac01b899b6 --- /dev/null +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/metric/ThroughputRegistry.java @@ -0,0 +1,32 @@ +package pl.allegro.tech.hermes.frontend.metric; + +import com.codahale.metrics.Meter; +import com.codahale.metrics.MetricRegistry; +import pl.allegro.tech.hermes.api.TopicName; +import pl.allegro.tech.hermes.common.metric.MetricsFacade; + +public class ThroughputRegistry { + + private final MetricsFacade metricsFacade; + private final MetricRegistry metricRegistry; + private final Meter globalThroughputMeter; + + public ThroughputRegistry(MetricsFacade metricsFacade, MetricRegistry metricRegistry) { + this.metricsFacade = metricsFacade; + this.metricRegistry = metricRegistry; + this.globalThroughputMeter = metricRegistry.meter("globalThroughputMeter"); + } + + public double getGlobalThroughputOneMinuteRate() { + return globalThroughputMeter.getOneMinuteRate(); + } + + public ThroughputMeter forTopic(TopicName topic) { + return new ThroughputMeter( + metricsFacade.topics().topicThroughputBytes(topic), + metricsFacade.topics().topicGlobalThroughputBytes(), + metricRegistry.meter(topic.qualifiedName() + "Throughput"), + globalThroughputMeter + ); + } +} diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/handlers/ThroughputLimiterFactory.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/handlers/ThroughputLimiterFactory.java index aae4d09ff2..da0abba8a5 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/handlers/ThroughputLimiterFactory.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/handlers/ThroughputLimiterFactory.java @@ -3,7 +3,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import pl.allegro.tech.hermes.common.metric.MetricsFacade; +import pl.allegro.tech.hermes.frontend.metric.ThroughputRegistry; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ThreadFactory; @@ -15,13 +15,13 @@ public class ThroughputLimiterFactory { private final ThroughputParameters throughputParameters; - private final MetricsFacade metricsFacade; + private final ThroughputRegistry throughputRegistry; private enum ThroughputLimiterType { UNLIMITED, FIXED, DYNAMIC } - public ThroughputLimiterFactory(ThroughputParameters throughputParameters, MetricsFacade metricsFacade) { + public ThroughputLimiterFactory(ThroughputParameters throughputParameters, ThroughputRegistry throughputRegistry) { this.throughputParameters = throughputParameters; - this.metricsFacade = metricsFacade; + this.throughputRegistry = throughputRegistry; } public ThroughputLimiter provide() { @@ -37,7 +37,7 @@ public ThroughputLimiter provide() { throughputParameters.getDynamicDesired(), throughputParameters.getDynamicIdle(), throughputParameters.getDynamicCheckInterval(), - metricsFacade.topics().topicGlobalThroughputBytes(), + throughputRegistry::getGlobalThroughputOneMinuteRate, getExecutor()); default: throw new IllegalArgumentException("Unknown throughput limiter type."); diff --git a/hermes-frontend/src/main/resources/application-integration.yaml b/hermes-frontend/src/main/resources/application-integration.yaml index 4af762bba7..1ab55e4739 100644 --- a/hermes-frontend/src/main/resources/application-integration.yaml +++ b/hermes-frontend/src/main/resources/application-integration.yaml @@ -17,12 +17,6 @@ frontend: http2Enabled: true handlers: idleTimeout: 2000ms - metrics: - metric-registry: - graphiteReporterEnabled: true - reportPeriod: 1s - graphite: - port: 18023 zookeeper: clusters: - diff --git a/hermes-frontend/src/main/resources/application-local.yaml b/hermes-frontend/src/main/resources/application-local.yaml index c98dc5d41d..1c78837c87 100644 --- a/hermes-frontend/src/main/resources/application-local.yaml +++ b/hermes-frontend/src/main/resources/application-local.yaml @@ -8,11 +8,6 @@ frontend: clusters: - datacenter: "dc" brokerList: "localhost:9092" - graphite: - host: "localhost" - metrics: - metric-registry: - graphiteReporterEnabled: true schema: cache: refreshAfterWrite: 1m diff --git a/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducerIntegrationTest.groovy b/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducerIntegrationTest.groovy index 401112312c..0a3abe9e4b 100644 --- a/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducerIntegrationTest.groovy +++ b/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducerIntegrationTest.groovy @@ -1,6 +1,5 @@ package pl.allegro.tech.hermes.frontend.producer.kafka -import com.codahale.metrics.MetricRegistry import com.jayway.awaitility.Awaitility import io.micrometer.core.instrument.simple.SimpleMeterRegistry import org.apache.commons.lang3.tuple.ImmutablePair @@ -17,7 +16,6 @@ import pl.allegro.tech.hermes.api.* import pl.allegro.tech.hermes.common.kafka.ConsumerGroupId import pl.allegro.tech.hermes.common.kafka.JsonToAvroMigrationKafkaNamesMapper import pl.allegro.tech.hermes.common.kafka.KafkaNamesMapper -import pl.allegro.tech.hermes.common.metric.HermesMetrics import pl.allegro.tech.hermes.common.metric.MetricsFacade import pl.allegro.tech.hermes.frontend.config.HTTPHeadersProperties import pl.allegro.tech.hermes.frontend.config.KafkaHeaderNameProperties @@ -26,7 +24,6 @@ import pl.allegro.tech.hermes.frontend.metric.CachedTopic import pl.allegro.tech.hermes.frontend.producer.BrokerLatencyReporter import pl.allegro.tech.hermes.frontend.publishing.avro.AvroMessage import pl.allegro.tech.hermes.frontend.server.CachedTopicsTestHelper -import pl.allegro.tech.hermes.metrics.PathsCompiler import pl.allegro.tech.hermes.test.helper.avro.AvroUser import pl.allegro.tech.hermes.test.helper.builder.TopicBuilder import pl.allegro.tech.hermes.test.helper.containers.ImageTags @@ -85,10 +82,7 @@ class LocalDatacenterMessageProducerIntegrationTest extends Specification { String datacenter = "dc" @Shared - MetricsFacade metricsFacade = new MetricsFacade( - new SimpleMeterRegistry(), - new HermesMetrics(new MetricRegistry(), new PathsCompiler("")) - ) + MetricsFacade metricsFacade = new MetricsFacade(new SimpleMeterRegistry()) def setupSpec() { kafkaContainer.start() diff --git a/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/publishing/metric/ThroughputRegistryTest.groovy b/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/publishing/metric/ThroughputRegistryTest.groovy new file mode 100644 index 0000000000..04cdf08154 --- /dev/null +++ b/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/publishing/metric/ThroughputRegistryTest.groovy @@ -0,0 +1,65 @@ +package pl.allegro.tech.hermes.frontend.publishing.metric + +import com.codahale.metrics.MetricRegistry +import com.jayway.awaitility.Duration +import io.micrometer.core.instrument.simple.SimpleMeterRegistry +import org.assertj.core.api.Assertions +import pl.allegro.tech.hermes.api.Topic +import pl.allegro.tech.hermes.common.metric.MetricsFacade +import pl.allegro.tech.hermes.frontend.metric.ThroughputMeter +import pl.allegro.tech.hermes.frontend.metric.ThroughputRegistry +import spock.lang.Shared +import spock.lang.Specification + +import static com.jayway.awaitility.Awaitility.await +import static pl.allegro.tech.hermes.test.helper.builder.TopicBuilder.topic + +class ThroughputRegistryTest extends Specification { + @Shared + Topic topicA = topic("group.topicA").build() + @Shared + Topic topicB = topic("group.topicB").build() + @Shared + Topic topicC = topic("group.topicC").build(); + + private final MetricsFacade metricsFacade = new MetricsFacade(new SimpleMeterRegistry()); + private final ThroughputRegistry throughputRegistry = new ThroughputRegistry(metricsFacade, new MetricRegistry()) + + def "topic throughput should be preserved for instances of the same topic"() { + given: "throughput meter for a topic with recorded value" + ThroughputMeter meter = throughputRegistry.forTopic(topicA.getName()) + assert meter.oneMinuteRate == 0.0d + + meter.increment(1024) + + when: "new throughput meter is obtained for the same topic" + meter = throughputRegistry.forTopic(topicA.getName()) + + then: "throughput is preserved" + await().atMost(Duration.TEN_SECONDS).until({ + Assertions.assertThat(meter.oneMinuteRate).isGreaterThan(0.0d) + }) + } + + def "global throughput should be shared for all topics"() { + given: "given throughput meters for two topics" + ThroughputMeter topicBMeter = throughputRegistry.forTopic(topicB.getName()) + ThroughputMeter topicCMeter = throughputRegistry.forTopic(topicC.getName()) + assert topicBMeter.oneMinuteRate == 0.0d + assert topicCMeter.oneMinuteRate == 0.0d + + when: "throughput for both meters is recorded" + topicBMeter.increment(1024) + topicCMeter.increment(1024) + + then: "global throughput is a sum of topic throughput" + await().atMost(Duration.TEN_SECONDS).until({ + def topicAValue = topicBMeter.oneMinuteRate + def topicBValue = topicCMeter.oneMinuteRate + def globalValue = throughputRegistry.globalThroughputOneMinuteRate + Assertions.assertThat(topicAValue).isGreaterThan(0.0d) + Assertions.assertThat(topicBValue).isGreaterThan(0.0d) + Assertions.assertThat(globalValue).isGreaterThan(topicAValue) + }) + } +} diff --git a/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/server/CachedTopicsTestHelper.groovy b/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/server/CachedTopicsTestHelper.groovy index 08cd79fc9f..a76d3d4eaa 100644 --- a/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/server/CachedTopicsTestHelper.groovy +++ b/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/server/CachedTopicsTestHelper.groovy @@ -7,26 +7,23 @@ import pl.allegro.tech.hermes.api.Topic import pl.allegro.tech.hermes.common.kafka.KafkaTopic import pl.allegro.tech.hermes.common.kafka.KafkaTopicName import pl.allegro.tech.hermes.common.kafka.KafkaTopics -import pl.allegro.tech.hermes.common.metric.HermesMetrics import pl.allegro.tech.hermes.common.metric.MetricsFacade import pl.allegro.tech.hermes.frontend.metric.CachedTopic -import pl.allegro.tech.hermes.metrics.PathsCompiler +import pl.allegro.tech.hermes.frontend.metric.ThroughputRegistry import pl.allegro.tech.hermes.test.helper.builder.TopicBuilder class CachedTopicsTestHelper { - static HermesMetrics hermesMetrics = new HermesMetrics( - new MetricRegistry(), new PathsCompiler("localhost")) - - static MetricsFacade micrometerHermesMetrics = new MetricsFacade(new SimpleMeterRegistry(), hermesMetrics) + static MetricsFacade micrometerHermesMetrics = new MetricsFacade(new SimpleMeterRegistry()) + static ThroughputRegistry registry = new ThroughputRegistry(micrometerHermesMetrics, new MetricRegistry()) static CachedTopic cachedTopic(String name) { def kafkaTopics = new KafkaTopics(new KafkaTopic(KafkaTopicName.valueOf(name), ContentType.JSON)) - return new CachedTopic(TopicBuilder.topic(name).build(), micrometerHermesMetrics, kafkaTopics) + return new CachedTopic(TopicBuilder.topic(name).build(), micrometerHermesMetrics, registry, kafkaTopics) } static CachedTopic cachedTopic(Topic topic) { def kafkaTopics = new KafkaTopics(new KafkaTopic(KafkaTopicName.valueOf(topic.qualifiedName), ContentType.JSON)) - return new CachedTopic(topic, micrometerHermesMetrics, kafkaTopics) + return new CachedTopic(topic, micrometerHermesMetrics, registry, kafkaTopics) } } diff --git a/hermes-frontend/src/test/java/pl/allegro/tech/hermes/frontend/buffer/BackupMessagesLoaderTest.java b/hermes-frontend/src/test/java/pl/allegro/tech/hermes/frontend/buffer/BackupMessagesLoaderTest.java index d4a8f5de74..c530cbc7f6 100644 --- a/hermes-frontend/src/test/java/pl/allegro/tech/hermes/frontend/buffer/BackupMessagesLoaderTest.java +++ b/hermes-frontend/src/test/java/pl/allegro/tech/hermes/frontend/buffer/BackupMessagesLoaderTest.java @@ -71,9 +71,8 @@ public void setUp() { tempDir = Files.createTempDir(); Timer micrometerTimer = new SimpleMeterRegistry().timer("broker-latency"); - com.codahale.metrics.Timer graphiteTimer = new com.codahale.metrics.Timer(); when(cachedTopic.getTopic()).thenReturn(topic); - when(cachedTopic.startBrokerLatencyTimer()).thenReturn(HermesTimerContext.from(micrometerTimer, graphiteTimer)); + when(cachedTopic.startBrokerLatencyTimer()).thenReturn(HermesTimerContext.from(micrometerTimer)); when(topicsCache.getTopic(topic.getQualifiedName())).thenReturn(Optional.of(cachedTopic)); when(producer.isTopicAvailable(cachedTopic)).thenReturn(true); } diff --git a/hermes-frontend/src/test/java/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducerTest.java b/hermes-frontend/src/test/java/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducerTest.java index 9d0abf3232..febb9c49ad 100644 --- a/hermes-frontend/src/test/java/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducerTest.java +++ b/hermes-frontend/src/test/java/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducerTest.java @@ -1,6 +1,5 @@ package pl.allegro.tech.hermes.frontend.producer.kafka; -import com.codahale.metrics.MetricRegistry; import io.micrometer.core.instrument.simple.SimpleMeterRegistry; import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.producer.MockProducer; @@ -16,18 +15,17 @@ import pl.allegro.tech.hermes.common.kafka.HTTPHeadersPropagationAsKafkaHeadersProperties; import pl.allegro.tech.hermes.common.kafka.KafkaNamesMapper; import pl.allegro.tech.hermes.common.kafka.NamespaceKafkaNamesMapper; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.frontend.cache.topic.TopicsCache; import pl.allegro.tech.hermes.frontend.config.HTTPHeadersProperties; import pl.allegro.tech.hermes.frontend.config.KafkaHeaderNameProperties; import pl.allegro.tech.hermes.frontend.config.SchemaProperties; import pl.allegro.tech.hermes.frontend.metric.CachedTopic; +import pl.allegro.tech.hermes.frontend.metric.ThroughputRegistry; import pl.allegro.tech.hermes.frontend.producer.BrokerLatencyReporter; import pl.allegro.tech.hermes.frontend.publishing.PublishingCallback; import pl.allegro.tech.hermes.frontend.publishing.message.JsonMessage; import pl.allegro.tech.hermes.frontend.publishing.message.Message; -import pl.allegro.tech.hermes.metrics.PathsCompiler; import java.time.Duration; import java.util.List; @@ -53,9 +51,7 @@ public class LocalDatacenterMessageProducerTest { private static final Message MESSAGE = new JsonMessage(MESSAGE_ID, CONTENT, TIMESTAMP, PARTITION_KEY, emptyMap()); private final ByteArraySerializer serializer = new ByteArraySerializer(); - @Mock - private HermesMetrics hermesMetrics = new HermesMetrics(new MetricRegistry(), new PathsCompiler("")); - private final MetricsFacade metricsFacade = new MetricsFacade(new SimpleMeterRegistry(), hermesMetrics); + private final MetricsFacade metricsFacade = new MetricsFacade(new SimpleMeterRegistry()); private final BrokerLatencyReporter brokerLatencyReporter = new BrokerLatencyReporter(false, metricsFacade, Duration.ZERO, Executors.newSingleThreadExecutor()); @@ -91,9 +87,12 @@ public class LocalDatacenterMessageProducerTest { private final SchemaProperties schemaProperties = new SchemaProperties(); + @Mock + private ThroughputRegistry throughputRegistry; + @Before public void before() { - cachedTopic = new CachedTopic(TOPIC, metricsFacade, kafkaNamesMapper.toKafkaTopics(TOPIC)); + cachedTopic = new CachedTopic(TOPIC, metricsFacade, throughputRegistry, kafkaNamesMapper.toKafkaTopics(TOPIC)); MessageToKafkaProducerRecordConverter messageConverter = new MessageToKafkaProducerRecordConverter(kafkaHeaderFactory, schemaProperties.isIdHeaderEnabled()); producer = new LocalDatacenterMessageProducer(kafkaMessageSenders, messageConverter); @@ -160,7 +159,7 @@ public void onEachPublished(Message message, Topic topic, String datacenter) { public void shouldUseEveryoneConfirmProducerForTopicWithAckAll() { //given Topic topic = topic("group.all").withAck(Topic.Ack.ALL).build(); - CachedTopic cachedTopic = new CachedTopic(topic, metricsFacade, + CachedTopic cachedTopic = new CachedTopic(topic, metricsFacade, throughputRegistry, kafkaNamesMapper.toKafkaTopics(topic)); //when diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ExternalMonitoringConfiguration.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ExternalMonitoringConfiguration.java index accd325bf2..ed3148d5b3 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ExternalMonitoringConfiguration.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ExternalMonitoringConfiguration.java @@ -13,9 +13,6 @@ import org.springframework.http.client.ClientHttpRequestFactory; import org.springframework.http.client.HttpComponentsClientHttpRequestFactory; import org.springframework.web.client.RestTemplate; -import pl.allegro.tech.hermes.management.infrastructure.graphite.CachingGraphiteClient; -import pl.allegro.tech.hermes.management.infrastructure.graphite.GraphiteClient; -import pl.allegro.tech.hermes.management.infrastructure.graphite.RestTemplateGraphiteClient; import pl.allegro.tech.hermes.management.infrastructure.prometheus.CachingPrometheusClient; import pl.allegro.tech.hermes.management.infrastructure.prometheus.PrometheusClient; import pl.allegro.tech.hermes.management.infrastructure.prometheus.RestTemplatePrometheusClient; @@ -28,20 +25,6 @@ @Configuration public class ExternalMonitoringConfiguration { - @Bean - @ConditionalOnProperty(value = "graphite.client.enabled", havingValue = "true") - public GraphiteClient graphiteClient(@Qualifier("monitoringRestTemplate") RestTemplate graphiteRestTemplate, - GraphiteMonitoringMetricsProperties graphiteClientProperties) { - RestTemplateGraphiteClient underlyingGraphiteClient = - new RestTemplateGraphiteClient(graphiteRestTemplate, URI.create(graphiteClientProperties.getExternalMonitoringUrl())); - return new CachingGraphiteClient( - underlyingGraphiteClient, - systemTicker(), - graphiteClientProperties.getCacheTtlSeconds(), - graphiteClientProperties.getCacheSize() - ); - } - @Bean @ConditionalOnProperty(value = "prometheus.client.enabled", havingValue = "true") public VictoriaMetricsMetricsProvider prometheusMetricsProvider(PrometheusClient prometheusClient, @@ -53,10 +36,10 @@ public VictoriaMetricsMetricsProvider prometheusMetricsProvider(PrometheusClient @Bean @ConditionalOnProperty(value = "prometheus.client.enabled", havingValue = "true") - public PrometheusClient prometheusClient(@Qualifier("monitoringRestTemplate") RestTemplate graphiteRestTemplate, + public PrometheusClient prometheusClient(@Qualifier("monitoringRestTemplate") RestTemplate monitoringRestTemplate, PrometheusMonitoringClientProperties clientProperties) { RestTemplatePrometheusClient underlyingPrometheusClient = - new RestTemplatePrometheusClient(graphiteRestTemplate, URI.create(clientProperties.getExternalMonitoringUrl())); + new RestTemplatePrometheusClient(monitoringRestTemplate, URI.create(clientProperties.getExternalMonitoringUrl())); return new CachingPrometheusClient( underlyingPrometheusClient, systemTicker(), diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/GraphiteMonitoringMetricsProperties.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/GraphiteMonitoringMetricsProperties.java deleted file mode 100644 index 4ae837902f..0000000000 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/GraphiteMonitoringMetricsProperties.java +++ /dev/null @@ -1,14 +0,0 @@ -package pl.allegro.tech.hermes.management.config; - -public class GraphiteMonitoringMetricsProperties extends ExternalMonitoringClientProperties { - - private String prefix = "stats.tech.hermes"; - - public String getPrefix() { - return prefix; - } - - public void setPrefix(String prefix) { - this.prefix = prefix; - } -} diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ManagementConfiguration.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ManagementConfiguration.java index 8277843789..c6945f814c 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ManagementConfiguration.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ManagementConfiguration.java @@ -1,6 +1,5 @@ package pl.allegro.tech.hermes.management.config; -import com.codahale.metrics.MetricRegistry; import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.InjectableValues; @@ -15,7 +14,6 @@ import org.springframework.context.annotation.Configuration; import pl.allegro.tech.hermes.api.Topic; import pl.allegro.tech.hermes.common.clock.ClockFactory; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.common.util.InetAddressInstanceIdResolver; import pl.allegro.tech.hermes.common.util.InstanceIdResolver; @@ -55,12 +53,6 @@ public ObjectMapper objectMapper() { return mapper; } - @Bean - @ConditionalOnMissingBean - public MetricRegistry metricRegistry() { - return new MetricRegistry(); - } - @Bean public InstanceIdResolver instanceIdResolver() { return new InetAddressInstanceIdResolver(); @@ -72,14 +64,8 @@ public PathsCompiler pathsCompiler(InstanceIdResolver instanceIdResolver) { } @Bean - public HermesMetrics hermesMetrics(MetricRegistry metricRegistry, - PathsCompiler pathsCompiler) { - return new HermesMetrics(metricRegistry, pathsCompiler); - } - - @Bean - public MetricsFacade micrometerHermesMetrics(MeterRegistry meterRegistry, HermesMetrics hermesMetrics) { - return new MetricsFacade(meterRegistry, hermesMetrics); + public MetricsFacade micrometerHermesMetrics(MeterRegistry meterRegistry) { + return new MetricsFacade(meterRegistry); } @Bean diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/MonitoringClientPropertiesConfiguration.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/MonitoringClientPropertiesConfiguration.java index a67745dc33..ed4b733b0d 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/MonitoringClientPropertiesConfiguration.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/MonitoringClientPropertiesConfiguration.java @@ -11,12 +11,6 @@ */ @Configuration public class MonitoringClientPropertiesConfiguration { - @Bean - @ConfigurationProperties("graphite.client") - @ConditionalOnProperty(value = "graphite.client.enabled", havingValue = "true") - public GraphiteMonitoringMetricsProperties graphiteMonitoringClientProperties() { - return new GraphiteMonitoringMetricsProperties(); - } @Bean @ConfigurationProperties("prometheus.client") diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/graphite/CachingGraphiteClient.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/graphite/CachingGraphiteClient.java deleted file mode 100644 index aeae87869d..0000000000 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/graphite/CachingGraphiteClient.java +++ /dev/null @@ -1,62 +0,0 @@ -package pl.allegro.tech.hermes.management.infrastructure.graphite; - -import com.google.common.base.Ticker; -import com.google.common.cache.CacheBuilder; -import com.google.common.cache.CacheLoader; -import com.google.common.cache.LoadingCache; -import pl.allegro.tech.hermes.api.MetricDecimalValue; -import pl.allegro.tech.hermes.management.infrastructure.metrics.MonitoringMetricsContainer; - -import java.util.Map; -import java.util.concurrent.ExecutionException; - -import static com.google.common.collect.Iterables.toArray; -import static java.util.Arrays.asList; -import static java.util.Arrays.stream; -import static java.util.Collections.singleton; -import static java.util.concurrent.TimeUnit.SECONDS; -import static java.util.function.Function.identity; -import static java.util.stream.Collectors.toMap; - -public class CachingGraphiteClient implements GraphiteClient { - - private final GraphiteClient underlyingGraphiteClient; - private final LoadingCache graphiteMetricsCache; - - public CachingGraphiteClient(GraphiteClient underlyingGraphiteClient, Ticker ticker, long cacheTtlInSeconds, long cacheSize) { - this.underlyingGraphiteClient = underlyingGraphiteClient; - this.graphiteMetricsCache = CacheBuilder.newBuilder() - .ticker(ticker) - .expireAfterWrite(cacheTtlInSeconds, SECONDS) - .maximumSize(cacheSize) - .build(new GraphiteMetricsCacheLoader()); - } - - @Override - public MonitoringMetricsContainer readMetrics(String... metricPaths) { - try { - Map graphiteMetrics = graphiteMetricsCache.getAll(asList(metricPaths)); - return MonitoringMetricsContainer.initialized(graphiteMetrics); - } catch (ExecutionException e) { - // should never happen because the loader does not throw any checked exceptions - throw new RuntimeException(e); - } - } - - private class GraphiteMetricsCacheLoader extends CacheLoader { - @Override - public MetricDecimalValue load(String metricPath) { - return loadAll(singleton(metricPath)).get(metricPath); - } - - @Override - public Map loadAll(Iterable metricPaths) { - String[] metricPathsArray = toArray(metricPaths, String.class); - MonitoringMetricsContainer metricsContainer = underlyingGraphiteClient.readMetrics(metricPathsArray); - return stream(metricPathsArray).collect(toMap( - identity(), - metricsContainer::metricValue - )); - } - } -} \ No newline at end of file diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/graphite/GraphiteClient.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/graphite/GraphiteClient.java deleted file mode 100644 index 489991f76f..0000000000 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/graphite/GraphiteClient.java +++ /dev/null @@ -1,8 +0,0 @@ -package pl.allegro.tech.hermes.management.infrastructure.graphite; - -import pl.allegro.tech.hermes.management.infrastructure.metrics.MonitoringMetricsContainer; - -public interface GraphiteClient { - - MonitoringMetricsContainer readMetrics(String... metricPaths); -} diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/graphite/GraphiteResponse.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/graphite/GraphiteResponse.java deleted file mode 100644 index 4b5fedfbe5..0000000000 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/graphite/GraphiteResponse.java +++ /dev/null @@ -1,29 +0,0 @@ -package pl.allegro.tech.hermes.management.infrastructure.graphite; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; - -import java.util.List; - -public class GraphiteResponse { - - private final String target; - private final List> datapoints; - - @JsonCreator - public GraphiteResponse( - @JsonProperty("target") String target, - @JsonProperty("datapoints") List> datapoints - ) { - this.target = target; - this.datapoints = datapoints; - } - - public String getTarget() { - return target; - } - - public List> getDatapoints() { - return datapoints; - } -} diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/graphite/RestTemplateGraphiteClient.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/graphite/RestTemplateGraphiteClient.java deleted file mode 100644 index baad2da445..0000000000 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/graphite/RestTemplateGraphiteClient.java +++ /dev/null @@ -1,95 +0,0 @@ -package pl.allegro.tech.hermes.management.infrastructure.graphite; - -import com.google.common.base.Strings; -import jakarta.ws.rs.core.UriBuilder; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.springframework.core.ParameterizedTypeReference; -import org.springframework.http.HttpEntity; -import org.springframework.http.HttpMethod; -import org.springframework.http.ResponseEntity; -import org.springframework.web.client.RestTemplate; -import pl.allegro.tech.hermes.api.MetricDecimalValue; -import pl.allegro.tech.hermes.management.infrastructure.metrics.MonitoringMetricsContainer; - -import java.io.UnsupportedEncodingException; -import java.net.URI; -import java.util.List; - -import static com.google.common.base.Preconditions.checkArgument; -import static org.apache.commons.lang.exception.ExceptionUtils.getRootCauseMessage; - -public class RestTemplateGraphiteClient implements GraphiteClient { - - private static final Logger logger = LoggerFactory.getLogger(RestTemplateGraphiteClient.class); - - private static final ParameterizedTypeReference> GRAPHITE_RESPONSE_TYPE = - new ParameterizedTypeReference<>() { - }; - - private static final String DEFAULT_VALUE = "0.0"; - - private static final String TARGET_PARAM = "target"; - - private final URI graphiteUri; - - private final RestTemplate restTemplate; - - public RestTemplateGraphiteClient(RestTemplate restTemplate, URI graphiteUri) { - this.restTemplate = restTemplate; - this.graphiteUri = graphiteUri; - } - - @Override - public MonitoringMetricsContainer readMetrics(String... metricPaths) { - try { - MonitoringMetricsContainer response = MonitoringMetricsContainer.createEmpty(); - queryGraphite(metricPaths).forEach(metric -> response.addMetricValue(metric.getTarget(), getFirstValue(metric))); - return response; - } catch (Exception exception) { - logger.warn("Unable to read from Graphite: {}", getRootCauseMessage(exception)); - return MonitoringMetricsContainer.unavailable(); - } - } - - private MetricDecimalValue getFirstValue(GraphiteResponse graphiteResponse) { - checkArgument(hasDatapoints(graphiteResponse), "Graphite format changed. Reexamine implementation."); - - String firstNotNullValue = DEFAULT_VALUE; - for (List datapoint : graphiteResponse.getDatapoints()) { - if (datapointValid(datapoint)) { - firstNotNullValue = datapoint.get(0); - break; - } - } - return MetricDecimalValue.of(firstNotNullValue); - } - - private boolean datapointValid(List value) { - return !value.isEmpty() && !Strings.isNullOrEmpty(value.get(0)) && !"null".equals(value.get(0)); - } - - private boolean hasDatapoints(GraphiteResponse graphiteResponse) { - return !graphiteResponse.getDatapoints().isEmpty() && !graphiteResponse.getDatapoints().get(0).isEmpty(); - } - - private List queryGraphite(String... queries) throws UnsupportedEncodingException { - UriBuilder builder = UriBuilder.fromUri(graphiteUri) - .path("render") - .queryParam("from", "-5minutes") - .queryParam("until", "-1minutes") - .queryParam("format", "json"); - - for (String query : queries) { - builder.queryParam(TARGET_PARAM, query); - } - - ResponseEntity> response = restTemplate.exchange( - builder.build(), - HttpMethod.GET, - HttpEntity.EMPTY, - GRAPHITE_RESPONSE_TYPE - ); - return response.getBody(); - } -} diff --git a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/graphite/CachingGraphiteClientTest.groovy b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/graphite/CachingGraphiteClientTest.groovy deleted file mode 100644 index cde1f26f27..0000000000 --- a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/graphite/CachingGraphiteClientTest.groovy +++ /dev/null @@ -1,54 +0,0 @@ -package pl.allegro.tech.hermes.management.infrastructure.graphite - -import pl.allegro.tech.hermes.management.infrastructure.metrics.MonitoringMetricsContainer -import pl.allegro.tech.hermes.test.helper.cache.FakeTicker -import spock.lang.Specification -import spock.lang.Subject - -import java.time.Duration - -import static pl.allegro.tech.hermes.api.MetricDecimalValue.of - -class CachingGraphiteClientTest extends Specification { - static final CACHE_TTL_IN_SECONDS = 30 - static final CACHE_SIZE = 100_000 - static final CACHE_TTL = Duration.ofSeconds(CACHE_TTL_IN_SECONDS) - - def underlyingClient = Mock(GraphiteClient) - def ticker = new FakeTicker() - - @Subject - def cachingClient = new CachingGraphiteClient(underlyingClient, ticker, CACHE_TTL_IN_SECONDS, CACHE_SIZE) - - def "should return metrics from the underlying client"() { - given: - underlyingClient.readMetrics("metric_1", "metric_2") >> MonitoringMetricsContainer.initialized([metric_1: of("1"), metric_2: of("2")]) - - when: - def metrics = cachingClient.readMetrics("metric_1", "metric_2") - - then: - metrics.metricValue("metric_1") == of("1") - metrics.metricValue("metric_2") == of("2") - } - - def "should return metrics from cache while TTL has not expired"() { - when: - cachingClient.readMetrics("metric_1", "metric_2") - ticker.advance(CACHE_TTL.minusSeconds(1)) - cachingClient.readMetrics("metric_1", "metric_2") - - then: - 1 * underlyingClient.readMetrics("metric_1", "metric_2") >> MonitoringMetricsContainer.initialized([metric_1: of("1"), metric_2: of("2")]) - } - - def "should get metrics from the underlying client after TTL expires"() { - when: - cachingClient.readMetrics("metric_1", "metric_2") - ticker.advance(CACHE_TTL.plusSeconds(1)) - cachingClient.readMetrics("metric_1", "metric_2") - - then: - 2 * underlyingClient.readMetrics("metric_1", "metric_2") >> MonitoringMetricsContainer.initialized([metric_1: of("1"), metric_2: of("2")]) - } -} diff --git a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/graphite/RestTemplateGraphiteClientTest.groovy b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/graphite/RestTemplateGraphiteClientTest.groovy deleted file mode 100644 index e73c9be9ef..0000000000 --- a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/graphite/RestTemplateGraphiteClientTest.groovy +++ /dev/null @@ -1,101 +0,0 @@ -package pl.allegro.tech.hermes.management.infrastructure.graphite - -import com.github.tomakehurst.wiremock.client.WireMock -import com.github.tomakehurst.wiremock.junit.WireMockRule -import jakarta.ws.rs.core.MediaType -import org.junit.Rule -import org.springframework.web.client.RestTemplate -import pl.allegro.tech.hermes.management.infrastructure.metrics.MonitoringMetricsContainer -import pl.allegro.tech.hermes.test.helper.util.Ports -import spock.lang.Specification - -import static pl.allegro.tech.hermes.api.MetricDecimalValue.of - -class RestTemplateGraphiteClientTest extends Specification { - - private static final int GRAPHITE_HTTP_PORT = Ports.nextAvailable() - - @Rule - WireMockRule wireMockRule = new WireMockRule(GRAPHITE_HTTP_PORT) - - private RestTemplateGraphiteClient client - - void setup() { - RestTemplate restTemplate = new RestTemplate(); - client = new RestTemplateGraphiteClient(restTemplate, URI.create("http://localhost:$GRAPHITE_HTTP_PORT")); - } - - def "should get metrics for path"() { - given: - mockGraphite([ - [ metric: 'metric1', data: ['10'] ], - [ metric: 'metric2', data: ['20'] ] - ]) - - when: - MonitoringMetricsContainer metrics = client.readMetrics("metric1", "metric2") - - then: - metrics.metricValue("metric1") == of("10") - metrics.metricValue("metric2") == of("20") - } - - def "should return default value when metric has no value"() { - given: - mockGraphite([[ metric: 'metric', data: [null] ]]) - - when: - MonitoringMetricsContainer metrics = client.readMetrics("metric"); - - then: - metrics.metricValue("metric1") == of("0.0") - } - - def "should return first notnull value"() { - given: - mockGraphite([ - [ metric: 'metric', data: [null, null, '13'] ], - ]) - - when: - MonitoringMetricsContainer metrics = client.readMetrics("metric"); - - then: - metrics.metricValue("metric") == of("13") - } - - def "should properly encode metric query strings"() { - given: - mockGraphite([ - [ metric: 'sumSeries%28stats.tech.hermes.%2A.m1_rate%29', data: ['13'] ], - ]) - - when: - MonitoringMetricsContainer metrics = client.readMetrics('sumSeries(stats.tech.hermes.*.m1_rate)'); - - then: - metrics.metricValue('sumSeries%28stats.tech.hermes.%2A.m1_rate%29') == of("13") - } - - private void mockGraphite(List queries) { - String targetParams = queries.collect({ "target=$it.metric" }).join('&') - String response = '[' + queries.collect({ jsonResponse(it.metric, it.data) }).join(',') + ']' - - mockGraphite(targetParams, response) - } - - private void mockGraphite(String targetParams, String jsonResponse) { - WireMock.stubFor(WireMock.get(WireMock.urlEqualTo(String.format("/render?from=-5minutes&until=-1minutes&format=json&%s", targetParams))) - .willReturn(WireMock.aResponse() - .withStatus(200) - .withHeader("Content-Type", MediaType.APPLICATION_JSON) - .withBody(jsonResponse))); - } - - private String jsonResponse(String query, List datapoints) { - long timestamp = System.currentTimeSeconds() - String datapointsString = datapoints.collect({ "[$it, $timestamp]" }).join(',') - return '{"target": "' + query + '", "datapoints": [' + datapointsString + '], "tags": []}' - } - -} diff --git a/hermes-metrics/build.gradle b/hermes-metrics/build.gradle index 867742ae82..d44403b978 100644 --- a/hermes-metrics/build.gradle +++ b/hermes-metrics/build.gradle @@ -3,7 +3,7 @@ plugins { } dependencies { - api group: 'io.dropwizard.metrics', name: 'metrics-graphite', version: versions.dropwizard_metrics + api group: 'io.dropwizard.metrics', name: 'metrics-core', version: versions.dropwizard_metrics api group: 'commons-lang', name: 'commons-lang', version: '2.6' api group: 'io.micrometer', name: 'micrometer-core', version: versions.micrometer_metrics diff --git a/hermes-metrics/src/main/java/pl/allegro/tech/hermes/metrics/DefaultHermesHistogram.java b/hermes-metrics/src/main/java/pl/allegro/tech/hermes/metrics/DefaultHermesHistogram.java index 33e8d9b14e..ffc1573d94 100644 --- a/hermes-metrics/src/main/java/pl/allegro/tech/hermes/metrics/DefaultHermesHistogram.java +++ b/hermes-metrics/src/main/java/pl/allegro/tech/hermes/metrics/DefaultHermesHistogram.java @@ -1,23 +1,20 @@ package pl.allegro.tech.hermes.metrics; +import io.micrometer.core.instrument.DistributionSummary; + public class DefaultHermesHistogram implements HermesHistogram { - private final io.micrometer.core.instrument.DistributionSummary micrometerHistogram; - private final com.codahale.metrics.Histogram graphiteHistogram; + private final DistributionSummary micrometerHistogram; - private DefaultHermesHistogram(io.micrometer.core.instrument.DistributionSummary micrometerHistogram, - com.codahale.metrics.Histogram graphiteHistogram) { + private DefaultHermesHistogram(io.micrometer.core.instrument.DistributionSummary micrometerHistogram) { this.micrometerHistogram = micrometerHistogram; - this.graphiteHistogram = graphiteHistogram; } - public static DefaultHermesHistogram of(io.micrometer.core.instrument.DistributionSummary micrometerHistogram, - com.codahale.metrics.Histogram graphiteHistogram) { - return new DefaultHermesHistogram(micrometerHistogram, graphiteHistogram); + public static DefaultHermesHistogram of(io.micrometer.core.instrument.DistributionSummary micrometerHistogram) { + return new DefaultHermesHistogram(micrometerHistogram); } @Override public void record(long value) { micrometerHistogram.record(value); - graphiteHistogram.update(value); } } diff --git a/hermes-metrics/src/main/java/pl/allegro/tech/hermes/metrics/HermesTimer.java b/hermes-metrics/src/main/java/pl/allegro/tech/hermes/metrics/HermesTimer.java index 9b518afd6e..8102c31c94 100644 --- a/hermes-metrics/src/main/java/pl/allegro/tech/hermes/metrics/HermesTimer.java +++ b/hermes-metrics/src/main/java/pl/allegro/tech/hermes/metrics/HermesTimer.java @@ -1,21 +1,19 @@ package pl.allegro.tech.hermes.metrics; +import io.micrometer.core.instrument.Timer; + public class HermesTimer { - private final io.micrometer.core.instrument.Timer micrometerTimer; - private final com.codahale.metrics.Timer graphiteTimer; + private final Timer micrometerTimer; - private HermesTimer(io.micrometer.core.instrument.Timer micrometerTimer, - com.codahale.metrics.Timer graphiteTimer) { + private HermesTimer(Timer micrometerTimer) { this.micrometerTimer = micrometerTimer; - this.graphiteTimer = graphiteTimer; } - public static HermesTimer from(io.micrometer.core.instrument.Timer micrometerTimer, - com.codahale.metrics.Timer graphiteTimer) { - return new HermesTimer(micrometerTimer, graphiteTimer); + public static HermesTimer from(io.micrometer.core.instrument.Timer micrometerTimer) { + return new HermesTimer(micrometerTimer); } public HermesTimerContext time() { - return HermesTimerContext.from(micrometerTimer, graphiteTimer); + return HermesTimerContext.from(micrometerTimer); } } \ No newline at end of file diff --git a/hermes-metrics/src/main/java/pl/allegro/tech/hermes/metrics/HermesTimerContext.java b/hermes-metrics/src/main/java/pl/allegro/tech/hermes/metrics/HermesTimerContext.java index b3a39a349b..65bb85d157 100644 --- a/hermes-metrics/src/main/java/pl/allegro/tech/hermes/metrics/HermesTimerContext.java +++ b/hermes-metrics/src/main/java/pl/allegro/tech/hermes/metrics/HermesTimerContext.java @@ -10,19 +10,17 @@ public class HermesTimerContext implements Closeable { private final Timer micrometerTimer; - private final com.codahale.metrics.Timer graphiteTimer; private final Clock clock; private final long startNanos; - private HermesTimerContext(Timer micrometerTimer, com.codahale.metrics.Timer graphiteTimer, Clock clock) { + private HermesTimerContext(Timer micrometerTimer, Clock clock) { this.micrometerTimer = micrometerTimer; - this.graphiteTimer = graphiteTimer; this.clock = clock; this.startNanos = clock.monotonicTime(); } - public static HermesTimerContext from(Timer micrometerTimer, com.codahale.metrics.Timer graphiteTimer) { - return new HermesTimerContext(micrometerTimer, graphiteTimer, Clock.SYSTEM); + public static HermesTimerContext from(Timer micrometerTimer) { + return new HermesTimerContext(micrometerTimer, Clock.SYSTEM); } @Override @@ -36,7 +34,6 @@ public Duration closeAndGet() { private long reportTimer() { long amount = clock.monotonicTime() - startNanos; - graphiteTimer.update(amount, TimeUnit.NANOSECONDS); micrometerTimer.record(amount, TimeUnit.NANOSECONDS); return amount; } diff --git a/hermes-metrics/src/main/java/pl/allegro/tech/hermes/metrics/PathContext.java b/hermes-metrics/src/main/java/pl/allegro/tech/hermes/metrics/PathContext.java index 642b4edb3e..d96b017d9a 100644 --- a/hermes-metrics/src/main/java/pl/allegro/tech/hermes/metrics/PathContext.java +++ b/hermes-metrics/src/main/java/pl/allegro/tech/hermes/metrics/PathContext.java @@ -7,37 +7,13 @@ public class PathContext { private final Optional group; private final Optional topic; private final Optional subscription; - private final Optional kafkaTopic; - private final Optional partition; - private final Optional kafkaCluster; - private final Optional httpCode; - private final Optional httpCodeFamily; - private final Optional executorName; - private final Optional oAuthProviderName; - private final Optional schemaRepoType; private PathContext(Optional group, Optional topic, - Optional subscription, - Optional kafkaTopic, - Optional partition, - Optional kafkaCluster, - Optional httpCode, - Optional httpCodeFamily, - Optional executorName, - Optional oAuthProviderName, - Optional schemaRepoType) { + Optional subscription) { this.group = group; this.topic = topic; this.subscription = subscription; - this.kafkaTopic = kafkaTopic; - this.partition = partition; - this.kafkaCluster = kafkaCluster; - this.httpCode = httpCode; - this.httpCodeFamily = httpCodeFamily; - this.executorName = executorName; - this.oAuthProviderName = oAuthProviderName; - this.schemaRepoType = schemaRepoType; } public Optional getGroup() { @@ -52,37 +28,6 @@ public Optional getSubscription() { return subscription; } - public Optional getKafkaTopic() { - return kafkaTopic; - } - - public Optional getPartition() { - return partition; - } - - public Optional getKafkaCluster() { - return kafkaCluster; - } - - public Optional getHttpCode() { - return httpCode; - } - - public Optional getHttpCodeFamily() { - return httpCodeFamily; - } - - public Optional getExecutorName() { - return executorName; - } - - public Optional getoAuthProviderName() { - return oAuthProviderName; - } - - public Optional getSchemaRepoType() { - return schemaRepoType; - } public static Builder pathContext() { return new Builder(); @@ -93,14 +38,6 @@ public static class Builder { private Optional group = Optional.empty(); private Optional topic = Optional.empty(); private Optional subscription = Optional.empty(); - private Optional kafkaTopic = Optional.empty(); - private Optional partition = Optional.empty(); - private Optional kafkaCluster = Optional.empty(); - private Optional httpCode = Optional.empty(); - private Optional httpCodeFamily = Optional.empty(); - private Optional executorName = Optional.empty(); - private Optional oAuthProviderName = Optional.empty(); - private Optional schemaRepoType = Optional.empty(); public Builder withGroup(String group) { this.group = Optional.of(group); @@ -117,49 +54,8 @@ public Builder withSubscription(String subscription) { return this; } - public Builder withKafkaTopic(String kafkaTopic) { - this.kafkaTopic = Optional.of(kafkaTopic); - return this; - } - - public Builder withPartition(int partition) { - this.partition = Optional.of(partition); - return this; - } - - public Builder withKafkaCluster(String kafkaCluster) { - this.kafkaCluster = Optional.of(kafkaCluster); - return this; - } - - public Builder withHttpCode(int httpCode) { - this.httpCode = Optional.of(httpCode); - return this; - } - - public Builder withHttpCodeFamily(String httpCodeFamily) { - this.httpCodeFamily = Optional.of(httpCodeFamily); - return this; - } - - public Builder withExecutorName(String executorName) { - this.executorName = Optional.of(executorName); - return this; - } - - public Builder withOAuthProvider(String oAuthProviderName) { - this.oAuthProviderName = Optional.of(oAuthProviderName); - return this; - } - - public Builder withSchemaRepoType(String schemaRepoType) { - this.schemaRepoType = Optional.of(schemaRepoType); - return this; - } - public PathContext build() { - return new PathContext(group, topic, subscription, kafkaTopic, partition, kafkaCluster, - httpCode, httpCodeFamily, executorName, oAuthProviderName, schemaRepoType); + return new PathContext(group, topic, subscription); } } } diff --git a/hermes-metrics/src/main/java/pl/allegro/tech/hermes/metrics/PathsCompiler.java b/hermes-metrics/src/main/java/pl/allegro/tech/hermes/metrics/PathsCompiler.java index b517f4dbeb..63c6ae4ca8 100644 --- a/hermes-metrics/src/main/java/pl/allegro/tech/hermes/metrics/PathsCompiler.java +++ b/hermes-metrics/src/main/java/pl/allegro/tech/hermes/metrics/PathsCompiler.java @@ -10,14 +10,6 @@ public class PathsCompiler { public static final String GROUP = "$group"; public static final String TOPIC = "$topic"; public static final String SUBSCRIPTION = "$subscription"; - public static final String KAFKA_TOPIC = "$kafka_topic"; - public static final String PARTITION = "$partition"; - public static final String KAFKA_CLUSTER = "$kafka_cluster"; - public static final String HTTP_CODE = "$http_code"; - public static final String HTTP_CODE_FAMILY = "$http_family_of_code"; - public static final String EXECUTOR_NAME = "$executor_name"; - public static final String OAUTH_PROVIDER_NAME = "$oauth_provider_name"; - public static final String SCHEMA_REPO_TYPE = "$schema_repo_type"; private final String hostname; @@ -35,14 +27,6 @@ public String compile(String path, PathContext context) { context.getGroup().ifPresent(g -> pathBuilder.replaceAll(GROUP, g)); context.getTopic().ifPresent(t -> pathBuilder.replaceAll(TOPIC, t)); context.getSubscription().ifPresent(s -> pathBuilder.replaceAll(SUBSCRIPTION, s)); - context.getKafkaTopic().ifPresent(k -> pathBuilder.replaceAll(KAFKA_TOPIC, k)); - context.getPartition().ifPresent(p -> pathBuilder.replaceAll(PARTITION, p.toString())); - context.getKafkaCluster().ifPresent(c -> pathBuilder.replaceAll(KAFKA_CLUSTER, c)); - context.getHttpCode().ifPresent(c -> pathBuilder.replaceAll(HTTP_CODE, c.toString())); - context.getHttpCodeFamily().ifPresent(cf -> pathBuilder.replaceAll(HTTP_CODE_FAMILY, cf)); - context.getExecutorName().ifPresent(c -> pathBuilder.replaceAll(EXECUTOR_NAME, c)); - context.getoAuthProviderName().ifPresent(c -> pathBuilder.replaceAll(OAUTH_PROVIDER_NAME, c)); - context.getSchemaRepoType().ifPresent(c -> pathBuilder.replaceAll(SCHEMA_REPO_TYPE, c)); pathBuilder.replaceAll(HOSTNAME, hostname); diff --git a/hermes-metrics/src/main/java/pl/allegro/tech/hermes/metrics/counters/DefaultHermesCounter.java b/hermes-metrics/src/main/java/pl/allegro/tech/hermes/metrics/counters/DefaultHermesCounter.java index cc9a4f69c5..c87629c534 100644 --- a/hermes-metrics/src/main/java/pl/allegro/tech/hermes/metrics/counters/DefaultHermesCounter.java +++ b/hermes-metrics/src/main/java/pl/allegro/tech/hermes/metrics/counters/DefaultHermesCounter.java @@ -5,16 +5,12 @@ public class DefaultHermesCounter implements HermesCounter { protected final io.micrometer.core.instrument.Counter micrometerCounter; - protected final com.codahale.metrics.Counter graphiteCounter; - protected DefaultHermesCounter(io.micrometer.core.instrument.Counter micrometerCounter, - com.codahale.metrics.Counter graphiteCounter) { + protected DefaultHermesCounter(io.micrometer.core.instrument.Counter micrometerCounter) { this.micrometerCounter = micrometerCounter; - this.graphiteCounter = graphiteCounter; } public void increment(long size) { this.micrometerCounter.increment((double) size); - this.graphiteCounter.inc(size); } } diff --git a/hermes-metrics/src/main/java/pl/allegro/tech/hermes/metrics/counters/HermesCounters.java b/hermes-metrics/src/main/java/pl/allegro/tech/hermes/metrics/counters/HermesCounters.java index c9329ca431..67b4e2f054 100644 --- a/hermes-metrics/src/main/java/pl/allegro/tech/hermes/metrics/counters/HermesCounters.java +++ b/hermes-metrics/src/main/java/pl/allegro/tech/hermes/metrics/counters/HermesCounters.java @@ -2,13 +2,7 @@ public class HermesCounters { - public static DefaultHermesCounter from(io.micrometer.core.instrument.Counter micrometerCounter, - com.codahale.metrics.Counter graphiteCounter) { - return new DefaultHermesCounter(micrometerCounter, graphiteCounter); - } - - public static MeterBackedHermesCounter from(io.micrometer.core.instrument.Counter micrometerCounter, - com.codahale.metrics.Meter graphiteMeter) { - return new MeterBackedHermesCounter(micrometerCounter, graphiteMeter); + public static DefaultHermesCounter from(io.micrometer.core.instrument.Counter micrometerCounter) { + return new DefaultHermesCounter(micrometerCounter); } } diff --git a/hermes-metrics/src/main/java/pl/allegro/tech/hermes/metrics/counters/MeterBackedHermesCounter.java b/hermes-metrics/src/main/java/pl/allegro/tech/hermes/metrics/counters/MeterBackedHermesCounter.java deleted file mode 100644 index be226355eb..0000000000 --- a/hermes-metrics/src/main/java/pl/allegro/tech/hermes/metrics/counters/MeterBackedHermesCounter.java +++ /dev/null @@ -1,26 +0,0 @@ -package pl.allegro.tech.hermes.metrics.counters; - - -import pl.allegro.tech.hermes.metrics.HermesCounter; -import pl.allegro.tech.hermes.metrics.HermesRateMeter; - -public class MeterBackedHermesCounter implements HermesCounter, HermesRateMeter { - protected final io.micrometer.core.instrument.Counter micrometerCounter; - protected final com.codahale.metrics.Meter graphiteMeter; - - protected MeterBackedHermesCounter(io.micrometer.core.instrument.Counter micrometerCounter, - com.codahale.metrics.Meter graphiteMeter) { - this.micrometerCounter = micrometerCounter; - this.graphiteMeter = graphiteMeter; - } - - public void increment(long size) { - this.micrometerCounter.increment((double) size); - this.graphiteMeter.mark(size); - } - - @Override - public double getOneMinuteRate() { - return graphiteMeter.getOneMinuteRate(); - } -} diff --git a/hermes-metrics/src/test/groovy/pl.allegro.tech.hermes.metrics/PathsCompilerTest.groovy b/hermes-metrics/src/test/groovy/pl.allegro.tech.hermes.metrics/PathsCompilerTest.groovy index 5319ec8b4c..f37d9425cc 100644 --- a/hermes-metrics/src/test/groovy/pl.allegro.tech.hermes.metrics/PathsCompilerTest.groovy +++ b/hermes-metrics/src/test/groovy/pl.allegro.tech.hermes.metrics/PathsCompilerTest.groovy @@ -21,14 +21,12 @@ class PathsCompilerTest extends Specification { def pathContext = pathContext().withGroup("group") .withTopic("topic") .withSubscription("subscription") - .withPartition(0) - .withHttpCode(201) .build() when: - def compiled = pathsCompiler.compile("hermes.$GROUP.$TOPIC.$SUBSCRIPTION.$PARTITION.$HTTP_CODE", pathContext) + def compiled = pathsCompiler.compile("hermes.$GROUP.$TOPIC.$SUBSCRIPTION", pathContext) then: - compiled == "hermes.group.topic.subscription.0.201" + compiled == "hermes.group.topic.subscription" } } diff --git a/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/metrics/TestMetricsFacadeFactory.java b/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/metrics/TestMetricsFacadeFactory.java index 1b415b44a9..8037c05db3 100644 --- a/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/metrics/TestMetricsFacadeFactory.java +++ b/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/metrics/TestMetricsFacadeFactory.java @@ -1,17 +1,11 @@ package pl.allegro.tech.hermes.test.helper.metrics; -import com.codahale.metrics.MetricRegistry; import io.micrometer.core.instrument.simple.SimpleMeterRegistry; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; import pl.allegro.tech.hermes.common.metric.MetricsFacade; -import pl.allegro.tech.hermes.metrics.PathsCompiler; public class TestMetricsFacadeFactory { public static MetricsFacade create() { - return new MetricsFacade( - new SimpleMeterRegistry(), - new HermesMetrics(new MetricRegistry(), new PathsCompiler("localhost")) - ); + return new MetricsFacade(new SimpleMeterRegistry()); } } diff --git a/hermes-tracker-elasticsearch/src/main/java/pl/allegro/tech/hermes/tracker/elasticsearch/ElasticsearchQueueCommitter.java b/hermes-tracker-elasticsearch/src/main/java/pl/allegro/tech/hermes/tracker/elasticsearch/ElasticsearchQueueCommitter.java index daed417560..8e5e12d232 100644 --- a/hermes-tracker-elasticsearch/src/main/java/pl/allegro/tech/hermes/tracker/elasticsearch/ElasticsearchQueueCommitter.java +++ b/hermes-tracker-elasticsearch/src/main/java/pl/allegro/tech/hermes/tracker/elasticsearch/ElasticsearchQueueCommitter.java @@ -1,6 +1,5 @@ package pl.allegro.tech.hermes.tracker.elasticsearch; -import com.codahale.metrics.Timer; import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.elasticsearch.action.bulk.BulkRequestBuilder; import org.elasticsearch.client.Client; diff --git a/hermes-tracker-elasticsearch/src/test/java/pl/allegro/tech/hermes/tracker/elasticsearch/consumers/ConsumersElasticsearchLogRepositoryTest.java b/hermes-tracker-elasticsearch/src/test/java/pl/allegro/tech/hermes/tracker/elasticsearch/consumers/ConsumersElasticsearchLogRepositoryTest.java index 3f4d5de1c0..3ed0cb3860 100644 --- a/hermes-tracker-elasticsearch/src/test/java/pl/allegro/tech/hermes/tracker/elasticsearch/consumers/ConsumersElasticsearchLogRepositoryTest.java +++ b/hermes-tracker-elasticsearch/src/test/java/pl/allegro/tech/hermes/tracker/elasticsearch/consumers/ConsumersElasticsearchLogRepositoryTest.java @@ -1,6 +1,5 @@ package pl.allegro.tech.hermes.tracker.elasticsearch.consumers; -import com.codahale.metrics.MetricRegistry; import io.micrometer.core.instrument.simple.SimpleMeterRegistry; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.index.query.BoolQueryBuilder; @@ -8,9 +7,7 @@ import org.testng.annotations.AfterSuite; import org.testng.annotations.BeforeSuite; import pl.allegro.tech.hermes.api.SentMessageTraceStatus; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; import pl.allegro.tech.hermes.common.metric.MetricsFacade; -import pl.allegro.tech.hermes.metrics.PathsCompiler; import pl.allegro.tech.hermes.tracker.consumers.AbstractLogRepositoryTest; import pl.allegro.tech.hermes.tracker.consumers.LogRepository; import pl.allegro.tech.hermes.tracker.elasticsearch.ElasticsearchResource; @@ -37,8 +34,7 @@ public class ConsumersElasticsearchLogRepositoryTest extends AbstractLogReposito private static final ConsumersIndexFactory indexFactory = new ConsumersDailyIndexFactory(clock); private static final FrontendIndexFactory frontendIndexFactory = new FrontendDailyIndexFactory(clock); private static final MetricsFacade metricsFacade = new MetricsFacade( - new SimpleMeterRegistry(), - new HermesMetrics(new MetricRegistry(), new PathsCompiler("")) + new SimpleMeterRegistry() ); diff --git a/hermes-tracker-elasticsearch/src/test/java/pl/allegro/tech/hermes/tracker/elasticsearch/frontend/FrontendElasticsearchLogRepositoryTest.java b/hermes-tracker-elasticsearch/src/test/java/pl/allegro/tech/hermes/tracker/elasticsearch/frontend/FrontendElasticsearchLogRepositoryTest.java index 05c2384763..294bc428ff 100644 --- a/hermes-tracker-elasticsearch/src/test/java/pl/allegro/tech/hermes/tracker/elasticsearch/frontend/FrontendElasticsearchLogRepositoryTest.java +++ b/hermes-tracker-elasticsearch/src/test/java/pl/allegro/tech/hermes/tracker/elasticsearch/frontend/FrontendElasticsearchLogRepositoryTest.java @@ -1,6 +1,5 @@ package pl.allegro.tech.hermes.tracker.elasticsearch.frontend; -import com.codahale.metrics.MetricRegistry; import io.micrometer.core.instrument.simple.SimpleMeterRegistry; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.index.query.BoolQueryBuilder; @@ -8,9 +7,7 @@ import org.testng.annotations.AfterSuite; import org.testng.annotations.BeforeSuite; import pl.allegro.tech.hermes.api.PublishedMessageTraceStatus; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; import pl.allegro.tech.hermes.common.metric.MetricsFacade; -import pl.allegro.tech.hermes.metrics.PathsCompiler; import pl.allegro.tech.hermes.tracker.elasticsearch.ElasticsearchResource; import pl.allegro.tech.hermes.tracker.elasticsearch.LogSchemaAware; import pl.allegro.tech.hermes.tracker.elasticsearch.SchemaManager; @@ -38,8 +35,7 @@ public class FrontendElasticsearchLogRepositoryTest extends AbstractLogRepositor private static final FrontendIndexFactory frontendIndexFactory = new FrontendDailyIndexFactory(clock); private static final ConsumersIndexFactory consumersIndexFactory = new ConsumersDailyIndexFactory(clock); private static final MetricsFacade metricsFacade = new MetricsFacade( - new SimpleMeterRegistry(), - new HermesMetrics(new MetricRegistry(), new PathsCompiler("")) + new SimpleMeterRegistry() ); private static final ElasticsearchResource elasticsearch = new ElasticsearchResource(); diff --git a/hermes-tracker-elasticsearch/src/test/java/pl/allegro/tech/hermes/tracker/elasticsearch/management/ElasticsearchLogRepositoryTest.java b/hermes-tracker-elasticsearch/src/test/java/pl/allegro/tech/hermes/tracker/elasticsearch/management/ElasticsearchLogRepositoryTest.java index e3755e8a85..a0606e8e1b 100644 --- a/hermes-tracker-elasticsearch/src/test/java/pl/allegro/tech/hermes/tracker/elasticsearch/management/ElasticsearchLogRepositoryTest.java +++ b/hermes-tracker-elasticsearch/src/test/java/pl/allegro/tech/hermes/tracker/elasticsearch/management/ElasticsearchLogRepositoryTest.java @@ -1,6 +1,5 @@ package pl.allegro.tech.hermes.tracker.elasticsearch.management; -import com.codahale.metrics.MetricRegistry; import com.google.common.collect.ImmutableMap; import io.micrometer.core.instrument.simple.SimpleMeterRegistry; import org.testng.annotations.AfterSuite; @@ -11,9 +10,7 @@ import pl.allegro.tech.hermes.api.PublishedMessageTraceStatus; import pl.allegro.tech.hermes.api.SentMessageTrace; import pl.allegro.tech.hermes.api.SentMessageTraceStatus; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; import pl.allegro.tech.hermes.common.metric.MetricsFacade; -import pl.allegro.tech.hermes.metrics.PathsCompiler; import pl.allegro.tech.hermes.tracker.consumers.MessageMetadata; import pl.allegro.tech.hermes.tracker.consumers.TestMessageMetadata; import pl.allegro.tech.hermes.tracker.elasticsearch.ElasticsearchResource; @@ -50,8 +47,7 @@ public class ElasticsearchLogRepositoryTest implements LogSchemaAware { private static final FrontendIndexFactory frontendIndexFactory = new FrontendDailyIndexFactory(clock); private static final ConsumersIndexFactory consumersIndexFactory = new ConsumersDailyIndexFactory(clock); private static final MetricsFacade metricsFacade = new MetricsFacade( - new SimpleMeterRegistry(), - new HermesMetrics(new MetricRegistry(), new PathsCompiler("")) + new SimpleMeterRegistry() ); private static final ElasticsearchResource elasticsearch = new ElasticsearchResource(); diff --git a/hermes-tracker-elasticsearch/src/test/java/pl/allegro/tech/hermes/tracker/elasticsearch/management/MultiElasticsearchLogRepositoryTest.java b/hermes-tracker-elasticsearch/src/test/java/pl/allegro/tech/hermes/tracker/elasticsearch/management/MultiElasticsearchLogRepositoryTest.java index 877a2e499a..0c9d814e42 100644 --- a/hermes-tracker-elasticsearch/src/test/java/pl/allegro/tech/hermes/tracker/elasticsearch/management/MultiElasticsearchLogRepositoryTest.java +++ b/hermes-tracker-elasticsearch/src/test/java/pl/allegro/tech/hermes/tracker/elasticsearch/management/MultiElasticsearchLogRepositoryTest.java @@ -1,6 +1,5 @@ package pl.allegro.tech.hermes.tracker.elasticsearch.management; -import com.codahale.metrics.MetricRegistry; import com.google.common.collect.ImmutableMap; import io.micrometer.core.instrument.simple.SimpleMeterRegistry; import org.testng.annotations.AfterSuite; @@ -11,9 +10,7 @@ import pl.allegro.tech.hermes.api.PublishedMessageTraceStatus; import pl.allegro.tech.hermes.api.SentMessageTrace; import pl.allegro.tech.hermes.api.SentMessageTraceStatus; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; import pl.allegro.tech.hermes.common.metric.MetricsFacade; -import pl.allegro.tech.hermes.metrics.PathsCompiler; import pl.allegro.tech.hermes.tracker.consumers.MessageMetadata; import pl.allegro.tech.hermes.tracker.consumers.TestMessageMetadata; import pl.allegro.tech.hermes.tracker.elasticsearch.ElasticsearchResource; @@ -51,8 +48,7 @@ public class MultiElasticsearchLogRepositoryTest implements LogSchemaAware { private static final FrontendIndexFactory frontendIndexFactory = new FrontendDailyIndexFactory(clock); private static final ConsumersIndexFactory consumersIndexFactory = new ConsumersDailyIndexFactory(clock); private static final MetricsFacade metricsFacade = new MetricsFacade( - new SimpleMeterRegistry(), - new HermesMetrics(new MetricRegistry(), new PathsCompiler("")) + new SimpleMeterRegistry() ); private static final ElasticsearchResource elasticsearch1 = new ElasticsearchResource(); diff --git a/hermes-tracker/src/main/java/pl/allegro/tech/hermes/tracker/QueueCommitter.java b/hermes-tracker/src/main/java/pl/allegro/tech/hermes/tracker/QueueCommitter.java index 3bae8d6131..51056ba5e4 100644 --- a/hermes-tracker/src/main/java/pl/allegro/tech/hermes/tracker/QueueCommitter.java +++ b/hermes-tracker/src/main/java/pl/allegro/tech/hermes/tracker/QueueCommitter.java @@ -1,6 +1,5 @@ package pl.allegro.tech.hermes.tracker; -import com.codahale.metrics.Timer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import pl.allegro.tech.hermes.metrics.HermesTimer; diff --git a/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/setup/HermesConsumersTestApp.java b/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/setup/HermesConsumersTestApp.java index e7d0541776..f6151023e0 100644 --- a/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/setup/HermesConsumersTestApp.java +++ b/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/setup/HermesConsumersTestApp.java @@ -77,7 +77,6 @@ private List createArgs() { "--consumer.commit.offset.period=" + Duration.ofSeconds(1), "--consumer.metrics.micrometer.reportPeriod=" + Duration.ofSeconds(5), "--consumer.schema.cache.enabled=true", - "--consumer.metrics.metric-registry.graphiteReporterEnabled=false", "--consumer.google.pubsub.sender.transportChannelProviderAddress=" + getGooglePubSubEndpoint() ); } From 8bfb4540762fd5507b468dd5424215cd430a791b Mon Sep 17 00:00:00 2001 From: Maciej Moscicki Date: Mon, 27 May 2024 13:12:23 +0200 Subject: [PATCH 57/87] Immediate fallback for MultiDatacentenderMessageProducer (#1861) --- .../InstrumentedExecutorServiceFactory.java | 46 ++- ...edExecutorServiceFactoryMetricsTest.groovy | 3 +- .../config/ConsumerSenderConfiguration.java | 10 +- .../FailFastKafkaProducerProperties.java | 33 -- .../FailFastLocalKafkaProducerProperties.java | 42 ++- ...FailFastRemoteKafkaProducerProperties.java | 8 +- .../config/FrontendProducerConfiguration.java | 41 ++- .../producer/kafka/KafkaChaosProperties.java | 40 +++ .../producer/kafka/KafkaMessageSender.java | 45 ++- .../kafka/KafkaMessageSendersFactory.java | 10 +- .../kafka/LocalDatacenterMessageProducer.java | 11 +- .../kafka/MultiDatacenterMessageProducer.java | 303 ++++++++++++------ ...enterMessageProducerIntegrationTest.groovy | 9 +- .../MultiDatacenterMessageProducerTest.groovy | 279 ++++++++++++++++ .../LocalDatacenterMessageProducerTest.java | 7 +- 15 files changed, 706 insertions(+), 181 deletions(-) create mode 100644 hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaChaosProperties.java create mode 100644 hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/producer/kafka/MultiDatacenterMessageProducerTest.groovy diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/executor/InstrumentedExecutorServiceFactory.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/executor/InstrumentedExecutorServiceFactory.java index 621876228e..08a69b7936 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/executor/InstrumentedExecutorServiceFactory.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/executor/InstrumentedExecutorServiceFactory.java @@ -33,23 +33,51 @@ public ExecutorService getExecutorService(String name, int size, boolean monitor return monitoringEnabled ? monitor(name, executor) : executor; } - public ScheduledExecutorService getScheduledExecutorService( - String name, int size, boolean monitoringEnabled - ) { - ThreadFactory threadFactory = new ThreadFactoryBuilder().setNameFormat(name + "-scheduled-executor-%d").build(); - ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(size, threadFactory); - return monitoringEnabled ? monitor(name, executor) : executor; + public class ScheduledExecutorServiceBuilder { + final String name; + final int size; + boolean monitoringEnabled = false; + boolean removeOnCancel = false; + + public ScheduledExecutorServiceBuilder(String name, int size) { + this.name = name; + this.size = size; + } + + public ScheduledExecutorServiceBuilder withMonitoringEnabled(boolean monitoringEnabled) { + this.monitoringEnabled = monitoringEnabled; + return this; + } + + public ScheduledExecutorServiceBuilder withRemoveOnCancel(boolean removeOnCancel) { + this.removeOnCancel = removeOnCancel; + return this; + } + + public ScheduledExecutorService create() { + ThreadFactory threadFactory = new ThreadFactoryBuilder().setNameFormat(name + "-scheduled-executor-%d").build(); + ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(size, threadFactory); + executor.setRemoveOnCancelPolicy(removeOnCancel); + return monitoringEnabled ? monitor(name, executor) : executor; + } + + private ScheduledExecutorService monitor(String threadPoolName, ScheduledExecutorService executor) { + return metricsFacade.executor().monitor(executor, threadPoolName); + } } - private ExecutorService monitor(String threadPoolName, ExecutorService executor) { - return metricsFacade.executor().monitor(executor, threadPoolName); + public ScheduledExecutorServiceBuilder scheduledExecutorBuilder( + String name, int size + ) { + return new ScheduledExecutorServiceBuilder(name, size); } - private ScheduledExecutorService monitor(String threadPoolName, ScheduledExecutorService executor) { + private ExecutorService monitor(String threadPoolName, ExecutorService executor) { return metricsFacade.executor().monitor(executor, threadPoolName); } + /** * Copy of {@link java.util.concurrent.Executors#newFixedThreadPool(int, java.util.concurrent.ThreadFactory)} * with configurable queue capacity. diff --git a/hermes-common/src/test/groovy/pl/allegro/tech/hermes/common/metric/executor/InstrumentedExecutorServiceFactoryMetricsTest.groovy b/hermes-common/src/test/groovy/pl/allegro/tech/hermes/common/metric/executor/InstrumentedExecutorServiceFactoryMetricsTest.groovy index 5794650177..7ad8cee715 100644 --- a/hermes-common/src/test/groovy/pl/allegro/tech/hermes/common/metric/executor/InstrumentedExecutorServiceFactoryMetricsTest.groovy +++ b/hermes-common/src/test/groovy/pl/allegro/tech/hermes/common/metric/executor/InstrumentedExecutorServiceFactoryMetricsTest.groovy @@ -45,7 +45,8 @@ class InstrumentedExecutorServiceFactoryMetricsTest extends Specification { def "should record metrics for scheduled executor service (monitoring enabled: #monitoringEnabled)"() { given: - ScheduledExecutorService executor = factory.getScheduledExecutorService("test-scheduled-executor", 10, monitoringEnabled) + ScheduledExecutorService executor = factory.scheduledExecutorBuilder("test-scheduled-executor", 10) + .withMonitoringEnabled(monitoringEnabled).create() when: ScheduledFuture task = executor.schedule({ println("scheduled task executed") }, 1, SECONDS) diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/ConsumerSenderConfiguration.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/ConsumerSenderConfiguration.java index 97d76ff8e8..50be00f5e7 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/ConsumerSenderConfiguration.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/ConsumerSenderConfiguration.java @@ -262,11 +262,11 @@ public EndpointAddressResolver interpolatingEndpointAddressResolver(UriInterpola @Bean public FutureAsyncTimeout futureAsyncTimeoutFactory(InstrumentedExecutorServiceFactory executorFactory, SenderAsyncTimeoutProperties senderAsyncTimeoutProperties) { - ScheduledExecutorService timeoutExecutorService = executorFactory.getScheduledExecutorService( - "async-timeout", - senderAsyncTimeoutProperties.getThreadPoolSize(), - senderAsyncTimeoutProperties.isThreadPoolMonitoringEnabled() - ); + ScheduledExecutorService timeoutExecutorService = executorFactory.scheduledExecutorBuilder( + "async-timeout", + senderAsyncTimeoutProperties.getThreadPoolSize() + ).withMonitoringEnabled(senderAsyncTimeoutProperties.isThreadPoolMonitoringEnabled()) + .create(); return new FutureAsyncTimeout(timeoutExecutorService); } } diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FailFastKafkaProducerProperties.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FailFastKafkaProducerProperties.java index e65eb59c12..01056b10d4 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FailFastKafkaProducerProperties.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FailFastKafkaProducerProperties.java @@ -16,8 +16,6 @@ public class FailFastKafkaProducerProperties { private FallbackSchedulerProperties fallbackScheduler = new FallbackSchedulerProperties(); - private ChaosSchedulerProperties chaosScheduler = new ChaosSchedulerProperties(); - public Duration getSpeculativeSendDelay() { return speculativeSendDelay; } @@ -50,14 +48,6 @@ public void setRemote(KafkaProducerParameters remote) { this.remote = remote; } - public ChaosSchedulerProperties getChaosScheduler() { - return chaosScheduler; - } - - public void setChaosScheduler(ChaosSchedulerProperties chaosScheduler) { - this.chaosScheduler = chaosScheduler; - } - public static class FallbackSchedulerProperties { private int threadPoolSize = 16; @@ -80,27 +70,4 @@ public void setThreadPoolMonitoringEnabled(boolean threadPoolMonitoringEnabled) this.threadPoolMonitoringEnabled = threadPoolMonitoringEnabled; } } - - public static class ChaosSchedulerProperties { - - private int threadPoolSize = 16; - - private boolean threadPoolMonitoringEnabled = false; - - public int getThreadPoolSize() { - return threadPoolSize; - } - - public void setThreadPoolSize(int threadPoolSize) { - this.threadPoolSize = threadPoolSize; - } - - public boolean isThreadPoolMonitoringEnabled() { - return threadPoolMonitoringEnabled; - } - - public void setThreadPoolMonitoringEnabled(boolean threadPoolMonitoringEnabled) { - this.threadPoolMonitoringEnabled = threadPoolMonitoringEnabled; - } - } } diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FailFastLocalKafkaProducerProperties.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FailFastLocalKafkaProducerProperties.java index 7d5b24ed02..f5c53b047c 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FailFastLocalKafkaProducerProperties.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FailFastLocalKafkaProducerProperties.java @@ -4,6 +4,44 @@ import java.time.Duration; + +/** + Kafka producer maintains a single connection to each broker, over which produce request are sent. + When producer request duration exceeds requestTimeout, producer closes the connection to the broker + that the request was sent to. This causes all inflight requests that were sent to that broker to be cancelled. + The number of inflight requests is configured by maxInflightRequestsPerConnection property. + + Let's assume that we have requestTimeout set to 500ms, maxInflightRequestsPerConnection set to 5, + and there are following inflight batches in the producer being sent to broker1: + + batchId | time spent in send buffer (socket) + ------------------------------------ + batch1 | 10ms + batch2 | 200ms + batch3 | 300ms + batch4 | 400ms + batch5 | 501ms + + Batch5 exceeded the requestTimeout so producer will close the connection to broker1. This causes batch5 to be marked + as failed but also causes batches 1-4 to be retried. This has the following consequences: + 1. Batches 1-4 will probably get duplicated - even tough they were cancelled, they were probably sent to the broker, + just haven't been ACKd yet. Retry would cause them to be sent once again resulting in duplicates. + 2. On retry, batches 1-4 will have a smaller time budget to complete. Part of their budget was already wasted + in send buffer + retryBackoff will be applied to them. They will have little time to complete on retry which can cause + them to be timed out, potentially resulting in a vicious circle. + 3. Connection to the broker must be reestablished which takes time. + + To avoid problems described above we actually set requestTimeout and deliveryTimeout to be much higher than the + maximum frontend request duration (frontend.handlers.maxPublishRequestDuration). This means that when + maxPublishRequestDuration is exceeded for a message we received, a client will receive 5xx even tough the + corresponding message is still being processed in the producer. The message will eventually be ACKd by Kafka so upon client-side + retry the message will be duplicated. This however, would likely also happen if the message was promptly timed-out by producer + before maxPublishRequestDuration elapsed - the message was likely already sent to Kafka, there just haven't been a response yet. + + So by using large requestTimeout we cause the first slow message to be duplicated (by client-side retry) but: + - we protect other inflight messages from being duplicated, + - we prevent connections from being frequently dropped and reestablished. + */ public class FailFastLocalKafkaProducerProperties implements KafkaProducerParameters { private Duration maxBlock = Duration.ofMillis(500); @@ -15,9 +53,9 @@ public class FailFastLocalKafkaProducerProperties implements KafkaProducerParame private Duration retryBackoff = Duration.ofMillis(50); - private Duration requestTimeout = Duration.ofMillis(500); + private Duration requestTimeout = Duration.ofSeconds(30); - private Duration deliveryTimeout = Duration.ofMillis(500); + private Duration deliveryTimeout = Duration.ofSeconds(30); private int batchSize = 16 * 1024; diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FailFastRemoteKafkaProducerProperties.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FailFastRemoteKafkaProducerProperties.java index 1265f343bb..b051486460 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FailFastRemoteKafkaProducerProperties.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FailFastRemoteKafkaProducerProperties.java @@ -4,6 +4,10 @@ import java.time.Duration; +/** + * See {@link pl.allegro.tech.hermes.frontend.config.FailFastLocalKafkaProducerProperties} + * for the explanation of default values used. + */ public class FailFastRemoteKafkaProducerProperties implements KafkaProducerParameters { private Duration maxBlock = Duration.ofMillis(250); @@ -15,9 +19,9 @@ public class FailFastRemoteKafkaProducerProperties implements KafkaProducerParam private Duration retryBackoff = Duration.ofMillis(50); - private Duration requestTimeout = Duration.ofMillis(250); + private Duration requestTimeout = Duration.ofSeconds(30); - private Duration deliveryTimeout = Duration.ofMillis(250); + private Duration deliveryTimeout = Duration.ofSeconds(30); private int batchSize = 16 * 1024; diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FrontendProducerConfiguration.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FrontendProducerConfiguration.java index f1265cd44e..fe70b502c2 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FrontendProducerConfiguration.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/FrontendProducerConfiguration.java @@ -9,11 +9,11 @@ import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.common.metric.executor.InstrumentedExecutorServiceFactory; import pl.allegro.tech.hermes.frontend.cache.topic.TopicsCache; -import pl.allegro.tech.hermes.frontend.config.FailFastKafkaProducerProperties.ChaosSchedulerProperties; import pl.allegro.tech.hermes.frontend.config.FailFastKafkaProducerProperties.FallbackSchedulerProperties; import pl.allegro.tech.hermes.frontend.producer.BrokerLatencyReporter; import pl.allegro.tech.hermes.frontend.producer.BrokerMessageProducer; import pl.allegro.tech.hermes.frontend.producer.kafka.FallbackToRemoteDatacenterAwareMessageProducer; +import pl.allegro.tech.hermes.frontend.producer.kafka.KafkaChaosProperties; import pl.allegro.tech.hermes.frontend.producer.kafka.KafkaHeaderFactory; import pl.allegro.tech.hermes.frontend.producer.kafka.KafkaMessageSenders; import pl.allegro.tech.hermes.frontend.producer.kafka.KafkaMessageSendersFactory; @@ -42,6 +42,7 @@ KafkaHeaderNameProperties.class, KafkaProducerProperties.class, FailFastKafkaProducerProperties.class, + KafkaChaosProperties.class, KafkaClustersProperties.class, HTTPHeadersProperties.class }) @@ -69,24 +70,19 @@ public BrokerMessageProducer multiDatacenterBrokerProducer(@Named("failFastKafka AdminReadinessService adminReadinessService, InstrumentedExecutorServiceFactory executorServiceFactory) { FallbackSchedulerProperties fallbackSchedulerProperties = kafkaProducerProperties.getFallbackScheduler(); - ScheduledExecutorService fallbackScheduler = executorServiceFactory.getScheduledExecutorService( + ScheduledExecutorService fallbackScheduler = executorServiceFactory.scheduledExecutorBuilder( "fallback-to-remote", - fallbackSchedulerProperties.getThreadPoolSize(), - fallbackSchedulerProperties.isThreadPoolMonitoringEnabled() - ); - ChaosSchedulerProperties chaosSchedulerProperties = kafkaProducerProperties.getChaosScheduler(); - ScheduledExecutorService chaosScheduler = executorServiceFactory.getScheduledExecutorService( - "chaos", - chaosSchedulerProperties.getThreadPoolSize(), - chaosSchedulerProperties.isThreadPoolMonitoringEnabled() - ); + fallbackSchedulerProperties.getThreadPoolSize() + ) + .withMonitoringEnabled(fallbackSchedulerProperties.isThreadPoolMonitoringEnabled()) + .withRemoveOnCancel(true) + .create(); return new MultiDatacenterMessageProducer( kafkaMessageSenders, adminReadinessService, messageConverter, kafkaProducerProperties.getSpeculativeSendDelay(), - fallbackScheduler, - chaosScheduler + fallbackScheduler ); } @@ -111,6 +107,17 @@ public KafkaMessageSenders failFastKafkaMessageSenders(FailFastKafkaProducerProp "failFast"); } + @Bean + public ScheduledExecutorService chaosScheduler(KafkaChaosProperties chaosProperties, InstrumentedExecutorServiceFactory executorServiceFactory) { + KafkaChaosProperties.ChaosSchedulerProperties chaosSchedulerProperties = chaosProperties.getChaosScheduler(); + return executorServiceFactory.scheduledExecutorBuilder( + "chaos", + chaosSchedulerProperties.getThreadPoolSize() + ) + .withMonitoringEnabled(chaosSchedulerProperties.isThreadPoolMonitoringEnabled()) + .create(); + } + @Bean(destroyMethod = "close") public KafkaMessageSendersFactory kafkaMessageSendersFactory(KafkaClustersProperties kafkaClustersProperties, KafkaProducerProperties kafkaProducerProperties, @@ -119,9 +126,12 @@ public KafkaMessageSendersFactory kafkaMessageSendersFactory(KafkaClustersProper LocalMessageStorageProperties localMessageStorageProperties, DatacenterNameProvider datacenterNameProvider, BrokerLatencyReporter brokerLatencyReporter, - MetricsFacade metricsFacade) { + MetricsFacade metricsFacade, + @Named("chaosScheduler") ScheduledExecutorService chaosScheduler + ) { KafkaProperties kafkaProperties = kafkaClustersProperties.toKafkaProperties(datacenterNameProvider); List remoteKafkaProperties = kafkaClustersProperties.toRemoteKafkaProperties(datacenterNameProvider); + return new KafkaMessageSendersFactory( kafkaProperties, remoteKafkaProperties, @@ -133,7 +143,8 @@ public KafkaMessageSendersFactory kafkaMessageSendersFactory(KafkaClustersProper topicLoadingProperties.getMetadata().getRetryInterval(), topicLoadingProperties.getMetadata().getThreadPoolSize(), localMessageStorageProperties.getBufferedSizeBytes(), - kafkaProducerProperties.getMetadataMaxAge() + kafkaProducerProperties.getMetadataMaxAge(), + chaosScheduler ); } diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaChaosProperties.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaChaosProperties.java new file mode 100644 index 0000000000..9d2ebbdd7e --- /dev/null +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaChaosProperties.java @@ -0,0 +1,40 @@ +package pl.allegro.tech.hermes.frontend.producer.kafka; + +import org.springframework.boot.context.properties.ConfigurationProperties; + +@ConfigurationProperties(prefix = "frontend.kafka.chaos") +public class KafkaChaosProperties { + private ChaosSchedulerProperties chaosScheduler = new ChaosSchedulerProperties(); + + public ChaosSchedulerProperties getChaosScheduler() { + return chaosScheduler; + } + + public void setChaosScheduler(ChaosSchedulerProperties chaosScheduler) { + this.chaosScheduler = chaosScheduler; + } + + public static class ChaosSchedulerProperties { + + private int threadPoolSize = 16; + + private boolean threadPoolMonitoringEnabled = false; + + public int getThreadPoolSize() { + return threadPoolSize; + } + + public void setThreadPoolSize(int threadPoolSize) { + this.threadPoolSize = threadPoolSize; + } + + public boolean isThreadPoolMonitoringEnabled() { + return threadPoolMonitoringEnabled; + } + + public void setThreadPoolMonitoringEnabled(boolean threadPoolMonitoringEnabled) { + this.threadPoolMonitoringEnabled = threadPoolMonitoringEnabled; + } + } + +} diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSender.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSender.java index b9c53dc048..6e00173e48 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSender.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSender.java @@ -7,7 +7,9 @@ import org.apache.kafka.common.Metric; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.InterruptException; +import org.apache.kafka.common.record.RecordBatch; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import pl.allegro.tech.hermes.api.Topic; @@ -22,6 +24,9 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.concurrent.RejectedExecutionException; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; import java.util.function.Predicate; import java.util.function.Supplier; import java.util.function.ToDoubleFunction; @@ -34,28 +39,64 @@ public class KafkaMessageSender { private final BrokerLatencyReporter brokerLatencyReporter; private final MetricsFacade metricsFacade; private final String datacenter; + private final ScheduledExecutorService chaosScheduler; KafkaMessageSender(Producer kafkaProducer, BrokerLatencyReporter brokerLatencyReporter, MetricsFacade metricsFacade, - String datacenter) { + String datacenter, + ScheduledExecutorService chaosScheduler) { this.producer = kafkaProducer; this.brokerLatencyReporter = brokerLatencyReporter; this.metricsFacade = metricsFacade; this.datacenter = datacenter; + this.chaosScheduler = chaosScheduler; } public String getDatacenter() { return datacenter; } + public void send(ProducerRecord producerRecord, + CachedTopic cachedTopic, + Message message, + Callback callback, + MultiDatacenterMessageProducer.ChaosExperiment experiment) { + if (experiment.enabled()) { + try { + chaosScheduler.schedule(() -> { + if (experiment.completeWithError()) { + var exception = new ChaosException(datacenter, experiment.delayInMillis(), message.getId()); + callback.onCompletion(exceptionalRecordMetadata(cachedTopic), exception); + } else { + send(producerRecord, cachedTopic, message, callback); + } + }, experiment.delayInMillis(), TimeUnit.MILLISECONDS); + } catch (RejectedExecutionException e) { + logger.warn("Failed while scheduling chaos experiment. Sending message to Kafka.", e); + send(producerRecord, cachedTopic, message, callback); + } + } else { + send(producerRecord, cachedTopic, message, callback); + } + } + public void send(ProducerRecord producerRecord, CachedTopic cachedTopic, Message message, Callback callback) { HermesTimerContext timer = cachedTopic.startBrokerLatencyTimer(); Callback meteredCallback = new MeteredCallback(timer, message, cachedTopic, callback); - producer.send(producerRecord, meteredCallback); + try { + producer.send(producerRecord, meteredCallback); + } catch (Exception e) { + callback.onCompletion(exceptionalRecordMetadata(cachedTopic), e); + } + } + + private static RecordMetadata exceptionalRecordMetadata(CachedTopic cachedTopic) { + var tp = new TopicPartition(cachedTopic.getKafkaTopics().getPrimary().name().asString(), RecordMetadata.UNKNOWN_PARTITION); + return new RecordMetadata(tp, -1, -1, RecordBatch.NO_TIMESTAMP, -1, -1); } List loadPartitionMetadataFor(String topic) { diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSendersFactory.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSendersFactory.java index b03b8bc387..cd10168f3c 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSendersFactory.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSendersFactory.java @@ -10,6 +10,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.ScheduledExecutorService; import static org.apache.kafka.clients.CommonClientConfigs.SECURITY_PROTOCOL_CONFIG; import static org.apache.kafka.clients.producer.ProducerConfig.ACKS_CONFIG; @@ -46,6 +47,7 @@ public class KafkaMessageSendersFactory { private final BrokerLatencyReporter brokerLatencyReporter; private final MetricsFacade metricsFacade; private final long bufferedSizeBytes; + private final ScheduledExecutorService chaosScheduler; public KafkaMessageSendersFactory(KafkaParameters kafkaParameters, List remoteKafkaParameters, @@ -57,7 +59,8 @@ public KafkaMessageSendersFactory(KafkaParameters kafkaParameters, Duration retryInterval, int threadPoolSize, long bufferedSizeBytes, - Duration metadataMaxAge) { + Duration metadataMaxAge, + ScheduledExecutorService chaosScheduler) { this.topicMetadataLoadingExecutor = new TopicMetadataLoadingExecutor(topicsCache, retryCount, retryInterval, threadPoolSize); this.localMinInSyncReplicasLoader = new MinInSyncReplicasLoader(localAdminClient, metadataMaxAge); this.bufferedSizeBytes = bufferedSizeBytes; @@ -65,6 +68,7 @@ public KafkaMessageSendersFactory(KafkaParameters kafkaParameters, this.remoteKafkaParameters = remoteKafkaParameters; this.metricsFacade = metricsFacade; this.brokerLatencyReporter = brokerLatencyReporter; + this.chaosScheduler = chaosScheduler; } @@ -123,11 +127,13 @@ private KafkaMessageSender sender(KafkaParameters kafkaParameter props.put(SECURITY_PROTOCOL_CONFIG, kafkaParameters.getAuthenticationProtocol()); props.put(SASL_JAAS_CONFIG, kafkaParameters.getJaasConfig()); } + return new KafkaMessageSender<>( new org.apache.kafka.clients.producer.KafkaProducer<>(props), brokerLatencyReporter, metricsFacade, - kafkaParameters.getDatacenter() + kafkaParameters.getDatacenter(), + chaosScheduler ); } diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducer.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducer.java index 94b11793fe..2fc42890c4 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducer.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducer.java @@ -26,14 +26,9 @@ public void send(Message message, CachedTopic cachedTopic, final PublishingCallb ProducerRecord producerRecord = messageConverter.convertToProducerRecord(message, cachedTopic.getKafkaTopics().getPrimary().name()); - try { - var producer = kafkaMessageSenders.get(cachedTopic.getTopic()); - Callback wrappedCallback = new SendCallback(message, cachedTopic, callback, producer.getDatacenter()); - producer.send(producerRecord, cachedTopic, message, wrappedCallback); - } catch (Exception e) { - // message didn't get to internal producer buffer and it will not be send to a broker - callback.onUnpublished(message, cachedTopic.getTopic(), e); - } + var producer = kafkaMessageSenders.get(cachedTopic.getTopic()); + Callback wrappedCallback = new SendCallback(message, cachedTopic, callback, producer.getDatacenter()); + producer.send(producerRecord, cachedTopic, message, wrappedCallback); } @Override diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/MultiDatacenterMessageProducer.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/MultiDatacenterMessageProducer.java index b60cccd65d..349380023c 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/MultiDatacenterMessageProducer.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/MultiDatacenterMessageProducer.java @@ -18,7 +18,9 @@ import java.util.HashMap; import java.util.Map; import java.util.Optional; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Future; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ThreadLocalRandom; @@ -37,20 +39,17 @@ public class MultiDatacenterMessageProducer implements BrokerMessageProducer { private final Duration speculativeSendDelay; private final AdminReadinessService adminReadinessService; private final ScheduledExecutorService fallbackScheduler; - private final ScheduledExecutorService chaosScheduler; public MultiDatacenterMessageProducer(KafkaMessageSenders kafkaMessageSenders, AdminReadinessService adminReadinessService, MessageToKafkaProducerRecordConverter messageConverter, Duration speculativeSendDelay, - ScheduledExecutorService fallbackScheduler, - ScheduledExecutorService chaosScheduler) { + ScheduledExecutorService fallbackScheduler) { this.messageConverter = messageConverter; this.kafkaMessageSenders = kafkaMessageSenders; this.speculativeSendDelay = speculativeSendDelay; this.adminReadinessService = adminReadinessService; this.fallbackScheduler = fallbackScheduler; - this.chaosScheduler = chaosScheduler; } @Override @@ -60,86 +59,115 @@ public void send(Message message, CachedTopic cachedTopic, PublishingCallback ca KafkaMessageSender localSender = kafkaMessageSenders.get(cachedTopic.getTopic()); Optional> remoteSender = getRemoteSender(cachedTopic); - final SendCallback sendCallback = remoteSender.isPresent() - ? SendCallback.withFallback(callback) - : SendCallback.withoutFallback(callback); - Map experiments = createChaosExperimentsPerDatacenter(cachedTopic.getTopic()); - fallbackScheduler.schedule(() -> { - if (!sendCallback.sent.get() && remoteSender.isPresent()) { - sendOrScheduleChaosExperiment( - remoteSender.get(), - producerRecord, - sendCallback, - cachedTopic, - message, - experiments.getOrDefault(remoteSender.get().getDatacenter(), ChaosExperiment.DISABLED) - ); - } - }, speculativeSendDelay.toMillis(), TimeUnit.MILLISECONDS); + if (remoteSender.isPresent()) { + sendWithFallback( + localSender, + remoteSender.get(), + producerRecord, + cachedTopic, + message, + experiments, + callback + ); + } else { + sendWithoutFallback( + localSender, + producerRecord, + cachedTopic, + message, + callback + ); + } + } + + private static class SendWithFallbackExecutionContext { + + private final AtomicBoolean fallbackExecuted = new AtomicBoolean(false); + private final AtomicBoolean sent = new AtomicBoolean(false); + private final AtomicInteger tries; + private final ConcurrentHashMap errors; + + private SendWithFallbackExecutionContext() { + this.tries = new AtomicInteger(2); + this.errors = new ConcurrentHashMap<>(2); + } + + public boolean tryTransitionToFallback() { + return fallbackExecuted.compareAndSet(false, true) && !sent.get(); + } + + boolean tryTransitionToUnpublished(String datacenter, Exception exception) { + errors.put(datacenter, exception); + return tries.decrementAndGet() == 0; + } + + public boolean tryTransitionToFirstSent() { + return sent.compareAndSet(false, true); + } + } + + /* + We first try to send message to local DC. If the local send fails we perform 'immediate' fallback to remote DC. + + Additionally, we schedule a 'speculative' fallback to remote DC to execute after 'speculativeSendDelay' elapses. + Speculative fallback decreases publication latency but may result in messages being duplicated across DCs. - sendOrScheduleChaosExperiment( - localSender, + If local DC send succeeds or fails before 'speculativeSendDelay' elapses we try to cancel the 'speculative' fallback if + it has not been executed yet. We guarantee that only one fallback executes - either 'immediate' or 'speculative'. + */ + private void sendWithFallback(KafkaMessageSender localSender, + KafkaMessageSender remoteSender, + ProducerRecord producerRecord, + CachedTopic cachedTopic, + Message message, + Map experiments, + PublishingCallback publishingCallback) { + + SendWithFallbackExecutionContext context = new SendWithFallbackExecutionContext(); + + FallbackRunnable fallback = new FallbackRunnable( + remoteSender, producerRecord, - sendCallback, cachedTopic, message, - experiments.getOrDefault(localSender.getDatacenter(), ChaosExperiment.DISABLED) + experiments.getOrDefault(remoteSender.getDatacenter(), ChaosExperiment.DISABLED), + publishingCallback, + context ); - } - private void send(KafkaMessageSender sender, - ProducerRecord producerRecord, - SendCallback callback, - CachedTopic cachedTopic, - Message message) { - String datacenter = sender.getDatacenter(); + Future speculativeFallback; try { - sender.send(producerRecord, cachedTopic, message, new DCAwareCallback( - message, - cachedTopic, - datacenter, - callback)); - } catch (Exception e) { - // message didn't get to internal producer buffer and it will not be sent to a broker - callback.onUnpublished(message, cachedTopic, datacenter, e); + speculativeFallback = fallbackScheduler.schedule(fallback, speculativeSendDelay.toMillis(), TimeUnit.MILLISECONDS); + } catch (RejectedExecutionException rejectedExecutionException) { + logger.warn("Failed to run schedule fallback for message: {}, topic: {}", message, cachedTopic.getQualifiedName(), rejectedExecutionException); + speculativeFallback = CompletableFuture.completedFuture(null); } - } - private void sendOrScheduleChaosExperiment(KafkaMessageSender sender, - ProducerRecord producerRecord, - SendCallback callback, - CachedTopic cachedTopic, - Message message, - ChaosExperiment experiment) { - if (experiment.enabled()) { - scheduleChaosExperiment(experiment, sender, producerRecord, callback, cachedTopic, message); - } else { - send(sender, producerRecord, callback, cachedTopic, message); - } + localSender.send( + producerRecord, + cachedTopic, + message, + new FallbackAwareLocalSendCallback( + message, cachedTopic, localSender.getDatacenter(), + context, publishingCallback, fallback, speculativeFallback + ), + experiments.getOrDefault(localSender.getDatacenter(), ChaosExperiment.DISABLED) + ); } - private void scheduleChaosExperiment(ChaosExperiment experiment, - KafkaMessageSender sender, - ProducerRecord producerRecord, - SendCallback callback, - CachedTopic cachedTopic, - Message message) { - try { - chaosScheduler.schedule(() -> { - if (experiment.completeWithError()) { - var datacenter = sender.getDatacenter(); - var exception = new ChaosException(datacenter, experiment.delayInMillis(), message.getId()); - callback.onUnpublished(message, cachedTopic, datacenter, exception); - } else { - send(sender, producerRecord, callback, cachedTopic, message); - } - }, experiment.delayInMillis(), TimeUnit.MILLISECONDS); - } catch (RejectedExecutionException e) { - logger.warn("Failed while scheduling chaos experiment. Sending message to Kafka.", e); - send(sender, producerRecord, callback, cachedTopic, message); - } + private void sendWithoutFallback(KafkaMessageSender sender, + ProducerRecord producerRecord, + CachedTopic cachedTopic, + Message message, + PublishingCallback callback) { + sender.send( + producerRecord, + cachedTopic, + message, + new LocalSendCallback(message, cachedTopic, sender.getDatacenter(), callback) + ); } private Map createChaosExperimentsPerDatacenter(Topic topic) { @@ -185,7 +213,7 @@ private ChaosExperiment createChaosExperimentForDatacenter(ChaosPolicy policy, b return new ChaosExperiment(true, policy.completeWithError(), delayMillis); } - private record ChaosExperiment(boolean enabled, boolean completeWithError, long delayInMillis) { + public record ChaosExperiment(boolean enabled, boolean completeWithError, long delayInMillis) { private static final ChaosExperiment DISABLED = new ChaosExperiment(false, false, 0); @@ -198,53 +226,132 @@ private Optional> getRemoteSender(CachedTopic .findFirst(); } - private record DCAwareCallback(Message message, CachedTopic cachedTopic, String datacenter, - SendCallback callback) implements Callback { + private record RemoteSendCallback(Message message, CachedTopic cachedTopic, + String datacenter, PublishingCallback callback, + SendWithFallbackExecutionContext state) implements Callback { @Override public void onCompletion(RecordMetadata metadata, Exception exception) { if (exception == null) { - callback.onPublished(message, cachedTopic, datacenter); + callback.onEachPublished(message, cachedTopic.getTopic(), datacenter); + if (state.tryTransitionToFirstSent()) { + callback.onPublished(message, cachedTopic.getTopic()); + } else { + cachedTopic.markMessageDuplicated(); + } } else { - callback.onUnpublished(message, cachedTopic, datacenter, exception); + if (state.tryTransitionToUnpublished(datacenter, exception)) { + callback.onUnpublished(message, cachedTopic.getTopic(), new MultiDCPublishException(state.errors)); + } } } } - private static class SendCallback { + private class FallbackAwareLocalSendCallback implements Callback { + private final Message message; + private final CachedTopic cachedTopic; + private final String datacenter; private final PublishingCallback callback; - private final AtomicBoolean sent = new AtomicBoolean(false); - private final AtomicInteger tries; - private final ConcurrentHashMap errors; - - private SendCallback(PublishingCallback callback, int tries) { + private final FallbackRunnable fallback; + private final Future speculativeFallback; + private final SendWithFallbackExecutionContext state; + + private FallbackAwareLocalSendCallback(Message message, CachedTopic cachedTopic, String datacenter, + SendWithFallbackExecutionContext state, + PublishingCallback callback, + FallbackRunnable fallback, Future speculativeFallback) { + this.message = message; + this.cachedTopic = cachedTopic; + this.datacenter = datacenter; this.callback = callback; - this.tries = new AtomicInteger(tries); - this.errors = new ConcurrentHashMap<>(tries); + this.fallback = fallback; + this.speculativeFallback = speculativeFallback; + this.state = state; } - static SendCallback withFallback(PublishingCallback callback) { - return new SendCallback(callback, 2); + @Override + public void onCompletion(RecordMetadata metadata, Exception exception) { + if (exception == null) { + cancelSpeculativeFallback(); + callback.onEachPublished(message, cachedTopic.getTopic(), datacenter); + if (state.tryTransitionToFirstSent()) { + callback.onPublished(message, cachedTopic.getTopic()); + } else { + cachedTopic.markMessageDuplicated(); + } + } else { + if (state.tryTransitionToUnpublished(datacenter, exception)) { + callback.onUnpublished(message, cachedTopic.getTopic(), new MultiDCPublishException(state.errors)); + } else { + fallback(); + } + } } - static SendCallback withoutFallback(PublishingCallback callback) { - return new SendCallback(callback, 1); + private void fallback() { + try { + speculativeFallback.cancel(false); + fallbackScheduler.execute(fallback); + } catch (RejectedExecutionException rejectedExecutionException) { + logger.warn("Failed to run immediate fallback for message: {}, topic: {}", message, cachedTopic.getQualifiedName(), rejectedExecutionException); + } } - private void onUnpublished(Message message, CachedTopic cachedTopic, String datacenter, Exception exception) { - errors.put(datacenter, exception); - if (tries.decrementAndGet() == 0) { - callback.onUnpublished(message, cachedTopic.getTopic(), new MultiDCPublishException(errors)); - } + private void cancelSpeculativeFallback() { + speculativeFallback.cancel(false); } + } - private void onPublished(Message message, CachedTopic cachedTopic, String datacenter) { - callback.onEachPublished(message, cachedTopic.getTopic(), datacenter); - if (sent.compareAndSet(false, true)) { - callback.onPublished(message, cachedTopic.getTopic()); + private record LocalSendCallback(Message message, CachedTopic cachedTopic, String datacenter, + PublishingCallback callback) implements Callback { + + @Override + public void onCompletion(RecordMetadata metadata, Exception exception) { + if (exception != null) { + callback.onUnpublished(message, cachedTopic.getTopic(), exception); } else { - cachedTopic.markMessageDuplicated(); + callback.onEachPublished(message, cachedTopic.getTopic(), datacenter); + callback.onPublished(message, cachedTopic.getTopic()); + } + } + } + + private class FallbackRunnable implements Runnable { + private final KafkaMessageSender remoteSender; + private final ProducerRecord producerRecord; + private final CachedTopic cachedTopic; + private final Message message; + private final ChaosExperiment experiment; + private final PublishingCallback callback; + private final SendWithFallbackExecutionContext context; + + public FallbackRunnable(KafkaMessageSender remoteSender, + ProducerRecord producerRecord, + CachedTopic cachedTopic, + Message message, + ChaosExperiment experiment, + PublishingCallback callback, + SendWithFallbackExecutionContext context + ) { + this.remoteSender = remoteSender; + this.producerRecord = producerRecord; + this.cachedTopic = cachedTopic; + this.message = message; + this.experiment = experiment; + this.callback = callback; + this.context = context; + } + + public void run() { + if (context.tryTransitionToFallback()) { + remoteSender.send( + producerRecord, + cachedTopic, + message, + new RemoteSendCallback(message, cachedTopic, remoteSender.getDatacenter(), callback, context), + experiment + ); } } } diff --git a/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducerIntegrationTest.groovy b/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducerIntegrationTest.groovy index 0a3abe9e4b..088ac3a204 100644 --- a/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducerIntegrationTest.groovy +++ b/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducerIntegrationTest.groovy @@ -31,6 +31,8 @@ import spock.lang.Shared import spock.lang.Specification import java.time.Duration +import java.util.concurrent.Executors +import java.util.concurrent.ScheduledExecutorService import java.util.stream.Collectors import static java.util.Collections.emptyList @@ -81,6 +83,9 @@ class LocalDatacenterMessageProducerIntegrationTest extends Specification { @Shared String datacenter = "dc" + @Shared + ScheduledExecutorService chaosScheduler = Executors.newSingleThreadScheduledExecutor(); + @Shared MetricsFacade metricsFacade = new MetricsFacade(new SimpleMeterRegistry()) @@ -105,8 +110,8 @@ class LocalDatacenterMessageProducerIntegrationTest extends Specification { topicMetadataLoadingExecutor, minInSyncReplicasLoader, new KafkaMessageSenders.Tuple( - new KafkaMessageSender(leaderConfirms, brokerLatencyReporter, metricsFacade, datacenter), - new KafkaMessageSender(everyoneConfirms, brokerLatencyReporter, metricsFacade, datacenter) + new KafkaMessageSender(leaderConfirms, brokerLatencyReporter, metricsFacade, datacenter, chaosScheduler), + new KafkaMessageSender(everyoneConfirms, brokerLatencyReporter, metricsFacade, datacenter, chaosScheduler) ), emptyList() ) diff --git a/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/producer/kafka/MultiDatacenterMessageProducerTest.groovy b/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/producer/kafka/MultiDatacenterMessageProducerTest.groovy new file mode 100644 index 0000000000..596105bc7a --- /dev/null +++ b/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/producer/kafka/MultiDatacenterMessageProducerTest.groovy @@ -0,0 +1,279 @@ +package pl.allegro.tech.hermes.frontend.producer.kafka + +import org.apache.kafka.clients.producer.Callback +import org.apache.kafka.clients.producer.ProducerRecord +import pl.allegro.tech.hermes.api.Topic +import pl.allegro.tech.hermes.frontend.metric.CachedTopic +import pl.allegro.tech.hermes.frontend.publishing.PublishingCallback +import pl.allegro.tech.hermes.frontend.publishing.message.JsonMessage +import pl.allegro.tech.hermes.frontend.publishing.message.Message +import pl.allegro.tech.hermes.frontend.readiness.AdminReadinessService +import pl.allegro.tech.hermes.frontend.server.CachedTopicsTestHelper +import pl.allegro.tech.hermes.test.helper.builder.TopicBuilder +import spock.lang.Specification +import spock.util.concurrent.PollingConditions + +import java.time.Duration +import java.util.concurrent.ExecutorService +import java.util.concurrent.Executors +import java.util.concurrent.ScheduledExecutorService +import java.util.concurrent.atomic.AtomicInteger + +import static java.util.Collections.emptyMap + +class MultiDatacenterMessageProducerTest extends Specification { + ScheduledExecutorService fallbackScheduler = Executors.newSingleThreadScheduledExecutor(); + ExecutorService testExecutor = Executors.newSingleThreadExecutor(); + + + Duration speculativeSendDelay = Duration.ofMillis(250); + Message message = new JsonMessage('id', [2] as byte[], 0L, null, emptyMap()); + + private static final String localDC = "LOCAL" + private static final String remoteDC = "REMOTE" + + class VerificationCallback implements PublishingCallback { + AtomicInteger onUnpublished = new AtomicInteger(0) + AtomicInteger onPublished = new AtomicInteger(0) + AtomicInteger publishedToLocal = new AtomicInteger(0) + AtomicInteger publishedToRemote = new AtomicInteger(0) + volatile String exception = "" + + @Override + void onUnpublished(Message message, Topic topic, Exception exception) { + onUnpublished.incrementAndGet(); + this.exception = exception; + } + + @Override + void onPublished(Message message, Topic topic) { + onPublished.incrementAndGet(); + } + + @Override + void onEachPublished(Message message, Topic topic, String datacenter) { + if (datacenter == localDC) { + publishedToLocal.incrementAndGet() + } else if (datacenter == remoteDC) { + publishedToRemote.incrementAndGet() + } + } + } + + def "should send to local DC only if the send is successful before speculativeSendDelay elapses"() { + given: + CachedTopic cachedTopic = CachedTopicsTestHelper.cachedTopic(TopicBuilder.topicWithRandomName().build()) + + def localSender = successfulSender(localDC) + def remoteSender = successfulSender(remoteDC) + + def producer = producer(localSender, remoteSender) + + PublishingCallback callback = new VerificationCallback(); + + when: + testExecutor.execute { producer.send(message, cachedTopic, callback) } + + then: + // wait until speculative send delay elapses + Thread.sleep(speculativeSendDelay.plusMillis(200).toMillis()) + callback.publishedToLocal.get() == 1 + callback.publishedToRemote.get() == 0 + callback.onPublished.get() == 1 + callback.onUnpublished.get() == 0 + + } + + def "should send to remote DC if local DC fails"() { + given: + CachedTopic cachedTopic = CachedTopicsTestHelper.cachedTopic(TopicBuilder.topicWithRandomName().build()) + + def localSender = failingSender(localDC) + def remoteSender = successfulSender(remoteDC) + + def producer = producer(localSender, remoteSender) + + + PublishingCallback callback = new VerificationCallback(); + + when: + testExecutor.execute { producer.send(message, cachedTopic, callback) } + + then: + new PollingConditions(timeout: 10).eventually { + callback.publishedToLocal.get() == 0 + callback.publishedToRemote.get() == 1 + callback.onPublished.get() == 1 + callback.onUnpublished.get() == 0 + } + } + + def "should send to remote DC when local does not respond before speculativeSendDelay elapses"() { + given: + CachedTopic cachedTopic = CachedTopicsTestHelper.cachedTopic(TopicBuilder.topicWithRandomName().build()) + + def localSender = stuckLocalSender() + def remoteSender = successfulSender(remoteDC) + + def producer = producer(localSender, remoteSender) + + PublishingCallback callback = new VerificationCallback(); + + when: + def start = System.currentTimeMillis() + testExecutor.execute { producer.send(message, cachedTopic, callback) } + + then: + new PollingConditions(timeout: 10).eventually { + callback.publishedToLocal.get() == 0 + callback.publishedToRemote.get() == 1 + callback.onPublished.get() == 1 + callback.onUnpublished.get() == 0 + System.currentTimeMillis() - start > speculativeSendDelay.toMillis() + } + } + + def "should publish to local DC and remote DC when local send is slower than speculativeSendDelay but it eventually succeeds"() { + given: + CachedTopic cachedTopic = CachedTopicsTestHelper.cachedTopic(TopicBuilder.topicWithRandomName().build()) + + def localSender = delayedSender(localDC, speculativeSendDelay.plusMillis(100).toMillis()) + def remoteSender = successfulSender(remoteDC) + + def producer = producer(localSender, remoteSender) + + PublishingCallback callback = new VerificationCallback(); + + when: + testExecutor.execute { producer.send(message, cachedTopic, callback) } + + then: + new PollingConditions(timeout: 10).eventually { + callback.publishedToLocal.get() == 1 + callback.publishedToRemote.get() == 1 + callback.onPublished.get() == 1 + callback.onUnpublished.get() == 0 + } + } + + def "should invoke onUnpublished when both DCs fail"() { + given: + CachedTopic cachedTopic = CachedTopicsTestHelper.cachedTopic(TopicBuilder.topicWithRandomName().build()) + + def localSender = failingSender(localDC, "network error") + def remoteSender = failingSender(remoteDC, "not leader or follower") + + def producer = producer(localSender, remoteSender) + + + PublishingCallback callback = new VerificationCallback(); + + when: + testExecutor.execute { producer.send(message, cachedTopic, callback) } + + then: + new PollingConditions(timeout: 10).eventually { + callback.publishedToLocal.get() == 0 + callback.publishedToRemote.get() == 0 + callback.onPublished.get() == 0 + callback.onUnpublished.get() == 1 + callback.exception.contains("[LOCAL]: RuntimeException: network error") + callback.exception.contains("[REMOTE]: RuntimeException: not leader or follower") + } + } + + def "should publish to remote DC once when both scheduled fallback (after speculativeSendDelay) and immediate fallback are run"() { + given: + CachedTopic cachedTopic = CachedTopicsTestHelper.cachedTopic(TopicBuilder.topicWithRandomName().build()) + + def localSender = delayedFailingSender(localDC, speculativeSendDelay.toMillis() + 5) + def remoteSender = successfulSender(remoteDC) + + def producer = producer(localSender, remoteSender) + + PublishingCallback callback = new VerificationCallback() + + when: + testExecutor.execute { producer.send(message, cachedTopic, callback) } + + then: + Thread.sleep(speculativeSendDelay.toMillis() + 100) + callback.publishedToLocal.get() == 0 + callback.publishedToRemote.get() == 1 + callback.onPublished.get() == 1 + callback.onUnpublished.get() == 0 + } + + + KafkaMessageSender stuckLocalSender() { + return Mock(KafkaMessageSender) { + getDatacenter() >> localDC + } + } + + KafkaMessageSender delayedSender(String sender, long delayMillis) { + return Mock(KafkaMessageSender) { + getDatacenter() >> sender + send(*_) >> (arguments) -> { + Thread.sleep(delayMillis) + callback(arguments).onCompletion(null, null) + } + } + } + + KafkaMessageSender delayedFailingSender(String sender, long delayMillis, String exceptionMsg = "fail") { + return Mock(KafkaMessageSender) { + getDatacenter() >> sender + send(*_) >> (arguments) -> { + Thread.sleep(delayMillis) + callback(arguments).onCompletion(null, new RuntimeException(exceptionMsg)) + } + } + } + + KafkaMessageSender failingSender(String sender, String exceptionMsg = "fail") { + return Mock(KafkaMessageSender) { + getDatacenter() >> sender + send(*_) >> { + arguments -> callback(arguments).onCompletion(null, new RuntimeException(exceptionMsg)) + } + } + } + + KafkaMessageSender successfulSender(String sender) { + return Mock(KafkaMessageSender) { + getDatacenter() >> sender + send(*_) >> { + arguments -> callback(arguments).onCompletion(null, null) + } + } + } + + // callback argument from pl.allegro.tech.hermes.frontend.producer.kafka.KafkaMessageSender#send + Callback callback(arguments) { + return (arguments[3] as Callback) + } + + MultiDatacenterMessageProducer producer(KafkaMessageSender localSender, KafkaMessageSender remoteSender) { + MessageToKafkaProducerRecordConverter messageConverter = Mock(MessageToKafkaProducerRecordConverter) { + convertToProducerRecord(*_) >> new ProducerRecord("topic", new byte[]{0x0}, new byte[]{0x0}) + } + + def adminReadinessService = Mock(AdminReadinessService) { + isDatacenterReady(_) >> true + } + + KafkaMessageSenders senders = new KafkaMessageSenders( + Mock(TopicMetadataLoadingExecutor), + Mock(MinInSyncReplicasLoader), + new KafkaMessageSenders.Tuple( + localSender, localSender + ), + [new KafkaMessageSenders.Tuple(remoteSender, remoteSender)] + ) + + return new MultiDatacenterMessageProducer( + senders, adminReadinessService, messageConverter, speculativeSendDelay, fallbackScheduler + ) + } +} diff --git a/hermes-frontend/src/test/java/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducerTest.java b/hermes-frontend/src/test/java/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducerTest.java index febb9c49ad..67bbfcd162 100644 --- a/hermes-frontend/src/test/java/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducerTest.java +++ b/hermes-frontend/src/test/java/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducerTest.java @@ -30,6 +30,7 @@ import java.time.Duration; import java.util.List; import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.atomic.AtomicBoolean; import static com.google.common.base.Charsets.UTF_8; @@ -55,10 +56,12 @@ public class LocalDatacenterMessageProducerTest { private final BrokerLatencyReporter brokerLatencyReporter = new BrokerLatencyReporter(false, metricsFacade, Duration.ZERO, Executors.newSingleThreadExecutor()); + private final ScheduledExecutorService chaosScheduler = Executors.newSingleThreadScheduledExecutor(); + private final MockProducer leaderConfirmsProducer = new MockProducer<>(true, serializer, serializer); private final MockProducer everyoneConfirmProducer = new MockProducer<>(true, serializer, serializer); - private final KafkaMessageSender leaderConfirmsProduceWrapper = new KafkaMessageSender<>(leaderConfirmsProducer, brokerLatencyReporter, metricsFacade, datacenter); - private final KafkaMessageSender everyoneConfirmsProduceWrapper = new KafkaMessageSender<>(everyoneConfirmProducer, brokerLatencyReporter, metricsFacade, datacenter); + private final KafkaMessageSender leaderConfirmsProduceWrapper = new KafkaMessageSender<>(leaderConfirmsProducer, brokerLatencyReporter, metricsFacade, datacenter, chaosScheduler); + private final KafkaMessageSender everyoneConfirmsProduceWrapper = new KafkaMessageSender<>(everyoneConfirmProducer, brokerLatencyReporter, metricsFacade, datacenter, chaosScheduler); private final KafkaHeaderNameProperties kafkaHeaderNameProperties = new KafkaHeaderNameProperties(); private final HTTPHeadersPropagationAsKafkaHeadersProperties httpHeadersPropagationAsKafkaHeadersProperties = From 0c7e112fc2e3b582614a852df30c4b0e878f90f3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20F=C4=85derski?= Date: Mon, 3 Jun 2024 13:35:36 +0200 Subject: [PATCH 58/87] Update dependencies (#1848) Update hermes dependencies --- build.gradle | 40 +++--- hermes-api/build.gradle | 10 +- hermes-benchmark/build.gradle | 9 +- hermes-client/build.gradle | 10 +- ...esClientMicrometerTaggedMetricsTest.groovy | 5 +- hermes-common/build.gradle | 16 +-- .../message/converter/AvroBinaryDecoders.java | 2 +- .../filtering/avro/AvroPathPredicate.java | 24 +++- .../zookeeper/ZookeeperBasedRepository.java | 2 +- .../zookeeper/ZookeeperPaths.java | 2 +- .../cache/HierarchicalCacheLevel.java | 2 +- .../ZookeeperInternalNotificationBus.java | 2 +- ...okeeperWorkloadConstraintsCacheTest.groovy | 3 +- ...erWorkloadConstraintsRepositoryTest.groovy | 4 +- .../cache/HierarchicalCacheTest.groovy | 20 +-- .../message/wrapper/SchemaAwareSerDeTest.java | 17 +-- .../ZookeeperCounterReporterTest.java | 4 +- .../ZookeeperCounterStorageTest.java | 6 +- hermes-consumers/build.gradle | 29 ++-- .../hermes/consumers/consumer/Message.java | 2 +- .../oauth/client/OAuthHttpClient.java | 5 +- .../consumers/consumer/rate/SendCounters.java | 5 +- .../googlepubsub/CompressionCodecFactory.java | 2 +- .../googlepubsub/MessageCompressor.java | 9 +- .../sender/http/ByteBufferEntity.java | 131 ------------------ .../sender/http/JettyMessageBatchSender.java | 6 +- .../sender/http/auth/BasicAuthProvider.java | 4 +- .../registry/ConsumerNodesRegistry.java | 4 +- .../tech/hermes/consumers/uri/UriUtils.java | 2 +- .../ConsumerProcessSupervisorTest.groovy | 10 +- .../consumer/ConsumerMessageSenderTest.java | 38 ++--- .../AvroToJsonMessageConverterTest.java | 2 +- .../calculator/OutputRateCalculatorTest.java | 2 +- .../rate/maxrate/MaxRateRegistryTest.java | 2 +- .../sender/MessageSenderFactoryTest.java | 9 +- .../sender/http/ByteBufferEntityTest.java | 104 -------------- .../sender/jms/JmsMessageSenderTest.java | 2 +- ...erpolatingEndpointAddressResolverTest.java | 18 ++- .../ConsumerTestRuntimeEnvironment.java | 61 ++++---- .../WorkloadSupervisorIntegrationTest.java | 13 +- hermes-frontend/build.gradle | 19 +-- .../message/MessageContentTypeEnforcer.java | 2 +- .../preview/MessagePreviewFactory.java | 2 +- ...enterMessageProducerIntegrationTest.groovy | 3 +- .../metric/ThroughputRegistryTest.groovy | 14 +- .../buffer/BackupMessagesLoaderTest.java | 4 +- .../LocalDatacenterMessageProducerTest.java | 4 +- hermes-management/build.gradle | 16 +-- .../api/auth/AllowAllSecurityProvider.java | 2 +- .../domain/filtering/FilteringService.java | 2 +- .../graphite/RestTemplateGraphiteClient.java | 0 .../HybridSubscriptionMetricsRepository.java | 3 +- hermes-metrics/build.gradle | 2 +- .../tech/hermes/metrics/PathsCompiler.java | 5 +- hermes-mock/build.gradle | 8 +- .../tech/hermes/mock/HermesMockExpect.java | 6 +- hermes-test-helper/build.gradle | 15 +- .../client/integration/HermesInitHelper.java | 14 +- .../client/integration/HermesTestClient.java | 40 +++--- .../endpoint/RemoteServiceEndpoint.java | 27 ++-- .../test/helper/endpoint/TimeoutAdjuster.java | 6 +- .../helper/oauth/server/OAuthTestServer.java | 2 +- .../tech/hermes/test/helper/retry/Retry.java | 28 ---- .../test/helper/retry/RetryListener.java | 18 --- .../helper/zookeeper/ZookeeperWaiter.java | 3 +- hermes-tracker-elasticsearch/build.gradle | 9 +- .../MultiElasticsearchLogRepository.java | 2 +- .../DailyIndexFactoryTest.groovy | 3 +- ...nsumersElasticsearchLogRepositoryTest.java | 22 +-- ...rontendElasticsearchLogRepositoryTest.java | 20 +-- .../ElasticsearchLogRepositoryTest.java | 37 +++-- .../MultiElasticsearchLogRepositoryTest.java | 34 ++--- .../consumers/AbstractLogRepositoryTest.java | 22 +-- .../frontend/AbstractLogRepositoryTest.java | 44 ++---- integration-tests/build.gradle | 3 +- .../setup/HermesExtension.java | 8 +- .../setup/HermesFrontendTestApp.java | 2 +- .../setup/HermesManagementTestApp.java | 4 +- .../subscriber/TestJmsSubscriber.java | 15 +- .../subscriber/TestSubscriber.java | 20 +-- .../management/TestSecurityProvider.java | 2 +- .../integrationtests/BatchDeliveryTest.java | 4 +- .../BatchRetryPolicyTest.java | 10 +- .../BroadcastDeliveryTest.java | 9 +- .../ConsumerProfilingTest.java | 16 +-- .../KafkaProducerMetricsTest.java | 4 +- .../KafkaRetransmissionServiceTest.java | 6 +- .../hermes/integrationtests/MetricsTest.java | 10 +- .../OAuthIntegrationTest.java | 4 +- .../PublishingAndConsumingTest.java | 5 +- .../PublishingAuthenticationTest.java | 14 +- .../integrationtests/PublishingAvroTest.java | 12 +- .../integrationtests/PublishingTest.java | 2 +- .../integrationtests/ReadinessCheckTest.java | 9 +- .../TopicAuthorizationTest.java | 16 +-- .../integrationtests/TopicBlacklistTest.java | 9 +- .../integrationtests/UndeliveredLogTest.java | 7 +- .../management/GroupManagementTest.java | 12 +- .../management/HealthCheckTest.java | 4 +- .../MessagePreviewIntegrationTest.java | 6 +- .../management/QueryEndpointTest.java | 10 +- .../SubscriptionManagementTest.java | 14 +- .../management/TopicManagementTest.java | 8 +- .../BrokerLatencyReportingTest.java | 7 +- .../HermesClientPublishingHttpsTest.java | 4 +- .../KafkaReadinessCheckTest.java | 8 +- .../MessageBufferLoadingTest.java | 4 +- .../TopicCreationRollbackTest.java | 6 +- 108 files changed, 524 insertions(+), 826 deletions(-) delete mode 100644 hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/sender/http/ByteBufferEntity.java delete mode 100644 hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/sender/http/ByteBufferEntityTest.java create mode 100644 hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/graphite/RestTemplateGraphiteClient.java delete mode 100644 hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/retry/Retry.java delete mode 100644 hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/retry/RetryListener.java diff --git a/build.gradle b/build.gradle index 5f99480281..0b15a7e9bd 100644 --- a/build.gradle +++ b/build.gradle @@ -49,26 +49,27 @@ allprojects { project.ext.versions = [ kafka : '3.6.2', - guava : '23.0', - jackson : '2.15.2', - jersey : '3.1.2', - jetty : '12.0.7', + guava : '33.1.0-jre', + jackson : '2.17.0', + jersey : '3.1.6', + jetty : '12.0.8', curator : '5.4.0', - dropwizard_metrics: '4.1.0', - micrometer_metrics: '1.11.1', - wiremock : '3.3.1', - spock : '2.4-M1-groovy-4.0', - groovy : '4.0.12', - avro : '1.9.1', + dropwizard_metrics: '4.2.25', + micrometer_metrics: '1.12.5', + wiremock : '3.5.2', + spock : '2.4-M4-groovy-4.0', + groovy : '4.0.21', + avro : '1.11.3', json2avro : '0.2.14', + // TODO: newest version requires subject alternative name in a certificate during host verification, current test cert does not have a one okhttp : '3.9.1', - undertow : '2.0.29.Final', - spring_web : '6.1.2', - failsafe : '2.3.1', - junit_jupiter : '5.9.1', + undertow : '2.3.12.Final', + spring_web : '6.1.6', + failsafe : '2.4.4', + junit_jupiter : '5.10.2', testcontainers : '1.19.8', - spring : '3.2.1', - assertj : '3.24.2' + spring : '3.2.4', + assertj : '3.25.3' ] repositories { @@ -77,15 +78,14 @@ allprojects { dependencies { implementation group: 'org.slf4j', name: 'slf4j-api', version: '2.0.4' - implementation group: 'org.apache.commons', name: 'commons-lang3', version: '3.9' + implementation group: 'org.apache.commons', name: 'commons-lang3', version: '3.14.0' testImplementation group: 'junit', name: 'junit', version: '4.11' testImplementation group: 'com.tngtech.java', name: 'junit-dataprovider', version: '1.10.0' testImplementation group: 'pl.pragmatists', name: 'JUnitParams', version: '1.0.2' - testImplementation group: 'org.mockito', name: 'mockito-all', version: '1.9.5' + testImplementation group: 'org.mockito', name: 'mockito-core', version: '5.11.0' testImplementation group: 'org.assertj', name: 'assertj-core', version: versions.assertj - testImplementation group: 'com.jayway.awaitility', name: 'awaitility', version: '1.6.1' - testImplementation group: 'com.googlecode.catch-exception', name: 'catch-exception', version: '1.2.0' + testImplementation group: 'org.awaitility', name: 'awaitility', version: '4.2.1' annotationProcessor group: 'org.springframework.boot', name: 'spring-boot-configuration-processor', version: versions.spring } diff --git a/hermes-api/build.gradle b/hermes-api/build.gradle index c6554f612b..54c06d0400 100644 --- a/hermes-api/build.gradle +++ b/hermes-api/build.gradle @@ -4,19 +4,19 @@ plugins { } dependencies { - api group: 'org.hibernate.validator', name: 'hibernate-validator', version: '8.0.0.Final' + api group: 'org.hibernate.validator', name: 'hibernate-validator', version: '8.0.1.Final' api group: 'jakarta.ws.rs', name: 'jakarta.ws.rs-api', version: '3.1.0' implementation group: 'com.fasterxml.jackson.core', name: 'jackson-annotations', version: versions.jackson api group: 'com.fasterxml.jackson.jakarta.rs', name: 'jackson-jakarta-rs-json-provider', version: versions.jackson api group: 'com.fasterxml.jackson.datatype', name: 'jackson-datatype-jsr310', version: versions.jackson implementation group: 'com.google.guava', name: 'guava', version: versions.guava - api group: 'com.damnhandy', name: 'handy-uri-templates', version: '2.0.2' + api group: 'com.damnhandy', name: 'handy-uri-templates', version: '2.1.8' api group: 'jakarta.xml.bind', name: 'jakarta.xml.bind-api', version: '4.0.0' - implementation group: 'com.sun.xml.bind', name: 'jaxb-core', version: '4.0.3' - implementation group: 'com.sun.xml.bind', name: 'jaxb-impl', version: '4.0.3' - implementation group: 'jakarta.annotation', name: 'jakarta.annotation-api', version: '2.1.1' + implementation group: 'com.sun.xml.bind', name: 'jaxb-core', version: '4.0.5' + implementation group: 'com.sun.xml.bind', name: 'jaxb-impl', version: '4.0.5' + implementation group: 'jakarta.annotation', name: 'jakarta.annotation-api', version: '3.0.0' testImplementation group: 'org.spockframework', name: 'spock-core', version: versions.spock diff --git a/hermes-benchmark/build.gradle b/hermes-benchmark/build.gradle index abd76e7967..6262b38a8a 100644 --- a/hermes-benchmark/build.gradle +++ b/hermes-benchmark/build.gradle @@ -1,5 +1,5 @@ plugins { - id 'me.champeau.jmh' version '0.6.8' + id 'me.champeau.jmh' version '0.7.2' } configurations { @@ -26,10 +26,9 @@ jmh { } dependencies { - jmh group: 'org.openjdk.jmh', name: 'jmh-core', version: '1.12' - jmh group: 'org.openjdk.jmh', name: 'jmh-generator-annprocess', version: '1.12' - jmh group: 'org.apache.httpcomponents', name: 'httpasyncclient', version: '4.1.1' - jmh group: 'org.spf4j', name: 'spf4j-jmh', version: '8.0.3' + jmh group: 'org.openjdk.jmh', name: 'jmh-core', version: '1.37' + jmh group: 'org.openjdk.jmh', name: 'jmh-generator-annprocess', version: '1.37' + jmh group: 'org.apache.httpcomponents', name: 'httpasyncclient', version: '4.1.5' jmh project(':hermes-frontend') jmh project(':hermes-test-helper') jmh project(':hermes-common') diff --git a/hermes-client/build.gradle b/hermes-client/build.gradle index 77f46f6722..8e2fa2c039 100644 --- a/hermes-client/build.gradle +++ b/hermes-client/build.gradle @@ -11,13 +11,13 @@ dependencies { compileOnly group: 'com.squareup.okhttp3', name: 'okhttp', version: versions.okhttp implementation group: 'net.jodah', name: 'failsafe', version: versions.failsafe - api group: 'io.projectreactor', name: 'reactor-core', version: '3.4.25' + api group: 'io.projectreactor', name: 'reactor-core', version: '3.6.5' testImplementation group: 'org.spockframework', name: 'spock-core', version: versions.spock testImplementation group: 'org.spockframework', name: 'spock-junit4', version: versions.spock testImplementation group: 'org.wiremock', name: 'wiremock-standalone', version: versions.wiremock - testImplementation group: 'javax.servlet', name: 'javax.servlet-api', version: '3.1.0' - testImplementation group: 'com.jayway.jsonpath', name: 'json-path', version: '2.5.0' + testImplementation group: 'jakarta.servlet', name: 'jakarta.servlet-api', version: '6.0.0' + testImplementation group: 'com.jayway.jsonpath', name: 'json-path', version: '2.9.0' testImplementation group: 'io.micrometer', name: 'micrometer-core', version: versions.micrometer_metrics testImplementation group: 'org.glassfish.jersey.core', name: 'jersey-client', version: versions.jersey @@ -26,6 +26,6 @@ dependencies { testImplementation group: 'org.springframework', name: 'spring-context', version: versions.spring_web testImplementation group: 'org.springframework', name: 'spring-webflux', version: versions.spring_web testImplementation group: 'com.squareup.okhttp3', name: 'okhttp', version: versions.okhttp - testImplementation group: 'io.projectreactor.netty', name: 'reactor-netty', version: '1.0.25' - testImplementation group: 'io.projectreactor', name: 'reactor-test', version: '3.4.25' + testImplementation group: 'io.projectreactor.netty', name: 'reactor-netty', version: '1.1.18' + testImplementation group: 'io.projectreactor', name: 'reactor-test', version: '3.6.5' } diff --git a/hermes-client/src/test/groovy/pl/allegro/tech/hermes/client/HermesClientMicrometerTaggedMetricsTest.groovy b/hermes-client/src/test/groovy/pl/allegro/tech/hermes/client/HermesClientMicrometerTaggedMetricsTest.groovy index 117c63054a..05b760eea9 100644 --- a/hermes-client/src/test/groovy/pl/allegro/tech/hermes/client/HermesClientMicrometerTaggedMetricsTest.groovy +++ b/hermes-client/src/test/groovy/pl/allegro/tech/hermes/client/HermesClientMicrometerTaggedMetricsTest.groovy @@ -7,7 +7,6 @@ import pl.allegro.tech.hermes.client.metrics.MicrometerTaggedMetricsProvider import spock.lang.Specification import java.time.Duration -import java.time.temporal.ChronoUnit import java.util.concurrent.CompletableFuture import java.util.concurrent.TimeUnit @@ -31,8 +30,8 @@ class HermesClientMicrometerTaggedMetricsTest extends Specification { then: metrics.counter("hermes-client.status", "code", String.valueOf(201), "topic", "com_group.topic").count() == 1 def timer = metrics.timer("hermes-client.latency", "topic", "com_group.topic") - timer.totalTime(TimeUnit.NANOSECONDS) >= Duration.ofMillis(100).get(ChronoUnit.NANOS) - timer.totalTime(TimeUnit.NANOSECONDS) < Duration.ofMillis(500).get(ChronoUnit.NANOS) + timer.totalTime(TimeUnit.NANOSECONDS) >= Duration.ofMillis(100).toNanos() + timer.totalTime(TimeUnit.NANOSECONDS) < Duration.ofMillis(1000).toNanos() } def "should close timer on exceptional completion and log failure metric"() { diff --git a/hermes-common/build.gradle b/hermes-common/build.gradle index 85b4ff6ee3..d1e850d4d6 100644 --- a/hermes-common/build.gradle +++ b/hermes-common/build.gradle @@ -25,21 +25,21 @@ dependencies { api group: 'tech.allegro.schema.json2avro', name: 'converter', version: versions.json2avro api group: 'org.apache.commons', name: 'commons-collections4', version: '4.4' - implementation group: 'commons-codec', name: 'commons-codec', version: '1.9' + implementation group: 'commons-codec', name: 'commons-codec', version: '1.16.1' implementation group: 'com.google.guava', name: 'guava', version: versions.guava api group: 'com.fasterxml.jackson.core', name: 'jackson-databind', version: versions.jackson api group: 'org.apache.avro', name: 'avro', version: versions.avro - api group: 'com.jayway.jsonpath', name: 'json-path', version: '2.5.0' + api group: 'com.jayway.jsonpath', name: 'json-path', version: '2.9.0' implementation group: 'io.dropwizard.metrics', name: 'metrics-core', version: versions.dropwizard_metrics - implementation group: 'com.google.code.findbugs', name: 'annotations', version: '3.0.0' + implementation group: 'com.google.code.findbugs', name: 'annotations', version: '3.0.1' api group: 'io.micrometer', name: 'micrometer-core', version: versions.micrometer_metrics api group: 'io.micrometer', name: 'micrometer-registry-prometheus', version: versions.micrometer_metrics - implementation group: 'org.slf4j', name: 'log4j-over-slf4j', version: '2.0.4' - implementation group: 'ch.qos.logback', name: 'logback-classic', version: '1.4.7' + implementation group: 'org.slf4j', name: 'log4j-over-slf4j', version: '2.0.13' + implementation group: 'ch.qos.logback', name: 'logback-classic', version: '1.4.14' api(group: 'org.apache.kafka', name: 'kafka-clients', version: versions.kafka) { exclude group: 'net.sf.jopt-simple' } @@ -48,12 +48,10 @@ dependencies { testImplementation project(':hermes-test-helper') - testImplementation group: 'javax.servlet', name: 'javax.servlet-api', version: '3.1.0' + testImplementation group: 'jakarta.servlet', name: 'jakarta.servlet-api', version: '6.0.0' testImplementation group: 'org.spockframework', name: 'spock-core', version: versions.spock testImplementation group: 'org.spockframework', name: 'spock-junit4', version: versions.spock - testImplementation (group: 'com.jayway.awaitility', name: 'awaitility-groovy', version: '1.7.0') { - exclude group: 'org.codehaus.groovy', module: 'groovy-all' - } + testImplementation group: 'org.awaitility', name: 'awaitility-groovy', version: '4.2.1' testRuntimeOnly group: 'org.junit.vintage', name: 'junit-vintage-engine', version: versions.junit_jupiter } diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/message/converter/AvroBinaryDecoders.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/message/converter/AvroBinaryDecoders.java index c77be92b29..1d092f655b 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/message/converter/AvroBinaryDecoders.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/message/converter/AvroBinaryDecoders.java @@ -5,7 +5,7 @@ import org.apache.avro.generic.GenericRecord; import org.apache.avro.io.BinaryDecoder; import org.apache.avro.io.DecoderFactory; -import org.apache.commons.lang.exception.ExceptionUtils; +import org.apache.commons.lang3.exception.ExceptionUtils; import tech.allegro.schema.json2avro.converter.AvroConversionException; import java.io.ByteArrayInputStream; diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/domain/filtering/avro/AvroPathPredicate.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/domain/filtering/avro/AvroPathPredicate.java index f49a22d7ba..7cd000a2e5 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/domain/filtering/avro/AvroPathPredicate.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/domain/filtering/avro/AvroPathPredicate.java @@ -1,5 +1,6 @@ package pl.allegro.tech.hermes.domain.filtering.avro; +import jakarta.annotation.Nullable; import org.apache.avro.Schema; import org.apache.avro.generic.GenericArray; import org.apache.avro.generic.GenericRecord; @@ -26,7 +27,7 @@ import static java.util.Collections.emptyList; import static java.util.Collections.emptyListIterator; import static java.util.Collections.singletonList; -import static org.apache.commons.lang.StringUtils.strip; +import static org.apache.commons.lang3.StringUtils.strip; import static pl.allegro.tech.hermes.common.message.converter.AvroRecordToBytesConverter.bytesToRecord; import static pl.allegro.tech.hermes.domain.filtering.FilteringException.check; @@ -62,7 +63,7 @@ public boolean test(final FilterableMessage message) { } private List select(final FilterableMessage message) throws IOException { - CompiledSchema compiledSchema = message.getSchema().get(); + CompiledSchema compiledSchema = message.getSchema().get(); return select(bytesToRecord(message.getData(), compiledSchema.getSchema())); } @@ -82,7 +83,7 @@ private List select(Object record, ListIterator iter) { if (arrayMatcher.matches()) { selector = arrayMatcher.group(GROUP_SELECTOR); - current = currentRecord.get(selector); + current = recordFieldValueOrNull(selector, currentRecord); if (!(current instanceof GenericArray)) { return emptyList(); } @@ -97,7 +98,7 @@ private List select(Object record, ListIterator iter) { } } else { - current = currentRecord.get(selector); + current = recordFieldValueOrNull(selector, currentRecord); } } else if (current instanceof HashMap) { Map currentRecord = (HashMap) current; @@ -115,9 +116,9 @@ private boolean isSupportedType(Object record) { private List selectMultipleArrayItems(ListIterator iter, GenericArray currentArray) { return currentArray.stream() - .map(item -> select(item, iter.hasNext() ? path.listIterator(iter.nextIndex()) : emptyListIterator())) - .flatMap(List::stream) - .collect(Collectors.toList()); + .map(item -> select(item, iter.hasNext() ? path.listIterator(iter.nextIndex()) : emptyListIterator())) + .flatMap(List::stream) + .collect(Collectors.toList()); } private Object selectSingleArrayItem(int idx, GenericArray currentArray) { @@ -139,6 +140,15 @@ private boolean matchResultsStream(Stream results) { } } + @Nullable + private Object recordFieldValueOrNull(String selector, GenericRecord record) { + Schema.Field field = record.getSchema().getField(selector); + if (field == null) { + return null; + } + return record.get(field.pos()); + } + private boolean matches(String value) { return pattern.matcher(value).matches(); } diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/infrastructure/zookeeper/ZookeeperBasedRepository.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/infrastructure/zookeeper/ZookeeperBasedRepository.java index b688aa314d..dcb958901c 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/infrastructure/zookeeper/ZookeeperBasedRepository.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/infrastructure/zookeeper/ZookeeperBasedRepository.java @@ -3,7 +3,7 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.JsonMappingException; import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.commons.lang.ArrayUtils; +import org.apache.commons.lang3.ArrayUtils; import org.apache.curator.framework.CuratorFramework; import org.apache.zookeeper.data.Stat; import org.slf4j.Logger; diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/infrastructure/zookeeper/ZookeeperPaths.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/infrastructure/zookeeper/ZookeeperPaths.java index b39596faca..9e04bcc64c 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/infrastructure/zookeeper/ZookeeperPaths.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/infrastructure/zookeeper/ZookeeperPaths.java @@ -1,7 +1,7 @@ package pl.allegro.tech.hermes.infrastructure.zookeeper; import com.google.common.base.Joiner; -import org.apache.commons.lang.StringUtils; +import org.apache.commons.lang3.StringUtils; import pl.allegro.tech.hermes.api.Subscription; import pl.allegro.tech.hermes.api.SubscriptionName; import pl.allegro.tech.hermes.api.TopicName; diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/infrastructure/zookeeper/cache/HierarchicalCacheLevel.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/infrastructure/zookeeper/cache/HierarchicalCacheLevel.java index 320a60d0a1..2e1a9795b3 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/infrastructure/zookeeper/cache/HierarchicalCacheLevel.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/infrastructure/zookeeper/cache/HierarchicalCacheLevel.java @@ -1,6 +1,6 @@ package pl.allegro.tech.hermes.infrastructure.zookeeper.cache; -import org.apache.commons.lang.ArrayUtils; +import org.apache.commons.lang3.ArrayUtils; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.recipes.cache.PathChildrenCache; import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent; diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/infrastructure/zookeeper/notifications/ZookeeperInternalNotificationBus.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/infrastructure/zookeeper/notifications/ZookeeperInternalNotificationBus.java index d7989ba7d0..ccab8542d7 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/infrastructure/zookeeper/notifications/ZookeeperInternalNotificationBus.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/infrastructure/zookeeper/notifications/ZookeeperInternalNotificationBus.java @@ -1,7 +1,7 @@ package pl.allegro.tech.hermes.infrastructure.zookeeper.notifications; import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.commons.lang.ArrayUtils; +import org.apache.commons.lang3.ArrayUtils; import org.apache.curator.framework.recipes.cache.ChildData; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/hermes-common/src/test/groovy/pl/allegro/tech/hermes/infrastructure/zookeeper/ZookeeperWorkloadConstraintsCacheTest.groovy b/hermes-common/src/test/groovy/pl/allegro/tech/hermes/infrastructure/zookeeper/ZookeeperWorkloadConstraintsCacheTest.groovy index 07cb8e1a60..e42121af43 100644 --- a/hermes-common/src/test/groovy/pl/allegro/tech/hermes/infrastructure/zookeeper/ZookeeperWorkloadConstraintsCacheTest.groovy +++ b/hermes-common/src/test/groovy/pl/allegro/tech/hermes/infrastructure/zookeeper/ZookeeperWorkloadConstraintsCacheTest.groovy @@ -5,6 +5,7 @@ import ch.qos.logback.classic.Logger import ch.qos.logback.classic.spi.ILoggingEvent import ch.qos.logback.core.read.ListAppender import com.fasterxml.jackson.databind.ObjectMapper +import org.awaitility.Awaitility import org.slf4j.LoggerFactory import pl.allegro.tech.hermes.api.Constraints import pl.allegro.tech.hermes.api.SubscriptionName @@ -14,8 +15,8 @@ import pl.allegro.tech.hermes.test.IntegrationTest import java.util.concurrent.TimeUnit -import static com.jayway.awaitility.Awaitility.await import static java.util.Collections.emptyMap +import static org.awaitility.Awaitility.await class ZookeeperWorkloadConstraintsCacheTest extends IntegrationTest { diff --git a/hermes-common/src/test/groovy/pl/allegro/tech/hermes/infrastructure/zookeeper/ZookeeperWorkloadConstraintsRepositoryTest.groovy b/hermes-common/src/test/groovy/pl/allegro/tech/hermes/infrastructure/zookeeper/ZookeeperWorkloadConstraintsRepositoryTest.groovy index 485def9354..98f069b7a3 100644 --- a/hermes-common/src/test/groovy/pl/allegro/tech/hermes/infrastructure/zookeeper/ZookeeperWorkloadConstraintsRepositoryTest.groovy +++ b/hermes-common/src/test/groovy/pl/allegro/tech/hermes/infrastructure/zookeeper/ZookeeperWorkloadConstraintsRepositoryTest.groovy @@ -2,6 +2,7 @@ package pl.allegro.tech.hermes.infrastructure.zookeeper import com.fasterxml.jackson.databind.ObjectMapper import org.apache.zookeeper.KeeperException +import org.awaitility.Awaitility import pl.allegro.tech.hermes.api.SubscriptionName import pl.allegro.tech.hermes.api.TopicName import pl.allegro.tech.hermes.api.Constraints @@ -13,7 +14,8 @@ import pl.allegro.tech.hermes.test.IntegrationTest import java.util.concurrent.TimeUnit -import static com.jayway.awaitility.Awaitility.await +import static org.awaitility.Awaitility.await + class ZookeeperWorkloadConstraintsRepositoryTest extends IntegrationTest { diff --git a/hermes-common/src/test/groovy/pl/allegro/tech/hermes/infrastructure/zookeeper/cache/HierarchicalCacheTest.groovy b/hermes-common/src/test/groovy/pl/allegro/tech/hermes/infrastructure/zookeeper/cache/HierarchicalCacheTest.groovy index 64411665ed..48b9c9490d 100644 --- a/hermes-common/src/test/groovy/pl/allegro/tech/hermes/infrastructure/zookeeper/cache/HierarchicalCacheTest.groovy +++ b/hermes-common/src/test/groovy/pl/allegro/tech/hermes/infrastructure/zookeeper/cache/HierarchicalCacheTest.groovy @@ -1,16 +1,16 @@ package pl.allegro.tech.hermes.infrastructure.zookeeper.cache -import com.jayway.awaitility.Duration -import com.jayway.awaitility.groovy.AwaitilityTrait +import org.awaitility.Awaitility import pl.allegro.tech.hermes.test.IntegrationTest +import java.time.Duration import java.util.concurrent.Executors -import static com.jayway.awaitility.Awaitility.await import static org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent.Type.CHILD_ADDED import static org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent.Type.CHILD_REMOVED +import static org.awaitility.Awaitility.await -class HierarchicalCacheTest extends IntegrationTest implements AwaitilityTrait { +class HierarchicalCacheTest extends IntegrationTest { private HierarchicalCache cache = new HierarchicalCache( zookeeper(), @@ -48,7 +48,7 @@ class HierarchicalCacheTest extends IntegrationTest implements AwaitilityTrait { .and().commit() then: - await().atMost(Duration.FIVE_SECONDS).until({ + await().atMost(Duration.ofSeconds(5)).until({ calledCallbacks.contains(new Tuple(CHILD_ADDED, '/hierarchicalCacheTest/groups/groupA', 'groupA')) }) @@ -59,7 +59,7 @@ class HierarchicalCacheTest extends IntegrationTest implements AwaitilityTrait { .and().commit() then: - await().atMost(Duration.FIVE_SECONDS).until({ + await().atMost(Duration.ofSeconds(5)).until({ calledCallbacks.contains(new Tuple(CHILD_ADDED, '/hierarchicalCacheTest/groups/groupA/topics/topicA', 'topicA')) }) @@ -67,7 +67,7 @@ class HierarchicalCacheTest extends IntegrationTest implements AwaitilityTrait { zookeeper().create().forPath('/hierarchicalCacheTest/groups/groupA/topics/topicA/subscriptions/subA', 'subA'.bytes) then: - await().atMost(Duration.FIVE_SECONDS).until({ + await().atMost(Duration.ofSeconds(5)).until({ calledCallbacks.contains(new Tuple(CHILD_ADDED, '/hierarchicalCacheTest/groups/groupA/topics/topicA/subscriptions/subA', 'subA')) }) @@ -89,7 +89,7 @@ class HierarchicalCacheTest extends IntegrationTest implements AwaitilityTrait { cache.start() then: - await().atMost(Duration.FIVE_SECONDS).until({ + await().atMost(Duration.ofSeconds(5)).until({ calledCallbacks.contains(new Tuple(CHILD_ADDED, '/hierarchicalCacheTest/groups/groupB', 'groupB')) && calledCallbacks.contains(new Tuple(CHILD_ADDED, '/hierarchicalCacheTest/groups/groupB/topics/topicB', 'topicB')) && calledCallbacks.contains(new Tuple(CHILD_ADDED, '/hierarchicalCacheTest/groups/groupB/topics/topicB/subscriptions/subB', 'subB')) @@ -110,7 +110,7 @@ class HierarchicalCacheTest extends IntegrationTest implements AwaitilityTrait { '/hierarchicalCacheTest/groups/groupC/topics/topicC/metrics/published', '123'.bytes) then: - await().atMost(Duration.FIVE_SECONDS).until({ + await().atMost(Duration.ofSeconds(5)).until({ calledCallbacks.contains( new Tuple(CHILD_REMOVED, '/hierarchicalCacheTest/groups/groupC/topics/topicC', '')) }) @@ -143,7 +143,7 @@ class HierarchicalCacheTest extends IntegrationTest implements AwaitilityTrait { '/hierarchicalCacheTest/workload/runtime/topic$subscription/hs-consumer1', 'AUTO_ASSIGNED'.bytes) then: - await().atMost(Duration.FIVE_SECONDS).until({ + await().atMost(Duration.ofSeconds(5)).until({ calledCallbacks.contains(new Tuple(CHILD_ADDED, '/hierarchicalCacheTest/workload/runtime/topic$subscription', '')) && !calledCallbacks.contains(new Tuple(CHILD_REMOVED, '/hierarchicalCacheTest/workload/runtime/topic$subscription', '')) }) diff --git a/hermes-common/src/test/java/pl/allegro/tech/hermes/common/message/wrapper/SchemaAwareSerDeTest.java b/hermes-common/src/test/java/pl/allegro/tech/hermes/common/message/wrapper/SchemaAwareSerDeTest.java index 3ffa2ba9e2..a11f0b8833 100644 --- a/hermes-common/src/test/java/pl/allegro/tech/hermes/common/message/wrapper/SchemaAwareSerDeTest.java +++ b/hermes-common/src/test/java/pl/allegro/tech/hermes/common/message/wrapper/SchemaAwareSerDeTest.java @@ -1,18 +1,17 @@ package pl.allegro.tech.hermes.common.message.wrapper; -import org.testng.annotations.Test; +import org.junit.Test; import pl.allegro.tech.hermes.schema.SchemaId; import pl.allegro.tech.hermes.test.helper.avro.AvroUser; -import java.io.IOException; - import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.Assert.assertThrows; public class SchemaAwareSerDeTest { static final AvroUser avro = new AvroUser("bob", 10, "red"); @Test - public void shouldSerialize() throws IOException { + public void shouldSerialize() { // given SchemaId id = SchemaId.valueOf(8); @@ -24,7 +23,7 @@ public void shouldSerialize() throws IOException { } @Test - public void shouldDeserialize() throws IOException { + public void shouldDeserialize() { // given byte[] serialized = SchemaAwareSerDe.serialize(SchemaId.valueOf(8), avro.asBytes()); @@ -36,11 +35,9 @@ public void shouldDeserialize() throws IOException { assertThat(deserialized.getPayload()).isEqualTo(avro.asBytes()); } - @Test(expectedExceptions = { DeserializationException.class }) - public void shouldThrowExceptionWhenDeserializingWithoutMagicByte() throws IOException { + @Test + public void shouldThrowExceptionWhenDeserializingWithoutMagicByte() { // when - SchemaAwareSerDe.deserialize(new byte[]{1, 2, 3}); - - // then exception is thrown + assertThrows(DeserializationException.class, () -> SchemaAwareSerDe.deserialize(new byte[]{1, 2, 3})); } } diff --git a/hermes-common/src/test/java/pl/allegro/tech/hermes/common/metric/counter/zookeeper/ZookeeperCounterReporterTest.java b/hermes-common/src/test/java/pl/allegro/tech/hermes/common/metric/counter/zookeeper/ZookeeperCounterReporterTest.java index 1e9fd3d017..a974a283b0 100644 --- a/hermes-common/src/test/java/pl/allegro/tech/hermes/common/metric/counter/zookeeper/ZookeeperCounterReporterTest.java +++ b/hermes-common/src/test/java/pl/allegro/tech/hermes/common/metric/counter/zookeeper/ZookeeperCounterReporterTest.java @@ -6,7 +6,7 @@ import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Mock; -import org.mockito.runners.MockitoJUnitRunner; +import org.mockito.junit.MockitoJUnitRunner; import pl.allegro.tech.hermes.api.SubscriptionName; import pl.allegro.tech.hermes.api.TopicName; import pl.allegro.tech.hermes.common.metric.MetricsFacade; @@ -14,7 +14,6 @@ import pl.allegro.tech.hermes.common.util.InstanceIdResolver; import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; @RunWith(MockitoJUnitRunner.class) public class ZookeeperCounterReporterTest { @@ -40,7 +39,6 @@ public class ZookeeperCounterReporterTest { @Before public void before() { - when(instanceIdResolver.resolve()).thenReturn("localhost.domain"); zookeeperCounterReporter = new ZookeeperCounterReporter(meterRegistry, counterStorage, ""); } diff --git a/hermes-common/src/test/java/pl/allegro/tech/hermes/common/metric/counter/zookeeper/ZookeeperCounterStorageTest.java b/hermes-common/src/test/java/pl/allegro/tech/hermes/common/metric/counter/zookeeper/ZookeeperCounterStorageTest.java index 4434989d2b..2d92eec12a 100644 --- a/hermes-common/src/test/java/pl/allegro/tech/hermes/common/metric/counter/zookeeper/ZookeeperCounterStorageTest.java +++ b/hermes-common/src/test/java/pl/allegro/tech/hermes/common/metric/counter/zookeeper/ZookeeperCounterStorageTest.java @@ -4,7 +4,8 @@ import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Mock; -import org.mockito.runners.MockitoJUnitRunner; +import org.mockito.Mockito; +import org.mockito.junit.MockitoJUnitRunner; import pl.allegro.tech.hermes.api.TopicName; import pl.allegro.tech.hermes.domain.subscription.SubscriptionNotExistsException; import pl.allegro.tech.hermes.domain.subscription.SubscriptionRepository; @@ -14,7 +15,6 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.verifyZeroInteractions; import static org.mockito.Mockito.when; @RunWith(MockitoJUnitRunner.class) @@ -88,6 +88,6 @@ public void shouldNotIncrementSharedCounterForNonExistingSubscription() { storage.setSubscriptionDeliveredCounter(topicName, subscriptionName, 1L); //then - verifyZeroInteractions(sharedCounter); + Mockito.verifyNoInteractions(sharedCounter); } } \ No newline at end of file diff --git a/hermes-consumers/build.gradle b/hermes-consumers/build.gradle index 292efde6fe..37bab685a3 100644 --- a/hermes-consumers/build.gradle +++ b/hermes-consumers/build.gradle @@ -18,41 +18,34 @@ dependencies { api group: 'org.springframework.boot', name: 'spring-boot-starter', version: versions.spring api group: 'org.eclipse.jetty', name: 'jetty-alpn-java-client', version: versions.jetty api group: 'org.eclipse.jetty.http2', name: 'jetty-http2-client-transport', version: versions.jetty - - implementation group: 'org.jctools', name: 'jctools-core', version: '1.2' - - api group: 'javax.jms', name: 'javax.jms-api', version: '2.0' - implementation group: 'joda-time', name: 'joda-time', version: '2.5' - + implementation group: 'org.jctools', name: 'jctools-core', version: '4.0.3' + api group: 'jakarta.jms', name: 'jakarta.jms-api', version: '3.1.0' + implementation group: 'joda-time', name: 'joda-time', version: '2.12.7' implementation(group: 'com.github.rholder', name: 'guava-retrying', version: '2.0.0') { exclude module: 'guava' } - - implementation group: 'org.agrona', name: 'agrona', version: '1.0.6' - + implementation group: 'org.agrona', name: 'agrona', version: '1.21.1' + // TODO: can we update it ? Which version of server our clients use ? implementation(group: 'org.hornetq', name: 'hornetq-jms-client', version: '2.4.1.Final') { exclude module: 'hornetq-native' } - - api(group: 'com.google.cloud', name: 'google-cloud-pubsub', version: '1.115.1') + api group: 'com.google.cloud', name: 'google-cloud-pubsub', version: '1.128.1' + api group: 'org.apache.httpcomponents.core5', name: 'httpcore5', version: '5.2.4' testImplementation project(':hermes-test-helper') testImplementation group: 'org.apache.curator', name: 'curator-test', version: versions.curator - testImplementation group: 'javax.servlet', name: 'javax.servlet-api', version: '3.1.0' + testImplementation group: 'jakarta.servlet', name: 'jakarta.servlet-api', version: '6.0.0' testImplementation project(':hermes-common') - testImplementation(group: 'com.jayway.awaitility', name: 'awaitility', version: '1.6.1') { - exclude group: 'com.jayway.jsonpath', module: 'json-path' - } - + testImplementation(group: 'org.awaitility', name: 'awaitility-groovy', version: '4.2.1') testImplementation group: 'tech.allegro.schema.json2avro', name: 'converter', version: versions.json2avro testImplementation group: 'org.spockframework', name: 'spock-core', version: versions.spock testImplementation group: 'org.spockframework', name: 'spock-junit4', version: versions.spock testRuntimeOnly group: 'org.junit.vintage', name: 'junit-vintage-engine', version: versions.junit_jupiter - sbeClasspath group: 'uk.co.real-logic', name: 'sbe-all', version: '1.12.8' + sbeClasspath group: 'uk.co.real-logic', name: 'sbe-all', version: '1.31.1' } def generatedPath = "${buildDir}/generated/java/" @@ -90,6 +83,6 @@ sourceSets { } } -compileJava{ +compileJava { dependsOn(generateSbeStubs) } diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/Message.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/Message.java index 973fdd4d5c..eec94009f1 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/Message.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/Message.java @@ -4,7 +4,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Sets; import org.apache.avro.Schema; -import org.apache.commons.lang.ArrayUtils; +import org.apache.commons.lang3.ArrayUtils; import pl.allegro.tech.hermes.api.ContentType; import pl.allegro.tech.hermes.api.Header; import pl.allegro.tech.hermes.api.SubscriptionPolicy; diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/oauth/client/OAuthHttpClient.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/oauth/client/OAuthHttpClient.java index cc0dc18849..e8a82866a7 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/oauth/client/OAuthHttpClient.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/oauth/client/OAuthHttpClient.java @@ -1,7 +1,6 @@ package pl.allegro.tech.hermes.consumers.consumer.oauth.client; import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.http.entity.ContentType; import org.eclipse.jetty.client.ContentResponse; import org.eclipse.jetty.client.HttpClient; import org.eclipse.jetty.client.Request; @@ -15,6 +14,8 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import static jakarta.ws.rs.core.MediaType.APPLICATION_FORM_URLENCODED; + public class OAuthHttpClient implements OAuthClient { private final HttpClient httpClient; @@ -42,7 +43,7 @@ private Request createHttpRequest(OAuthTokenRequest request) { .method(HttpMethod.POST) .headers(headers -> { headers.add(HttpHeader.KEEP_ALIVE, "true"); - headers.add(HttpHeader.CONTENT_TYPE, ContentType.APPLICATION_FORM_URLENCODED.toString()); + headers.add(HttpHeader.CONTENT_TYPE, APPLICATION_FORM_URLENCODED); }); addParamIfNotNull(httpRequest, OAuthTokenRequest.Param.GRANT_TYPE, request.getGrantType()); addParamIfNotNull(httpRequest, OAuthTokenRequest.Param.SCOPE, request.getScope()); diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/rate/SendCounters.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/rate/SendCounters.java index f81f9648c6..af891d932d 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/rate/SendCounters.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/rate/SendCounters.java @@ -1,13 +1,14 @@ package pl.allegro.tech.hermes.consumers.consumer.rate; import com.google.common.util.concurrent.AtomicDouble; -import org.apache.commons.lang.math.Fraction; +import org.apache.commons.lang3.math.Fraction; import java.time.Clock; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; -import static org.apache.commons.lang.math.Fraction.getFraction; +import static org.apache.commons.lang3.math.Fraction.getFraction; + public class SendCounters { diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/sender/googlepubsub/CompressionCodecFactory.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/sender/googlepubsub/CompressionCodecFactory.java index ba46970280..b6152b1b8d 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/sender/googlepubsub/CompressionCodecFactory.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/sender/googlepubsub/CompressionCodecFactory.java @@ -84,7 +84,7 @@ static class ZstandardCodecFactory extends CompressionCodecFactory { @Override Codec createInstance() { - return new ZstandardCodec(compressionLevel, true); + return new ZstandardCodec(compressionLevel, true, ZstandardCodec.DEFAULT_USE_BUFFERPOOL); } } } diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/sender/googlepubsub/MessageCompressor.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/sender/googlepubsub/MessageCompressor.java index bf933b18cd..ae2ed0579e 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/sender/googlepubsub/MessageCompressor.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/sender/googlepubsub/MessageCompressor.java @@ -17,15 +17,18 @@ byte[] compress(byte[] data) throws IOException { Codec codec = codecFactory.createInstance(); ByteBuffer compressed = codec.compress(ByteBuffer.wrap(data)); + byte[] compressedBytes = new byte[compressed.limit()]; + compressed.get(compressedBytes); - return compressed.array(); + return compressedBytes; } byte[] decompress(byte[] data) throws IOException { Codec codec = codecFactory.createInstance(); ByteBuffer decompressed = codec.decompress(ByteBuffer.wrap(data)); - - return decompressed.array(); + byte[] decompressedBytes = new byte[decompressed.limit()]; + decompressed.get(decompressedBytes); + return decompressedBytes; } } diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/sender/http/ByteBufferEntity.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/sender/http/ByteBufferEntity.java deleted file mode 100644 index 44435a681b..0000000000 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/sender/http/ByteBufferEntity.java +++ /dev/null @@ -1,131 +0,0 @@ -package pl.allegro.tech.hermes.consumers.consumer.sender.http; - -/* - * ==================================================================== - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you 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. - * ==================================================================== - * - * This software consists of voluntary contributions made by many - * individuals on behalf of the Apache Software Foundation. For more - * information on the Apache Software Foundation, please see - * . - * - */ - -import org.apache.http.annotation.Contract; -import org.apache.http.entity.AbstractHttpEntity; -import org.apache.http.entity.ContentType; -import org.apache.http.util.Args; - -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.nio.ByteBuffer; -import java.nio.channels.Channels; -import java.nio.channels.WritableByteChannel; - -/* - * CAUTION! - * This implementation comes from PR that was merged directly into hc.core5 - * which is not backwards compatible with http-components version 4. We need - * entire http stack and not just the core so we cannot use core5 yet. This - * is the last implementation that works on version 4. If it is good enough - * for hc.core5 then it should be ok for out use case. - * - * PR: https://github.com/apache/httpcore/pull/13 - * -* */ - -/** - * An entity that delivers the contents of a {@link ByteBuffer}. - */ -@Contract(threading = org.apache.http.annotation.ThreadingBehavior.UNSAFE) -public class ByteBufferEntity extends AbstractHttpEntity implements Cloneable { - - private final ByteBuffer buffer; - - private class ByteBufferInputStream extends InputStream { - - ByteBufferInputStream() { - buffer.position(0); - } - - public int read() throws IOException { - if (!buffer.hasRemaining()) { - return -1; - } - return buffer.get() & 0xFF; - } - - public int read(byte[] bytes, int off, int len) throws IOException { - if (!buffer.hasRemaining()) { - return -1; - } - - len = Math.min(len, buffer.remaining()); - buffer.get(bytes, off, len); - return len; - } - } - - public ByteBufferEntity(final ByteBuffer buffer, final ContentType contentType) { - super(); - Args.notNull(buffer, "Source byte buffer"); - this.buffer = buffer; - if (contentType != null) { - setContentType(contentType.toString()); - } - } - - public ByteBufferEntity(final ByteBuffer buffer) { - this(buffer, null); - } - - @Override - public boolean isRepeatable() { - return true; - } - - @Override - public long getContentLength() { - return buffer.limit(); - } - - @Override - public InputStream getContent() throws IOException, UnsupportedOperationException { - return new ByteBufferInputStream(); - } - - @Override - public void writeTo(OutputStream outstream) throws IOException { - Args.notNull(outstream, "Output stream"); - WritableByteChannel channel = Channels.newChannel(outstream); - channel.write(buffer); - outstream.flush(); - } - - @Override - public boolean isStreaming() { - return false; - } - - @Override - public Object clone() throws CloneNotSupportedException { - return super.clone(); - } -} diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/sender/http/JettyMessageBatchSender.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/sender/http/JettyMessageBatchSender.java index 386a449feb..f9abdf262f 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/sender/http/JettyMessageBatchSender.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/sender/http/JettyMessageBatchSender.java @@ -1,7 +1,6 @@ package pl.allegro.tech.hermes.consumers.consumer.sender.http; -import org.apache.http.entity.ContentType; -import org.apache.http.protocol.HTTP; +import org.apache.hc.core5.http.ContentType; import org.eclipse.jetty.client.ContentResponse; import org.eclipse.jetty.client.Request; import org.slf4j.Logger; @@ -22,6 +21,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeoutException; +import static jakarta.ws.rs.core.HttpHeaders.CONTENT_TYPE; import static pl.allegro.tech.hermes.api.AvroMediaType.AVRO_BINARY; import static pl.allegro.tech.hermes.api.ContentType.AVRO; import static pl.allegro.tech.hermes.common.http.MessageMetadataHeaders.BATCH_ID; @@ -81,7 +81,7 @@ private HttpRequestHeaders buildHeaders(MessageBatch batch, HttpRequestHeaders b ContentType contentType = getMediaType(batch.getContentType()); headers.put(BATCH_ID.getName(), batch.getId()); - headers.put(HTTP.CONTENT_TYPE, contentType.getMimeType()); + headers.put(CONTENT_TYPE, contentType.getMimeType()); headers.put(RETRY_COUNT.getName(), Integer.toString(batch.getRetryCounter())); if (batch.hasSubscriptionIdentityHeaders()) { diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/sender/http/auth/BasicAuthProvider.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/sender/http/auth/BasicAuthProvider.java index 761f06af35..ddde76854d 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/sender/http/auth/BasicAuthProvider.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/sender/http/auth/BasicAuthProvider.java @@ -1,9 +1,9 @@ package pl.allegro.tech.hermes.consumers.consumer.sender.http.auth; -import org.apache.commons.codec.binary.Base64; import pl.allegro.tech.hermes.api.EndpointAddress; import java.nio.charset.StandardCharsets; +import java.util.Base64; import java.util.Optional; public class BasicAuthProvider implements HttpAuthorizationProvider { @@ -12,7 +12,7 @@ public class BasicAuthProvider implements HttpAuthorizationProvider { public BasicAuthProvider(EndpointAddress endpoint) { String credentials = endpoint.getUsername() + ":" + endpoint.getPassword(); - String encodedCredentials = Base64.encodeBase64String(credentials.getBytes(StandardCharsets.UTF_8)); + String encodedCredentials = Base64.getEncoder().encodeToString(credentials.getBytes(StandardCharsets.UTF_8)); this.token = "Basic " + encodedCredentials; } diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/registry/ConsumerNodesRegistry.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/registry/ConsumerNodesRegistry.java index af6346de5e..fead0874cc 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/registry/ConsumerNodesRegistry.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/registry/ConsumerNodesRegistry.java @@ -1,6 +1,6 @@ package pl.allegro.tech.hermes.consumers.registry; -import org.apache.commons.lang.StringUtils; +import org.apache.commons.lang3.StringUtils; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.recipes.cache.PathChildrenCache; import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent; @@ -23,7 +23,7 @@ import java.util.concurrent.TimeUnit; import static java.util.stream.Collectors.toList; -import static org.apache.commons.lang.StringUtils.substringAfterLast; +import static org.apache.commons.lang3.StringUtils.substringAfterLast; import static org.apache.zookeeper.CreateMode.EPHEMERAL; public class ConsumerNodesRegistry extends PathChildrenCache implements PathChildrenCacheListener { diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/uri/UriUtils.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/uri/UriUtils.java index 654fd535c0..287068cd3f 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/uri/UriUtils.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/uri/UriUtils.java @@ -1,7 +1,7 @@ package pl.allegro.tech.hermes.consumers.uri; import com.google.common.base.Splitter; -import org.apache.commons.lang.StringUtils; +import org.apache.commons.lang3.StringUtils; import java.net.URI; import java.util.List; diff --git a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/supervisor/process/ConsumerProcessSupervisorTest.groovy b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/supervisor/process/ConsumerProcessSupervisorTest.groovy index 98a087a9d3..a40fff24f1 100644 --- a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/supervisor/process/ConsumerProcessSupervisorTest.groovy +++ b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/supervisor/process/ConsumerProcessSupervisorTest.groovy @@ -1,10 +1,10 @@ package pl.allegro.tech.hermes.consumers.supervisor.process -import com.jayway.awaitility.Awaitility -import com.jayway.awaitility.core.ConditionFactory import io.micrometer.core.instrument.MeterRegistry import io.micrometer.core.instrument.search.Search import io.micrometer.core.instrument.simple.SimpleMeterRegistry +import org.awaitility.Awaitility +import org.awaitility.core.ConditionFactory import pl.allegro.tech.hermes.api.DeliveryType import pl.allegro.tech.hermes.api.Subscription import pl.allegro.tech.hermes.api.SubscriptionName @@ -106,7 +106,7 @@ class ConsumerProcessSupervisorTest extends Specification { } then: - await().until { + await().untilAsserted { supervisor.run() assert !supervisor.runningSubscriptionsStatus().isEmpty() assert supervisor.runningSubscriptionsStatus().first().signalTimesheet[signals.start.type] == currentTime @@ -125,7 +125,7 @@ class ConsumerProcessSupervisorTest extends Specification { } then: - await().until { + await().untilAsserted { assert consumer.tearDownCount > 0 assert consumer.wasInterrupted } @@ -141,7 +141,7 @@ class ConsumerProcessSupervisorTest extends Specification { runAndWait(supervisor) then: - await().until { + await().untilAsserted { assert !supervisor.runningSubscriptionsStatus().isEmpty() signalsToPass.forEach { assert supervisor.runningSubscriptionsStatus().first().signalTimesheet[it.type] == currentTime diff --git a/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSenderTest.java b/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSenderTest.java index d5ca9aac60..31cd6539b1 100644 --- a/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSenderTest.java +++ b/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSenderTest.java @@ -4,7 +4,7 @@ import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Mock; -import org.mockito.runners.MockitoJUnitRunner; +import org.mockito.junit.MockitoJUnitRunner; import pl.allegro.tech.hermes.api.Subscription; import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.common.metric.SubscriptionMetrics; @@ -32,14 +32,14 @@ import java.util.concurrent.Executors; import static org.assertj.core.api.Assertions.assertThat; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.eq; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.timeout; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.verifyZeroInteractions; +import static org.mockito.Mockito.verifyNoInteractions; import static org.mockito.Mockito.when; import static pl.allegro.tech.hermes.api.SubscriptionOAuthPolicy.passwordGrantOAuthPolicy; import static pl.allegro.tech.hermes.api.SubscriptionPolicy.Builder.subscriptionPolicy; @@ -134,10 +134,10 @@ public void shouldHandleSuccessfulSending() { verifySemaphoreReleased(); verifyLatencyTimersCountedTimes(subscription, 1, 1); verifyRateLimiterAcquireTimersCountedTimes(subscription, 1, 1); - verifyZeroInteractions(errorHandler); - verifyZeroInteractions(failedMeter); + verifyNoInteractions(errorHandler); + verifyNoInteractions(failedMeter); verifyRateLimiterAcquired(); - verifyZeroInteractions(retries); + verifyNoInteractions(retries); } @Test @@ -171,11 +171,11 @@ public void shouldDiscardMessageWhenTTLIsExceeded() { // then verify(errorHandler, timeout(1000)).handleDiscarded(eq(message), eq(subscription), any(MessageSendingResult.class)); verifySemaphoreReleased(); - verifyZeroInteractions(successHandler); + verifyNoInteractions(successHandler); verifyLatencyTimersCountedTimes(subscription, 1, 1); verifyRateLimiterAcquireTimersCountedTimes(subscription, 1, 1); verifyRateLimiterAcquired(); - verifyZeroInteractions(retries); + verifyNoInteractions(retries); } @Test @@ -190,11 +190,11 @@ public void shouldNotKeepTryingToSendMessageFailedWithStatusCode4xx() { // then verify(errorHandler, timeout(1000)).handleDiscarded(eq(message), eq(subscription), any(MessageSendingResult.class)); verifySemaphoreReleased(); - verifyZeroInteractions(successHandler); + verifyNoInteractions(successHandler); verifyLatencyTimersCountedTimes(subscription, 1, 1); verifyRateLimiterAcquireTimersCountedTimes(subscription, 1, 1); verifyRateLimiterAcquired(); - verifyZeroInteractions(retries); + verifyNoInteractions(retries); } @Test @@ -276,11 +276,11 @@ public void shouldNotRetryOnRetryAfterAboveTtl() { // then verify(errorHandler, timeout(1000)).handleDiscarded(eq(message), eq(subscription), any(MessageSendingResult.class)); verifySemaphoreReleased(); - verifyZeroInteractions(successHandler); + verifyNoInteractions(successHandler); verifyLatencyTimersCountedTimes(subscription, 1, 1); verifyRateLimiterAcquireTimersCountedTimes(subscription, 1, 1); verifyRateLimiterAcquired(); - verifyZeroInteractions(retries); + verifyNoInteractions(retries); } @Test @@ -301,7 +301,7 @@ public void shouldDeliverToModifiedEndpoint() { // then verify(otherMessageSender, timeout(1000)).send(message); verifyRateLimiterAcquired(); - verifyZeroInteractions(retries); + verifyNoInteractions(retries); } @Test @@ -322,7 +322,7 @@ public void shouldDeliverToNewSenderAfterModifiedTimeout() { // then verify(otherMessageSender, timeout(1000)).send(message); verifyRateLimiterAcquired(); - verifyZeroInteractions(retries); + verifyNoInteractions(retries); } @Test @@ -348,7 +348,7 @@ public void shouldDelaySendingMessageForHalfSecond() { long sendingTime = System.currentTimeMillis() - sendingStartTime; assertThat(sendingTime).isGreaterThanOrEqualTo(500); verifyRateLimiterAcquired(); - verifyZeroInteractions(retries); + verifyNoInteractions(retries); } @Test @@ -374,7 +374,7 @@ public void shouldCalculateSendingDelayBasingOnPublishingTimestamp() { long sendingTime = System.currentTimeMillis() - sendingStartTime; assertThat(sendingTime).isLessThan(300); verifyRateLimiterAcquired(); - verifyZeroInteractions(retries); + verifyNoInteractions(retries); } @Test @@ -394,7 +394,7 @@ public void shouldIncreaseRetryBackoffExponentially() throws InterruptedExceptio Thread.sleep(backoff + (long) multiplier * backoff - 100); // then - verifyZeroInteractions(successHandler); + verifyNoInteractions(successHandler); verifyRateLimiterAcquired(expectedNumberOfFailures); verifyRetryCounted(expectedNumberOfFailures); } @@ -438,7 +438,7 @@ public void shouldIgnoreExponentialRetryBackoffAfterExceededTtl() throws Interru Thread.sleep(backoff + (long) multiplier * backoff + 1000); //then - verifyZeroInteractions(successHandler); + verifyNoInteractions(successHandler); verifyRateLimiterAcquired(2); verifyRetryCounted(); } diff --git a/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/converter/AvroToJsonMessageConverterTest.java b/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/converter/AvroToJsonMessageConverterTest.java index 9c759cdba2..d53c82e68d 100644 --- a/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/converter/AvroToJsonMessageConverterTest.java +++ b/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/converter/AvroToJsonMessageConverterTest.java @@ -2,7 +2,7 @@ import org.junit.Test; import org.junit.runner.RunWith; -import org.mockito.runners.MockitoJUnitRunner; +import org.mockito.junit.MockitoJUnitRunner; import pl.allegro.tech.hermes.api.Topic; import pl.allegro.tech.hermes.consumers.consumer.Message; import pl.allegro.tech.hermes.schema.CompiledSchema; diff --git a/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/rate/calculator/OutputRateCalculatorTest.java b/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/rate/calculator/OutputRateCalculatorTest.java index 92117be21b..e6e80c18d1 100644 --- a/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/rate/calculator/OutputRateCalculatorTest.java +++ b/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/rate/calculator/OutputRateCalculatorTest.java @@ -3,7 +3,7 @@ import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; -import org.mockito.runners.MockitoJUnitRunner; +import org.mockito.junit.MockitoJUnitRunner; import pl.allegro.tech.hermes.api.SubscriptionPolicy; import pl.allegro.tech.hermes.consumers.config.RateProperties; import pl.allegro.tech.hermes.consumers.consumer.rate.maxrate.NegotiatedMaxRateProvider; diff --git a/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/rate/maxrate/MaxRateRegistryTest.java b/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/rate/maxrate/MaxRateRegistryTest.java index d821d0fcfc..d7a2e132a5 100644 --- a/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/rate/maxrate/MaxRateRegistryTest.java +++ b/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/rate/maxrate/MaxRateRegistryTest.java @@ -22,7 +22,7 @@ import java.util.Collections; import java.util.concurrent.TimeUnit; -import static com.jayway.awaitility.Awaitility.await; +import static org.awaitility.Awaitility.await; import static org.junit.Assert.assertEquals; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; diff --git a/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/sender/MessageSenderFactoryTest.java b/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/sender/MessageSenderFactoryTest.java index b439fff7a8..f96e21422a 100644 --- a/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/sender/MessageSenderFactoryTest.java +++ b/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/sender/MessageSenderFactoryTest.java @@ -2,7 +2,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; -import com.googlecode.catchexception.CatchException; import org.junit.Test; import org.mockito.Mockito; import pl.allegro.tech.hermes.api.EndpointAddress; @@ -12,8 +11,8 @@ import java.util.Set; -import static com.googlecode.catchexception.CatchException.catchException; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.Assert.assertThrows; import static pl.allegro.tech.hermes.test.helper.builder.SubscriptionBuilder.subscription; public class MessageSenderFactoryTest { @@ -43,12 +42,8 @@ public void shouldGetProtocolNotSupportedExceptionWhenPassingUnknownUri() { MessageSenderFactory factory = new MessageSenderFactory(ImmutableList.of()); Subscription subscription = subscription("group.topic", "subscription", "unknown://localhost:8080/test").build(); - // when - catchException(factory).create(subscription, resilientMessageSender); - // then - assertThat(CatchException.caughtException()) - .isInstanceOf(EndpointProtocolNotSupportedException.class); + assertThrows(EndpointProtocolNotSupportedException.class, () -> factory.create(subscription, resilientMessageSender)); } private ProtocolMessageSenderProvider protocolMessageSenderProviderReturning(Object createdMessageSender, String protocol) { diff --git a/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/sender/http/ByteBufferEntityTest.java b/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/sender/http/ByteBufferEntityTest.java deleted file mode 100644 index ea6fefce87..0000000000 --- a/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/sender/http/ByteBufferEntityTest.java +++ /dev/null @@ -1,104 +0,0 @@ -package pl.allegro.tech.hermes.consumers.consumer.sender.http; - -/* - * ==================================================================== - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you 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. - * ==================================================================== - * - * This software consists of voluntary contributions made by many - * individuals on behalf of the Apache Software Foundation. For more - * information on the Apache Software Foundation, please see - * . - * - */ - -/* - * CAUTION! - * This implementation comes from PR that was merged directly into hc.core5 - * which is not backwards compatible with http-components version 4. We need - * entire http stack and not just the core so we cannot use core5 yet. This - * is the last implementation that works on version 4. If it is good enough - * for hc.core5 then it should be ok for out use case. - * - * PR: https://github.com/apache/httpcore/pull/13 - * -* */ - -import org.apache.http.Consts; -import org.junit.Assert; -import org.junit.Test; - -import java.io.ByteArrayOutputStream; -import java.nio.ByteBuffer; - -/** - * Unit tests for {@link ByteBufferEntity}. - * - */ -public class ByteBufferEntityTest { - - @Test - public void testBasics() throws Exception { - final ByteBuffer bytes = ByteBuffer.wrap("Message content".getBytes(Consts.ASCII)); - final ByteBufferEntity httpentity = new ByteBufferEntity(bytes); - - Assert.assertEquals(bytes.capacity(), httpentity.getContentLength()); - Assert.assertNotNull(httpentity.getContent()); - Assert.assertTrue(httpentity.isRepeatable()); - Assert.assertFalse(httpentity.isStreaming()); - } - - - @Test(expected = IllegalArgumentException.class) - public void testIllegalConstructorNullByteArray() throws Exception { - new ByteBufferEntity(null); - } - - @Test - public void testWriteTo() throws Exception { - final ByteBuffer bytes = ByteBuffer.wrap("Message content".getBytes(Consts.ASCII)); - final ByteBufferEntity httpentity = new ByteBufferEntity(bytes); - - ByteArrayOutputStream out = new ByteArrayOutputStream(); - httpentity.writeTo(out); - byte[] bytes2 = out.toByteArray(); - Assert.assertNotNull(bytes2); - Assert.assertEquals(bytes.capacity(), bytes2.length); - bytes.position(0); - for (int i = 0; i < bytes2.length; i++) { - Assert.assertEquals(bytes.get(i), bytes2[i]); - } - - out = new ByteArrayOutputStream(); - httpentity.writeTo(out); - bytes2 = out.toByteArray(); - Assert.assertNotNull(bytes2); - Assert.assertEquals(bytes.capacity(), bytes2.length); - bytes.position(0); - for (int i = 0; i < bytes.capacity(); i++) { - Assert.assertEquals(bytes.get(i), bytes2[i]); - } - - try { - httpentity.writeTo(null); - Assert.fail("IllegalArgumentException should have been thrown"); - } catch (final IllegalArgumentException ex) { - // expected - } - } -} \ No newline at end of file diff --git a/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/sender/jms/JmsMessageSenderTest.java b/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/sender/jms/JmsMessageSenderTest.java index 3c4f6032e5..7255e9a62e 100644 --- a/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/sender/jms/JmsMessageSenderTest.java +++ b/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/sender/jms/JmsMessageSenderTest.java @@ -8,7 +8,7 @@ import org.mockito.InjectMocks; import org.mockito.Mock; import org.mockito.Spy; -import org.mockito.runners.MockitoJUnitRunner; +import org.mockito.junit.MockitoJUnitRunner; import pl.allegro.tech.hermes.consumers.consumer.Message; import pl.allegro.tech.hermes.consumers.consumer.sender.MessageSendingResult; diff --git a/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/sender/resolver/InterpolatingEndpointAddressResolverTest.java b/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/sender/resolver/InterpolatingEndpointAddressResolverTest.java index b69b4b2410..5f4c432aa8 100644 --- a/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/sender/resolver/InterpolatingEndpointAddressResolverTest.java +++ b/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/sender/resolver/InterpolatingEndpointAddressResolverTest.java @@ -1,6 +1,5 @@ package pl.allegro.tech.hermes.consumers.consumer.sender.resolver; -import com.googlecode.catchexception.CatchException; import org.junit.Test; import pl.allegro.tech.hermes.api.EndpointAddress; import pl.allegro.tech.hermes.api.EndpointAddressResolverMetadata; @@ -11,8 +10,8 @@ import java.net.URI; import java.nio.charset.StandardCharsets; -import static com.googlecode.catchexception.CatchException.catchException; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.Assert.assertThrows; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; import static pl.allegro.tech.hermes.consumers.test.MessageBuilder.withTestMessage; @@ -41,19 +40,18 @@ public void shouldUseInterpolatorToInterpolateURI() throws InterpolationExceptio } @Test - @SuppressWarnings("unchecked") - public void shouldThrowResolvingExceptionWhenInterpolationFails() throws EndpointAddressResolutionException, InterpolationException { + public void shouldThrowResolvingExceptionWhenInterpolationFails() throws InterpolationException { // given EndpointAddress address = EndpointAddress.of("http://localhost/{a}"); Message message = withTestMessage().withContent("content", StandardCharsets.UTF_8).build(); when(interpolator.interpolate(address, message)).thenThrow(InterpolationException.class); - // when - catchException(resolver).resolve(EndpointAddress.of("http://localhost/{a}"), - withTestMessage().withContent("content", StandardCharsets.UTF_8).build(), metadata); - // then - assertThat(CatchException.caughtException()) - .isInstanceOf(EndpointAddressResolutionException.class); + assertThrows(EndpointAddressResolutionException.class, + () -> resolver.resolve( + EndpointAddress.of("http://localhost/{a}"), + withTestMessage().withContent("content", StandardCharsets.UTF_8).build(), + metadata) + ); } } diff --git a/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/supervisor/workload/ConsumerTestRuntimeEnvironment.java b/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/supervisor/workload/ConsumerTestRuntimeEnvironment.java index 05057dbc78..4018066ac8 100644 --- a/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/supervisor/workload/ConsumerTestRuntimeEnvironment.java +++ b/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/supervisor/workload/ConsumerTestRuntimeEnvironment.java @@ -56,9 +56,9 @@ import java.util.function.Supplier; import java.util.stream.IntStream; -import static com.jayway.awaitility.Awaitility.await; -import static com.jayway.awaitility.Duration.ONE_SECOND; import static java.util.stream.Collectors.toList; +import static org.assertj.core.api.Assertions.assertThat; +import static org.awaitility.Awaitility.await; import static org.mockito.Mockito.mock; import static pl.allegro.tech.hermes.test.helper.builder.SubscriptionBuilder.subscription; import static pl.allegro.tech.hermes.test.helper.builder.TopicBuilder.topic; @@ -154,13 +154,13 @@ private WorkloadSupervisor createConsumer(String consumerId, new ZookeeperSubscriptionIdProvider(curator, zookeeperPaths), new CommonConsumerProperties()); ConsumerAssignmentCache consumerAssignmentCache = - new ConsumerAssignmentCache( - curator, - workloadProperties.getNodeId(), - kafkaProperties.getClusterName(), - zookeeperPaths, - subscriptionIds - ); + new ConsumerAssignmentCache( + curator, + workloadProperties.getNodeId(), + kafkaProperties.getClusterName(), + zookeeperPaths, + subscriptionIds + ); try { consumerAssignmentCache.start(); } catch (Exception e) { @@ -168,22 +168,22 @@ private WorkloadSupervisor createConsumer(String consumerId, } ClusterAssignmentCache clusterAssignmentCache = - new ClusterAssignmentCache( - curator, - kafkaProperties.getClusterName(), - zookeeperPaths, - subscriptionIds, - nodesRegistry - ); + new ClusterAssignmentCache( + curator, + kafkaProperties.getClusterName(), + zookeeperPaths, + subscriptionIds, + nodesRegistry + ); ConsumerAssignmentRegistry consumerAssignmentRegistry = - new ConsumerAssignmentRegistry( - curator, - workloadProperties.getRegistryBinaryEncoderAssignmentsBufferSizeBytes(), - kafkaProperties.getClusterName(), - zookeeperPaths, - subscriptionIds - ); + new ConsumerAssignmentRegistry( + curator, + workloadProperties.getRegistryBinaryEncoderAssignmentsBufferSizeBytes(), + kafkaProperties.getClusterName(), + zookeeperPaths, + subscriptionIds + ); return new WorkloadSupervisor( consumersSupervisor, @@ -270,8 +270,9 @@ void kill(WorkloadSupervisor node) { consumerZookeeperConnections.get(node.consumerId()).close(); } - void killAll() { + void cleanState() { consumerZookeeperConnections.values().forEach(CuratorFramework::close); + subscriptionsCaches.clear(); } private WorkloadSupervisor startNode(WorkloadSupervisor workloadSupervisor) { @@ -285,7 +286,7 @@ private WorkloadSupervisor startNode(WorkloadSupervisor workloadSupervisor) { } void waitForRegistration(String consumerId) { - await().atMost(adjust(ONE_SECOND)).until(() -> isRegistered(consumerId)); + await().atMost(adjust(Duration.ofSeconds(1))).until(() -> isRegistered(consumerId)); } private boolean isRegistered(String nodeId) { @@ -297,9 +298,7 @@ private boolean isRegistered(String nodeId) { } void awaitUntilAssignmentExists(SubscriptionName subscription, WorkloadSupervisor node) { - await().atMost(adjust(ONE_SECOND)).until(() -> { - node.assignedSubscriptions().contains(subscription); - }); + await().atMost(adjust(Duration.ofSeconds(2))).until(() -> node.assignedSubscriptions().contains(subscription)); } List createSubscription(int howMany) { @@ -321,11 +320,11 @@ private SubscriptionName createSubscription(SubscriptionName subscriptionName) { topicRepository.createTopic(topic(subscription.getTopicName()).build()); } subscriptionRepository.createSubscription(subscription); - await().atMost(adjust(ONE_SECOND)).until( + await().atMost(adjust(Duration.ofSeconds(2))).untilAsserted( () -> { - subscriptionRepository.subscriptionExists(subscription.getTopicName(), subscription.getName()); + assertThat(subscriptionRepository.subscriptionExists(subscription.getTopicName(), subscription.getName())).isTrue(); subscriptionsCaches.forEach(subscriptionsCache -> - subscriptionsCache.listActiveSubscriptionNames().contains(subscriptionName)); + assertThat(subscriptionsCache.listActiveSubscriptionNames().contains(subscriptionName)).isTrue()); } ); return subscription.getQualifiedName(); diff --git a/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/supervisor/workload/WorkloadSupervisorIntegrationTest.java b/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/supervisor/workload/WorkloadSupervisorIntegrationTest.java index 4c510823ef..8e609aad90 100644 --- a/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/supervisor/workload/WorkloadSupervisorIntegrationTest.java +++ b/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/supervisor/workload/WorkloadSupervisorIntegrationTest.java @@ -14,10 +14,9 @@ import java.time.Duration; import java.util.List; -import static com.jayway.awaitility.Awaitility.await; -import static com.jayway.awaitility.Duration.FIVE_SECONDS; import static org.assertj.core.api.Assertions.assertThat; -import static org.mockito.Matchers.any; +import static org.awaitility.Awaitility.await; +import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -30,7 +29,7 @@ public class WorkloadSupervisorIntegrationTest extends ZookeeperBaseTest { @Before public void setup() throws Exception { - runtime.killAll(); + runtime.cleanState(); deleteData("/hermes"); createPath("/hermes/groups"); } @@ -67,8 +66,8 @@ public void shouldElectNewLeaderAfterShutdown() { runtime.kill(leader); // then - await().atMost(adjust(FIVE_SECONDS)).until(() -> runtime.findLeader(supervisors) != leader); - await().atMost(adjust(FIVE_SECONDS)).until(() -> !leader.isLeader()); + await().atMost(adjust(Duration.ofSeconds(5))).until(() -> runtime.findLeader(supervisors) != leader); + await().atMost(adjust(Duration.ofSeconds(5))).until(() -> !leader.isLeader()); } @Test @@ -137,7 +136,7 @@ public void shouldRecreateMissingConsumer() throws Exception { monitor.start(); // then - await().atMost(FIVE_SECONDS).until( + await().atMost(Duration.ofSeconds(5)).untilAsserted( () -> verify(consumerFactory, times(2)).createConsumer(any())); shutdown(supervisor); diff --git a/hermes-frontend/build.gradle b/hermes-frontend/build.gradle index 5423631ef9..dcb9018c40 100644 --- a/hermes-frontend/build.gradle +++ b/hermes-frontend/build.gradle @@ -13,24 +13,19 @@ dependencies { api group: 'org.springframework.boot', name: 'spring-boot-starter', version: versions.spring api group: 'io.undertow', name: 'undertow-core', version: versions.undertow - implementation group: 'org.apache.commons', name: 'commons-lang3', version: '3.0' - api (group: 'net.openhft', name: 'chronicle-map', version: '3.22.9') { + // Did not update that as we're trying to abandon buffers + api(group: 'net.openhft', name: 'chronicle-map', version: '3.22.9') { exclude group: 'net.openhft', module: 'chronicle-analytics' } - implementation group: 'commons-io', name: 'commons-io', version: '2.4' + implementation group: 'commons-io', name: 'commons-io', version: '2.16.1' implementation group: 'net.jodah', name: 'failsafe', version: versions.failsafe + testImplementation project(':hermes-test-helper') + testImplementation group: 'org.spockframework', name: 'spock-core', version: versions.spock testImplementation group: 'org.apache.groovy', name: 'groovy-json', version: versions.groovy - - testImplementation(group: 'com.jayway.awaitility', name: 'awaitility-groovy', version: '1.7.0') { - exclude group: 'org.codehaus.groovy', module: 'groovy-all' - } - - testImplementation project(':hermes-test-helper') - testImplementation(group: 'com.jayway.awaitility', name: 'awaitility', version: '1.6.1') { - exclude group: 'com.jayway.jsonpath', module: 'json-path' - } + testImplementation group: 'org.awaitility', name: 'awaitility-groovy', version: '4.2.1' + testImplementation group: 'org.awaitility', name: 'awaitility', version: '4.2.1' testImplementation group: 'org.testcontainers', name: 'spock', version: versions.testcontainers testImplementation group: 'org.testcontainers', name: 'kafka', version: versions.testcontainers testRuntimeOnly group: 'org.junit.vintage', name: 'junit-vintage-engine', version: versions.junit_jupiter diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/message/MessageContentTypeEnforcer.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/message/MessageContentTypeEnforcer.java index ad1d7c7913..72dd78f57d 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/message/MessageContentTypeEnforcer.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/message/MessageContentTypeEnforcer.java @@ -1,7 +1,7 @@ package pl.allegro.tech.hermes.frontend.publishing.message; import org.apache.avro.Schema; -import org.apache.commons.lang.StringUtils; +import org.apache.commons.lang3.StringUtils; import pl.allegro.tech.hermes.api.Topic; import pl.allegro.tech.hermes.common.message.wrapper.UnsupportedContentTypeException; import tech.allegro.schema.json2avro.converter.JsonAvroConverter; diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/preview/MessagePreviewFactory.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/preview/MessagePreviewFactory.java index 1087d45e28..fae6b9205e 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/preview/MessagePreviewFactory.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/preview/MessagePreviewFactory.java @@ -1,6 +1,6 @@ package pl.allegro.tech.hermes.frontend.publishing.preview; -import org.apache.commons.lang.ArrayUtils; +import org.apache.commons.lang3.ArrayUtils; import pl.allegro.tech.hermes.domain.topic.preview.MessagePreview; import pl.allegro.tech.hermes.frontend.publishing.message.Message; import pl.allegro.tech.hermes.frontend.publishing.message.MessageToJsonConverter; diff --git a/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducerIntegrationTest.groovy b/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducerIntegrationTest.groovy index 088ac3a204..46b1913121 100644 --- a/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducerIntegrationTest.groovy +++ b/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducerIntegrationTest.groovy @@ -1,6 +1,6 @@ package pl.allegro.tech.hermes.frontend.producer.kafka -import com.jayway.awaitility.Awaitility +import com.codahale.metrics.MetricRegistry import io.micrometer.core.instrument.simple.SimpleMeterRegistry import org.apache.commons.lang3.tuple.ImmutablePair import org.apache.kafka.clients.admin.AdminClient @@ -9,6 +9,7 @@ import org.apache.kafka.clients.consumer.KafkaConsumer import org.apache.kafka.clients.consumer.OffsetAndMetadata import org.apache.kafka.clients.producer.KafkaProducer import org.apache.kafka.common.TopicPartition +import org.awaitility.Awaitility import org.testcontainers.containers.KafkaContainer import org.testcontainers.containers.wait.strategy.Wait import org.testcontainers.spock.Testcontainers diff --git a/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/publishing/metric/ThroughputRegistryTest.groovy b/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/publishing/metric/ThroughputRegistryTest.groovy index 04cdf08154..b5d5d397a0 100644 --- a/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/publishing/metric/ThroughputRegistryTest.groovy +++ b/hermes-frontend/src/test/groovy/pl/allegro/tech/hermes/frontend/publishing/metric/ThroughputRegistryTest.groovy @@ -1,9 +1,9 @@ package pl.allegro.tech.hermes.frontend.publishing.metric import com.codahale.metrics.MetricRegistry -import com.jayway.awaitility.Duration import io.micrometer.core.instrument.simple.SimpleMeterRegistry import org.assertj.core.api.Assertions +import org.awaitility.Awaitility import pl.allegro.tech.hermes.api.Topic import pl.allegro.tech.hermes.common.metric.MetricsFacade import pl.allegro.tech.hermes.frontend.metric.ThroughputMeter @@ -11,7 +11,8 @@ import pl.allegro.tech.hermes.frontend.metric.ThroughputRegistry import spock.lang.Shared import spock.lang.Specification -import static com.jayway.awaitility.Awaitility.await +import java.time.Duration + import static pl.allegro.tech.hermes.test.helper.builder.TopicBuilder.topic class ThroughputRegistryTest extends Specification { @@ -36,9 +37,9 @@ class ThroughputRegistryTest extends Specification { meter = throughputRegistry.forTopic(topicA.getName()) then: "throughput is preserved" - await().atMost(Duration.TEN_SECONDS).until({ + Awaitility.await().atMost(Duration.ofSeconds(10)).untilAsserted { Assertions.assertThat(meter.oneMinuteRate).isGreaterThan(0.0d) - }) + } } def "global throughput should be shared for all topics"() { @@ -53,13 +54,14 @@ class ThroughputRegistryTest extends Specification { topicCMeter.increment(1024) then: "global throughput is a sum of topic throughput" - await().atMost(Duration.TEN_SECONDS).until({ + Awaitility.await().atMost(Duration.ofSeconds(10)).untilAsserted { def topicAValue = topicBMeter.oneMinuteRate def topicBValue = topicCMeter.oneMinuteRate def globalValue = throughputRegistry.globalThroughputOneMinuteRate Assertions.assertThat(topicAValue).isGreaterThan(0.0d) Assertions.assertThat(topicBValue).isGreaterThan(0.0d) Assertions.assertThat(globalValue).isGreaterThan(topicAValue) - }) + } } + } diff --git a/hermes-frontend/src/test/java/pl/allegro/tech/hermes/frontend/buffer/BackupMessagesLoaderTest.java b/hermes-frontend/src/test/java/pl/allegro/tech/hermes/frontend/buffer/BackupMessagesLoaderTest.java index c530cbc7f6..528b0a0e38 100644 --- a/hermes-frontend/src/test/java/pl/allegro/tech/hermes/frontend/buffer/BackupMessagesLoaderTest.java +++ b/hermes-frontend/src/test/java/pl/allegro/tech/hermes/frontend/buffer/BackupMessagesLoaderTest.java @@ -35,8 +35,8 @@ import static java.time.LocalDateTime.now; import static java.time.ZoneOffset.UTC; import static org.assertj.core.api.Assertions.assertThat; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.eq; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; diff --git a/hermes-frontend/src/test/java/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducerTest.java b/hermes-frontend/src/test/java/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducerTest.java index 67bbfcd162..3866830892 100644 --- a/hermes-frontend/src/test/java/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducerTest.java +++ b/hermes-frontend/src/test/java/pl/allegro/tech/hermes/frontend/producer/kafka/LocalDatacenterMessageProducerTest.java @@ -10,7 +10,7 @@ import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Mock; -import org.mockito.runners.MockitoJUnitRunner; +import org.mockito.junit.MockitoJUnitRunner; import pl.allegro.tech.hermes.api.Topic; import pl.allegro.tech.hermes.common.kafka.HTTPHeadersPropagationAsKafkaHeadersProperties; import pl.allegro.tech.hermes.common.kafka.KafkaNamesMapper; @@ -34,10 +34,10 @@ import java.util.concurrent.atomic.AtomicBoolean; import static com.google.common.base.Charsets.UTF_8; -import static com.jayway.awaitility.Awaitility.await; import static java.util.Collections.emptyList; import static java.util.Collections.emptyMap; import static org.assertj.core.api.Assertions.assertThat; +import static org.awaitility.Awaitility.await; import static pl.allegro.tech.hermes.test.helper.builder.TopicBuilder.topic; @RunWith(MockitoJUnitRunner.class) diff --git a/hermes-management/build.gradle b/hermes-management/build.gradle index f1105eadcb..f6b966dbdd 100644 --- a/hermes-management/build.gradle +++ b/hermes-management/build.gradle @@ -15,27 +15,25 @@ dependencies { api group: 'org.springframework.boot', name: 'spring-boot-starter-web', version: versions.spring api group: 'org.springframework.boot', name: 'spring-boot-starter-actuator', version: versions.spring api group: 'org.springframework.boot', name: 'spring-boot-starter-jersey', version: versions.spring - implementation group: 'net.sf.jopt-simple', name: 'jopt-simple', version: '4.8' + implementation group: 'net.sf.jopt-simple', name: 'jopt-simple', version: '5.0.4' implementation group: 'org.glassfish.jersey.ext', name: 'jersey-mvc-freemarker', version: versions.jersey - implementation (group: 'io.swagger', name: 'swagger-jersey2-jaxrs', version: '1.6.3') { + implementation (group: 'io.swagger', name: 'swagger-jersey2-jaxrs', version: '1.6.14') { exclude group: 'javax.validation', module: 'validation-api' } implementation group: 'org.apache.kafka', name: 'kafka-clients', version: versions.kafka - implementation group: 'commons-codec', name: 'commons-codec', version: '1.9' - implementation (group: 'com.github.fge', name: 'json-schema-validator', version: '2.2.6') { - exclude group: 'net.sf.jopt-simple' - } + implementation group: 'commons-codec', name: 'commons-codec', version: '1.16.1' + implementation group: 'com.github.java-json-tools', name: 'json-schema-validator', version: '2.2.14' implementation group: 'commons-jxpath', name: 'commons-jxpath', version: '1.3' - implementation group: 'org.apache.httpcomponents.client5', name: 'httpclient5', version: '5.2.1' + implementation group: 'org.apache.httpcomponents.client5', name: 'httpclient5', version: '5.3.1' - api group: 'org.javers', name: 'javers-core', version: '5.2.5' + api group: 'org.javers', name: 'javers-core', version: '7.4.2' implementation group: 'com.fasterxml.jackson.datatype', name: 'jackson-datatype-jsr310', version: versions.jackson - implementation group: 'commons-io', name: 'commons-io', version: '2.6' + implementation group: 'commons-io', name: 'commons-io', version: '2.16.1' testImplementation project(':hermes-test-helper') testImplementation group: 'org.springframework.boot', name: 'spring-boot-starter-test', version: versions.spring diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/api/auth/AllowAllSecurityProvider.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/api/auth/AllowAllSecurityProvider.java index 214e99c5fd..395429516c 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/api/auth/AllowAllSecurityProvider.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/api/auth/AllowAllSecurityProvider.java @@ -2,7 +2,7 @@ import jakarta.ws.rs.container.ContainerRequestContext; import jakarta.ws.rs.core.SecurityContext; -import org.apache.commons.lang.NotImplementedException; +import org.apache.commons.lang3.NotImplementedException; import java.security.Principal; diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/filtering/FilteringService.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/filtering/FilteringService.java index 0cf4e1a8f0..09cb1148f9 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/filtering/FilteringService.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/filtering/FilteringService.java @@ -1,7 +1,7 @@ package pl.allegro.tech.hermes.management.domain.filtering; import org.apache.avro.Schema; -import org.apache.commons.lang.exception.ExceptionUtils; +import org.apache.commons.lang3.exception.ExceptionUtils; import org.springframework.stereotype.Component; import pl.allegro.tech.hermes.api.MessageFiltersVerificationInput; import pl.allegro.tech.hermes.api.MessageFiltersVerificationResult; diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/graphite/RestTemplateGraphiteClient.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/graphite/RestTemplateGraphiteClient.java new file mode 100644 index 0000000000..e69de29bb2 diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridSubscriptionMetricsRepository.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridSubscriptionMetricsRepository.java index 8b82e66be7..712a5bba17 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridSubscriptionMetricsRepository.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridSubscriptionMetricsRepository.java @@ -14,7 +14,8 @@ import java.util.function.Supplier; -import static org.apache.commons.lang.exception.ExceptionUtils.getRootCauseMessage; +import static org.apache.commons.lang3.exception.ExceptionUtils.getRootCauseMessage; + @Component public class HybridSubscriptionMetricsRepository implements SubscriptionMetricsRepository { diff --git a/hermes-metrics/build.gradle b/hermes-metrics/build.gradle index d44403b978..f06e7f39ad 100644 --- a/hermes-metrics/build.gradle +++ b/hermes-metrics/build.gradle @@ -4,7 +4,7 @@ plugins { dependencies { api group: 'io.dropwizard.metrics', name: 'metrics-core', version: versions.dropwizard_metrics - api group: 'commons-lang', name: 'commons-lang', version: '2.6' + api group: 'org.apache.commons', name: 'commons-text', version: '1.12.0' api group: 'io.micrometer', name: 'micrometer-core', version: versions.micrometer_metrics testImplementation group: 'org.spockframework', name: 'spock-core', version: versions.spock diff --git a/hermes-metrics/src/main/java/pl/allegro/tech/hermes/metrics/PathsCompiler.java b/hermes-metrics/src/main/java/pl/allegro/tech/hermes/metrics/PathsCompiler.java index 63c6ae4ca8..c69e333740 100644 --- a/hermes-metrics/src/main/java/pl/allegro/tech/hermes/metrics/PathsCompiler.java +++ b/hermes-metrics/src/main/java/pl/allegro/tech/hermes/metrics/PathsCompiler.java @@ -1,6 +1,7 @@ package pl.allegro.tech.hermes.metrics; -import org.apache.commons.lang.text.StrBuilder; + +import org.apache.commons.text.TextStringBuilder; public class PathsCompiler { @@ -22,7 +23,7 @@ public String compile(String path) { } public String compile(String path, PathContext context) { - StrBuilder pathBuilder = new StrBuilder(path); + TextStringBuilder pathBuilder = new TextStringBuilder(path); context.getGroup().ifPresent(g -> pathBuilder.replaceAll(GROUP, g)); context.getTopic().ifPresent(t -> pathBuilder.replaceAll(TOPIC, t)); diff --git a/hermes-mock/build.gradle b/hermes-mock/build.gradle index e49c6a112b..f0cd9e2906 100644 --- a/hermes-mock/build.gradle +++ b/hermes-mock/build.gradle @@ -5,9 +5,9 @@ plugins { } dependencies { - implementation group: 'junit', name: 'junit', version: '4.11' + implementation group: 'junit', name: 'junit', version: '4.13.2' api group: 'org.wiremock', name: 'wiremock-standalone', version: versions.wiremock - implementation group: 'com.jayway.awaitility', name: 'awaitility', version: '1.6.1' + implementation group: 'org.awaitility', name: 'awaitility', version: '4.2.1' api group: 'org.apache.avro', name: 'avro', version: versions.avro implementation group: 'tech.allegro.schema.json2avro', name: 'converter', version: versions.json2avro implementation group: 'org.junit.jupiter', name: 'junit-jupiter-api', version: versions.junit_jupiter @@ -18,9 +18,9 @@ dependencies { testImplementation group: 'org.apache.groovy', name: 'groovy-json', version: versions.groovy testImplementation group: 'org.glassfish.jersey.core', name: 'jersey-client', version: versions.jersey testImplementation group: 'org.glassfish.jersey.inject', name: 'jersey-hk2', version: versions.jersey - testImplementation group: 'org.junit.jupiter', name :'junit-jupiter', version: versions.junit_jupiter + testImplementation group: 'org.junit.jupiter', name: 'junit-jupiter', version: versions.junit_jupiter testImplementation group: 'org.springframework', name: 'spring-test', version: versions.spring_web - testRuntimeOnly group: 'org.junit.vintage', name : 'junit-vintage-engine', version: versions.junit_jupiter + testRuntimeOnly group: 'org.junit.vintage', name: 'junit-vintage-engine', version: versions.junit_jupiter } test { diff --git a/hermes-mock/src/main/java/pl/allegro/tech/hermes/mock/HermesMockExpect.java b/hermes-mock/src/main/java/pl/allegro/tech/hermes/mock/HermesMockExpect.java index 1872ad4364..7f503235a3 100644 --- a/hermes-mock/src/main/java/pl/allegro/tech/hermes/mock/HermesMockExpect.java +++ b/hermes-mock/src/main/java/pl/allegro/tech/hermes/mock/HermesMockExpect.java @@ -1,15 +1,15 @@ package pl.allegro.tech.hermes.mock; -import com.jayway.awaitility.core.ConditionTimeoutException; import org.apache.avro.Schema; +import org.awaitility.core.ConditionTimeoutException; import java.util.List; import java.util.function.Predicate; import java.util.function.Supplier; -import static com.jayway.awaitility.Awaitility.await; import static java.util.concurrent.TimeUnit.SECONDS; import static java.util.stream.Collectors.toList; +import static org.awaitility.Awaitility.await; public class HermesMockExpect { private final HermesMockHelper hermesMockHelper; @@ -64,7 +64,7 @@ private void assertMessages(int count, Supplier> messages) { private void expectMessages(String topicName, int count) { try { - await().atMost(awaitSeconds, SECONDS).until(() -> hermesMockHelper.verifyRequest(count, topicName)); + await().atMost(awaitSeconds, SECONDS).untilAsserted(() -> hermesMockHelper.verifyRequest(count, topicName)); } catch (ConditionTimeoutException ex) { throw new HermesMockException("Hermes mock did not receive " + count + " messages.", ex); } diff --git a/hermes-test-helper/build.gradle b/hermes-test-helper/build.gradle index f02b8c2241..b24683e815 100644 --- a/hermes-test-helper/build.gradle +++ b/hermes-test-helper/build.gradle @@ -10,7 +10,7 @@ dependencies { implementation group: 'org.glassfish.jersey.core', name: 'jersey-client', version: versions.jersey implementation group: 'org.glassfish.jersey.inject', name: 'jersey-hk2', version: versions.jersey implementation group: 'org.glassfish.jersey.ext', name: 'jersey-proxy-client', version: versions.jersey - api group: 'commons-io', name: 'commons-io', version: '2.4' + api group: 'commons-io', name: 'commons-io', version: '2.16.1' api group: 'org.wiremock', name: 'wiremock-standalone', version: versions.wiremock api (group: 'org.apache.curator', name: 'curator-test', version: versions.curator) { exclude module: 'slf4j-log4j12' @@ -24,14 +24,13 @@ dependencies { exclude module: 'slf4j-log4j12' exclude module: 'log4j' } - implementation group: 'com.google.code.findbugs', name: 'annotations', version: '3.0.0' - implementation group: 'com.jayway.awaitility', name: 'awaitility', version: '1.6.1' + implementation group: 'com.github.spotbugs', name: 'spotbugs-annotations', version: '4.8.4' + implementation group: 'org.awaitility', name: 'awaitility-groovy', version: '4.2.1' implementation group: 'org.assertj', name: 'assertj-core', version: versions.assertj - api group: 'net.javacrumbs.json-unit', name: 'json-unit-fluent', version: '1.5.5' - implementation group: 'junit', name: 'junit', version: '4.11' - api group: 'org.testng', name: 'testng', version: '7.4.0' - implementation group: 'org.apache.httpcomponents', name: 'httpclient', version: '4.5.13' - implementation group: 'com.jayway.jsonpath', name: 'json-path', version: '2.5.0' + api group: 'net.javacrumbs.json-unit', name: 'json-unit-fluent', version: '3.2.7' + implementation group: 'org.junit.jupiter', name: 'junit-jupiter-api', version: '5.10.2' + implementation group: 'org.apache.httpcomponents.client5', name: 'httpclient5', version: '5.3.1' + implementation group: 'com.jayway.jsonpath', name: 'json-path', version: '2.9.0' implementation group: 'com.fasterxml.jackson.datatype', name: 'jackson-datatype-jsr310', version: versions.jackson implementation group: 'org.springframework', name: 'spring-test', version: versions.spring_web implementation group: 'org.springframework', name: 'spring-webflux', version: versions.spring_web diff --git a/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/client/integration/HermesInitHelper.java b/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/client/integration/HermesInitHelper.java index fb47861480..d90c793d07 100644 --- a/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/client/integration/HermesInitHelper.java +++ b/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/client/integration/HermesInitHelper.java @@ -1,14 +1,14 @@ package pl.allegro.tech.hermes.test.helper.client.integration; -import com.jayway.awaitility.Duration; +import java.time.Duration; import pl.allegro.tech.hermes.api.Group; import pl.allegro.tech.hermes.api.OAuthProvider; import pl.allegro.tech.hermes.api.Subscription; import pl.allegro.tech.hermes.api.Topic; import pl.allegro.tech.hermes.api.TopicWithSchema; -import static com.jayway.awaitility.Awaitility.waitAtMost; import static org.assertj.core.api.Assertions.assertThat; +import static org.awaitility.Awaitility.waitAtMost; public class HermesInitHelper { @@ -61,13 +61,13 @@ public Group createGroup(Group group) { } private void waitUntilGroupCreated(String groupName) { - waitAtMost(Duration.ONE_MINUTE) + waitAtMost(Duration.ofMinutes(1)) .until(() -> managementTestClient.getGroups().contains(groupName)); } private void waitUntilTopicCreated(String topicQualifiedName) { - waitAtMost(Duration.ONE_MINUTE) - .until(() -> managementTestClient.getTopic(topicQualifiedName) + waitAtMost(Duration.ofMinutes(1)) + .untilAsserted(() -> managementTestClient.getTopic(topicQualifiedName) .expectStatus() .is2xxSuccessful()); } @@ -81,8 +81,8 @@ public Subscription createSubscription(Subscription subscription) { } public void waitUntilSubscriptionIsActive(Subscription subscription) { - waitAtMost(Duration.TEN_SECONDS) - .until(() -> { + waitAtMost(Duration.ofSeconds(10)) + .untilAsserted(() -> { Subscription sub = managementTestClient.getSubscription(subscription.getQualifiedTopicName(), subscription.getName()) .expectStatus() .is2xxSuccessful() diff --git a/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/client/integration/HermesTestClient.java b/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/client/integration/HermesTestClient.java index 43c183f27a..7ac2fcd6df 100644 --- a/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/client/integration/HermesTestClient.java +++ b/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/client/integration/HermesTestClient.java @@ -1,7 +1,8 @@ package pl.allegro.tech.hermes.test.helper.client.integration; -import com.jayway.awaitility.Duration; import jakarta.ws.rs.core.Response; +import java.time.Duration; +import org.assertj.core.api.Assertions; import org.springframework.http.HttpHeaders; import org.springframework.test.web.reactive.server.WebTestClient; import org.springframework.util.MultiValueMap; @@ -21,7 +22,8 @@ import java.io.IOException; import java.util.List; -import static com.jayway.awaitility.Awaitility.waitAtMost; +import static org.assertj.core.api.Assertions.assertThat; +import static org.awaitility.Awaitility.waitAtMost; import static pl.allegro.tech.hermes.test.helper.endpoint.TimeoutAdjuster.adjust; public class HermesTestClient { @@ -61,7 +63,7 @@ public WebTestClient.ResponseSpec saveSchema(String topicQualifiedName, String s public void ensureSchemaSaved(String topicQualifiedName, boolean validate, String schema) { managementTestClient.saveSchema(topicQualifiedName, validate, schema) .expectStatus().isCreated(); - waitAtMost(adjust(Duration.ONE_MINUTE)).until(() -> + waitAtMost(adjust(Duration.ofMinutes(1))).untilAsserted(() -> managementTestClient.getSchema(topicQualifiedName).expectStatus().isOk() ); } @@ -106,33 +108,37 @@ public WebTestClient.ResponseSpec suspendSubscription(Topic topic, String subscr } public void waitUntilSubscriptionActivated(String topicQualifiedName, String subscriptionName) { - waitAtMost(Duration.TEN_SECONDS) - .until(() -> { - managementTestClient.getSubscription(topicQualifiedName, subscriptionName) + waitAtMost(Duration.ofSeconds(10)) + .untilAsserted(() -> { + assertThat(managementTestClient.getSubscription(topicQualifiedName, subscriptionName) .expectStatus() .is2xxSuccessful() .expectBody(Subscription.class) - .returnResult().getResponseBody().getState().equals(Subscription.State.ACTIVE); - managementTestClient.getConsumerGroupsDescription(topicQualifiedName, subscriptionName).expectBodyList(ConsumerGroup.class).returnResult().getResponseBody() + .returnResult().getResponseBody().getState()) + .isEqualTo(Subscription.State.ACTIVE); + assertThat(managementTestClient.getConsumerGroupsDescription(topicQualifiedName, subscriptionName) + .expectBodyList(ConsumerGroup.class).returnResult().getResponseBody() .get(0) - .getState() - .equals("Stable"); + .getState()) + .isEqualTo("Stable"); } ); } public void waitUntilSubscriptionSuspended(String topicQualifiedName, String subscriptionName) { - waitAtMost(Duration.TEN_SECONDS) - .until(() -> { - managementTestClient.getSubscription(topicQualifiedName, subscriptionName) + waitAtMost(Duration.ofSeconds(10)) + .untilAsserted(() -> { + assertThat(managementTestClient.getSubscription(topicQualifiedName, subscriptionName) .expectStatus() .is2xxSuccessful() .expectBody(Subscription.class) - .returnResult().getResponseBody().getState().equals(Subscription.State.SUSPENDED); - managementTestClient.getConsumerGroupsDescription(topicQualifiedName, subscriptionName).expectBodyList(ConsumerGroup.class).returnResult().getResponseBody() + .returnResult().getResponseBody().getState()) + .isEqualTo(Subscription.State.SUSPENDED); + assertThat(managementTestClient.getConsumerGroupsDescription(topicQualifiedName, subscriptionName) + .expectBodyList(ConsumerGroup.class).returnResult().getResponseBody() .get(0) - .getState() - .equals("Empty"); + .getState()) + .isEqualTo("Empty"); } ); } diff --git a/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/endpoint/RemoteServiceEndpoint.java b/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/endpoint/RemoteServiceEndpoint.java index da4c479bc6..64adf9918a 100644 --- a/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/endpoint/RemoteServiceEndpoint.java +++ b/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/endpoint/RemoteServiceEndpoint.java @@ -4,7 +4,7 @@ import com.github.tomakehurst.wiremock.client.WireMock; import com.github.tomakehurst.wiremock.verification.LoggedRequest; import com.google.common.collect.Iterables; -import com.jayway.awaitility.Duration; +import java.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import pl.allegro.tech.hermes.test.helper.message.TestMessage; @@ -22,10 +22,10 @@ import static com.github.tomakehurst.wiremock.client.WireMock.post; import static com.github.tomakehurst.wiremock.client.WireMock.urlEqualTo; import static com.github.tomakehurst.wiremock.stubbing.Scenario.STARTED; -import static com.jayway.awaitility.Awaitility.await; import static java.util.stream.Collectors.toList; import static jakarta.ws.rs.core.Response.Status.MOVED_PERMANENTLY; import static org.assertj.core.api.Assertions.assertThat; +import static org.awaitility.Awaitility.await; import static pl.allegro.tech.hermes.test.helper.endpoint.TimeoutAdjuster.adjust; public class RemoteServiceEndpoint { @@ -77,9 +77,9 @@ public void expectMessages(List messages) { receivedRequests.clear(); expectedMessages = messages; messages.forEach(m -> listener - .register( - post(urlEqualTo(path)) - .willReturn(aResponse().withStatus(returnedStatusCode).withFixedDelay(delay)))); + .register( + post(urlEqualTo(path)) + .willReturn(aResponse().withStatus(returnedStatusCode).withFixedDelay(delay)))); } public void redirectMessage(String message) { @@ -88,10 +88,10 @@ public void redirectMessage(String message) { expectedMessages = Collections.singletonList(message); listener.register( - post(urlEqualTo(path)) - .willReturn(aResponse() - .withStatus(MOVED_PERMANENTLY.getStatusCode()) - .withHeader("Location", "http://localhost:" + service.port()))); + post(urlEqualTo(path)) + .willReturn(aResponse() + .withStatus(MOVED_PERMANENTLY.getStatusCode()) + .withHeader("Location", "http://localhost:" + service.port()))); } public void retryMessage(String message, int delay) { @@ -149,7 +149,7 @@ public void setReturnedStatusCode(int statusCode) { public void waitUntilReceived(long seconds) { logger.info("Expecting to receive {} messages", expectedMessages.size()); - await().atMost(adjust(new Duration(seconds, TimeUnit.SECONDS))).until(() -> + await().atMost(adjust(Duration.ofSeconds(seconds))).untilAsserted(() -> assertThat(receivedRequests.size()).isGreaterThanOrEqualTo(expectedMessages.size())); synchronized (receivedRequests) { assertThat(receivedRequests.stream().map(LoggedRequest::getBodyAsString).collect(toList())).containsAll(expectedMessages); @@ -157,7 +157,8 @@ public void waitUntilReceived(long seconds) { } public void waitUntilReceived(long seconds, int numberOfExpectedMessages) { - waitUntilReceived(seconds, numberOfExpectedMessages, body -> {}); + waitUntilReceived(seconds, numberOfExpectedMessages, body -> { + }); } public void waitUntilReceived(Consumer requestBodyConsumer) { @@ -170,7 +171,7 @@ public void waitUntilRequestReceived(Consumer requestConsumer) { public void waitUntilReceived(long seconds, int numberOfExpectedMessages, Consumer requestBodyConsumer) { logger.info("Expecting to receive {} messages", numberOfExpectedMessages); - await().atMost(adjust(new Duration(seconds, TimeUnit.SECONDS))).until(() -> + await().atMost(adjust(Duration.ofSeconds(seconds))).untilAsserted(() -> assertThat(receivedRequests.size()).isGreaterThanOrEqualTo(numberOfExpectedMessages)); synchronized (receivedRequests) { receivedRequests.forEach(requestBodyConsumer); @@ -179,7 +180,7 @@ public void waitUntilReceived(long seconds, int numberOfExpectedMessages, Consum public void waitUntilReceived(Duration duration, int numberOfExpectedMessages, Consumer requestBodyConsumer) { logger.info("Expecting to receive {} messages", numberOfExpectedMessages); - await().atMost(duration).until(() -> assertThat(receivedRequests.size()).isEqualTo(numberOfExpectedMessages)); + await().atMost(duration).untilAsserted(() -> assertThat(receivedRequests.size()).isEqualTo(numberOfExpectedMessages)); synchronized (receivedRequests) { receivedRequests.forEach(requestBodyConsumer); } diff --git a/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/endpoint/TimeoutAdjuster.java b/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/endpoint/TimeoutAdjuster.java index 3a94138ee9..2d06f09c44 100644 --- a/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/endpoint/TimeoutAdjuster.java +++ b/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/endpoint/TimeoutAdjuster.java @@ -1,8 +1,6 @@ package pl.allegro.tech.hermes.test.helper.endpoint; -import com.jayway.awaitility.Duration; - -import java.util.concurrent.TimeUnit; +import java.time.Duration; import static java.lang.Double.parseDouble; @@ -14,6 +12,6 @@ public static long adjust(long value) { } public static Duration adjust(Duration duration) { - return new Duration(adjust(duration.getValueInMS()), TimeUnit.MILLISECONDS); + return Duration.ofMillis(adjust(duration.toMillis())); } } diff --git a/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/oauth/server/OAuthTestServer.java b/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/oauth/server/OAuthTestServer.java index e7fdfafaa6..acb3b9e767 100644 --- a/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/oauth/server/OAuthTestServer.java +++ b/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/oauth/server/OAuthTestServer.java @@ -11,7 +11,7 @@ import static com.github.tomakehurst.wiremock.client.WireMock.equalTo; import static com.github.tomakehurst.wiremock.client.WireMock.post; import static com.github.tomakehurst.wiremock.client.WireMock.urlPathEqualTo; -import static org.apache.http.HttpStatus.SC_OK; +import static org.apache.hc.core5.http.HttpStatus.SC_OK; public class OAuthTestServer { diff --git a/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/retry/Retry.java b/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/retry/Retry.java deleted file mode 100644 index b0411a4235..0000000000 --- a/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/retry/Retry.java +++ /dev/null @@ -1,28 +0,0 @@ -package pl.allegro.tech.hermes.test.helper.retry; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.testng.IRetryAnalyzer; -import org.testng.ITestResult; - -import static java.lang.Integer.parseInt; - -public class Retry implements IRetryAnalyzer { - private int retryCount = 0; - private static final int maxRetryCount = parseInt(System.getProperty("tests.retry.count", "2")); - private static final Logger logger = LoggerFactory.getLogger(Retry.class); - - @Override - public boolean retry(ITestResult result) { - logger.error("Retrying test {}.{}", result.getTestClass().getName(), result.getMethod().getMethodName(), result.getThrowable()); - if (isRetryAvailable()) { - retryCount++; - return true; - } - return false; - } - - public boolean isRetryAvailable() { - return retryCount < maxRetryCount; - } -} diff --git a/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/retry/RetryListener.java b/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/retry/RetryListener.java deleted file mode 100644 index 7049dab36c..0000000000 --- a/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/retry/RetryListener.java +++ /dev/null @@ -1,18 +0,0 @@ -package pl.allegro.tech.hermes.test.helper.retry; - -import org.testng.IRetryAnalyzer; -import org.testng.ITestResult; -import org.testng.Reporter; -import org.testng.TestListenerAdapter; - -public class RetryListener extends TestListenerAdapter { - - @Override - public void onTestFailure(ITestResult result) { - IRetryAnalyzer analyzer = result.getMethod().getRetryAnalyzer(result); - if (analyzer instanceof Retry) { - result.setStatus(((Retry) analyzer).isRetryAvailable() ? ITestResult.SKIP : ITestResult.FAILURE); - Reporter.setCurrentTestResult(result); - } - } -} diff --git a/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/zookeeper/ZookeeperWaiter.java b/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/zookeeper/ZookeeperWaiter.java index 937caffeaa..3e9fcb0bea 100644 --- a/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/zookeeper/ZookeeperWaiter.java +++ b/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/zookeeper/ZookeeperWaiter.java @@ -5,7 +5,8 @@ import java.util.concurrent.TimeUnit; -import static com.jayway.awaitility.Awaitility.await; +import static org.awaitility.Awaitility.await; + public class ZookeeperWaiter { diff --git a/hermes-tracker-elasticsearch/build.gradle b/hermes-tracker-elasticsearch/build.gradle index 9641184d82..6e18bada83 100644 --- a/hermes-tracker-elasticsearch/build.gradle +++ b/hermes-tracker-elasticsearch/build.gradle @@ -4,15 +4,16 @@ plugins { dependencies { implementation project(':hermes-common') implementation project(':hermes-tracker') - implementation 'org.slf4j:slf4j-api:2.0.4' + implementation 'org.slf4j:slf4j-api:2.0.13' + // TODO: client has to have the same major version as backend: https://www.elastic.co/guide/en/elasticsearch/client/java-api/current/client.html + // we need to use test-containers for elasticsearch server to able to use newer client. api group: 'org.elasticsearch.client', name: 'transport', version: '6.8.12' testImplementation project(path: ":hermes-tracker", configuration: "testArtifacts") testImplementation project(path: ":hermes-test-helper") testImplementation group: 'org.spockframework', name: 'spock-core', version: versions.spock testImplementation group: 'org.spockframework', name: 'spock-junit4', version: versions.spock + // this should be migrated to testcontainers testImplementation 'pl.allegro.tech:embedded-elasticsearch:2.10.0' testRuntimeOnly group: 'org.junit.vintage', name: 'junit-vintage-engine', version: versions.junit_jupiter -} - -test.useTestNG() \ No newline at end of file +} \ No newline at end of file diff --git a/hermes-tracker-elasticsearch/src/main/java/pl/allegro/tech/hermes/tracker/elasticsearch/management/MultiElasticsearchLogRepository.java b/hermes-tracker-elasticsearch/src/main/java/pl/allegro/tech/hermes/tracker/elasticsearch/management/MultiElasticsearchLogRepository.java index b49e5d0afc..d5aaf7a0ef 100644 --- a/hermes-tracker-elasticsearch/src/main/java/pl/allegro/tech/hermes/tracker/elasticsearch/management/MultiElasticsearchLogRepository.java +++ b/hermes-tracker-elasticsearch/src/main/java/pl/allegro/tech/hermes/tracker/elasticsearch/management/MultiElasticsearchLogRepository.java @@ -12,7 +12,7 @@ public class MultiElasticsearchLogRepository implements LogRepository, LogSchemaAware { - private List elasticsearchLogRepositories; + private final List elasticsearchLogRepositories; public MultiElasticsearchLogRepository(List elasticClients) { elasticsearchLogRepositories = elasticClients.stream() diff --git a/hermes-tracker-elasticsearch/src/test/groovy/pl/allegro/tech/hermes/tracker/elasticsearch/DailyIndexFactoryTest.groovy b/hermes-tracker-elasticsearch/src/test/groovy/pl/allegro/tech/hermes/tracker/elasticsearch/DailyIndexFactoryTest.groovy index bd4d0adcd1..c76f236fbe 100644 --- a/hermes-tracker-elasticsearch/src/test/groovy/pl/allegro/tech/hermes/tracker/elasticsearch/DailyIndexFactoryTest.groovy +++ b/hermes-tracker-elasticsearch/src/test/groovy/pl/allegro/tech/hermes/tracker/elasticsearch/DailyIndexFactoryTest.groovy @@ -9,7 +9,6 @@ import java.time.Clock import static java.time.LocalDate.of import static java.time.ZoneId.systemDefault import static java.time.ZoneOffset.UTC -import static org.assertj.core.api.Assertions.assertThat class DailyIndexFactoryTest extends Specification { @@ -18,7 +17,7 @@ class DailyIndexFactoryTest extends Specification { def "should create daily index"() { expect: - assertThat(indexFactory.createIndex()).endsWith("_2000_01_01") + indexFactory.createIndex().endsWith("_2000_01_01") where: indexFactory << [new FrontendDailyIndexFactory(clock), new ConsumersDailyIndexFactory(clock)] diff --git a/hermes-tracker-elasticsearch/src/test/java/pl/allegro/tech/hermes/tracker/elasticsearch/consumers/ConsumersElasticsearchLogRepositoryTest.java b/hermes-tracker-elasticsearch/src/test/java/pl/allegro/tech/hermes/tracker/elasticsearch/consumers/ConsumersElasticsearchLogRepositoryTest.java index 3ed0cb3860..20d0aa0e6d 100644 --- a/hermes-tracker-elasticsearch/src/test/java/pl/allegro/tech/hermes/tracker/elasticsearch/consumers/ConsumersElasticsearchLogRepositoryTest.java +++ b/hermes-tracker-elasticsearch/src/test/java/pl/allegro/tech/hermes/tracker/elasticsearch/consumers/ConsumersElasticsearchLogRepositoryTest.java @@ -4,8 +4,8 @@ import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.index.query.BoolQueryBuilder; import org.elasticsearch.index.query.QueryBuilder; -import org.testng.annotations.AfterSuite; -import org.testng.annotations.BeforeSuite; +import org.junit.AfterClass; +import org.junit.BeforeClass; import pl.allegro.tech.hermes.api.SentMessageTraceStatus; import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.tracker.consumers.AbstractLogRepositoryTest; @@ -17,12 +17,12 @@ import pl.allegro.tech.hermes.tracker.elasticsearch.frontend.FrontendIndexFactory; import java.time.Clock; +import java.time.Duration; import java.time.LocalDate; import java.time.ZoneId; import java.time.ZoneOffset; -import static com.jayway.awaitility.Awaitility.await; -import static com.jayway.awaitility.Duration.ONE_MINUTE; +import static org.awaitility.Awaitility.await; import static org.elasticsearch.index.query.QueryBuilders.boolQuery; import static org.elasticsearch.index.query.QueryBuilders.termQuery; @@ -38,17 +38,17 @@ public class ConsumersElasticsearchLogRepositoryTest extends AbstractLogReposito ); - private static ElasticsearchResource elasticsearch = new ElasticsearchResource(); - private SchemaManager schemaManager; + private static final ElasticsearchResource elasticsearch = new ElasticsearchResource(); + private static SchemaManager schemaManager; - @BeforeSuite - public void before() throws Throwable { + @BeforeClass + public static void beforeAll() throws Throwable { elasticsearch.before(); schemaManager = new SchemaManager(elasticsearch.client(), frontendIndexFactory, indexFactory, false); } - @AfterSuite - public void after() { + @AfterClass + public static void afterAll() { elasticsearch.after(); } @@ -73,7 +73,7 @@ protected void awaitUntilBatchMessageIsPersisted(String topic, String subscripti } private void awaitUntilPersisted(QueryBuilder query) { - await().atMost(ONE_MINUTE).until(() -> { + await().atMost(Duration.ofMinutes(1)).until(() -> { SearchResponse response = elasticsearch.client().prepareSearch(indexFactory.createIndex()) .setTypes(SchemaManager.SENT_TYPE) .setQuery(query) diff --git a/hermes-tracker-elasticsearch/src/test/java/pl/allegro/tech/hermes/tracker/elasticsearch/frontend/FrontendElasticsearchLogRepositoryTest.java b/hermes-tracker-elasticsearch/src/test/java/pl/allegro/tech/hermes/tracker/elasticsearch/frontend/FrontendElasticsearchLogRepositoryTest.java index 294bc428ff..329824b414 100644 --- a/hermes-tracker-elasticsearch/src/test/java/pl/allegro/tech/hermes/tracker/elasticsearch/frontend/FrontendElasticsearchLogRepositoryTest.java +++ b/hermes-tracker-elasticsearch/src/test/java/pl/allegro/tech/hermes/tracker/elasticsearch/frontend/FrontendElasticsearchLogRepositoryTest.java @@ -4,8 +4,8 @@ import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.index.query.BoolQueryBuilder; import org.elasticsearch.index.query.QueryBuilder; -import org.testng.annotations.AfterSuite; -import org.testng.annotations.BeforeSuite; +import org.junit.AfterClass; +import org.junit.BeforeClass; import pl.allegro.tech.hermes.api.PublishedMessageTraceStatus; import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.tracker.elasticsearch.ElasticsearchResource; @@ -17,12 +17,12 @@ import pl.allegro.tech.hermes.tracker.frontend.LogRepository; import java.time.Clock; +import java.time.Duration; import java.time.LocalDate; import java.time.ZoneId; import java.time.ZoneOffset; -import static com.jayway.awaitility.Awaitility.await; -import static com.jayway.awaitility.Duration.ONE_MINUTE; +import static org.awaitility.Awaitility.await; import static org.elasticsearch.index.query.QueryBuilders.boolQuery; import static org.elasticsearch.index.query.QueryBuilders.matchQuery; import static org.elasticsearch.index.query.QueryBuilders.termQuery; @@ -40,16 +40,16 @@ public class FrontendElasticsearchLogRepositoryTest extends AbstractLogRepositor private static final ElasticsearchResource elasticsearch = new ElasticsearchResource(); - private SchemaManager schemaManager; + private static SchemaManager schemaManager; - @BeforeSuite - public void before() throws Throwable { + @BeforeClass + public static void beforeAll() throws Throwable { elasticsearch.before(); schemaManager = new SchemaManager(elasticsearch.client(), frontendIndexFactory, consumersIndexFactory, false); } - @AfterSuite - public void after() { + @AfterClass + public static void afterAll() { elasticsearch.after(); } @@ -110,7 +110,7 @@ private BoolQueryBuilder getQuery(String topic, } private void awaitUntilMessageIsIndexed(QueryBuilder query) { - await().atMost(ONE_MINUTE).until(() -> { + await().atMost(Duration.ofMinutes(1)).until(() -> { SearchResponse response = elasticsearch.client().prepareSearch(frontendIndexFactory.createIndex()) .setTypes(SchemaManager.PUBLISHED_TYPE) .setQuery(query) diff --git a/hermes-tracker-elasticsearch/src/test/java/pl/allegro/tech/hermes/tracker/elasticsearch/management/ElasticsearchLogRepositoryTest.java b/hermes-tracker-elasticsearch/src/test/java/pl/allegro/tech/hermes/tracker/elasticsearch/management/ElasticsearchLogRepositoryTest.java index a0606e8e1b..0f36d0a768 100644 --- a/hermes-tracker-elasticsearch/src/test/java/pl/allegro/tech/hermes/tracker/elasticsearch/management/ElasticsearchLogRepositoryTest.java +++ b/hermes-tracker-elasticsearch/src/test/java/pl/allegro/tech/hermes/tracker/elasticsearch/management/ElasticsearchLogRepositoryTest.java @@ -2,9 +2,11 @@ import com.google.common.collect.ImmutableMap; import io.micrometer.core.instrument.simple.SimpleMeterRegistry; -import org.testng.annotations.AfterSuite; -import org.testng.annotations.BeforeSuite; -import org.testng.annotations.Test; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Ignore; +import org.junit.Test; import pl.allegro.tech.hermes.api.MessageTrace; import pl.allegro.tech.hermes.api.PublishedMessageTrace; import pl.allegro.tech.hermes.api.PublishedMessageTraceStatus; @@ -25,6 +27,7 @@ import pl.allegro.tech.hermes.tracker.management.LogRepository; import java.time.Clock; +import java.time.Duration; import java.time.LocalDate; import java.time.ZoneId; import java.time.ZoneOffset; @@ -32,9 +35,8 @@ import java.util.List; import java.util.Map; -import static com.jayway.awaitility.Awaitility.await; -import static com.jayway.awaitility.Duration.ONE_MINUTE; import static org.assertj.core.api.Assertions.assertThat; +import static org.awaitility.Awaitility.await; import static pl.allegro.tech.hermes.api.SentMessageTraceStatus.DISCARDED; import static pl.allegro.tech.hermes.common.http.ExtraRequestHeadersCollector.extraRequestHeadersCollector; @@ -56,9 +58,18 @@ public class ElasticsearchLogRepositoryTest implements LogSchemaAware { private FrontendElasticsearchLogRepository frontendLogRepository; private ConsumersElasticsearchLogRepository consumersLogRepository; - @BeforeSuite - public void before() throws Throwable { + @BeforeClass + public static void beforeAll() throws Throwable { elasticsearch.before(); + } + + @AfterClass + public static void afterAll() { + elasticsearch.after(); + } + + @Before + public void setUp() { SchemaManager schemaManager = new SchemaManager(elasticsearch.client(), frontendIndexFactory, consumersIndexFactory, false); logRepository = new ElasticsearchLogRepository(elasticsearch.client(), schemaManager); @@ -73,13 +84,9 @@ public void before() throws Throwable { .build(); } - @AfterSuite - public void after() { - elasticsearch.after(); - } - // TODO: figure out why this test sometimes *consistently* fails on CI - @Test(enabled = false) + @Ignore + @Test public void shouldGetLastUndelivered() throws Exception { //given String topic = "elasticsearch.lastUndelivered"; @@ -120,7 +127,7 @@ public void shouldGetMessageStatus() { private List fetchLastUndelivered(String topic, String subscription) { final List lastUndelivered = new ArrayList<>(); - await().atMost(ONE_MINUTE).until(() -> { + await().atMost(Duration.ofMinutes(1)).until(() -> { lastUndelivered.clear(); lastUndelivered.addAll(logRepository.getLastUndeliveredMessages(topic, subscription, 1)); return lastUndelivered.size() == 1; @@ -131,7 +138,7 @@ private List fetchLastUndelivered(String topic, String subscri private List fetchMessageStatus(MessageMetadata messageMetadata) { List status = new ArrayList<>(); - await().atMost(ONE_MINUTE).until(() -> { + await().atMost(Duration.ofMinutes(1)).until(() -> { status.clear(); status.addAll(logRepository.getMessageStatus(messageMetadata.getTopic(), messageMetadata.getSubscription(), messageMetadata.getMessageId())); diff --git a/hermes-tracker-elasticsearch/src/test/java/pl/allegro/tech/hermes/tracker/elasticsearch/management/MultiElasticsearchLogRepositoryTest.java b/hermes-tracker-elasticsearch/src/test/java/pl/allegro/tech/hermes/tracker/elasticsearch/management/MultiElasticsearchLogRepositoryTest.java index 0c9d814e42..917dbf971c 100644 --- a/hermes-tracker-elasticsearch/src/test/java/pl/allegro/tech/hermes/tracker/elasticsearch/management/MultiElasticsearchLogRepositoryTest.java +++ b/hermes-tracker-elasticsearch/src/test/java/pl/allegro/tech/hermes/tracker/elasticsearch/management/MultiElasticsearchLogRepositoryTest.java @@ -2,9 +2,10 @@ import com.google.common.collect.ImmutableMap; import io.micrometer.core.instrument.simple.SimpleMeterRegistry; -import org.testng.annotations.AfterSuite; -import org.testng.annotations.BeforeSuite; -import org.testng.annotations.Test; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; import pl.allegro.tech.hermes.api.MessageTrace; import pl.allegro.tech.hermes.api.PublishedMessageTrace; import pl.allegro.tech.hermes.api.PublishedMessageTraceStatus; @@ -15,7 +16,6 @@ import pl.allegro.tech.hermes.tracker.consumers.TestMessageMetadata; import pl.allegro.tech.hermes.tracker.elasticsearch.ElasticsearchResource; import pl.allegro.tech.hermes.tracker.elasticsearch.LogSchemaAware; -import pl.allegro.tech.hermes.tracker.elasticsearch.SchemaManager; import pl.allegro.tech.hermes.tracker.elasticsearch.consumers.ConsumersDailyIndexFactory; import pl.allegro.tech.hermes.tracker.elasticsearch.consumers.ConsumersElasticsearchLogRepository; import pl.allegro.tech.hermes.tracker.elasticsearch.consumers.ConsumersIndexFactory; @@ -25,6 +25,7 @@ import pl.allegro.tech.hermes.tracker.management.LogRepository; import java.time.Clock; +import java.time.Duration; import java.time.LocalDate; import java.time.ZoneId; import java.time.ZoneOffset; @@ -33,9 +34,8 @@ import java.util.List; import java.util.Map; -import static com.jayway.awaitility.Awaitility.await; -import static com.jayway.awaitility.Duration.ONE_MINUTE; import static org.assertj.core.api.Assertions.assertThat; +import static org.awaitility.Awaitility.await; import static pl.allegro.tech.hermes.common.http.ExtraRequestHeadersCollector.extraRequestHeadersCollector; @@ -58,12 +58,20 @@ public class MultiElasticsearchLogRepositoryTest implements LogSchemaAware { private FrontendElasticsearchLogRepository frontendLogRepository; private ConsumersElasticsearchLogRepository consumersLogRepository; - @BeforeSuite - public void before() throws Throwable { + @BeforeClass + public static void beforeAll() throws Throwable { elasticsearch1.before(); elasticsearch2.before(); + } + + @AfterClass + public static void afterAll() { + elasticsearch1.after(); + elasticsearch2.after(); + } - SchemaManager schemaManager = new SchemaManager(elasticsearch1.client(), frontendIndexFactory, consumersIndexFactory, false); + @Before + public void setUp() { logRepository = new MultiElasticsearchLogRepository(Arrays.asList(elasticsearch1.client(), elasticsearch2.client())); frontendLogRepository = new FrontendElasticsearchLogRepository.Builder( @@ -77,12 +85,6 @@ public void before() throws Throwable { .build(); } - @AfterSuite - public void after() { - elasticsearch1.after(); - elasticsearch2.after(); - } - @Test public void shouldGetMessageStatus() { // given @@ -104,7 +106,7 @@ public void shouldGetMessageStatus() { private List fetchMessageStatus(MessageMetadata messageMetadata) { List status = new ArrayList<>(); - await().atMost(ONE_MINUTE).until(() -> { + await().atMost(Duration.ofMinutes(1)).until(() -> { status.clear(); status.addAll(logRepository.getMessageStatus(messageMetadata.getTopic(), messageMetadata.getSubscription(), messageMetadata.getMessageId())); diff --git a/hermes-tracker/src/test/java/pl/allegro/tech/hermes/tracker/consumers/AbstractLogRepositoryTest.java b/hermes-tracker/src/test/java/pl/allegro/tech/hermes/tracker/consumers/AbstractLogRepositoryTest.java index 8bc4b6fa2d..9144196063 100644 --- a/hermes-tracker/src/test/java/pl/allegro/tech/hermes/tracker/consumers/AbstractLogRepositoryTest.java +++ b/hermes-tracker/src/test/java/pl/allegro/tech/hermes/tracker/consumers/AbstractLogRepositoryTest.java @@ -1,35 +1,21 @@ package pl.allegro.tech.hermes.tracker.consumers; -import org.testng.ITestContext; -import org.testng.ITestNGMethod; -import org.testng.annotations.BeforeSuite; -import org.testng.annotations.BeforeTest; -import org.testng.annotations.Listeners; -import org.testng.annotations.Test; +import org.junit.Before; +import org.junit.Test; import pl.allegro.tech.hermes.api.SentMessageTraceStatus; -import pl.allegro.tech.hermes.test.helper.retry.Retry; -import pl.allegro.tech.hermes.test.helper.retry.RetryListener; import static pl.allegro.tech.hermes.api.SentMessageTraceStatus.DISCARDED; import static pl.allegro.tech.hermes.api.SentMessageTraceStatus.INFLIGHT; import static pl.allegro.tech.hermes.api.SentMessageTraceStatus.SUCCESS; -@Listeners({RetryListener.class}) public abstract class AbstractLogRepositoryTest { private static final String SUBSCRIPTION = "subscription"; private LogRepository logRepository; - @BeforeSuite - public void setUpRetry(ITestContext context) { - for (ITestNGMethod method : context.getAllTestMethods()) { - method.setRetryAnalyzerClass(Retry.class); - } - } - - @BeforeTest - public void setUp() throws Exception { + @Before + public void setUp() { logRepository = createLogRepository(); } diff --git a/hermes-tracker/src/test/java/pl/allegro/tech/hermes/tracker/frontend/AbstractLogRepositoryTest.java b/hermes-tracker/src/test/java/pl/allegro/tech/hermes/tracker/frontend/AbstractLogRepositoryTest.java index a609522a57..8ec5d8ff22 100644 --- a/hermes-tracker/src/test/java/pl/allegro/tech/hermes/tracker/frontend/AbstractLogRepositoryTest.java +++ b/hermes-tracker/src/test/java/pl/allegro/tech/hermes/tracker/frontend/AbstractLogRepositoryTest.java @@ -1,32 +1,18 @@ package pl.allegro.tech.hermes.tracker.frontend; import com.google.common.collect.ImmutableMap; -import org.testng.ITestContext; -import org.testng.ITestNGMethod; -import org.testng.annotations.BeforeSuite; -import org.testng.annotations.BeforeTest; -import org.testng.annotations.Listeners; -import org.testng.annotations.Test; -import pl.allegro.tech.hermes.test.helper.retry.Retry; -import pl.allegro.tech.hermes.test.helper.retry.RetryListener; +import org.junit.Before; +import org.junit.Test; import java.util.Map; import static java.lang.System.currentTimeMillis; -@Listeners({RetryListener.class}) public abstract class AbstractLogRepositoryTest { private LogRepository logRepository; - @BeforeSuite - public void setUpRetry(ITestContext context) { - for (ITestNGMethod method : context.getAllTestMethods()) { - method.setRetryAnalyzerClass(Retry.class); - } - } - - @BeforeTest + @Before public void setup() { logRepository = createRepository(); } @@ -80,13 +66,13 @@ public void shouldLogInflight() throws Exception { } protected abstract void awaitUntilSuccessMessageIsPersisted( - String topic, - String id, - String remoteHostname, - String storageDatacenter, - String... extraRequestHeadersKeywords + String topic, + String id, + String remoteHostname, + String storageDatacenter, + String... extraRequestHeadersKeywords ) - throws Exception; + throws Exception; protected abstract void awaitUntilInflightMessageIsPersisted( String topic, @@ -98,11 +84,11 @@ protected abstract void awaitUntilInflightMessageIsPersisted( protected abstract void awaitUntilErrorMessageIsPersisted( - String topic, - String id, - String reason, - String remoteHostname, - String... extraRequestHeadersKeywords + String topic, + String id, + String reason, + String remoteHostname, + String... extraRequestHeadersKeywords ) - throws Exception; + throws Exception; } \ No newline at end of file diff --git a/integration-tests/build.gradle b/integration-tests/build.gradle index 12df7da2e9..19fca04359 100644 --- a/integration-tests/build.gradle +++ b/integration-tests/build.gradle @@ -16,9 +16,10 @@ dependencies { testImplementation group: 'com.squareup.okhttp3', name: 'okhttp', version: versions.okhttp testImplementation group: 'org.springframework', name: 'spring-webflux', version: versions.spring_web testImplementation group: 'org.springframework', name: 'spring-test', version: versions.spring_web - testImplementation group: 'org.eclipse.jetty', name: 'jetty-reactive-httpclient', version: '4.0.2' + testImplementation group: 'org.eclipse.jetty', name: 'jetty-reactive-httpclient', version: '4.0.3' testImplementation group: 'org.awaitility', name: 'awaitility', version: '4.2.0' testImplementation group: 'org.reactivestreams', name: 'reactive-streams', version: '1.0.4' + // TODO: can we update it ? Which version of server our clients use ? testImplementation(group: 'org.hornetq', name: 'hornetq-jms-server', version: '2.4.1.Final') { exclude module: 'hornetq-native' } diff --git a/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/setup/HermesExtension.java b/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/setup/HermesExtension.java index 7f1ce40cef..ff1e4bb4ed 100644 --- a/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/setup/HermesExtension.java +++ b/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/setup/HermesExtension.java @@ -1,6 +1,5 @@ package pl.allegro.tech.hermes.integrationtests.setup; -import com.jayway.awaitility.Duration; import org.assertj.core.api.Assertions; import org.junit.jupiter.api.extension.AfterAllCallback; import org.junit.jupiter.api.extension.BeforeAllCallback; @@ -27,10 +26,11 @@ import pl.allegro.tech.hermes.test.helper.containers.ZookeeperContainer; import pl.allegro.tech.hermes.test.helper.environment.HermesTestApp; +import java.time.Duration; import java.util.List; import java.util.stream.Stream; -import static com.jayway.awaitility.Awaitility.waitAtMost; +import static org.awaitility.Awaitility.waitAtMost; import static pl.allegro.tech.hermes.integrationtests.setup.HermesManagementTestApp.AUDIT_EVENT_PATH; import static pl.allegro.tech.hermes.test.helper.endpoint.TimeoutAdjuster.adjust; @@ -113,7 +113,7 @@ private void removeSubscriptions() { service.removeSubscription(subscription.getTopicName(), subscription.getName(), testUser); } - waitAtMost(adjust(Duration.ONE_MINUTE)).until(() -> + waitAtMost(adjust(Duration.ofMinutes(1))).untilAsserted(() -> Assertions.assertThat(service.getAllSubscriptions().size()).isEqualTo(0) ); } @@ -125,7 +125,7 @@ private void removeTopics() { service.removeTopicWithSchema(topic, testUser); } - waitAtMost(adjust(Duration.ONE_MINUTE)).until(() -> + waitAtMost(adjust(Duration.ofMinutes(1))).untilAsserted(() -> Assertions.assertThat(service.getAllTopics().size()).isEqualTo(0) ); } diff --git a/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/setup/HermesFrontendTestApp.java b/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/setup/HermesFrontendTestApp.java index 5a4aa9d67c..2a13efbaa9 100644 --- a/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/setup/HermesFrontendTestApp.java +++ b/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/setup/HermesFrontendTestApp.java @@ -111,7 +111,7 @@ private List createArgs() { args.put(KAFKA_PRODUCER_METADATA_MAX_AGE, metadataMaxAge); - args.put(FRONTEND_FORCE_TOPIC_MAX_MESSAGE_SIZE,true); + args.put(FRONTEND_FORCE_TOPIC_MAX_MESSAGE_SIZE, true); args.put(FRONTEND_IDLE_TIMEOUT, Duration.ofSeconds(2)); args.put(FRONTEND_THROUGHPUT_TYPE, "fixed"); diff --git a/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/setup/HermesManagementTestApp.java b/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/setup/HermesManagementTestApp.java index ffacfc1430..16ebf4bbe5 100644 --- a/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/setup/HermesManagementTestApp.java +++ b/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/setup/HermesManagementTestApp.java @@ -23,8 +23,8 @@ import java.util.Map; import java.util.concurrent.TimeUnit; -import static com.jayway.awaitility.Awaitility.waitAtMost; import static org.assertj.core.api.Assertions.assertThat; +import static org.awaitility.Awaitility.waitAtMost; import static pl.allegro.tech.hermes.infrastructure.dc.DefaultDatacenterNameProvider.DEFAULT_DC_NAME; import static pl.allegro.tech.hermes.test.helper.endpoint.TimeoutAdjuster.adjust; @@ -79,7 +79,7 @@ private void waitUntilReady() { .build(); HttpClient httpClient = HttpClient.newHttpClient(); - waitAtMost(adjust(240), TimeUnit.SECONDS).until(() -> { + waitAtMost(adjust(240), TimeUnit.SECONDS).untilAsserted(() -> { try { HttpResponse response = httpClient.send(request, HttpResponse.BodyHandlers.ofString()); assertThat(response.body()).isEqualTo("readWrite"); diff --git a/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/subscriber/TestJmsSubscriber.java b/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/subscriber/TestJmsSubscriber.java index b99c97d766..dabfedda37 100644 --- a/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/subscriber/TestJmsSubscriber.java +++ b/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/subscriber/TestJmsSubscriber.java @@ -1,24 +1,23 @@ package pl.allegro.tech.hermes.integrationtests.subscriber; -import com.jayway.awaitility.Duration; import org.hornetq.api.core.TransportConfiguration; import org.hornetq.api.jms.HornetQJMSClient; import org.hornetq.api.jms.JMSFactoryType; import org.hornetq.core.remoting.impl.netty.NettyConnectorFactory; +import java.nio.charset.StandardCharsets; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; import javax.jms.ConnectionFactory; import javax.jms.JMSContext; import javax.jms.JMSException; import javax.jms.Message; import javax.jms.Topic; -import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import static com.jayway.awaitility.Awaitility.await; -import static java.util.concurrent.TimeUnit.SECONDS; import static org.assertj.core.api.Assertions.assertThat; +import static org.awaitility.Awaitility.await; import static pl.allegro.tech.hermes.test.helper.endpoint.TimeoutAdjuster.adjust; public class TestJmsSubscriber { @@ -80,7 +79,7 @@ public void waitUntilMessageWithHeaderReceived(String headerName, String headerV } private void awaitWithSyncRequests(Runnable runnable) { - await().atMost(adjust(new Duration(DEFAULT_WAIT_TIME_IN_SEC, SECONDS))).until(() -> { + await().atMost(adjust(Duration.ofSeconds(DEFAULT_WAIT_TIME_IN_SEC))).untilAsserted(() -> { synchronized (receivedRequests) { runnable.run(); } diff --git a/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/subscriber/TestSubscriber.java b/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/subscriber/TestSubscriber.java index d502051294..59d2ace7fa 100644 --- a/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/subscriber/TestSubscriber.java +++ b/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/subscriber/TestSubscriber.java @@ -2,9 +2,9 @@ import com.github.tomakehurst.wiremock.verification.LoggedRequest; import com.google.common.collect.Streams; -import com.jayway.awaitility.Duration; import java.net.URI; +import java.time.Duration; import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -13,9 +13,8 @@ import java.util.function.Consumer; import java.util.stream.Collectors; -import static com.jayway.awaitility.Awaitility.await; -import static java.util.concurrent.TimeUnit.SECONDS; import static org.assertj.core.api.Assertions.assertThat; +import static org.awaitility.Awaitility.await; import static pl.allegro.tech.hermes.test.helper.endpoint.TimeoutAdjuster.adjust; public class TestSubscriber { @@ -53,15 +52,16 @@ public void waitUntilReceived(String body) { ).isNotEmpty()); } - public void waitUntilAnyMessageReceived() { - await().atMost(adjust(new Duration(DEFAULT_WAIT_TIME_IN_SEC, SECONDS))).until(() -> - assertThat(receivedRequests.size()).isPositive()); - } public void waitUntilReceived(Duration duration, int numberOfExpectedMessages) { - await().atMost(adjust(duration)).until(() -> + await().atMost(adjust(duration)).untilAsserted(() -> assertThat(receivedRequests.size()).isEqualTo(numberOfExpectedMessages)); } + public void waitUntilAnyMessageReceived() { + await().atMost(adjust(Duration.ofSeconds(DEFAULT_WAIT_TIME_IN_SEC))).untilAsserted(() -> + assertThat(receivedRequests.size()).isPositive()); + } + public void waitUntilRequestReceived(Consumer requestConsumer) { waitUntilAnyMessageReceived(); @@ -71,7 +71,7 @@ public void waitUntilRequestReceived(Consumer requestConsumer) { } public void waitUntilRequestsReceived(Consumer> requestsConsumer) { - await().atMost(adjust(new Duration(DEFAULT_WAIT_TIME_IN_SEC, SECONDS))).until( + await().atMost(adjust(Duration.ofSeconds(DEFAULT_WAIT_TIME_IN_SEC))).untilAsserted( () -> { synchronized (receivedRequests) { requestsConsumer.accept(receivedRequests); @@ -100,7 +100,7 @@ public java.time.Duration durationBetweenFirstAndLastRequest() { } private void awaitWithSyncRequests(Runnable runnable) { - await().atMost(adjust(new Duration(DEFAULT_WAIT_TIME_IN_SEC, SECONDS))).until(() -> { + await().atMost(adjust(Duration.ofSeconds(DEFAULT_WAIT_TIME_IN_SEC))).untilAsserted(() -> { synchronized (receivedRequests) { runnable.run(); } diff --git a/integration-tests/src/common/java/pl/allegro/tech/hermes/management/TestSecurityProvider.java b/integration-tests/src/common/java/pl/allegro/tech/hermes/management/TestSecurityProvider.java index 80553233f4..afcc375b62 100644 --- a/integration-tests/src/common/java/pl/allegro/tech/hermes/management/TestSecurityProvider.java +++ b/integration-tests/src/common/java/pl/allegro/tech/hermes/management/TestSecurityProvider.java @@ -2,7 +2,7 @@ import jakarta.ws.rs.container.ContainerRequestContext; import jakarta.ws.rs.core.SecurityContext; -import org.apache.commons.lang.NotImplementedException; +import org.apache.commons.lang3.NotImplementedException; import pl.allegro.tech.hermes.api.OwnerId; import pl.allegro.tech.hermes.management.api.auth.SecurityProvider; diff --git a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/BatchDeliveryTest.java b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/BatchDeliveryTest.java index bbf7d0f26e..1a700ebe59 100644 --- a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/BatchDeliveryTest.java +++ b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/BatchDeliveryTest.java @@ -1,7 +1,6 @@ package pl.allegro.tech.hermes.integrationtests; import com.fasterxml.jackson.databind.ObjectMapper; -import com.jayway.awaitility.Duration; import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.RegisterExtension; @@ -16,6 +15,7 @@ import pl.allegro.tech.hermes.test.helper.message.TestMessage; import java.io.IOException; +import java.time.Duration; import java.util.List; import java.util.Map; @@ -297,7 +297,7 @@ public void shouldTimeoutRequestToSlowlyRespondingClient() { // then // first request is retried because of timeout (with socket / idle timeout only, the request wouldn't be timed out because // there are chunks of response every 500ms which is smaller than 1s timeout) - subscriber.waitUntilReceived(Duration.FIVE_SECONDS, 2); + subscriber.waitUntilReceived(Duration.ofSeconds(5), 2); Assertions.assertThat(subscriber.getLastReceivedRequest().getHeader("Hermes-Retry-Count")).isEqualTo("1"); } diff --git a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/BatchRetryPolicyTest.java b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/BatchRetryPolicyTest.java index eefa8aeceb..77c9a42074 100644 --- a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/BatchRetryPolicyTest.java +++ b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/BatchRetryPolicyTest.java @@ -3,7 +3,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.github.tomakehurst.wiremock.stubbing.Scenario; import com.github.tomakehurst.wiremock.verification.LoggedRequest; -import com.jayway.awaitility.Duration; import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.RegisterExtension; @@ -17,15 +16,16 @@ import pl.allegro.tech.hermes.test.helper.message.TestMessage; import java.io.IOException; +import java.time.Duration; import java.util.List; import java.util.Map; import static com.github.tomakehurst.wiremock.client.WireMock.aResponse; import static com.github.tomakehurst.wiremock.client.WireMock.containing; import static com.github.tomakehurst.wiremock.client.WireMock.post; -import static org.apache.http.HttpStatus.SC_BAD_REQUEST; -import static org.apache.http.HttpStatus.SC_CREATED; -import static org.apache.http.HttpStatus.SC_INTERNAL_SERVER_ERROR; +import static org.apache.hc.core5.http.HttpStatus.SC_BAD_REQUEST; +import static org.apache.hc.core5.http.HttpStatus.SC_CREATED; +import static org.apache.hc.core5.http.HttpStatus.SC_INTERNAL_SERVER_ERROR; import static pl.allegro.tech.hermes.api.BatchSubscriptionPolicy.Builder.batchSubscriptionPolicy; import static pl.allegro.tech.hermes.test.helper.builder.SubscriptionBuilder.subscription; import static pl.allegro.tech.hermes.test.helper.builder.TopicBuilder.topicWithRandomName; @@ -91,7 +91,7 @@ public void shouldNotRetryIfRequestSuccessful() { hermes.api().publishUntilSuccess(topic.getQualifiedName(), message.body()); //then - subscriber.waitUntilReceived(Duration.FIVE_SECONDS, 1); + subscriber.waitUntilReceived(Duration.ofSeconds(5), 1); } @Test diff --git a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/BroadcastDeliveryTest.java b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/BroadcastDeliveryTest.java index 699d64d5d0..b6c51c3903 100644 --- a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/BroadcastDeliveryTest.java +++ b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/BroadcastDeliveryTest.java @@ -1,6 +1,5 @@ package pl.allegro.tech.hermes.integrationtests; -import com.jayway.awaitility.Duration; import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.RegisterExtension; @@ -14,14 +13,14 @@ import pl.allegro.tech.hermes.integrationtests.subscriber.TestSubscribersExtension; import pl.allegro.tech.hermes.test.helper.message.TestMessage; +import java.time.Duration; import java.util.List; import java.util.stream.Stream; -import static com.jayway.awaitility.Awaitility.waitAtMost; -import static com.jayway.awaitility.Duration.TEN_SECONDS; import static java.util.stream.Collectors.joining; import static java.util.stream.Collectors.toList; import static org.assertj.core.api.Assertions.assertThat; +import static org.awaitility.Awaitility.waitAtMost; import static pl.allegro.tech.hermes.api.SubscriptionPolicy.Builder.subscriptionPolicy; import static pl.allegro.tech.hermes.test.helper.builder.SubscriptionBuilder.subscription; import static pl.allegro.tech.hermes.test.helper.builder.TopicBuilder.topicWithRandomName; @@ -75,7 +74,7 @@ public void shouldPublishAndRetryOnlyForUndeliveredConsumers() { // then subscribers.forEach(s -> s.waitUntilReceived(message.body())); - retryingSubscriber.waitUntilReceived(Duration.ONE_MINUTE, 2); + retryingSubscriber.waitUntilReceived(Duration.ofMinutes(1), 2); Assertions.assertThat(retryingSubscriber.getLastReceivedRequest().getHeader("Hermes-Retry-Count")).isEqualTo("1"); } @@ -100,7 +99,7 @@ public void shouldNotRetryForBadRequestsFromConsumers() { // then subscribers.forEach(s -> s.waitUntilReceived(message.body())); - waitAtMost(TEN_SECONDS).until(() -> { + waitAtMost(Duration.ofSeconds(10)).untilAsserted(() -> { long discarded = hermes.api() .getSubscriptionMetrics(topic.getQualifiedName(), "subscription") .expectBody(SubscriptionMetrics.class).returnResult().getResponseBody().getDiscarded(); diff --git a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/ConsumerProfilingTest.java b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/ConsumerProfilingTest.java index f7c6eb1a42..e673317c2e 100644 --- a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/ConsumerProfilingTest.java +++ b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/ConsumerProfilingTest.java @@ -3,7 +3,6 @@ import ch.qos.logback.classic.Logger; import ch.qos.logback.classic.spi.ILoggingEvent; import ch.qos.logback.core.read.ListAppender; -import com.jayway.awaitility.Duration; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; @@ -20,10 +19,11 @@ import pl.allegro.tech.hermes.integrationtests.subscriber.TestSubscribersExtension; import pl.allegro.tech.hermes.test.helper.message.TestMessage; +import java.time.Duration; import java.util.List; -import static com.jayway.awaitility.Awaitility.waitAtMost; import static org.assertj.core.api.Assertions.assertThat; +import static org.awaitility.Awaitility.waitAtMost; import static pl.allegro.tech.hermes.test.helper.builder.SubscriptionBuilder.subscriptionWithRandomName; import static pl.allegro.tech.hermes.test.helper.builder.TopicBuilder.topicWithRandomName; @@ -87,7 +87,7 @@ public void shouldProfileEmptyRun() { .withProfilingEnabled(true).build()); // then - waitAtMost(Duration.TEN_SECONDS).until(() -> { + waitAtMost(Duration.ofSeconds(10)).untilAsserted(() -> { List logsList = listAppender.list.stream() .filter(log -> log.getFormattedMessage().contains(subscription.getQualifiedName().toString())).toList(); assertThat(logsList).hasSizeGreaterThan(0); @@ -137,7 +137,7 @@ public void shouldProfileSuccessfulMessageProcessing() { subscriber.waitUntilReceived(message.body()); // then - waitAtMost(Duration.TEN_SECONDS).until(() -> { + waitAtMost(Duration.ofSeconds(10)).untilAsserted(() -> { List logsList = listAppender.list.stream() .filter(log -> log.getFormattedMessage().contains(ConsumerRun.DELIVERED.name())).toList(); assertThat(logsList).hasSizeGreaterThan(0); @@ -172,7 +172,7 @@ public void shouldProfileDiscardedMessageProcessing() { subscriber.waitUntilReceived(message.body()); // then - waitAtMost(Duration.TEN_SECONDS).until(() -> { + waitAtMost(Duration.ofSeconds(10)).untilAsserted(() -> { List logsList = listAppender.list.stream() .filter(log -> log.getFormattedMessage().contains(ConsumerRun.DISCARDED.name())).toList(); assertThat(logsList).hasSizeGreaterThan(0); @@ -204,10 +204,10 @@ public void shouldProfileRetriedMessageProcessing() { hermes.api().publishUntilSuccess(topic.getQualifiedName(), message.body()); // when - subscriber.waitUntilReceived(Duration.FIVE_SECONDS, 2); + subscriber.waitUntilReceived(Duration.ofSeconds(5), 2); // then - waitAtMost(Duration.TEN_SECONDS).until(() -> { + waitAtMost(Duration.ofSeconds(10)).untilAsserted(() -> { List retriedLogsList = listAppender.list.stream() .filter(log -> log.getFormattedMessage().contains(ConsumerRun.RETRIED.name())).toList(); assertThat(retriedLogsList).hasSizeGreaterThan(0); @@ -228,7 +228,7 @@ public void shouldProfileRetriedMessageProcessing() { // and - waitAtMost(Duration.TEN_SECONDS).until(() -> { + waitAtMost(Duration.ofSeconds(10)).untilAsserted(() -> { List processedLogsList = listAppender.list.stream() .filter(log -> log.getFormattedMessage().contains(ConsumerRun.DELIVERED.name())).toList(); assertThat(processedLogsList).hasSizeGreaterThan(0); diff --git a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/KafkaProducerMetricsTest.java b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/KafkaProducerMetricsTest.java index 5f3a98a155..71555d636d 100644 --- a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/KafkaProducerMetricsTest.java +++ b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/KafkaProducerMetricsTest.java @@ -8,7 +8,7 @@ import java.util.concurrent.TimeUnit; -import static com.jayway.awaitility.Awaitility.await; +import static org.awaitility.Awaitility.await; import static pl.allegro.tech.hermes.integrationtests.assertions.HermesAssertions.assertThatMetrics; import static pl.allegro.tech.hermes.test.helper.builder.TopicBuilder.topicWithRandomName; @@ -28,7 +28,7 @@ public void shouldRegisterSendMetrics() { hermes.api().publish(topic.getQualifiedName(), "hello world"); // then - await().atMost(10, TimeUnit.SECONDS).until(() -> + await().atMost(10, TimeUnit.SECONDS).untilAsserted(() -> assertMetricsContainTotalSendMetric().withValue(initialMetricValue + 2.0)); } diff --git a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/KafkaRetransmissionServiceTest.java b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/KafkaRetransmissionServiceTest.java index 883e408a87..513d3e39c2 100644 --- a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/KafkaRetransmissionServiceTest.java +++ b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/KafkaRetransmissionServiceTest.java @@ -2,7 +2,6 @@ import com.google.common.collect.ImmutableListMultimap; import com.google.common.collect.Multimaps; -import com.jayway.awaitility.Duration; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.RegisterExtension; import org.springframework.test.web.reactive.server.WebTestClient; @@ -21,15 +20,16 @@ import pl.allegro.tech.hermes.test.helper.message.TestMessage; import java.time.Clock; +import java.time.Duration; import java.time.OffsetDateTime; import java.util.ArrayList; import java.util.List; import java.util.function.Predicate; import java.util.stream.Collectors; -import static com.jayway.awaitility.Awaitility.waitAtMost; import static java.util.stream.IntStream.range; import static org.assertj.core.api.Assertions.assertThat; +import static org.awaitility.Awaitility.waitAtMost; import static pl.allegro.tech.hermes.api.PatchData.patchData; import static pl.allegro.tech.hermes.consumers.supervisor.process.Signal.SignalType.COMMIT; import static pl.allegro.tech.hermes.test.helper.builder.SubscriptionBuilder.subscriptionWithRandomName; @@ -155,7 +155,7 @@ private void publishAndConsumeMessages(List messages, Topic topic, TestS private void waitUntilConsumerCommitsOffset(String topicQualifiedName, String subscription) { long currentTime = clock.millis(); - until(Duration.ONE_MINUTE, topicQualifiedName, subscription, sub -> + until(Duration.ofMinutes(1), topicQualifiedName, subscription, sub -> sub.getSignalTimesheet().getOrDefault(COMMIT, 0L) > currentTime); } diff --git a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/MetricsTest.java b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/MetricsTest.java index 359c7cb239..a10cec2ba3 100644 --- a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/MetricsTest.java +++ b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/MetricsTest.java @@ -1,6 +1,5 @@ package pl.allegro.tech.hermes.integrationtests; -import com.jayway.awaitility.Duration; import org.junit.jupiter.api.Order; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.RegisterExtension; @@ -20,12 +19,13 @@ import pl.allegro.tech.hermes.integrationtests.subscriber.TestSubscribersExtension; import pl.allegro.tech.hermes.test.helper.message.TestMessage; +import java.time.Duration; import java.util.Map; import java.util.UUID; -import static com.jayway.awaitility.Awaitility.waitAtMost; import static java.lang.Integer.MAX_VALUE; import static org.assertj.core.api.Assertions.assertThat; +import static org.awaitility.Awaitility.waitAtMost; import static pl.allegro.tech.hermes.api.BatchSubscriptionPolicy.Builder.batchSubscriptionPolicy; import static pl.allegro.tech.hermes.api.SubscriptionPolicy.Builder.subscriptionPolicy; import static pl.allegro.tech.hermes.integrationtests.assertions.HermesAssertions.assertThatMetrics; @@ -61,7 +61,7 @@ public void shouldIncreaseTopicMetricsAfterMessageHasBeenPublished() { TestMessage message = TestMessage.simple(); int attempts = hermes.api().publishUntilSuccess(topic.getQualifiedName(), message.body()); - waitAtMost(Duration.TEN_SECONDS).until(() -> { + waitAtMost(Duration.ofSeconds(10)).untilAsserted(() -> { // when WebTestClient.ResponseSpec response = hermes.api().getTopicMetrics(topic.getQualifiedName()); @@ -87,7 +87,7 @@ public void shouldIncreaseSubscriptionDeliveredMetricsAfterMessageDelivered() { hermes.api().publishUntilSuccess(topic.getQualifiedName(), message.body()); subscriber.waitUntilReceived(message.body()); - waitAtMost(Duration.TEN_SECONDS).until(() -> { + waitAtMost(Duration.ofSeconds(10)).untilAsserted(() -> { // when WebTestClient.ResponseSpec response = hermes.api().getSubscriptionMetrics(topic.getQualifiedName(), subscription.getName()); @@ -240,7 +240,7 @@ public void shouldReportMetricForFilteredSubscription() { // then subscriber.waitUntilReceived(unfiltered.body()); - waitAtMost(Duration.TEN_SECONDS).until(() -> + waitAtMost(Duration.ofSeconds(10)).untilAsserted(() -> hermes.api().getConsumersMetrics() .expectStatus() .isOk() diff --git a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/OAuthIntegrationTest.java b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/OAuthIntegrationTest.java index be76f491ef..886cd85e45 100644 --- a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/OAuthIntegrationTest.java +++ b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/OAuthIntegrationTest.java @@ -18,8 +18,8 @@ import static com.github.tomakehurst.wiremock.client.WireMock.equalTo; import static com.github.tomakehurst.wiremock.client.WireMock.not; import static com.github.tomakehurst.wiremock.client.WireMock.post; -import static org.apache.http.HttpStatus.SC_OK; -import static org.apache.http.HttpStatus.SC_UNAUTHORIZED; +import static org.apache.hc.core5.http.HttpStatus.SC_OK; +import static org.apache.hc.core5.http.HttpStatus.SC_UNAUTHORIZED; import static pl.allegro.tech.hermes.api.SubscriptionOAuthPolicy.clientCredentialsGrantOAuthPolicy; import static pl.allegro.tech.hermes.api.SubscriptionOAuthPolicy.passwordGrantOAuthPolicy; import static pl.allegro.tech.hermes.test.helper.builder.OAuthProviderBuilder.oAuthProvider; diff --git a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/PublishingAndConsumingTest.java b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/PublishingAndConsumingTest.java index 940cded8c9..f154ff77da 100644 --- a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/PublishingAndConsumingTest.java +++ b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/PublishingAndConsumingTest.java @@ -26,9 +26,8 @@ import java.util.Map; import java.util.UUID; -import static com.jayway.awaitility.Awaitility.waitAtMost; -import static com.jayway.awaitility.Duration.TEN_SECONDS; import static jakarta.ws.rs.core.Response.Status.CREATED; +import static org.awaitility.Awaitility.waitAtMost; import static pl.allegro.tech.hermes.api.SubscriptionPolicy.Builder.subscriptionPolicy; import static pl.allegro.tech.hermes.integrationtests.assertions.HermesAssertions.assertThat; import static pl.allegro.tech.hermes.test.helper.builder.SubscriptionBuilder.subscription; @@ -110,7 +109,7 @@ public void shouldTreatMessageWithInvalidInterpolationAsUndelivered() { hermes.api().publishUntilSuccess(topic.getQualifiedName(), message.body()); // then - waitAtMost(TEN_SECONDS).until(() -> { + waitAtMost(Duration.ofSeconds(10)).untilAsserted(() -> { long discarded = hermes.api() .getSubscriptionMetrics(topic.getQualifiedName(), "subscription") .expectBody(SubscriptionMetrics.class).returnResult().getResponseBody().getDiscarded(); diff --git a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/PublishingAuthenticationTest.java b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/PublishingAuthenticationTest.java index e3838f6b77..fb6ec99094 100644 --- a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/PublishingAuthenticationTest.java +++ b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/PublishingAuthenticationTest.java @@ -1,7 +1,5 @@ package pl.allegro.tech.hermes.integrationtests; -import com.jayway.awaitility.Duration; -import org.apache.commons.codec.binary.Base64; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.RegisterExtension; import org.springframework.http.HttpHeaders; @@ -12,9 +10,11 @@ import pl.allegro.tech.hermes.utils.Headers; import java.nio.charset.StandardCharsets; +import java.time.Duration; +import java.util.Base64; import java.util.Map; -import static com.jayway.awaitility.Awaitility.waitAtMost; +import static org.awaitility.Awaitility.waitAtMost; import static pl.allegro.tech.hermes.frontend.FrontendConfigurationProperties.AUTH_PASSWORD; import static pl.allegro.tech.hermes.frontend.FrontendConfigurationProperties.AUTH_USERNAME; import static pl.allegro.tech.hermes.frontend.FrontendConfigurationProperties.FRONTEND_AUTHENTICATION_ENABLED; @@ -42,7 +42,7 @@ public void shouldAuthenticateUsingBasicAuth() { //given Topic topic = hermes.initHelper().createTopic(topicWithRandomName().build()); - waitAtMost(Duration.TEN_SECONDS).until(() -> { + waitAtMost(Duration.ofSeconds(10)).untilAsserted(() -> { // when WebTestClient.ResponseSpec response = hermes.api().publish( topic.getQualifiedName(), @@ -60,7 +60,7 @@ public void shouldNotAuthenticateUserWithInvalidCredentials() { //given Topic topic = hermes.initHelper().createTopic(topicWithRandomName().build()); - waitAtMost(Duration.TEN_SECONDS).until(() -> { + waitAtMost(Duration.ofSeconds(10)).untilAsserted(() -> { // when WebTestClient.ResponseSpec response = hermes.api().publish( topic.getQualifiedName(), @@ -78,7 +78,7 @@ public void shouldNotAuthenticateUserWithoutCredentials() { //given Topic topic = hermes.initHelper().createTopic(topicWithRandomName().build()); - waitAtMost(Duration.TEN_SECONDS).until(() -> { + waitAtMost(Duration.ofSeconds(10)).untilAsserted(() -> { // when WebTestClient.ResponseSpec response = hermes.api().publish(topic.getQualifiedName(), MESSAGE); @@ -90,7 +90,7 @@ public void shouldNotAuthenticateUserWithoutCredentials() { private static HttpHeaders createAuthorizationHeader(String username, String password) { String credentials = username + ":" + password; Map headers = Map.of( - "Authorization", "Basic " + Base64.encodeBase64String(credentials.getBytes(StandardCharsets.UTF_8)) + "Authorization", "Basic " + Base64.getEncoder().encodeToString(credentials.getBytes(StandardCharsets.UTF_8)) ); return Headers.createHeaders(headers); } diff --git a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/PublishingAvroTest.java b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/PublishingAvroTest.java index c89dbf5070..dcf6cb2a05 100644 --- a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/PublishingAvroTest.java +++ b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/PublishingAvroTest.java @@ -1,6 +1,5 @@ package pl.allegro.tech.hermes.integrationtests; -import com.jayway.awaitility.Duration; import net.javacrumbs.jsonunit.core.Option; import org.apache.avro.Schema; import org.junit.jupiter.api.Test; @@ -22,14 +21,15 @@ import pl.allegro.tech.hermes.test.helper.message.TestMessage; import java.time.Clock; +import java.time.Duration; import java.util.Map; import java.util.UUID; -import static com.jayway.awaitility.Awaitility.waitAtMost; import static jakarta.ws.rs.core.MediaType.TEXT_PLAIN; import static java.util.Collections.singletonMap; import static net.javacrumbs.jsonunit.fluent.JsonFluentAssert.assertThatJson; import static org.assertj.core.api.Assertions.assertThat; +import static org.awaitility.Awaitility.waitAtMost; import static pl.allegro.tech.hermes.api.AvroMediaType.AVRO_JSON; import static pl.allegro.tech.hermes.api.ContentType.AVRO; import static pl.allegro.tech.hermes.api.ContentType.JSON; @@ -227,7 +227,7 @@ public void shouldGetBadRequestForJsonNotMatchingWithAvroSchemaAndAvroContentTyp String message = "{\"__metadata\":null,\"name\":\"john\",\"age\":\"string instead of int\"}"; // when / then - waitAtMost(Duration.TEN_SECONDS).until(() -> { + waitAtMost(Duration.ofSeconds(10)).untilAsserted(() -> { WebTestClient.ResponseSpec response = hermes.api().publish(topic.getQualifiedName(), message, createHeaders(Map.of("Content-Type", AVRO_JSON))); response.expectStatus().isBadRequest(); response.expectBody(String.class).isEqualTo( @@ -539,7 +539,7 @@ private void assertBodyDeserializesIntoUser(String body, AvroUser user) { } private void waitUntilSubscriptionContentTypeChanged(Topic topic, String subscription, ContentType expected) { - waitAtMost(adjust(Duration.TEN_SECONDS)).until(() -> { + waitAtMost(adjust(Duration.ofSeconds(10))).until(() -> { ContentType actual = hermes.api().getSubscription(topic.getQualifiedName(), subscription).getContentType(); logger.info("Expecting {} subscription endpoint address. Actual {}", expected, actual); return expected.equals(actual); @@ -548,7 +548,7 @@ private void waitUntilSubscriptionContentTypeChanged(Topic topic, String subscri private void waitUntilConsumerCommitsOffset(Topic topic, String subscription) { long currentTime = clock.millis(); - waitAtMost(adjust(Duration.ONE_MINUTE)).until(() -> + waitAtMost(adjust(Duration.ofMinutes(1))).until(() -> hermes.api().getRunningSubscriptionsStatus().stream() .filter(sub -> sub.getQualifiedName().equals(topic.getQualifiedName() + "$" + subscription)) .anyMatch(sub -> sub.getSignalTimesheet().getOrDefault(COMMIT, 0L) > currentTime)); @@ -556,7 +556,7 @@ private void waitUntilConsumerCommitsOffset(Topic topic, String subscription) { } private void waitUntilConsumersUpdateSubscription(final long currentTime, Topic topic, String subscription) { - waitAtMost(adjust(Duration.TEN_SECONDS)).until(() -> + waitAtMost(adjust(Duration.ofSeconds(10))).until(() -> hermes.api().getRunningSubscriptionsStatus().stream() .filter(sub -> sub.getQualifiedName().equals(topic.getQualifiedName() + "$" + subscription)) .anyMatch(sub -> sub.getSignalTimesheet().getOrDefault(UPDATE_SUBSCRIPTION, 0L) > currentTime)); diff --git a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/PublishingTest.java b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/PublishingTest.java index e36db02694..54ab67272b 100644 --- a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/PublishingTest.java +++ b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/PublishingTest.java @@ -1,6 +1,6 @@ package pl.allegro.tech.hermes.integrationtests; -import org.apache.commons.lang.StringUtils; +import org.apache.commons.lang3.StringUtils; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.RegisterExtension; import org.springframework.http.HttpStatus; diff --git a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/ReadinessCheckTest.java b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/ReadinessCheckTest.java index bb3a914cea..e4f65f4b33 100644 --- a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/ReadinessCheckTest.java +++ b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/ReadinessCheckTest.java @@ -1,11 +1,12 @@ package pl.allegro.tech.hermes.integrationtests; -import com.jayway.awaitility.Duration; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.RegisterExtension; import pl.allegro.tech.hermes.integrationtests.setup.HermesExtension; -import static com.jayway.awaitility.Awaitility.waitAtMost; +import java.time.Duration; + +import static org.awaitility.Awaitility.waitAtMost; import static pl.allegro.tech.hermes.infrastructure.dc.DefaultDatacenterNameProvider.DEFAULT_DC_NAME; public class ReadinessCheckTest { @@ -19,7 +20,7 @@ public void shouldRespectReadinessStatusSetByAdmin() { hermes.api().setReadiness(DEFAULT_DC_NAME, false).expectStatus().isAccepted(); // then - waitAtMost(Duration.FIVE_SECONDS).until(() -> + waitAtMost(Duration.ofSeconds(5)).untilAsserted(() -> hermes.api() .getFrontendReadiness() .expectStatus().is5xxServerError() @@ -30,7 +31,7 @@ public void shouldRespectReadinessStatusSetByAdmin() { hermes.api().setReadiness(DEFAULT_DC_NAME, true).expectStatus().isAccepted(); // then - waitAtMost(Duration.FIVE_SECONDS).until(() -> + waitAtMost(Duration.ofSeconds(5)).untilAsserted(() -> hermes.api() .getFrontendReadiness() .expectStatus().isOk() diff --git a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/TopicAuthorizationTest.java b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/TopicAuthorizationTest.java index f394a94c8c..9f0fc17700 100644 --- a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/TopicAuthorizationTest.java +++ b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/TopicAuthorizationTest.java @@ -1,7 +1,5 @@ package pl.allegro.tech.hermes.integrationtests; -import com.jayway.awaitility.Duration; -import org.apache.commons.codec.binary.Base64; import org.junit.jupiter.api.extension.RegisterExtension; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.MethodSource; @@ -14,10 +12,12 @@ import pl.allegro.tech.hermes.utils.Headers; import java.nio.charset.StandardCharsets; +import java.time.Duration; +import java.util.Base64; import java.util.Map; import java.util.stream.Stream; -import static com.jayway.awaitility.Awaitility.waitAtMost; +import static org.awaitility.Awaitility.waitAtMost; import static pl.allegro.tech.hermes.frontend.FrontendConfigurationProperties.AUTH_PASSWORD; import static pl.allegro.tech.hermes.frontend.FrontendConfigurationProperties.AUTH_USERNAME; import static pl.allegro.tech.hermes.frontend.FrontendConfigurationProperties.FRONTEND_AUTHENTICATION_ENABLED; @@ -46,7 +46,7 @@ public void shouldPublishWhenAuthenticated(Topic topic) { // given hermes.initHelper().createTopic(topic); - waitAtMost(Duration.TEN_SECONDS).until(() -> { + waitAtMost(Duration.ofSeconds(10)).untilAsserted(() -> { // when WebTestClient.ResponseSpec response = hermes.api().publish( topic.getQualifiedName(), @@ -92,7 +92,7 @@ public void shouldPublishAsGuestWhenAuthIsNotRequired(Topic topic) { // given hermes.initHelper().createTopic(topic); - waitAtMost(Duration.TEN_SECONDS).until(() -> { + waitAtMost(Duration.ofSeconds(10)).untilAsserted(() -> { // when WebTestClient.ResponseSpec response = hermes.api().publish(topic.getQualifiedName(), MESSAGE); @@ -121,7 +121,7 @@ public void shouldNotPublishAsGuestWhenAuthIsRequired(Topic topic) { // given hermes.initHelper().createTopic(topic); - waitAtMost(Duration.TEN_SECONDS).until(() -> { + waitAtMost(Duration.ofSeconds(10)).untilAsserted(() -> { // when WebTestClient.ResponseSpec response = hermes.api().publish(topic.getQualifiedName(), MESSAGE); @@ -150,7 +150,7 @@ public void shouldNotPublishWithoutPermissionWhenAuthenticated(Topic topic) { // given hermes.initHelper().createTopic(topic); - waitAtMost(Duration.TEN_SECONDS).until(() -> { + waitAtMost(Duration.ofSeconds(10)).untilAsserted(() -> { // when WebTestClient.ResponseSpec response = hermes.api().publish( topic.getQualifiedName(), @@ -187,7 +187,7 @@ static Stream notPublishWithoutPermissionWhenAuthenticatedTopics() { private static HttpHeaders createAuthorizationHeader(String username, String password) { String credentials = username + ":" + password; Map headers = Map.of( - "Authorization", "Basic " + Base64.encodeBase64String(credentials.getBytes(StandardCharsets.UTF_8)) + "Authorization", "Basic " + Base64.getEncoder().encodeToString(credentials.getBytes(StandardCharsets.UTF_8)) ); return Headers.createHeaders(headers); } diff --git a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/TopicBlacklistTest.java b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/TopicBlacklistTest.java index 025ff765e3..3b98dd7d54 100644 --- a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/TopicBlacklistTest.java +++ b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/TopicBlacklistTest.java @@ -1,6 +1,5 @@ package pl.allegro.tech.hermes.integrationtests; -import com.jayway.awaitility.Duration; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.RegisterExtension; import org.springframework.test.web.reactive.server.WebTestClient; @@ -8,8 +7,10 @@ import pl.allegro.tech.hermes.integrationtests.setup.HermesExtension; import pl.allegro.tech.hermes.test.helper.message.TestMessage; -import static com.jayway.awaitility.Awaitility.waitAtMost; +import java.time.Duration; + import static org.assertj.core.api.Assertions.assertThat; +import static org.awaitility.Awaitility.waitAtMost; import static pl.allegro.tech.hermes.api.BlacklistStatus.BLACKLISTED; import static pl.allegro.tech.hermes.api.BlacklistStatus.NOT_BLACKLISTED; import static pl.allegro.tech.hermes.test.helper.builder.TopicBuilder.topicWithRandomName; @@ -27,7 +28,7 @@ public void shouldRefuseMessageOnBlacklistedTopic() { // when hermes.api().blacklistTopic(topic.getQualifiedName()); - waitAtMost(Duration.TEN_SECONDS).until(() -> { + waitAtMost(Duration.ofSeconds(10)).untilAsserted(() -> { WebTestClient.ResponseSpec response = hermes.api().publish(topic.getQualifiedName(), message.body()); // then @@ -44,7 +45,7 @@ public void shouldAcceptMessageOnUnblacklistedTopic() { // when hermes.api().unblacklistTopic(topic.getQualifiedName()); - waitAtMost(Duration.TEN_SECONDS).until(() -> { + waitAtMost(Duration.ofSeconds(10)).untilAsserted(() -> { WebTestClient.ResponseSpec response = hermes.api().publish(topic.getQualifiedName(), message.body()); // then diff --git a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/UndeliveredLogTest.java b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/UndeliveredLogTest.java index 9c70941508..cd7b0e7341 100644 --- a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/UndeliveredLogTest.java +++ b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/UndeliveredLogTest.java @@ -1,6 +1,5 @@ package pl.allegro.tech.hermes.integrationtests; -import com.jayway.awaitility.Duration; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.RegisterExtension; import pl.allegro.tech.hermes.api.Subscription; @@ -8,7 +7,9 @@ import pl.allegro.tech.hermes.integrationtests.setup.HermesExtension; import pl.allegro.tech.hermes.test.helper.message.TestMessage; -import static com.jayway.awaitility.Awaitility.waitAtMost; +import java.time.Duration; + +import static org.awaitility.Awaitility.waitAtMost; import static pl.allegro.tech.hermes.test.helper.builder.SubscriptionBuilder.subscriptionWithRandomName; import static pl.allegro.tech.hermes.test.helper.builder.TopicBuilder.topicWithRandomName; @@ -29,7 +30,7 @@ public void shouldLogUndeliveredMessage() { hermes.api().publishUntilSuccess(topic.getQualifiedName(), TestMessage.simple().body()); // then - waitAtMost(Duration.TEN_SECONDS).until(() -> + waitAtMost(Duration.ofSeconds(10)).untilAsserted(() -> hermes.api().getLatestUndeliveredMessage(topic.getQualifiedName(), subscription.getName()).expectStatus().is2xxSuccessful() ); } diff --git a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/GroupManagementTest.java b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/GroupManagementTest.java index 74ff23d653..b61e16cbef 100644 --- a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/GroupManagementTest.java +++ b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/GroupManagementTest.java @@ -1,6 +1,5 @@ package pl.allegro.tech.hermes.integrationtests.management; -import com.jayway.awaitility.Duration; import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.RegisterExtension; @@ -10,10 +9,11 @@ import pl.allegro.tech.hermes.integrationtests.setup.HermesExtension; import pl.allegro.tech.hermes.management.TestSecurityProvider; +import java.time.Duration; import java.util.stream.Stream; -import static com.jayway.awaitility.Awaitility.waitAtMost; import static org.assertj.core.api.Assertions.assertThat; +import static org.awaitility.Awaitility.waitAtMost; import static pl.allegro.tech.hermes.api.ErrorCode.GROUP_NAME_IS_INVALID; import static pl.allegro.tech.hermes.api.ErrorCode.GROUP_NOT_EMPTY; import static pl.allegro.tech.hermes.integrationtests.management.TopicManagementTest.getErrorCode; @@ -111,8 +111,8 @@ public void shouldRemoveGroup() { // then response.expectStatus().isOk(); - waitAtMost(Duration.TEN_SECONDS) - .until(() -> Assertions.assertThat(hermes.api().getGroups()).doesNotContain(group.getGroupName())); + waitAtMost(Duration.ofSeconds(10)) + .untilAsserted(() -> Assertions.assertThat(hermes.api().getGroups()).doesNotContain(group.getGroupName())); } @Test @@ -127,8 +127,8 @@ public void shouldAllowNonAdminUserToRemoveGroup() { // then response.expectStatus().isOk(); - waitAtMost(Duration.TEN_SECONDS) - .until(() -> Assertions.assertThat(hermes.api().getGroups()).doesNotContain(group.getGroupName())); + waitAtMost(Duration.ofSeconds(10)) + .untilAsserted(() -> Assertions.assertThat(hermes.api().getGroups()).doesNotContain(group.getGroupName())); // cleanup TestSecurityProvider.reset(); diff --git a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/HealthCheckTest.java b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/HealthCheckTest.java index 0702d57d04..ca1447cbfa 100644 --- a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/HealthCheckTest.java +++ b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/HealthCheckTest.java @@ -7,7 +7,7 @@ import java.util.concurrent.TimeUnit; -import static com.jayway.awaitility.Awaitility.await; +import static org.awaitility.Awaitility.await; public class HealthCheckTest { @@ -20,7 +20,7 @@ public void shouldManagementBeHealthy() { WebTestClient.ResponseSpec response = hermes.api().getManagementHealth(); // when & then - await().atMost(5, TimeUnit.SECONDS).until(() -> + await().atMost(5, TimeUnit.SECONDS).untilAsserted(() -> response.expectStatus().isOk()); } } diff --git a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/MessagePreviewIntegrationTest.java b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/MessagePreviewIntegrationTest.java index 1206a16e80..026de4a9fc 100644 --- a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/MessagePreviewIntegrationTest.java +++ b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/MessagePreviewIntegrationTest.java @@ -9,12 +9,12 @@ import pl.allegro.tech.hermes.integrationtests.setup.HermesExtension; import pl.allegro.tech.hermes.test.helper.avro.AvroUser; +import java.time.Duration; import java.util.List; -import java.util.concurrent.TimeUnit; -import static com.jayway.awaitility.Awaitility.await; import static net.javacrumbs.jsonunit.fluent.JsonFluentAssert.assertThatJson; import static org.assertj.core.api.Assertions.assertThat; +import static org.awaitility.Awaitility.await; import static pl.allegro.tech.hermes.api.ContentType.AVRO; import static pl.allegro.tech.hermes.api.TopicWithSchema.topicWithSchema; import static pl.allegro.tech.hermes.test.helper.builder.TopicBuilder.topicWithRandomName; @@ -37,7 +37,7 @@ public void shouldReturnAvroMessageWithSchemaAwareSerialization() { hermes.api().publishAvroUntilSuccess(topic.getQualifiedName(), avroUser.asBytes()); - await().atMost(10, TimeUnit.SECONDS).until(() -> { + await().atMost(Duration.ofSeconds(10)).untilAsserted(() -> { // when List previews = hermes.api().getPreview(topic.getQualifiedName()) .expectStatus().isOk() diff --git a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/QueryEndpointTest.java b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/QueryEndpointTest.java index fb4bd6045a..12f9badb8b 100644 --- a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/QueryEndpointTest.java +++ b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/QueryEndpointTest.java @@ -1,6 +1,5 @@ package pl.allegro.tech.hermes.integrationtests.management; -import com.jayway.awaitility.Duration; import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Order; import org.junit.jupiter.api.Test; @@ -22,17 +21,18 @@ import pl.allegro.tech.hermes.test.helper.avro.AvroUserSchemaLoader; import pl.allegro.tech.hermes.test.helper.builder.SubscriptionBuilder; +import java.time.Duration; import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.stream.Collectors; import java.util.stream.Stream; -import static com.jayway.awaitility.Awaitility.waitAtMost; import static java.time.Duration.ofMinutes; import static java.util.Arrays.asList; import static java.util.stream.Collectors.toList; import static org.assertj.core.api.Assertions.assertThat; +import static org.awaitility.Awaitility.waitAtMost; import static org.junit.jupiter.params.provider.Arguments.arguments; import static pl.allegro.tech.hermes.api.ContentType.AVRO; import static pl.allegro.tech.hermes.api.ContentType.JSON; @@ -442,7 +442,7 @@ public void shouldQueryTopicsMetrics(String topicName1, String topicName2, Strin .map(topicName -> group.getGroupName() + "." + topicName) .collect(toList()); - waitAtMost(adjust(Duration.ONE_MINUTE)).until(() -> { + waitAtMost(adjust(Duration.ofMinutes(1))).untilAsserted(() -> { // when List found = hermes.api().queryTopicMetrics(query) .expectStatus().isOk() @@ -486,7 +486,7 @@ public void shouldQuerySubscriptionsMetrics() { .build() ); - waitAtMost(adjust(Duration.ONE_MINUTE)).until(() -> { + waitAtMost(adjust(Duration.ofMinutes(1))).untilAsserted(() -> { // when final List allSubscriptions = hermes.api() .querySubscriptionMetrics(queryGetAllSubscriptionsMetrics) @@ -529,7 +529,7 @@ public void shouldHandleUnavailableSubscriptionsMetrics() { String queryGetSubscriptionsMetricsWithPositiveRate = "{\"query\": {\"rate\": {\"gt\": 0}}}"; prometheus.stubDelay(ofMinutes(10)); - waitAtMost(adjust(Duration.ONE_MINUTE)).until(() -> { + waitAtMost(adjust(Duration.ofMinutes(1))).untilAsserted(() -> { // when List allSubscriptions = hermes.api() .querySubscriptionMetrics(queryGetAllSubscriptionsMetrics) diff --git a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/SubscriptionManagementTest.java b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/SubscriptionManagementTest.java index 0f79a77283..7e9da9c1bf 100644 --- a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/SubscriptionManagementTest.java +++ b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/SubscriptionManagementTest.java @@ -1,7 +1,6 @@ package pl.allegro.tech.hermes.integrationtests.management; import com.google.common.collect.ImmutableMap; -import com.jayway.awaitility.Duration; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Order; import org.junit.jupiter.api.Test; @@ -28,14 +27,15 @@ import pl.allegro.tech.hermes.management.TestSecurityProvider; import pl.allegro.tech.hermes.test.helper.message.TestMessage; +import java.time.Duration; import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.Objects; import java.util.stream.Stream; -import static com.jayway.awaitility.Awaitility.waitAtMost; import static org.assertj.core.api.Assertions.assertThat; +import static org.awaitility.Awaitility.waitAtMost; import static pl.allegro.tech.hermes.api.PatchData.patchData; import static pl.allegro.tech.hermes.api.SubscriptionHealth.Status.NO_DATA; import static pl.allegro.tech.hermes.api.SubscriptionHealth.Status.UNHEALTHY; @@ -200,7 +200,7 @@ public void shouldUpdateSubscriptionEndpoint() { // then response.expectStatus().isOk(); - waitAtMost(Duration.TEN_SECONDS) + waitAtMost(Duration.ofSeconds(10)) .until(() -> hermes.api().getSubscriptionResponse(topic.getQualifiedName(), subscription.getName()) .expectStatus() .is2xxSuccessful() @@ -705,13 +705,13 @@ public void shouldMoveOffsetsToTheEnd() { hermes.api().deleteSubscription(topic.getQualifiedName(), subscription.getName()); // when - waitAtMost(Duration.TEN_SECONDS) - .until(() -> hermes.api() + waitAtMost(Duration.ofSeconds(10)) + .untilAsserted(() -> hermes.api() .moveOffsetsToTheEnd(topic.getQualifiedName(), subscription.getName()).expectStatus().isOk()); // then - waitAtMost(Duration.TEN_SECONDS) - .until(() -> assertThat(allConsumerGroupOffsetsMovedToTheEnd(subscription)).isTrue()); + waitAtMost(Duration.ofSeconds(10)) + .untilAsserted(() -> assertThat(allConsumerGroupOffsetsMovedToTheEnd(subscription)).isTrue()); } private boolean allConsumerGroupOffsetsMovedToTheEnd(Subscription subscription) { diff --git a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/TopicManagementTest.java b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/TopicManagementTest.java index f23c76d5df..d097604b40 100644 --- a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/TopicManagementTest.java +++ b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/TopicManagementTest.java @@ -2,7 +2,6 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; -import com.jayway.awaitility.Duration; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.RegisterExtension; @@ -22,6 +21,7 @@ import pl.allegro.tech.hermes.management.TestSecurityProvider; import pl.allegro.tech.hermes.test.helper.avro.AvroUserSchemaLoader; +import java.time.Duration; import java.util.Arrays; import java.util.List; import java.util.Map; @@ -29,8 +29,8 @@ import java.util.Set; import java.util.stream.Stream; -import static com.jayway.awaitility.Awaitility.waitAtMost; import static org.assertj.core.api.Assertions.assertThat; +import static org.awaitility.Awaitility.waitAtMost; import static pl.allegro.tech.hermes.api.ContentType.AVRO; import static pl.allegro.tech.hermes.api.ContentType.JSON; import static pl.allegro.tech.hermes.api.PatchData.patchData; @@ -143,7 +143,7 @@ public void shouldRemoveTopic() { // then response.expectStatus().isOk(); - waitAtMost(Duration.TEN_SECONDS).until(() -> assertThat(getGroupTopicsList(topic.getName().getGroupName())).isEmpty()); + waitAtMost(Duration.ofSeconds(10)).untilAsserted(() -> assertThat(getGroupTopicsList(topic.getName().getGroupName())).isEmpty()); } @Test @@ -157,7 +157,7 @@ public void shouldUnblacklistTopicWhileDeleting() { // then response.expectStatus().isOk(); - waitAtMost(Duration.TEN_SECONDS).until(() -> assertThat(getGroupTopicsList(topic.getName().getGroupName())).isEmpty()); + waitAtMost(Duration.ofSeconds(10)).untilAsserted(() -> assertThat(getGroupTopicsList(topic.getName().getGroupName())).isEmpty()); assertThat(hermes.api().isTopicBlacklisted(topic.getQualifiedName()).isBlacklisted()).isFalse(); } diff --git a/integration-tests/src/slowIntegrationTest/java/pl/allegro/tech/hermes/integrationtests/BrokerLatencyReportingTest.java b/integration-tests/src/slowIntegrationTest/java/pl/allegro/tech/hermes/integrationtests/BrokerLatencyReportingTest.java index fed2d0e62f..e53cf88ea7 100644 --- a/integration-tests/src/slowIntegrationTest/java/pl/allegro/tech/hermes/integrationtests/BrokerLatencyReportingTest.java +++ b/integration-tests/src/slowIntegrationTest/java/pl/allegro/tech/hermes/integrationtests/BrokerLatencyReportingTest.java @@ -1,6 +1,5 @@ package pl.allegro.tech.hermes.integrationtests; -import com.jayway.awaitility.Duration; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Order; @@ -14,7 +13,9 @@ import pl.allegro.tech.hermes.test.helper.client.integration.FrontendTestClient; import pl.allegro.tech.hermes.test.helper.message.TestMessage; -import static com.jayway.awaitility.Awaitility.waitAtMost; +import java.time.Duration; + +import static org.awaitility.Awaitility.waitAtMost; import static pl.allegro.tech.hermes.integrationtests.assertions.HermesAssertions.assertThatMetrics; import static pl.allegro.tech.hermes.test.helper.builder.TopicBuilder.topicWithRandomName; @@ -56,7 +57,7 @@ public void shouldReportBrokerLatencyMetrics() { frontendTestClient.publishUntilSuccess(topic.getQualifiedName(), message.body()); // then - waitAtMost(Duration.FIVE_SECONDS).until(() -> { + waitAtMost(Duration.ofSeconds(5)).untilAsserted(() -> { frontendTestClient.getMetrics() .expectStatus() .isOk() diff --git a/integration-tests/src/slowIntegrationTest/java/pl/allegro/tech/hermes/integrationtests/HermesClientPublishingHttpsTest.java b/integration-tests/src/slowIntegrationTest/java/pl/allegro/tech/hermes/integrationtests/HermesClientPublishingHttpsTest.java index 45a05999af..35bacd5d11 100644 --- a/integration-tests/src/slowIntegrationTest/java/pl/allegro/tech/hermes/integrationtests/HermesClientPublishingHttpsTest.java +++ b/integration-tests/src/slowIntegrationTest/java/pl/allegro/tech/hermes/integrationtests/HermesClientPublishingHttpsTest.java @@ -1,6 +1,5 @@ package pl.allegro.tech.hermes.integrationtests; -import com.jayway.awaitility.Duration; import okhttp3.OkHttpClient; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; @@ -22,6 +21,7 @@ import pl.allegro.tech.hermes.test.helper.message.TestMessage; import java.net.URI; +import java.time.Duration; import javax.net.ssl.X509TrustManager; import static org.assertj.core.api.Assertions.assertThat; @@ -71,7 +71,7 @@ public void shouldCommunicateWithHermesUsingHttp2() { OkHttpHermesSender okHttpHermesSender = new OkHttpHermesSender(getOkHttpClientWithSslContextConfigured()); HermesClient client = hermesClient(okHttpHermesSender) .withRetries(5) - .withRetrySleep(Duration.FIVE_SECONDS.getValueInMS(), Duration.TEN_SECONDS.getValueInMS()) + .withRetrySleep(Duration.ofSeconds(5).toMillis(), Duration.ofSeconds(10).toMillis()) .withURI(URI.create("https://localhost:" + frontend.getSSLPort())) .build(); diff --git a/integration-tests/src/slowIntegrationTest/java/pl/allegro/tech/hermes/integrationtests/KafkaReadinessCheckTest.java b/integration-tests/src/slowIntegrationTest/java/pl/allegro/tech/hermes/integrationtests/KafkaReadinessCheckTest.java index 14eb1feeda..25873bdf84 100644 --- a/integration-tests/src/slowIntegrationTest/java/pl/allegro/tech/hermes/integrationtests/KafkaReadinessCheckTest.java +++ b/integration-tests/src/slowIntegrationTest/java/pl/allegro/tech/hermes/integrationtests/KafkaReadinessCheckTest.java @@ -20,8 +20,8 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -import static com.jayway.awaitility.Awaitility.await; import static java.util.concurrent.TimeUnit.SECONDS; +import static org.awaitility.Awaitility.await; import static org.springframework.http.HttpStatus.SERVICE_UNAVAILABLE; import static pl.allegro.tech.hermes.api.Topic.Ack.ALL; import static pl.allegro.tech.hermes.integrationtests.assertions.HermesAssertions.assertThat; @@ -77,7 +77,7 @@ public void shouldNotBeReadyUntilKafkaClusterIsUp() { // then await().atMost(5, SECONDS) - .until(() -> getStatusReady(hermesFrontend).expectStatus().is2xxSuccessful()); + .untilAsserted(() -> getStatusReady(hermesFrontend).expectStatus().is2xxSuccessful()); getStatusHealth(hermesFrontend).expectStatus().is2xxSuccessful(); // cleanup @@ -129,7 +129,7 @@ public void shouldNotBeReadyUntilThereAreNoUnderReplicatedPartitions() throws Ex // then await().atMost(5, SECONDS) - .until(() -> getStatusReady(hermesFrontend).expectStatus().is2xxSuccessful()); + .untilAsserted(() -> getStatusReady(hermesFrontend).expectStatus().is2xxSuccessful()); getStatusHealth(hermesFrontend).expectStatus().is2xxSuccessful(); // cleanup @@ -173,7 +173,7 @@ public void shouldNotBeReadyUntilThereAreNoOfflinePartitions() throws Exception // then await().atMost(5, SECONDS) - .until(() -> getStatusReady(hermesFrontend).expectStatus().is2xxSuccessful()); + .untilAsserted(() -> getStatusReady(hermesFrontend).expectStatus().is2xxSuccessful()); getStatusHealth(hermesFrontend).expectStatus().is2xxSuccessful(); // cleanup diff --git a/integration-tests/src/slowIntegrationTest/java/pl/allegro/tech/hermes/integrationtests/MessageBufferLoadingTest.java b/integration-tests/src/slowIntegrationTest/java/pl/allegro/tech/hermes/integrationtests/MessageBufferLoadingTest.java index bc1bb936b5..7d679f80dc 100644 --- a/integration-tests/src/slowIntegrationTest/java/pl/allegro/tech/hermes/integrationtests/MessageBufferLoadingTest.java +++ b/integration-tests/src/slowIntegrationTest/java/pl/allegro/tech/hermes/integrationtests/MessageBufferLoadingTest.java @@ -27,12 +27,12 @@ import java.time.Clock; import java.util.Collections; -import static com.jayway.awaitility.Awaitility.await; import static jakarta.ws.rs.core.Response.Status.ACCEPTED; import static java.nio.charset.Charset.defaultCharset; import static java.time.Instant.now; import static java.util.concurrent.TimeUnit.SECONDS; import static org.assertj.core.api.Assertions.assertThat; +import static org.awaitility.Awaitility.await; import static pl.allegro.tech.hermes.api.ContentType.JSON; import static pl.allegro.tech.hermes.frontend.FrontendConfigurationProperties.MESSAGES_LOCAL_STORAGE_DIRECTORY; import static pl.allegro.tech.hermes.frontend.FrontendConfigurationProperties.MESSAGES_LOCAL_STORAGE_ENABLED; @@ -92,7 +92,7 @@ public void shouldBackupMessage() { publisher.publishUntilStatus(topic.getQualifiedName(), "message", ACCEPTED.getStatusCode()); // then - await().atMost(10, SECONDS).until(() -> assertThat(backupRepository.findAll()).hasSize(1)); + await().atMost(10, SECONDS).untilAsserted(() -> assertThat(backupRepository.findAll()).hasSize(1)); } finally { // after diff --git a/integration-tests/src/slowIntegrationTest/java/pl/allegro/tech/hermes/integrationtests/TopicCreationRollbackTest.java b/integration-tests/src/slowIntegrationTest/java/pl/allegro/tech/hermes/integrationtests/TopicCreationRollbackTest.java index b74ff26aac..ab8f26c935 100644 --- a/integration-tests/src/slowIntegrationTest/java/pl/allegro/tech/hermes/integrationtests/TopicCreationRollbackTest.java +++ b/integration-tests/src/slowIntegrationTest/java/pl/allegro/tech/hermes/integrationtests/TopicCreationRollbackTest.java @@ -1,6 +1,5 @@ package pl.allegro.tech.hermes.integrationtests; -import com.jayway.awaitility.Duration; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; @@ -13,11 +12,12 @@ import pl.allegro.tech.hermes.test.helper.containers.KafkaContainerCluster; import pl.allegro.tech.hermes.test.helper.containers.ZookeeperContainer; +import java.time.Duration; import java.util.Map; import java.util.stream.Stream; -import static com.jayway.awaitility.Awaitility.waitAtMost; import static org.assertj.core.api.Assertions.assertThat; +import static org.awaitility.Awaitility.waitAtMost; import static pl.allegro.tech.hermes.api.TopicWithSchema.topicWithSchema; import static pl.allegro.tech.hermes.infrastructure.dc.DefaultDatacenterNameProvider.DEFAULT_DC_NAME; import static pl.allegro.tech.hermes.test.helper.builder.TopicBuilder.topic; @@ -72,7 +72,7 @@ public void topicCreationRollbackShouldNotDeleteTopicOnBroker() { hermesApi.createGroup(Group.from(groupName)); brokerOperations1.createTopic(qualifiedTopicName); - waitAtMost(Duration.ONE_MINUTE).until(() -> assertThat(brokerOperations1.topicExists(qualifiedTopicName)).isTrue()); + waitAtMost(Duration.ofMinutes(1)).untilAsserted(() -> assertThat(brokerOperations1.topicExists(qualifiedTopicName)).isTrue()); // when hermesApi.createTopic((topicWithSchema(topic(groupName, topicName).build()))); From 5c14ec2479455503829b0f6a6da68329828aeb70 Mon Sep 17 00:00:00 2001 From: Maciej Moscicki Date: Mon, 10 Jun 2024 12:49:03 +0200 Subject: [PATCH 59/87] Improve ACK docs (#1866) --- docs/docs/configuration/buffer-persistence.md | 4 +- docs/docs/user/publishing.md | 65 +++++++++++++++---- .../components/console-alert/ConsoleAlert.vue | 11 +++- .../topic/use-create-topic/useCreateTopic.ts | 2 +- .../composables/topic/use-form-topic/types.ts | 1 + .../topic/use-form-topic/useFormTopic.ts | 1 + hermes-console/src/dummy/topic-form.ts | 3 +- hermes-console/src/i18n/en-US/index.ts | 8 +-- hermes-console/src/i18n/en-US/topic-form.ts | 6 ++ .../src/views/topic/topic-form/TopicForm.vue | 11 ++++ 10 files changed, 90 insertions(+), 22 deletions(-) diff --git a/docs/docs/configuration/buffer-persistence.md b/docs/docs/configuration/buffer-persistence.md index a4e3b9ca66..875776f508 100644 --- a/docs/docs/configuration/buffer-persistence.md +++ b/docs/docs/configuration/buffer-persistence.md @@ -1,4 +1,4 @@ -# Publishing buffer persistence +# Publishing buffer persistence [deprecated] Hermes Frontend API has option to register callbacks triggered during different phases of message lifetime: @@ -15,7 +15,7 @@ to disk. Map structure is continuously persisted to disk, as it is stored in off When Hermes Frontend starts up it scans filesystem in search of existing persisted map. If found, it is read and any persisted events are sent to Message Store. This way recovering after crash is fully automatic. If Hermes process or -server crashes, nothing is lost. +server crashes, events that were flushed to disk are recovered. There is additional protection against flooding subscribers with outdated events. When reading events from persisted storage, Hermes filters out messages older than N hours, where N is a system parameter and is set to 3 days by default. diff --git a/docs/docs/user/publishing.md b/docs/docs/user/publishing.md index 034d2ce623..4889a49fcb 100644 --- a/docs/docs/user/publishing.md +++ b/docs/docs/user/publishing.md @@ -134,22 +134,21 @@ Failure statuses: Each topic can define level of acknowledgement (ACK): * leader ACK - only one Kafka node (leader) needs to acknowledge reception of message -* all ACK - all nodes that hold copy of message need to acknowledge reception of message +* all ACK - at least [min.insync.replicas](https://kafka.apache.org/documentation/#brokerconfigs_min.insync.replicas) nodes must acknowledge reception of message -For most of the topic leader ACK is enough. This guarantees roughly 99.999..% reception rate. Only in rare cases, during -Kafka cluster rebalancing or nodes outage Kafka might confirm that message was received, while it was not saved and it -will be lost. +ACK configuration has the following consequences: -What does it mean in practice? Numbers differ per case and they are affected by multiple factors like frequency of -rebalancing taking place on Kafka clusters, Kafka version etc. In our production environment using ACK leader means we falsely -believe message was received by Kafka once per 20 million events. This is a very rough estimate that should show you -the scale, if you need numbers to base your decision on - please conduct own measurements. +- with `ACK leader` message writes are replicated asynchronously, thus the acknowledgment latency will be low. However, message write may be lost +when there is a topic leadership change - e.g. due to rebalance or broker restart. +- with `ACK all` messages writes are synchronously replicated to replicas. Write acknowledgement latency will be much higher than with leader ACK, +it will also have higher variance due to tail latency. However, messages will be persisted as long as the whole replica set does not go down simultaneously. -If you need 100% guarantee that message was saved, force all replicas to send ACK. The downside of this is much longer -response times, they tend to vary a lot as well. Thanks to Hermes buffering (described in paragraphs below), we are able -to guarantee some sane response times to our clients even in *ACK all* mode. +Publishers are advised to select topic ACK level based on their latency and durability requirements. -## Buffering +Hermes also provides a feature called Buffering (described in paragraphs below) which provides consistent write latency +despite long Kafka response times. Note that, however, this mode may decrease message durability for `ACK all` setting. + +## Buffering [deprecated] Hermes administrator can set maximum time, for which Hermes will wait for Kafka acknowledgment. By default, it is set to 65ms. After that time, **202** response is sent to client. Event is kept in Kafka producer buffer and it's delivery will @@ -161,7 +160,7 @@ Kafka is back online. ### Buffer persistence -By default events are buffered in memory only. This raises the question about what happens in case of Hermes node failure +By default, events are buffered in memory only. This raises the question about what happens in case of Hermes node failure (or force kill of process). Hermes Frontend API exposes callbacks that can be used to implement persistence model of buffered events. @@ -169,6 +168,46 @@ Default implementation uses [OpenHFT ChronicleMap](https://github.com/OpenHFT/Ch to disk. Map structure is continuously persisted to disk, as it is stored in offheap memory as [memory mapped file](https://en.wikipedia.org/wiki/Memory-mapped_file). +Using buffering with ACK all setting means that durability of events may be lowered when **202** status code is received. If Hermes instance +is killed before message is spilled to disk or the data on disk becomes corrupted, the message is gone. Thus `ACK all` with **202** status code +is similar to `ACK leader` because a single node failure could cause the message be lost. + +### Deprecation notice +The buffering mechanism in Hermes is considered deprecated and is set to be removed in the future. + +## Remote DC fallback + +Hermes supports a remote datacenter fallback mechanism for [multi datacenter deployments](https://hermes-pubsub.readthedocs.io/en/latest/configuration/kafka-and-zookeeper/#multiple-kafka-and-zookeeper-clusters). + +Fallback is configured on per topic basis, using a `fallbackToRemoteDatacenterEnabled` property: + +```http request +PUT /topics/my.group.my-topic + +{ + "fallbackToRemoteDatacenterEnabled": true, +} +``` + +Using this setting automatically disables buffering mechanism for a topic. + +When using this setting for a topic, Hermes will try to send a message to a local datacenter Kafka first and will fall back to remote datacenter Kafka +if the local send fails. + +Hermes also provides a speculative fallback mechanism which will send messages to remote Kafka if the local Kafka is not responding in a timely manner. +Speculative send is performed after `frontend.kafka.fail-fast-producer.speculativeSendDelay` elapses. + +When using remote DC fallback, Hermes attempts to send a message to Kafka for the duration of `frontend.handlers.maxPublishRequestDuration` property. If after +`maxPublishRequestDuration` Hermes has not received an acknowledgment from Kafka, it will respond with **500** status code to the client. + +Table below summarizes remote fallback configuration options: + +| Option | Scope | Default value | +|--------------------------------------------------------|--------|---------------| +| fallbackToRemoteDatacenterEnabled | topic | false | +| frontend.kafka.fail-fast-producer.speculativeSendDelay | global | 250ms | +| frontend.handlers.maxPublishRequestDuration | global | 500ms | + ## Partition assignment `Partition-Key` header can be used by publishers to specify Kafka `key` which will be used for partition assignment for a message. This will ensure that all messages with given `Partition-Key` will be sent to the same Kafka partition. diff --git a/hermes-console/src/components/console-alert/ConsoleAlert.vue b/hermes-console/src/components/console-alert/ConsoleAlert.vue index 98ab23e19e..8b71ad7936 100644 --- a/hermes-console/src/components/console-alert/ConsoleAlert.vue +++ b/hermes-console/src/components/console-alert/ConsoleAlert.vue @@ -4,6 +4,8 @@ title?: string; text: string; type: 'error' | 'success' | 'warning' | 'info'; + link?: string; + linkDescription?: string; }>(); @@ -15,7 +17,14 @@ :type="props.type" border="start" :icon="icon ?? `\$${type}`" - > + > + {{ linkDescription }} + diff --git a/hermes-console/src/composables/topic/use-create-topic/useCreateTopic.ts b/hermes-console/src/composables/topic/use-create-topic/useCreateTopic.ts index 995bf2134d..cb2ca34f15 100644 --- a/hermes-console/src/composables/topic/use-create-topic/useCreateTopic.ts +++ b/hermes-console/src/composables/topic/use-create-topic/useCreateTopic.ts @@ -103,7 +103,7 @@ function initializeForm(form: Ref): void { trackingEnabled: false, contentType: loadedConfig.value.topic.defaults.contentType, maxMessageSize: defaultMaxMessageSize, - ack: loadedConfig.value.topic.defaults.ack, + ack: '', schema: '', }; } diff --git a/hermes-console/src/composables/topic/use-form-topic/types.ts b/hermes-console/src/composables/topic/use-form-topic/types.ts index 55d2ef1a5b..57c27946ad 100644 --- a/hermes-console/src/composables/topic/use-form-topic/types.ts +++ b/hermes-console/src/composables/topic/use-form-topic/types.ts @@ -53,6 +53,7 @@ export interface FormValidators { retentionTimeDuration: FieldValidator[]; maxMessageSize: FieldValidator[]; offlineRetentionTime: FieldValidator[]; + ack: FieldValidator[]; } export interface RawDataSources { diff --git a/hermes-console/src/composables/topic/use-form-topic/useFormTopic.ts b/hermes-console/src/composables/topic/use-form-topic/useFormTopic.ts index adb5f29dd7..f5ac01b391 100644 --- a/hermes-console/src/composables/topic/use-form-topic/useFormTopic.ts +++ b/hermes-console/src/composables/topic/use-form-topic/useFormTopic.ts @@ -41,6 +41,7 @@ function formValidators(): FormValidators { retentionTimeDuration: [required(), min(0), max(7)], maxMessageSize: [required(), min(0)], offlineRetentionTime: [required(), min(0)], + ack: [required()], }; } diff --git a/hermes-console/src/dummy/topic-form.ts b/hermes-console/src/dummy/topic-form.ts index f6f667fb36..aeca54e7c9 100644 --- a/hermes-console/src/dummy/topic-form.ts +++ b/hermes-console/src/dummy/topic-form.ts @@ -46,6 +46,7 @@ export const dummyTopicFormValidator = { retentionTimeDuration: [required(), min(0), max(7)], maxMessageSize: [required(), min(0)], offlineRetentionTime: [required(), min(0)], + ack: [required()], }; export const dummyContentTypes = [ @@ -123,7 +124,7 @@ export const dummyInitializedTopicForm = { trackingEnabled: false, contentType: dummyAppConfig.topic.defaults.contentType, maxMessageSize: defaultMaxMessageSize, - ack: dummyAppConfig.topic.defaults.ack, + ack: '', schema: '', }; diff --git a/hermes-console/src/i18n/en-US/index.ts b/hermes-console/src/i18n/en-US/index.ts index da8fd9da5f..37ce1b5b14 100644 --- a/hermes-console/src/i18n/en-US/index.ts +++ b/hermes-console/src/i18n/en-US/index.ts @@ -327,10 +327,10 @@ const en_US = { modificationDate: 'Modification date', tooltips: { acknowledgement: - 'Specifies the strength of guarantees that acknowledged message was indeed persisted. In ' + - '"Leader" mode ACK is required only from topic leader, which is fast and gives 99.99999% guarantee. It might ' + - 'be not enough when cluster is unstable. "All" mode means message needs to be saved on all replicas before ' + - 'sending ACK, which is quite slow but gives 100% guarantee that message has been persisted.', + 'Specifies the strength of guarantees that acknowledged message was indeed persisted. ' + + 'With `ACK leader` message writes are replicated asynchronously, thus the acknowledgment latency will be low. However, message write may be lost when there is a topic leadership change - e.g. due to rebalance or broker restart. ' + + 'With `ACK all` messages writes are synchronously replicated to replicas. Write acknowledgement latency will be much higher than with leader ACK,' + + ' it will also have higher variance due to tail latency. However, messages will be persisted as long as the whole replica set does not go down simultaneously.', retentionTime: 'For how many hours/days message is available for subscribers after being published.', authorizedPublishers: diff --git a/hermes-console/src/i18n/en-US/topic-form.ts b/hermes-console/src/i18n/en-US/topic-form.ts index 478898b459..a857db23fc 100644 --- a/hermes-console/src/i18n/en-US/topic-form.ts +++ b/hermes-console/src/i18n/en-US/topic-form.ts @@ -32,6 +32,12 @@ const messages = { days: 'DAYS', }, ack: 'Kafka ACK level', + ackHelpTitle: 'ACK level is very important', + ackHelpText: + 'Set ACK level according to your durability and latency requirements, see: ', + ackHelpLink: + 'https://hermes-pubsub.readthedocs.io/en/latest/user/publishing/#acknowledgment-level', + ackHelpLinkDescription: 'ACK docs.', contentType: 'Content type', maxMessageSize: { label: 'Max message size', diff --git a/hermes-console/src/views/topic/topic-form/TopicForm.vue b/hermes-console/src/views/topic/topic-form/TopicForm.vue index 5c47f9fc9d..6359edad8d 100644 --- a/hermes-console/src/views/topic/topic-form/TopicForm.vue +++ b/hermes-console/src/views/topic/topic-form/TopicForm.vue @@ -218,8 +218,19 @@ :label="$t('topicForm.fields.retentionTime.duration')" /> + + + From 5587796345532d13a17faed10c4481ea5bb2a08c Mon Sep 17 00:00:00 2001 From: Maciej Moscicki Date: Wed, 12 Jun 2024 10:55:58 +0200 Subject: [PATCH 60/87] transition to error state upon failing publishing handler (#1868) --- .../hermes/frontend/publishing/handlers/PublishingHandler.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/handlers/PublishingHandler.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/handlers/PublishingHandler.java index 9f795fb3b5..aa20a45520 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/handlers/PublishingHandler.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/publishing/handlers/PublishingHandler.java @@ -35,6 +35,9 @@ public void handleRequest(HttpServerExchange exchange) { try { handle(exchange); } catch (RuntimeException e) { + AttachmentContent attachment = exchange.getAttachment(AttachmentContent.KEY); + MessageState messageState = attachment.getMessageState(); + messageState.setErrorInSendingToKafka(); messageErrorProcessor.sendAndLog(exchange, "Exception while publishing message to a broker.", e); } }); From b594e7e10ee49e36eba3c7380a1fa1e81d77c105 Mon Sep 17 00:00:00 2001 From: Maciej Moscicki Date: Wed, 12 Jun 2024 16:12:21 +0200 Subject: [PATCH 61/87] make message thread safe (#1870) --- .../hermes/consumers/consumer/Message.java | 108 ++++++++++-------- 1 file changed, 63 insertions(+), 45 deletions(-) diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/Message.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/Message.java index eec94009f1..c0d5c5df17 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/Message.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/Message.java @@ -24,33 +24,35 @@ import static java.util.stream.Collectors.toList; +/** + * Implementation note: this class is partially mutable and may be accessed from multiple + * threads involved in message lifecycle, it must be thread safe. + */ public class Message implements FilterableMessage { - private String id; - private PartitionOffset partitionOffset; + private final String id; + private final PartitionOffset partitionOffset; - private String topic; - private String subscription; - private boolean hasSubscriptionIdentityHeaders; - private ContentType contentType; - private Optional> schema; + private final String topic; + private final String subscription; + private final boolean hasSubscriptionIdentityHeaders; + private final ContentType contentType; + private final Optional> schema; - private long publishingTimestamp; - private long readingTimestamp; - private byte[] data; + private final long publishingTimestamp; + private final long readingTimestamp; + private final byte[] data; private int retryCounter = 0; - private long partitionAssignmentTerm = -1; - private Map externalMetadata = Collections.emptyMap(); + private final long partitionAssignmentTerm; + private final Map externalMetadata; - private List
additionalHeaders = Collections.emptyList(); + private final List
additionalHeaders; - private Set succeededUris = Sets.newHashSet(); + private final Set succeededUris = Sets.newHashSet(); private long currentMessageBackoff = -1; - private Message() {} - public Message(String id, String topic, byte[] content, @@ -118,12 +120,12 @@ public boolean isTtlExceeded(long ttlMillis) { return currentTimestamp > readingTimestamp + ttlMillis; } - public void incrementRetryCounter(Collection succeededUris) { + public synchronized void incrementRetryCounter(Collection succeededUris) { this.retryCounter++; this.succeededUris.addAll(succeededUris.stream().map(URI::toString).collect(toList())); } - public int getRetryCounter() { + public synchronized int getRetryCounter() { return retryCounter; } @@ -138,14 +140,14 @@ public String getId() { @Override public Map getExternalMetadata() { - return Collections.unmodifiableMap(externalMetadata); + return externalMetadata; } public List
getAdditionalHeaders() { - return Collections.unmodifiableList(additionalHeaders); + return additionalHeaders; } - public long updateAndGetCurrentMessageBackoff(SubscriptionPolicy subscriptionPolicy) { + public synchronized long updateAndGetCurrentMessageBackoff(SubscriptionPolicy subscriptionPolicy) { if (currentMessageBackoff == -1) { currentMessageBackoff = subscriptionPolicy.getMessageBackoff(); } else { @@ -184,7 +186,7 @@ public PartitionOffset getPartitionOffset() { return partitionOffset; } - public boolean hasNotBeenSentTo(String uri) { + public synchronized boolean hasNotBeenSentTo(String uri) { return !succeededUris.contains(uri); } @@ -201,64 +203,80 @@ public String getSubscription() { } public static class Builder { + private String id; + private PartitionOffset partitionOffset; + + private String topic; + private String subscription; + private boolean hasSubscriptionIdentityHeaders; + private ContentType contentType; + private Optional> schema; + + private long publishingTimestamp; + private long readingTimestamp; + private byte[] data; + + private long partitionAssignmentTerm = -1; + private Map externalMetadata = Collections.emptyMap(); - private final Message message; + private List
additionalHeaders = Collections.emptyList(); public Builder() { - message = new Message(); } public Builder fromMessage(Message message) { - this.message.id = message.getId(); - this.message.data = message.getData(); - this.message.contentType = message.getContentType(); - this.message.topic = message.getTopic(); - this.message.subscription = message.getSubscription(); - this.message.hasSubscriptionIdentityHeaders = message.hasSubscriptionIdentityHeaders(); - this.message.publishingTimestamp = message.getPublishingTimestamp(); - this.message.readingTimestamp = message.getReadingTimestamp(); - this.message.partitionOffset = message.partitionOffset; - this.message.partitionAssignmentTerm = message.partitionAssignmentTerm; - this.message.externalMetadata = message.getExternalMetadata(); - this.message.additionalHeaders = message.getAdditionalHeaders(); - this.message.schema = message.getSchema(); + this.id = message.getId(); + this.data = message.getData(); + this.contentType = message.getContentType(); + this.topic = message.getTopic(); + this.subscription = message.getSubscription(); + this.hasSubscriptionIdentityHeaders = message.hasSubscriptionIdentityHeaders(); + this.publishingTimestamp = message.getPublishingTimestamp(); + this.readingTimestamp = message.getReadingTimestamp(); + this.partitionOffset = message.partitionOffset; + this.partitionAssignmentTerm = message.partitionAssignmentTerm; + this.externalMetadata = message.getExternalMetadata(); + this.additionalHeaders = message.getAdditionalHeaders(); + this.schema = message.getSchema(); return this; } - public Builder withData(byte [] data) { - this.message.data = data; + public Builder withData(byte[] data) { + this.data = data; return this; } public Builder withSchema(CompiledSchema schema) { - this.message.schema = Optional.of(schema); + this.schema = Optional.of(schema); return this; } public Builder withExternalMetadata(Map externalMetadata) { - this.message.externalMetadata = ImmutableMap.copyOf(externalMetadata); + this.externalMetadata = ImmutableMap.copyOf(externalMetadata); return this; } public Builder withAdditionalHeaders(List
additionalHeaders) { - this.message.additionalHeaders = ImmutableList.copyOf(additionalHeaders); + this.additionalHeaders = ImmutableList.copyOf(additionalHeaders); return this; } public Builder withContentType(ContentType contentType) { - this.message.contentType = contentType; + this.contentType = contentType; return this; } public Builder withNoSchema() { - this.message.schema = Optional.empty(); + this.schema = Optional.empty(); return this; } public Message build() { - return message; + return new Message( + id, topic, data, contentType, schema, publishingTimestamp, readingTimestamp, partitionOffset, partitionAssignmentTerm, externalMetadata, additionalHeaders, subscription, hasSubscriptionIdentityHeaders + ); } } } From a478894c8706573e11508aac54cd34463f7a4c2e Mon Sep 17 00:00:00 2001 From: Mateusz <76775507+szczygiel-m@users.noreply.github.com> Date: Fri, 14 Jun 2024 12:08:25 +0200 Subject: [PATCH 62/87] Added measurement point to consumer profiling (#1869) --- .../tech/hermes/consumers/consumer/ConsumerMessageSender.java | 1 + .../tech/hermes/consumers/consumer/profiling/Measurement.java | 1 + .../tech/hermes/integrationtests/ConsumerProfilingTest.java | 3 +++ 3 files changed, 5 insertions(+) diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSender.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSender.java index eb37f61a85..151235730d 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSender.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSender.java @@ -118,6 +118,7 @@ public void sendAsync(Message message, ConsumerProfiler profiler) { } private void sendAsync(Message message, int delayMillis, ConsumerProfiler profiler) { + profiler.measure(Measurement.SCHEDULE_MESSAGE_SENDING); retrySingleThreadExecutor.schedule(() -> sendMessage(message, profiler), delayMillis, TimeUnit.MILLISECONDS); } diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/profiling/Measurement.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/profiling/Measurement.java index 2de07271fb..a601674687 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/profiling/Measurement.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/profiling/Measurement.java @@ -8,6 +8,7 @@ public class Measurement { public static final String MESSAGE_CONVERSION = "messageConverter.convert"; public static final String OFFER_INFLIGHT_OFFSET = "offsetQueue.offerInflightOffset"; public static final String TRACKERS_LOG_INFLIGHT = "trackers.logInflight"; + public static final String SCHEDULE_MESSAGE_SENDING = "retrySingleThreadExecutor.schedule"; public static final String ACQUIRE_RATE_LIMITER = "acquireRateLimiter"; public static final String MESSAGE_SENDER_SEND = "messageSender.send"; public static final String HANDLERS = "handlers"; diff --git a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/ConsumerProfilingTest.java b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/ConsumerProfilingTest.java index e673317c2e..64148f67d0 100644 --- a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/ConsumerProfilingTest.java +++ b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/ConsumerProfilingTest.java @@ -148,6 +148,7 @@ public void shouldProfileSuccessfulMessageProcessing() { Measurement.MESSAGE_CONVERSION, Measurement.OFFER_INFLIGHT_OFFSET, Measurement.TRACKERS_LOG_INFLIGHT, + Measurement.SCHEDULE_MESSAGE_SENDING, Measurement.ACQUIRE_RATE_LIMITER, Measurement.MESSAGE_SENDER_SEND, Measurement.HANDLERS, @@ -183,6 +184,7 @@ public void shouldProfileDiscardedMessageProcessing() { Measurement.MESSAGE_CONVERSION, Measurement.OFFER_INFLIGHT_OFFSET, Measurement.TRACKERS_LOG_INFLIGHT, + Measurement.SCHEDULE_MESSAGE_SENDING, Measurement.ACQUIRE_RATE_LIMITER, Measurement.MESSAGE_SENDER_SEND, Measurement.HANDLERS, @@ -218,6 +220,7 @@ public void shouldProfileRetriedMessageProcessing() { Measurement.MESSAGE_CONVERSION, Measurement.OFFER_INFLIGHT_OFFSET, Measurement.TRACKERS_LOG_INFLIGHT, + Measurement.SCHEDULE_MESSAGE_SENDING, Measurement.ACQUIRE_RATE_LIMITER, Measurement.MESSAGE_SENDER_SEND, Measurement.HANDLERS, From 5754069191c3378613268b1f19122656c8586439 Mon Sep 17 00:00:00 2001 From: Maciej Moscicki Date: Thu, 20 Jun 2024 13:13:42 +0200 Subject: [PATCH 63/87] downgrade kafka clients (#1872) --- build.gradle | 2 +- .../tech/hermes/frontend/producer/kafka/KafkaMessageSender.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/build.gradle b/build.gradle index 0b15a7e9bd..0c855d7a33 100644 --- a/build.gradle +++ b/build.gradle @@ -48,7 +48,7 @@ allprojects { targetCompatibility = JavaVersion.VERSION_17 project.ext.versions = [ - kafka : '3.6.2', + kafka : '2.8.2', guava : '33.1.0-jre', jackson : '2.17.0', jersey : '3.1.6', diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSender.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSender.java index 6e00173e48..fd62a9142c 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSender.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/producer/kafka/KafkaMessageSender.java @@ -96,7 +96,7 @@ public void send(ProducerRecord producerRecord, private static RecordMetadata exceptionalRecordMetadata(CachedTopic cachedTopic) { var tp = new TopicPartition(cachedTopic.getKafkaTopics().getPrimary().name().asString(), RecordMetadata.UNKNOWN_PARTITION); - return new RecordMetadata(tp, -1, -1, RecordBatch.NO_TIMESTAMP, -1, -1); + return new RecordMetadata(tp, -1, -1, RecordBatch.NO_TIMESTAMP, -1L, -1, -1); } List loadPartitionMetadataFor(String topic) { From 104e710ba3812182734da836e23f774e7311e2ca Mon Sep 17 00:00:00 2001 From: Maciej Moscicki Date: Mon, 24 Jun 2024 15:27:27 +0200 Subject: [PATCH 64/87] InflightSize and retentionTime validation improvements (#1874) * fix inflight validation * fix RetentionTime validation * remove redundant e2e test --- .../tech/hermes/api/RetentionTime.java | 9 +- .../tech/hermes/api/SubscriptionPolicy.java | 3 - .../validator/SubscriptionValidator.java | 38 ++++ .../topic/validator/TopicValidator.java | 49 +++++ .../InflightSizeValidatorTest.groovy | 99 +++++++++- ...lidatorWithRealApiPreconditionsTest.groovy | 173 ++++++++++++++++-- .../SubscriptionManagementTest.java | 40 ---- 7 files changed, 344 insertions(+), 67 deletions(-) diff --git a/hermes-api/src/main/java/pl/allegro/tech/hermes/api/RetentionTime.java b/hermes-api/src/main/java/pl/allegro/tech/hermes/api/RetentionTime.java index bf6478b702..762465ce9b 100644 --- a/hermes-api/src/main/java/pl/allegro/tech/hermes/api/RetentionTime.java +++ b/hermes-api/src/main/java/pl/allegro/tech/hermes/api/RetentionTime.java @@ -2,18 +2,21 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; -import jakarta.validation.constraints.Max; import jakarta.validation.constraints.Min; -import pl.allegro.tech.hermes.api.constraints.AdminPermitted; +import java.util.EnumSet; import java.util.Objects; +import java.util.Set; import java.util.concurrent.TimeUnit; public class RetentionTime { private static final TimeUnit DEFAULT_UNIT = TimeUnit.DAYS; + public static RetentionTime MAX = new RetentionTime(7, TimeUnit.DAYS); + public static Set allowedUnits = EnumSet.of(TimeUnit.SECONDS, TimeUnit.MINUTES, TimeUnit.HOURS, TimeUnit.DAYS); + + @Min(0) - @Max(value = 7, groups = AdminPermitted.class) private final int duration; private final TimeUnit retentionUnit; diff --git a/hermes-api/src/main/java/pl/allegro/tech/hermes/api/SubscriptionPolicy.java b/hermes-api/src/main/java/pl/allegro/tech/hermes/api/SubscriptionPolicy.java index b1f33c0a78..275013ece1 100644 --- a/hermes-api/src/main/java/pl/allegro/tech/hermes/api/SubscriptionPolicy.java +++ b/hermes-api/src/main/java/pl/allegro/tech/hermes/api/SubscriptionPolicy.java @@ -5,8 +5,6 @@ import jakarta.annotation.Nullable; import jakarta.validation.constraints.Max; import jakarta.validation.constraints.Min; -import jakarta.validation.constraints.Null; -import pl.allegro.tech.hermes.api.constraints.AdminPermitted; import pl.allegro.tech.hermes.api.helpers.Patch; import java.util.Map; @@ -42,7 +40,6 @@ public class SubscriptionPolicy { private int socketTimeout = DEFAULT_SOCKET_TIMEOUT; @Min(1) - @Null(groups = AdminPermitted.class) private Integer inflightSize; @Min(0) diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/subscription/validator/SubscriptionValidator.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/subscription/validator/SubscriptionValidator.java index 2735eda153..095eecef1c 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/subscription/validator/SubscriptionValidator.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/subscription/validator/SubscriptionValidator.java @@ -1,6 +1,7 @@ package pl.allegro.tech.hermes.management.domain.subscription.validator; import pl.allegro.tech.hermes.api.Subscription; +import pl.allegro.tech.hermes.api.SubscriptionPolicy; import pl.allegro.tech.hermes.api.Topic; import pl.allegro.tech.hermes.domain.subscription.SubscriptionAlreadyExistsException; import pl.allegro.tech.hermes.domain.subscription.SubscriptionRepository; @@ -11,6 +12,7 @@ import pl.allegro.tech.hermes.management.domain.topic.TopicService; import java.util.List; +import java.util.Objects; public class SubscriptionValidator { @@ -45,6 +47,7 @@ public void checkCreation(Subscription toCheck, RequestUser createdBy) { checkOwner(toCheck); checkEndpoint(toCheck); checkPermissionsToManageSubscription(toCheck, createdBy); + ensureCreatedSubscriptionInflightIsValid(toCheck, createdBy); Topic topic = topicService.getTopicDetails(toCheck.getTopicName()); checkFilters(toCheck, topic); checkIfSubscribingToTopicIsAllowed(toCheck, topic, createdBy); @@ -58,6 +61,7 @@ public void checkModification(Subscription toCheck, RequestUser modifiedBy, Subs checkOwner(toCheck); checkEndpoint(toCheck); checkPermissionsToManageSubscription(toCheck, modifiedBy); + ensureUpdatedSubscriptionInflightIsValid(previous, toCheck, modifiedBy); Topic topic = topicService.getTopicDetails(toCheck.getTopicName()); checkFilters(toCheck, topic); if (!toCheck.getEndpoint().equals(previous.getEndpoint())) { @@ -111,4 +115,38 @@ private void checkPermissionsToManageSubscription(Subscription toCheck, RequestU ); } } + + private void ensureCreatedSubscriptionInflightIsValid(Subscription subscription, RequestUser requester) { + if (requester.isAdmin()) { + return; + } + SubscriptionPolicy subscriptionPolicy = subscription.getSerialSubscriptionPolicy(); + if (subscriptionPolicy == null) { + return; + } + if (subscriptionPolicy.getInflightSize() != null) { + throw new SubscriptionValidationException( + "Inflight size can't be set by non admin users" + ); + } + } + + private void ensureUpdatedSubscriptionInflightIsValid(Subscription previous, Subscription updated, RequestUser requester) { + if (requester.isAdmin()) { + return; + } + + SubscriptionPolicy updatedSubscriptionPolicy = updated.getSerialSubscriptionPolicy(); + if (updatedSubscriptionPolicy == null) { + return; + } + Integer updatedInflight = updatedSubscriptionPolicy.getInflightSize(); + + SubscriptionPolicy previousSubscriptionPolicy = previous.getSerialSubscriptionPolicy(); + Integer previousInflight = previousSubscriptionPolicy == null ? null : previousSubscriptionPolicy.getInflightSize(); + + if (!Objects.equals(previousInflight, updatedInflight)) { + throw new SubscriptionValidationException(String.format("Inflight size can't be changed by non admin users. Changed from: %s, to: %s", previousInflight, updatedInflight)); + } + } } diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/validator/TopicValidator.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/validator/TopicValidator.java index b1a425a9cc..eaa69ef2cd 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/validator/TopicValidator.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/validator/TopicValidator.java @@ -6,6 +6,7 @@ import pl.allegro.tech.hermes.api.PublishingChaosPolicy; import pl.allegro.tech.hermes.api.PublishingChaosPolicy.ChaosMode; import pl.allegro.tech.hermes.api.PublishingChaosPolicy.ChaosPolicy; +import pl.allegro.tech.hermes.api.RetentionTime; import pl.allegro.tech.hermes.api.Topic; import pl.allegro.tech.hermes.management.api.validator.ApiPreconditions; import pl.allegro.tech.hermes.management.domain.auth.RequestUser; @@ -15,6 +16,9 @@ import pl.allegro.tech.hermes.schema.SchemaNotFoundException; import pl.allegro.tech.hermes.schema.SchemaRepository; +import java.util.Arrays; +import java.util.concurrent.TimeUnit; + @Component public class TopicValidator { @@ -59,6 +63,8 @@ public void ensureCreatedTopicIsValid(Topic created, RequestUser createdBy, Crea if (!creatorRights.allowedToManage(created)) { throw new TopicValidationException("Provide an owner that includes you, you would not be able to manage this topic later"); } + + ensureCreatedTopicRetentionTimeValid(created, createdBy); } public void ensureUpdatedTopicIsValid(Topic updated, Topic previous, RequestUser modifiedBy) { @@ -91,6 +97,49 @@ public void ensureUpdatedTopicIsValid(Topic updated, Topic previous, RequestUser } else if (migrationFromJsonTypeFlagChangedToFalse(updated, previous)) { throw new TopicValidationException("Cannot migrate back to JSON!"); } + + ensureUpdatedTopicRetentionTimeValid(updated, previous, modifiedBy); + } + + private void ensureCreatedTopicRetentionTimeValid(Topic created, RequestUser modifiedBy) { + if (modifiedBy.isAdmin()) { + return; + } + + checkTopicRetentionTimeUnit(created.getRetentionTime().getRetentionUnit()); + + long seconds = created.getRetentionTime().getRetentionUnit().toSeconds(created.getRetentionTime().getDuration()); + + checkTopicRetentionLimit(seconds); + } + + private void ensureUpdatedTopicRetentionTimeValid(Topic updated, Topic previous, RequestUser modifiedBy) { + if (modifiedBy.isAdmin()) { + return; + } + + checkTopicRetentionTimeUnit(updated.getRetentionTime().getRetentionUnit()); + + long updatedSeconds = updated.getRetentionTime().getRetentionUnit().toSeconds(updated.getRetentionTime().getDuration()); + long previousSeconds = previous.getRetentionTime().getRetentionUnit().toSeconds(previous.getRetentionTime().getDuration()); + + if (updatedSeconds == previousSeconds) { + return; + } + + checkTopicRetentionLimit(updatedSeconds); + } + + private void checkTopicRetentionTimeUnit(TimeUnit toCheck) { + if (!RetentionTime.allowedUnits.contains(toCheck)) { + throw new TopicValidationException("Retention time unit must be one of: " + Arrays.toString(RetentionTime.allowedUnits.toArray())); + } + } + + private void checkTopicRetentionLimit(long retentionSeconds) { + if (retentionSeconds > RetentionTime.MAX.getRetentionUnit().toSeconds(RetentionTime.MAX.getDuration())) { + throw new TopicValidationException("Retention time larger than 7 days can't be configured by non admin users"); + } } private boolean contentTypeChanged(Topic updated, Topic previous) { diff --git a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/domain/subscription/validator/InflightSizeValidatorTest.groovy b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/domain/subscription/validator/InflightSizeValidatorTest.groovy index 811447481f..74c534654e 100644 --- a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/domain/subscription/validator/InflightSizeValidatorTest.groovy +++ b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/domain/subscription/validator/InflightSizeValidatorTest.groovy @@ -49,8 +49,8 @@ class InflightSizeValidatorTest extends Specification { subscriptionValidator.checkCreation(subscription, regularUser) then: - def exception = thrown(ConstraintViolationException) - exception.message == "serialSubscriptionPolicy.inflightSize: must be null" + def exception = thrown(SubscriptionValidationException) + exception.message == "Inflight size can't be set by non admin users" } def "creating subscription with inflight size should be allowed for admin users"() { @@ -91,8 +91,101 @@ class InflightSizeValidatorTest extends Specification { user << [regularUser, admin] } - private static Subscription subscriptionWithInflight(Integer inflightSize) { + def "changing inflight should not be allowed for users"() { + given: + def previous = subscriptionWithInflight(previousInflight) + def updated = subscriptionWithInflight(updatedInflight) + when: + subscriptionValidator.checkModification(updated, regularUser, previous) + + then: + def exception = thrown(SubscriptionValidationException) + exception.message == message + + where: + previousInflight | updatedInflight || message + null | 60 || "Inflight size can't be changed by non admin users. Changed from: null, to: 60" + 60 | null || "Inflight size can't be changed by non admin users. Changed from: 60, to: null" + 60 | 120 || "Inflight size can't be changed by non admin users. Changed from: 60, to: 120" + } + + def "changing inflight should be allowed for admins"() { + given: + def previous = subscriptionWithInflight(previousInflight) + def updated = subscriptionWithInflight(updatedInflight) + when: + subscriptionValidator.checkModification(updated, admin, previous) + + then: + noExceptionThrown() + + where: + previousInflight | updatedInflight + null | 60 + 60 | null + 60 | 120 + } + + def "updating subscription with non default inflight should be allowed for all users"() { + given: + def previous = subscriptionWithInflight(120, "lorem ipsum") + def updated = subscriptionWithInflight(120, "dolor sit amet") + + when: + subscriptionValidator.checkModification(updated, user, previous) + + then: + noExceptionThrown() + + where: + user << [regularUser, admin] + } + + def "resetting subscription policy should be allowed for all users"() { + def previous = subscriptionWithInflight(120) + def updated = subscriptionWithInflight(null) + updated.serialSubscriptionPolicy = null + + when: + subscriptionValidator.checkModification(updated, user, previous) + + then: + noExceptionThrown() + + where: + user << [regularUser, admin] + } + + def "changing subscription policy from null to policy with not null inflight should not be allowed for regular user"() { + def previous = subscriptionWithInflight(null) + previous.serialSubscriptionPolicy = null + + def updated = subscriptionWithInflight(120) + + when: + subscriptionValidator.checkModification(updated, regularUser, previous) + + then: + def exception = thrown(SubscriptionValidationException) + exception.message == "Inflight size can't be changed by non admin users. Changed from: null, to: 120" + } + + def "changing subscription policy from null to policy with not null inflight should be allowed for admin"() { + def previous = subscriptionWithInflight(null) + previous.serialSubscriptionPolicy = null + + def updated = subscriptionWithInflight(120) + + when: + subscriptionValidator.checkModification(updated, admin, previous) + + then: + noExceptionThrown() + } + + private static Subscription subscriptionWithInflight(Integer inflightSize, String description = "lorem ipsum") { return subscription("group.topic", "subscription") + .withDescription(description) .withSubscriptionPolicy( SubscriptionPolicy.Builder.subscriptionPolicy() .withInflightSize(inflightSize) diff --git a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/domain/topic/validator/TopicValidatorWithRealApiPreconditionsTest.groovy b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/domain/topic/validator/TopicValidatorWithRealApiPreconditionsTest.groovy index bca9772132..14aa7ec92e 100644 --- a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/domain/topic/validator/TopicValidatorWithRealApiPreconditionsTest.groovy +++ b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/domain/topic/validator/TopicValidatorWithRealApiPreconditionsTest.groovy @@ -1,6 +1,5 @@ package pl.allegro.tech.hermes.management.domain.topic.validator -import jakarta.validation.ConstraintViolationException import pl.allegro.tech.hermes.api.RetentionTime import pl.allegro.tech.hermes.management.api.validator.ApiPreconditions import pl.allegro.tech.hermes.management.domain.auth.TestRequestUser @@ -10,14 +9,17 @@ import spock.lang.Specification import spock.lang.Subject import spock.lang.Unroll +import java.util.concurrent.TimeUnit + import static java.util.concurrent.TimeUnit.DAYS +import static java.util.concurrent.TimeUnit.HOURS +import static java.util.concurrent.TimeUnit.MINUTES +import static java.util.concurrent.TimeUnit.SECONDS import static pl.allegro.tech.hermes.test.helper.builder.TopicBuilder.topic class TopicValidatorWithRealApiPreconditionsTest extends Specification { private static MANAGEABLE = { true } - private static retentionTime7Days = new RetentionTime(7, DAYS) - private static retentionTime8Days = new RetentionTime(8, DAYS) private static regularUser = new TestRequestUser("regularUser", false) private static admin = new TestRequestUser("admin", true) @@ -30,10 +32,10 @@ class TopicValidatorWithRealApiPreconditionsTest extends Specification { def topicValidator = new TopicValidator(ownerDescriptorValidator, contentTypeWhitelistValidator, topicLabelsValidator, schemaRepository, new ApiPreconditions()) @Unroll - def "creating topic with 7 days retention time should be valid"() { + def "creating and updating topic with up to 7 days retention time should be valid"() { given: def topic = topic('group.topic') - .withRetentionTime(retentionTime7Days) + .withRetentionTime(retentionTime) .build() when: @@ -43,26 +45,47 @@ class TopicValidatorWithRealApiPreconditionsTest extends Specification { noExceptionThrown() where: - user << [regularUser, admin] + retentionTime | user + new RetentionTime(1, MINUTES) | admin + new RetentionTime(1, MINUTES) | regularUser + new RetentionTime(1337, MINUTES) | admin + new RetentionTime(1337, MINUTES) | regularUser + new RetentionTime(24, HOURS) | admin + new RetentionTime(24, HOURS) | regularUser + new RetentionTime(72, HOURS) | admin + new RetentionTime(72, HOURS) | regularUser + new RetentionTime(1, DAYS) | admin + new RetentionTime(1, DAYS) | regularUser + new RetentionTime(7, DAYS) | admin + new RetentionTime(7, DAYS) | regularUser } - def "creating topic with 8 days retention time should be invalid for regular user"() { + def "creating topic with over 7 days of retention time should be invalid for regular user"() { given: def topic = topic('group.topic') - .withRetentionTime(retentionTime8Days) + .withRetentionTime(retentionTime) .build() when: topicValidator.ensureCreatedTopicIsValid(topic, regularUser, MANAGEABLE) then: - thrown ConstraintViolationException + def exception = thrown(TopicValidationException) + exception.message == "Retention time larger than 7 days can't be configured by non admin users" + + where: + retentionTime << [ + new RetentionTime(8, DAYS), + new RetentionTime(7 * 24 + 1, HOURS), + new RetentionTime(7 * 24 * 60 + 1, MINUTES), + new RetentionTime(7 * 24 * 60 * 60 + 1, SECONDS) + ] } - def "creating topic with 8 days retention time should be valid for admin"() { + def "creating topic with over 7 days retention time should be valid for admin"() { given: def topic = topic('group.topic') - .withRetentionTime(retentionTime8Days) + .withRetentionTime(retentionTime) .build() when: @@ -70,14 +93,22 @@ class TopicValidatorWithRealApiPreconditionsTest extends Specification { then: noExceptionThrown() + + where: + retentionTime << [ + new RetentionTime(8, DAYS), + new RetentionTime(7 * 24 + 1, HOURS), + new RetentionTime(7 * 24 * 60 + 1, MINUTES), + new RetentionTime(7 * 24 * 60 * 60 + 1, SECONDS) + ] } @Unroll - def "updating topic with 7 days retention time should be valid"() { + def "updating topic with up to 7 days of retention time should be valid"() { given: def existingTopic = topic('group.topic').build() def updatedTopic = topic('group.topic') - .withRetentionTime(retentionTime7Days) + .withRetentionTime(retentionTime) .build() when: @@ -87,28 +118,112 @@ class TopicValidatorWithRealApiPreconditionsTest extends Specification { noExceptionThrown() where: - user << [regularUser, admin] + retentionTime | user + new RetentionTime(1, MINUTES) | admin + new RetentionTime(1, MINUTES) | regularUser + new RetentionTime(1337, MINUTES) | admin + new RetentionTime(1337, MINUTES) | regularUser + new RetentionTime(24, HOURS) | admin + new RetentionTime(24, HOURS) | regularUser + new RetentionTime(72, HOURS) | admin + new RetentionTime(72, HOURS) | regularUser + new RetentionTime(1, DAYS) | admin + new RetentionTime(1, DAYS) | regularUser + new RetentionTime(7, DAYS) | admin + new RetentionTime(7, DAYS) | regularUser } - def "updating topic with 8 days retention time should be invalid for regular user"() { + def "updating topic with over 7 days of retention time should be invalid for regular user"() { given: def existingTopic = topic('group.topic').build() def updatedTopic = topic('group.topic') - .withRetentionTime(retentionTime8Days) + .withRetentionTime(retentionTime) .build() when: topicValidator.ensureUpdatedTopicIsValid(updatedTopic, existingTopic, regularUser) then: - thrown ConstraintViolationException + def exception = thrown(TopicValidationException) + exception.message == "Retention time larger than 7 days can't be configured by non admin users" + + where: + retentionTime << [ + new RetentionTime(8, DAYS), + new RetentionTime(7 * 24 + 1, HOURS), + new RetentionTime(7 * 24 * 60 + 1, MINUTES), + new RetentionTime(7 * 24 * 60 * 60 + 1, SECONDS) + ] } def "updating topic with 8 days retention time should be valid for admin"() { given: def existingTopic = topic('group.topic').build() def updatedTopic = topic('group.topic') - .withRetentionTime(retentionTime8Days) + .withRetentionTime(retentionTime) + .build() + + when: + topicValidator.ensureUpdatedTopicIsValid(updatedTopic, existingTopic, admin) + + then: + noExceptionThrown() + + where: + retentionTime << [ + new RetentionTime(8, DAYS), + new RetentionTime(7 * 24 + 1, HOURS), + new RetentionTime(7 * 24 * 60 + 1, MINUTES), + new RetentionTime(7 * 24 * 60 * 60 + 1, SECONDS) + ] + } + + def "updating topic without modifying retention time already exceeding maximum should be valid"() { + given: + def existingTopic = topic('group.topic') + .withRetentionTime(new RetentionTime(8, DAYS)) + .build() + def updatedTopic = topic('group.topic') + .withRetentionTime(new RetentionTime(8, DAYS)) + .withDescription("lorem ipsum") + .build() + + when: + topicValidator.ensureUpdatedTopicIsValid(updatedTopic, existingTopic, user) + + then: + noExceptionThrown() + + where: + user << [regularUser, admin] + } + + def "updating topic with modifying retention time already exceeding maximum should be invalid for regular user"() { + given: + def existingTopic = topic('group.topic') + .withRetentionTime(new RetentionTime(8, DAYS)) + .build() + def updatedTopic = topic('group.topic') + .withRetentionTime(new RetentionTime(12, DAYS)) + .withDescription("lorem ipsum") + .build() + + when: + topicValidator.ensureUpdatedTopicIsValid(updatedTopic, existingTopic, regularUser) + + then: + def exception = thrown(TopicValidationException) + exception.message == "Retention time larger than 7 days can't be configured by non admin users" + } + + def "updating topic with modifying retention time already exceeding maximum should be valid for admin"() { + given: + def existingTopic = topic('group.topic') + .withRetentionTime(new RetentionTime(8, DAYS)) + .build() + def updatedTopic = topic('group.topic') + .withRetentionTime(new RetentionTime(12, DAYS)) + .withDescription("lorem ipsum") .build() when: @@ -117,4 +232,26 @@ class TopicValidatorWithRealApiPreconditionsTest extends Specification { then: noExceptionThrown() } + + @Unroll + def "creating a topic with timeUnit smaller than seconds should be invalid"() { + given: + def topic = topic('group.topic') + .withRetentionTime(retentionTime) + .build() + + when: + topicValidator.ensureCreatedTopicIsValid(topic, regularUser, MANAGEABLE) + + then: + def exception = thrown(TopicValidationException) + exception.message == "Retention time unit must be one of: [SECONDS, MINUTES, HOURS, DAYS]" + + where: + retentionTime << [ + new RetentionTime(1, TimeUnit.MICROSECONDS), + new RetentionTime(1, TimeUnit.MILLISECONDS), + new RetentionTime(1, TimeUnit.NANOSECONDS) + ] + } } diff --git a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/SubscriptionManagementTest.java b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/SubscriptionManagementTest.java index 7e9da9c1bf..a5629282be 100644 --- a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/SubscriptionManagementTest.java +++ b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/SubscriptionManagementTest.java @@ -644,46 +644,6 @@ public void shouldReturnInflightSizeWhenSetToNonNullValue() { assertThat(response.getSerialSubscriptionPolicy().getInflightSize()).isEqualTo(42); } - @Test - public void shouldNotAllowNonAdminUserToSetInflightSize() { - // given - Topic topic = hermes.initHelper().createTopic(topicWithRandomName().build()); - Subscription subscription = hermes.initHelper().createSubscription(subscriptionWithRandomName(topic.getName()).build()); - TestSecurityProvider.setUserIsAdmin(false); - - PatchData patchData = patchData().set("subscriptionPolicy", ImmutableMap.builder() - .put("inflightSize", 100) - .build() - ).build(); - - // when - WebTestClient.ResponseSpec response = hermes.api().updateSubscription(topic, subscription.getName(), patchData); - - //then - response.expectStatus().isBadRequest(); - assertThat(response.expectBody(String.class).returnResult().getResponseBody()) - .contains("Subscription.serialSubscriptionPolicy.inflightSize must be null"); - } - - @Test - public void shouldAllowAdminUserToSetInflightSize() { - // given - Topic topic = hermes.initHelper().createTopic(topicWithRandomName().build()); - Subscription subscription = hermes.initHelper().createSubscription(subscriptionWithRandomName(topic.getName()).build()); - TestSecurityProvider.setUserIsAdmin(true); - - PatchData patchData = patchData().set("subscriptionPolicy", ImmutableMap.builder() - .put("inflightSize", 100) - .build() - ).build(); - - // when - WebTestClient.ResponseSpec response = hermes.api().updateSubscription(topic, subscription.getName(), patchData); - - //then - response.expectStatus().isOk(); - } - @Test public void shouldMoveOffsetsToTheEnd() { // given From 012581affcb0a732c5b0b212932a7eb95db701bb Mon Sep 17 00:00:00 2001 From: Szymon Marcinkiewicz Date: Tue, 25 Jun 2024 08:26:47 +0200 Subject: [PATCH 65/87] Fix topic search redirect link (#1871) --- .../views/search/topic-search-results/TopicSearchResults.vue | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hermes-console/src/views/search/topic-search-results/TopicSearchResults.vue b/hermes-console/src/views/search/topic-search-results/TopicSearchResults.vue index 6a4ade1f5c..10241862b2 100644 --- a/hermes-console/src/views/search/topic-search-results/TopicSearchResults.vue +++ b/hermes-console/src/views/search/topic-search-results/TopicSearchResults.vue @@ -15,7 +15,7 @@ } function onTopicBlankClick(topic: Topic) { const group = groupName(topic.name); - window.open(`/ui/groups/${group}//topics/${topic.name}`, '_blank'); + window.open(`/ui/groups/${group}/topics/${topic.name}`, '_blank'); } From 2673b98b138cffeb6591291dcc8aae3c2d85783e Mon Sep 17 00:00:00 2001 From: Maciej Moscicki Date: Tue, 25 Jun 2024 13:00:39 +0200 Subject: [PATCH 66/87] Allow for 'empty' broadcast send attempt (#1876) --- .../hermes/consumers/consumer/Message.java | 4 ++ .../http/JettyBroadCastMessageSender.java | 32 ++++++++++--- .../JettyBroadCastMessageSenderTest.groovy | 45 ++++++++++++++++++- 3 files changed, 75 insertions(+), 6 deletions(-) diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/Message.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/Message.java index c0d5c5df17..c0ed185669 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/Message.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/Message.java @@ -138,6 +138,10 @@ public String getId() { return id; } + public synchronized Set getSucceededUris() { + return succeededUris; + } + @Override public Map getExternalMetadata() { return externalMetadata; diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/sender/http/JettyBroadCastMessageSender.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/sender/http/JettyBroadCastMessageSender.java index 73cabc57e8..31e86f7a45 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/sender/http/JettyBroadCastMessageSender.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/sender/http/JettyBroadCastMessageSender.java @@ -17,8 +17,12 @@ import pl.allegro.tech.hermes.consumers.consumer.sender.resolver.ResolvableEndpointAddress; import java.net.URI; +import java.net.URISyntaxException; +import java.util.ArrayList; import java.util.Collections; +import java.util.HashSet; import java.util.List; +import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.function.Function; import java.util.stream.Collectors; @@ -58,17 +62,35 @@ public CompletableFuture send(Message message) { private CompletableFuture> sendMessage(Message message) { try { - List> results = collectResults(message); + Set> results = collectResults(message); return mergeResults(results); } catch (EndpointAddressResolutionException exception) { return CompletableFuture.completedFuture(Collections.singletonList(exceptionMapper.apply(exception))); } } - private List> collectResults( + private Set> collectResults( Message message ) throws EndpointAddressResolutionException { + var currentResults = sendPendingMessages(message); + var results = new HashSet<>(currentResults); + + // add previously succeeded uris to the result set so that successful uris from all attempts are retained. + // this way a MessageSendingResult can be considered successful even when the last send attempt + // did not send to any uri, e.g. because all uris returned by endpoint resolver were already sent to in the past. + for (String succeededUri : message.getSucceededUris()) { + try { + var uri = new URI(succeededUri); + var result = MessageSendingResult.succeededResult(uri); + results.add(CompletableFuture.completedFuture(result)); + } catch (URISyntaxException exception) { + logger.error("Error while parsing already sent broadcast URI {}", succeededUri, exception); + } + } + return results; + } + private Set> sendPendingMessages(Message message) throws EndpointAddressResolutionException { final HttpRequestData requestData = new HttpRequestDataBuilder() .withRawAddress(endpoint.getRawAddress()) .build(); @@ -80,16 +102,16 @@ private List> collectResults( if (resolvedUris.isEmpty()) { logger.debug("Empty resolved URIs for message: {}", message.getId()); - return Collections.emptyList(); + return Collections.emptySet(); } else { return resolvedUris.stream() .map(uri -> requestFactory.buildRequest(message, uri, headers)) .map(this::processResponse) - .collect(Collectors.toList()); + .collect(Collectors.toSet()); } } - private CompletableFuture> mergeResults(List> results) { + private CompletableFuture> mergeResults(Set> results) { return CompletableFuture.allOf(results.toArray(new CompletableFuture[results.size()])) .thenApply(v -> results.stream() .map(CompletableFuture::join) diff --git a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/sender/http/JettyBroadCastMessageSenderTest.groovy b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/sender/http/JettyBroadCastMessageSenderTest.groovy index 06d1bf7832..10688bb01a 100644 --- a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/sender/http/JettyBroadCastMessageSenderTest.groovy +++ b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/sender/http/JettyBroadCastMessageSenderTest.groovy @@ -150,7 +150,7 @@ class JettyBroadCastMessageSenderTest extends Specification { future.get(1, TimeUnit.SECONDS).succeeded() } - def "should return not succeeded and retry later when endpoint resolver return no hosts"() { + def "should return not succeeded and retry later when endpoint resolver return no hosts and no message was sent previously"() { given: def address = Stub(ResolvableEndpointAddress) { resolveAllFor(_ as Message) >> [] @@ -173,6 +173,49 @@ class JettyBroadCastMessageSenderTest extends Specification { messageSendingResult.isRetryLater() } + def "should return succeeded when endpoint resolver return no hosts and but message was sent previously"() { + given: + Message message = testMessage() + message.incrementRetryCounter([serviceEndpoints[0].url]) + def address = Stub(ResolvableEndpointAddress) { + resolveAllFor(_ as Message) >> [] + + getRawAddress() >> endpoint + } + + def httpRequestFactory = new DefaultHttpRequestFactory(client, 1000, 1000, new DefaultHttpMetadataAppender()) + MessageSender messageSender = new JettyBroadCastMessageSender(httpRequestFactory, address, + requestHeadersProvider, resultHandlersProvider, Mock(ResilientMessageSender)) + + when: + def future = messageSender.send(message) + + then: + MessageSendingResult messageSendingResult = future.get(1, TimeUnit.SECONDS) + + messageSendingResult.succeeded() + } + + + def "should return succeeded when endpoint resolver returns the same urls that the message was already sent to"() { + given: "a message that was sent" + ConsumerRateLimiter rateLimiter = Mock(ConsumerRateLimiter) { + 0 * registerSuccessfulSending() + } + + serviceEndpoints.forEach { endpoint -> endpoint.expectMessages(TEST_MESSAGE_CONTENT) } + + Message message = testMessage() + message.incrementRetryCounter(serviceEndpoints.collect { it.url }) + + when: + def future = getSender(rateLimiter).send(message) + + then: + MessageSendingResult messageSendingResult = future.get(1, TimeUnit.SECONDS) + messageSendingResult.succeeded() + } + def cleanupSpec() { wireMockServers.forEach { it.stop() } client.stop() From 1e634b41ac0ee8b50c11b1ad3a36cc2d0acafd98 Mon Sep 17 00:00:00 2001 From: Maciej Moscicki Date: Tue, 2 Jul 2024 12:10:53 +0200 Subject: [PATCH 67/87] allow configuration of default value for remoteDatacenterFallbackEnabled (#1875) * allow configuration of default value for remoteDatacenterFallbackEnabled * remove redundant test --- .../pl/allegro/tech/hermes/api/Topic.java | 12 ++++-- .../tech/hermes/api/TopicWithSchema.java | 4 +- .../pl/allegro/tech/hermes/api/TopicTest.java | 37 +++++++++++++++++-- .../di/factories/ObjectMapperFactory.java | 9 +++-- .../tech/hermes/test/IntegrationTest.groovy | 2 +- .../common/di/ObjectMapperFactoryTest.java | 2 +- .../consumers/config/CommonConfiguration.java | 6 ++- .../frontend/config/CommonConfiguration.java | 7 ++-- .../config/TopicDefaultsProperties.java | 16 ++++++++ .../config/ManagementConfiguration.java | 3 +- .../management/config/TopicProperties.java | 10 +++++ .../management/domain/topic/TopicService.java | 1 - 12 files changed, 90 insertions(+), 19 deletions(-) create mode 100644 hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/TopicDefaultsProperties.java diff --git a/hermes-api/src/main/java/pl/allegro/tech/hermes/api/Topic.java b/hermes-api/src/main/java/pl/allegro/tech/hermes/api/Topic.java index 3cc67c855f..277857a63f 100644 --- a/hermes-api/src/main/java/pl/allegro/tech/hermes/api/Topic.java +++ b/hermes-api/src/main/java/pl/allegro/tech/hermes/api/Topic.java @@ -37,7 +37,8 @@ public class Topic { private boolean jsonToAvroDryRunEnabled = false; @NotNull private Ack ack; - private boolean fallbackToRemoteDatacenterEnabled; + public static final String DEFAULT_FALLBACK_TO_REMOTE_DATACENTER_KEY = "defaultFallbackToRemoteDatacenterEnabled"; + private final boolean fallbackToRemoteDatacenterEnabled; private PublishingChaosPolicy chaos; @NotNull private ContentType contentType; @@ -58,8 +59,10 @@ public class Topic { private Instant modifiedAt; public Topic(TopicName name, String description, OwnerId owner, RetentionTime retentionTime, - boolean migratedFromJsonType, Ack ack, boolean fallbackToRemoteDatacenterEnabled, PublishingChaosPolicy chaos, - boolean trackingEnabled, ContentType contentType, boolean jsonToAvroDryRunEnabled, + boolean migratedFromJsonType, Ack ack, + @JacksonInject(value = DEFAULT_FALLBACK_TO_REMOTE_DATACENTER_KEY, useInput = OptBoolean.TRUE) + Boolean fallbackToRemoteDatacenterEnabled, + PublishingChaosPolicy chaos, boolean trackingEnabled, ContentType contentType, boolean jsonToAvroDryRunEnabled, @JacksonInject(value = DEFAULT_SCHEMA_ID_SERIALIZATION_ENABLED_KEY, useInput = OptBoolean.TRUE) Boolean schemaIdAwareSerializationEnabled, int maxMessageSize, PublishingAuth publishingAuth, boolean subscribingRestricted, @@ -93,7 +96,8 @@ public Topic( @JsonProperty("retentionTime") RetentionTime retentionTime, @JsonProperty("jsonToAvroDryRun") boolean jsonToAvroDryRunEnabled, @JsonProperty("ack") Ack ack, - @JsonProperty("fallbackToRemoteDatacenterEnabled") boolean fallbackToRemoteDatacenterEnabled, + @JacksonInject(value = DEFAULT_FALLBACK_TO_REMOTE_DATACENTER_KEY, useInput = OptBoolean.TRUE) + @JsonProperty("fallbackToRemoteDatacenterEnabled") Boolean fallbackToRemoteDatacenterEnabled, @JsonProperty("chaos") PublishingChaosPolicy chaos, @JsonProperty("trackingEnabled") boolean trackingEnabled, @JsonProperty("migratedFromJsonType") boolean migratedFromJsonType, diff --git a/hermes-api/src/main/java/pl/allegro/tech/hermes/api/TopicWithSchema.java b/hermes-api/src/main/java/pl/allegro/tech/hermes/api/TopicWithSchema.java index a8f6496198..2719cc0e5f 100644 --- a/hermes-api/src/main/java/pl/allegro/tech/hermes/api/TopicWithSchema.java +++ b/hermes-api/src/main/java/pl/allegro/tech/hermes/api/TopicWithSchema.java @@ -32,7 +32,9 @@ public TopicWithSchema(@JsonProperty("schema") String schema, @JsonProperty("retentionTime") RetentionTime retentionTime, @JsonProperty("jsonToAvroDryRun") boolean jsonToAvroDryRunEnabled, @JsonProperty("ack") Ack ack, - @JsonProperty("fallbackToRemoteDatacenterEnabled") boolean fallbackToRemoteDatacenterEnabled, + @JsonProperty("fallbackToRemoteDatacenterEnabled") + @JacksonInject(value = DEFAULT_FALLBACK_TO_REMOTE_DATACENTER_KEY, useInput = OptBoolean.TRUE) + boolean fallbackToRemoteDatacenterEnabled, @JsonProperty("chaos") PublishingChaosPolicy chaos, @JsonProperty("trackingEnabled") boolean trackingEnabled, @JsonProperty("migratedFromJsonType") boolean migratedFromJsonType, diff --git a/hermes-api/src/test/java/pl/allegro/tech/hermes/api/TopicTest.java b/hermes-api/src/test/java/pl/allegro/tech/hermes/api/TopicTest.java index 2c74da0365..27185dbeed 100644 --- a/hermes-api/src/test/java/pl/allegro/tech/hermes/api/TopicTest.java +++ b/hermes-api/src/test/java/pl/allegro/tech/hermes/api/TopicTest.java @@ -9,7 +9,7 @@ public class TopicTest { - private final ObjectMapper objectMapper = createObjectMapper(); + private final ObjectMapper objectMapper = createObjectMapper(false); @Test public void shouldDeserializeTopicWithDefaults() throws Exception { @@ -65,11 +65,42 @@ public void shouldSkippedDeserializedOldSchemaVersionId() throws Exception { assertThat(topic.getName().getName()).isEqualTo("bar"); } - private ObjectMapper createObjectMapper() { + @Test + public void shouldDeserializeFallbackToRemoteDatacenterWithDefaults() throws Exception { + // given + String json = "{\"name\":\"foo.bar\", \"description\": \"description\"}"; + + // when + Topic topic = objectMapper.readValue(json, Topic.class); + + // then + assertThat(topic.isFallbackToRemoteDatacenterEnabled()).isEqualTo(false); + + // and when + Topic topic2 = createObjectMapper(true).readValue(json, Topic.class); + + // then + assertThat(topic2.isFallbackToRemoteDatacenterEnabled()).isEqualTo(true); + } + + @Test + public void shouldDeserializeFallbackToRemoteDatacenter() throws Exception { + // given + String json = "{\"name\":\"foo.bar\", \"description\": \"description\", \"fallbackToRemoteDatacenterEnabled\": true}"; + + // when + Topic topic = objectMapper.readValue(json, Topic.class); + + // then + assertThat(topic.isFallbackToRemoteDatacenterEnabled()).isEqualTo(true); + } + + private ObjectMapper createObjectMapper(boolean fallbackToRemoteDatacenterEnabled) { ObjectMapper mapper = new ObjectMapper(); final InjectableValues defaultSchemaIdAwareSerializationEnabled = new InjectableValues - .Std().addValue(Topic.DEFAULT_SCHEMA_ID_SERIALIZATION_ENABLED_KEY, true); + .Std().addValue(Topic.DEFAULT_SCHEMA_ID_SERIALIZATION_ENABLED_KEY, true) + .addValue(Topic.DEFAULT_FALLBACK_TO_REMOTE_DATACENTER_KEY, fallbackToRemoteDatacenterEnabled); mapper.setInjectableValues(defaultSchemaIdAwareSerializationEnabled); mapper.disable(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES); diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/di/factories/ObjectMapperFactory.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/di/factories/ObjectMapperFactory.java index 368e8cf0bd..c2b9d89743 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/di/factories/ObjectMapperFactory.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/di/factories/ObjectMapperFactory.java @@ -11,9 +11,11 @@ public class ObjectMapperFactory { private final boolean schemaIdSerializationEnabled; + private final boolean fallbackToRemoteDatacenterEnabled; - public ObjectMapperFactory(boolean schemaIdSerializationEnabled) { + public ObjectMapperFactory(boolean schemaIdSerializationEnabled, boolean fallbackToRemoteDatacenterEnabled) { this.schemaIdSerializationEnabled = schemaIdSerializationEnabled; + this.fallbackToRemoteDatacenterEnabled = fallbackToRemoteDatacenterEnabled; } public ObjectMapper provide() { @@ -23,8 +25,9 @@ public ObjectMapper provide() { objectMapper.disable(SerializationFeature.WRITE_NULL_MAP_VALUES); objectMapper.registerModule(new JavaTimeModule()); - final InjectableValues defaultSchemaIdAwareSerializationEnabled = new InjectableValues - .Std().addValue(Topic.DEFAULT_SCHEMA_ID_SERIALIZATION_ENABLED_KEY, schemaIdSerializationEnabled); + final InjectableValues defaultSchemaIdAwareSerializationEnabled = new InjectableValues.Std() + .addValue(Topic.DEFAULT_SCHEMA_ID_SERIALIZATION_ENABLED_KEY, schemaIdSerializationEnabled) + .addValue(Topic.DEFAULT_FALLBACK_TO_REMOTE_DATACENTER_KEY, fallbackToRemoteDatacenterEnabled); objectMapper.setInjectableValues(defaultSchemaIdAwareSerializationEnabled); return objectMapper; diff --git a/hermes-common/src/test/groovy/pl/allegro/tech/hermes/test/IntegrationTest.groovy b/hermes-common/src/test/groovy/pl/allegro/tech/hermes/test/IntegrationTest.groovy index 2f4a09a291..95bf3b028f 100644 --- a/hermes-common/src/test/groovy/pl/allegro/tech/hermes/test/IntegrationTest.groovy +++ b/hermes-common/src/test/groovy/pl/allegro/tech/hermes/test/IntegrationTest.groovy @@ -31,7 +31,7 @@ abstract class IntegrationTest extends Specification { protected RepositoryWaiter wait = new RepositoryWaiter(zookeeperResource.curator(), paths) - protected ObjectMapper mapper = new ObjectMapperFactory(true).provide() + protected ObjectMapper mapper = new ObjectMapperFactory(true, false).provide() protected ZookeeperGroupRepository groupRepository = new ZookeeperGroupRepository(zookeeper(), mapper, paths) diff --git a/hermes-common/src/test/java/pl/allegro/tech/hermes/common/di/ObjectMapperFactoryTest.java b/hermes-common/src/test/java/pl/allegro/tech/hermes/common/di/ObjectMapperFactoryTest.java index ae1846c956..7b10a31a30 100644 --- a/hermes-common/src/test/java/pl/allegro/tech/hermes/common/di/ObjectMapperFactoryTest.java +++ b/hermes-common/src/test/java/pl/allegro/tech/hermes/common/di/ObjectMapperFactoryTest.java @@ -14,7 +14,7 @@ public class ObjectMapperFactoryTest { @Before public void init() { - ObjectMapperFactory factory = new ObjectMapperFactory(false); + ObjectMapperFactory factory = new ObjectMapperFactory(false, false); mapper = factory.provide(); } diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/CommonConfiguration.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/CommonConfiguration.java index 2ea5eab12e..725e4108cb 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/CommonConfiguration.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/CommonConfiguration.java @@ -176,7 +176,11 @@ public ZookeeperAdminCache zookeeperAdminCache(ZookeeperPaths zookeeperPaths, @Bean public ObjectMapper objectMapper(SchemaProperties schemaProperties) { - return new ObjectMapperFactory(schemaProperties.isIdSerializationEnabled()).provide(); + return new ObjectMapperFactory( + schemaProperties.isIdSerializationEnabled(), + /* fallbackToRemoteDatacenter is frontend specific property, we so don't expose consumer side property for it */ + false + ).provide(); } diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/CommonConfiguration.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/CommonConfiguration.java index d5e95e5c9f..ce996a7052 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/CommonConfiguration.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/CommonConfiguration.java @@ -83,7 +83,8 @@ ZookeeperClustersProperties.class, KafkaClustersProperties.class, ContentRootProperties.class, - DatacenterNameProperties.class + DatacenterNameProperties.class, + TopicDefaultsProperties.class }) public class CommonConfiguration { @@ -183,8 +184,8 @@ public ZookeeperAdminCache zookeeperAdminCache(ZookeeperPaths zookeeperPaths, } @Bean - public ObjectMapper objectMapper(SchemaProperties schemaProperties) { - return new ObjectMapperFactory(schemaProperties.isIdSerializationEnabled()).provide(); + public ObjectMapper objectMapper(SchemaProperties schemaProperties, TopicDefaultsProperties topicDefaults) { + return new ObjectMapperFactory(schemaProperties.isIdSerializationEnabled(), topicDefaults.isFallbackToRemoteDatacenterEnabled()).provide(); } @Bean diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/TopicDefaultsProperties.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/TopicDefaultsProperties.java new file mode 100644 index 0000000000..ac3692840a --- /dev/null +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/TopicDefaultsProperties.java @@ -0,0 +1,16 @@ +package pl.allegro.tech.hermes.frontend.config; + +import org.springframework.boot.context.properties.ConfigurationProperties; + +@ConfigurationProperties(prefix = "frontend.topic.defaults") +public class TopicDefaultsProperties { + private boolean fallbackToRemoteDatacenterEnabled = false; + + public boolean isFallbackToRemoteDatacenterEnabled() { + return fallbackToRemoteDatacenterEnabled; + } + + public void setFallbackToRemoteDatacenterEnabled(boolean fallbackToRemoteDatacenterEnabled) { + this.fallbackToRemoteDatacenterEnabled = fallbackToRemoteDatacenterEnabled; + } +} diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ManagementConfiguration.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ManagementConfiguration.java index c6945f814c..e4cc665d9a 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ManagementConfiguration.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ManagementConfiguration.java @@ -46,7 +46,8 @@ public ObjectMapper objectMapper() { final InjectableValues defaultSchemaIdAwareSerializationEnabled = new InjectableValues.Std().addValue( Topic.DEFAULT_SCHEMA_ID_SERIALIZATION_ENABLED_KEY, - topicProperties.isDefaultSchemaIdAwareSerializationEnabled()); + topicProperties.isDefaultSchemaIdAwareSerializationEnabled()) + .addValue(Topic.DEFAULT_FALLBACK_TO_REMOTE_DATACENTER_KEY, topicProperties.isDefaultFallbackToRemoteDatacenterEnabled()); mapper.setInjectableValues(defaultSchemaIdAwareSerializationEnabled); diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/TopicProperties.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/TopicProperties.java index 11b2fe876b..77dd98a1a7 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/TopicProperties.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/TopicProperties.java @@ -34,6 +34,8 @@ public class TopicProperties { private boolean defaultSchemaIdAwareSerializationEnabled = false; + private boolean defaultFallbackToRemoteDatacenterEnabled = false; + private boolean avroContentTypeMetadataRequired = true; /** @@ -155,6 +157,14 @@ public void setDefaultSchemaIdAwareSerializationEnabled(boolean defaultSchemaIdA this.defaultSchemaIdAwareSerializationEnabled = defaultSchemaIdAwareSerializationEnabled; } + public boolean isDefaultFallbackToRemoteDatacenterEnabled() { + return defaultFallbackToRemoteDatacenterEnabled; + } + + public void setDefaultFallbackToRemoteDatacenterEnabled(boolean defaultFallbackToRemoteDatacenterEnabled) { + this.defaultFallbackToRemoteDatacenterEnabled = defaultFallbackToRemoteDatacenterEnabled; + } + public boolean isAvroContentTypeMetadataRequired() { return avroContentTypeMetadataRequired; } diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/TopicService.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/TopicService.java index 55ba060834..220fbfd09b 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/TopicService.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/TopicService.java @@ -5,7 +5,6 @@ import org.slf4j.LoggerFactory; import org.springframework.beans.factory.annotation.Autowired; import org.springframework.stereotype.Component; -import pl.allegro.tech.hermes.api.ContentType; import pl.allegro.tech.hermes.api.MessageTextPreview; import pl.allegro.tech.hermes.api.OwnerId; import pl.allegro.tech.hermes.api.PatchData; From 5cc4fe18ba83c1a94abeef65fa237d3e23847b05 Mon Sep 17 00:00:00 2001 From: "maciej.moscicki" Date: Wed, 3 Jul 2024 13:49:16 +0200 Subject: [PATCH 68/87] improve fallback to remote dc validation --- .../topic/validator/TopicValidator.java | 14 +++++++---- .../management/TopicManagementTest.java | 23 +++++++++++-------- 2 files changed, 22 insertions(+), 15 deletions(-) diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/validator/TopicValidator.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/validator/TopicValidator.java index eaa69ef2cd..fd1e22d8da 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/validator/TopicValidator.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/validator/TopicValidator.java @@ -9,6 +9,7 @@ import pl.allegro.tech.hermes.api.RetentionTime; import pl.allegro.tech.hermes.api.Topic; import pl.allegro.tech.hermes.management.api.validator.ApiPreconditions; +import pl.allegro.tech.hermes.management.config.TopicProperties; import pl.allegro.tech.hermes.management.domain.auth.RequestUser; import pl.allegro.tech.hermes.management.domain.owner.validator.OwnerIdValidator; import pl.allegro.tech.hermes.management.domain.topic.CreatorRights; @@ -27,18 +28,21 @@ public class TopicValidator { private final TopicLabelsValidator topicLabelsValidator; private final SchemaRepository schemaRepository; private final ApiPreconditions apiPreconditions; + private final TopicProperties topicProperties; @Autowired public TopicValidator(OwnerIdValidator ownerIdValidator, ContentTypeValidator contentTypeValidator, TopicLabelsValidator topicLabelsValidator, SchemaRepository schemaRepository, - ApiPreconditions apiPreconditions) { + ApiPreconditions apiPreconditions, + TopicProperties topicProperties) { this.ownerIdValidator = ownerIdValidator; this.contentTypeValidator = contentTypeValidator; this.topicLabelsValidator = topicLabelsValidator; this.schemaRepository = schemaRepository; this.apiPreconditions = apiPreconditions; + this.topicProperties = topicProperties; } public void ensureCreatedTopicIsValid(Topic created, RequestUser createdBy, CreatorRights creatorRights) { @@ -47,8 +51,8 @@ public void ensureCreatedTopicIsValid(Topic created, RequestUser createdBy, Crea checkContentType(created); checkTopicLabels(created); - if (created.isFallbackToRemoteDatacenterEnabled() && !createdBy.isAdmin()) { - throw new TopicValidationException("User is not allowed to enable fallback to remote datacenter"); + if ((created.isFallbackToRemoteDatacenterEnabled() != topicProperties.isDefaultFallbackToRemoteDatacenterEnabled()) && !createdBy.isAdmin()) { + throw new TopicValidationException("User is not allowed to set non-default fallback to remote datacenter for this topic"); } if (created.getChaos().mode() != ChaosMode.DISABLED && !createdBy.isAdmin()) { @@ -72,8 +76,8 @@ public void ensureUpdatedTopicIsValid(Topic updated, Topic previous, RequestUser checkOwner(updated); checkTopicLabels(updated); - if (!previous.isFallbackToRemoteDatacenterEnabled() && updated.isFallbackToRemoteDatacenterEnabled() && !modifiedBy.isAdmin()) { - throw new TopicValidationException("User is not allowed to enable fallback to remote datacenter"); + if ((previous.isFallbackToRemoteDatacenterEnabled() != updated.isFallbackToRemoteDatacenterEnabled()) && !modifiedBy.isAdmin()) { + throw new TopicValidationException("User is not allowed to update fallback to remote datacenter for this topic"); } if (!previous.getChaos().equals(updated.getChaos()) && !modifiedBy.isAdmin()) { diff --git a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/TopicManagementTest.java b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/TopicManagementTest.java index d097604b40..461492ccd9 100644 --- a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/TopicManagementTest.java +++ b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/TopicManagementTest.java @@ -603,7 +603,7 @@ public void shouldCreateTopicEvenIfExistsInBrokers() { } @Test - public void shouldNotAllowNonAdminUserCreateTopicWithFallbackToRemoteDatacenterEnabled() { + public void shouldNotAllowNonAdminUserCreateTopicWithNonDefaultFallbackToRemoteDatacenter() { // given TestSecurityProvider.setUserIsAdmin(false); TopicWithSchema topic = topicWithSchema( @@ -619,11 +619,11 @@ public void shouldNotAllowNonAdminUserCreateTopicWithFallbackToRemoteDatacenterE //then response.expectStatus().isBadRequest(); assertThat(response.expectBody(String.class).returnResult().getResponseBody()) - .contains("User is not allowed to enable fallback to remote datacenter"); + .contains("User is not allowed to set non-default fallback to remote datacenter"); } @Test - public void shouldAllowAdminUserCreateTopicWithFallbackToRemoteDatacenterEnabled() { + public void shouldAllowAdminUserCreateTopicWithNonDefaultFallbackToRemoteDatacenterEnabled() { // given TestSecurityProvider.setUserIsAdmin(true); TopicWithSchema topic = topicWithSchema( @@ -641,11 +641,12 @@ public void shouldAllowAdminUserCreateTopicWithFallbackToRemoteDatacenterEnabled } @Test - public void shouldNotAllowNonAdminUserToEnableFallbackToRemoteDatacenter() { + public void shouldNotAllowNonAdminUserToChangeFallbackToRemoteDatacenter() { // given - Topic topic = hermes.initHelper().createTopic(topicWithRandomName().build()); + Topic topic = hermes.initHelper().createTopic(topicWithRandomName() + .withFallbackToRemoteDatacenterEnabled().build()); TestSecurityProvider.setUserIsAdmin(false); - PatchData patchData = PatchData.from(ImmutableMap.of("fallbackToRemoteDatacenterEnabled", true)); + PatchData patchData = PatchData.from(ImmutableMap.of("fallbackToRemoteDatacenterEnabled", false)); // when WebTestClient.ResponseSpec response = hermes.api().updateTopic(topic.getQualifiedName(), patchData); @@ -653,15 +654,17 @@ public void shouldNotAllowNonAdminUserToEnableFallbackToRemoteDatacenter() { //then response.expectStatus().isBadRequest(); assertThat(response.expectBody(String.class).returnResult().getResponseBody()) - .contains("User is not allowed to enable fallback to remote datacenter"); + .contains("User is not allowed to update fallback to remote datacenter for this topic"); } @Test - public void shouldAllowAdminUserToEnableFallbackToRemoteDatacenter() { + public void shouldAllowAdminUserToChangeFallbackToRemoteDatacenter() { // given - Topic topic = hermes.initHelper().createTopic(topicWithRandomName().build()); + Topic topic = hermes.initHelper().createTopic(topicWithRandomName() + .withFallbackToRemoteDatacenterEnabled() + .build()); TestSecurityProvider.setUserIsAdmin(true); - PatchData patchData = PatchData.from(ImmutableMap.of("fallbackToRemoteDatacenterEnabled", true)); + PatchData patchData = PatchData.from(ImmutableMap.of("fallbackToRemoteDatacenterEnabled", false)); // when WebTestClient.ResponseSpec response = hermes.api().updateTopic(topic.getQualifiedName(), patchData); From 19beb5856ad4184fdf98057f5bd61aa8cc2d7a0e Mon Sep 17 00:00:00 2001 From: Maciej Moscicki Date: Wed, 3 Jul 2024 14:07:07 +0200 Subject: [PATCH 69/87] Revert accidental "improve fallback to remote dc validation" (#1877) This reverts commit 5cc4fe18ba83c1a94abeef65fa237d3e23847b05. --- .../topic/validator/TopicValidator.java | 14 ++++------- .../management/TopicManagementTest.java | 23 ++++++++----------- 2 files changed, 15 insertions(+), 22 deletions(-) diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/validator/TopicValidator.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/validator/TopicValidator.java index fd1e22d8da..eaa69ef2cd 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/validator/TopicValidator.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/validator/TopicValidator.java @@ -9,7 +9,6 @@ import pl.allegro.tech.hermes.api.RetentionTime; import pl.allegro.tech.hermes.api.Topic; import pl.allegro.tech.hermes.management.api.validator.ApiPreconditions; -import pl.allegro.tech.hermes.management.config.TopicProperties; import pl.allegro.tech.hermes.management.domain.auth.RequestUser; import pl.allegro.tech.hermes.management.domain.owner.validator.OwnerIdValidator; import pl.allegro.tech.hermes.management.domain.topic.CreatorRights; @@ -28,21 +27,18 @@ public class TopicValidator { private final TopicLabelsValidator topicLabelsValidator; private final SchemaRepository schemaRepository; private final ApiPreconditions apiPreconditions; - private final TopicProperties topicProperties; @Autowired public TopicValidator(OwnerIdValidator ownerIdValidator, ContentTypeValidator contentTypeValidator, TopicLabelsValidator topicLabelsValidator, SchemaRepository schemaRepository, - ApiPreconditions apiPreconditions, - TopicProperties topicProperties) { + ApiPreconditions apiPreconditions) { this.ownerIdValidator = ownerIdValidator; this.contentTypeValidator = contentTypeValidator; this.topicLabelsValidator = topicLabelsValidator; this.schemaRepository = schemaRepository; this.apiPreconditions = apiPreconditions; - this.topicProperties = topicProperties; } public void ensureCreatedTopicIsValid(Topic created, RequestUser createdBy, CreatorRights creatorRights) { @@ -51,8 +47,8 @@ public void ensureCreatedTopicIsValid(Topic created, RequestUser createdBy, Crea checkContentType(created); checkTopicLabels(created); - if ((created.isFallbackToRemoteDatacenterEnabled() != topicProperties.isDefaultFallbackToRemoteDatacenterEnabled()) && !createdBy.isAdmin()) { - throw new TopicValidationException("User is not allowed to set non-default fallback to remote datacenter for this topic"); + if (created.isFallbackToRemoteDatacenterEnabled() && !createdBy.isAdmin()) { + throw new TopicValidationException("User is not allowed to enable fallback to remote datacenter"); } if (created.getChaos().mode() != ChaosMode.DISABLED && !createdBy.isAdmin()) { @@ -76,8 +72,8 @@ public void ensureUpdatedTopicIsValid(Topic updated, Topic previous, RequestUser checkOwner(updated); checkTopicLabels(updated); - if ((previous.isFallbackToRemoteDatacenterEnabled() != updated.isFallbackToRemoteDatacenterEnabled()) && !modifiedBy.isAdmin()) { - throw new TopicValidationException("User is not allowed to update fallback to remote datacenter for this topic"); + if (!previous.isFallbackToRemoteDatacenterEnabled() && updated.isFallbackToRemoteDatacenterEnabled() && !modifiedBy.isAdmin()) { + throw new TopicValidationException("User is not allowed to enable fallback to remote datacenter"); } if (!previous.getChaos().equals(updated.getChaos()) && !modifiedBy.isAdmin()) { diff --git a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/TopicManagementTest.java b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/TopicManagementTest.java index 461492ccd9..d097604b40 100644 --- a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/TopicManagementTest.java +++ b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/TopicManagementTest.java @@ -603,7 +603,7 @@ public void shouldCreateTopicEvenIfExistsInBrokers() { } @Test - public void shouldNotAllowNonAdminUserCreateTopicWithNonDefaultFallbackToRemoteDatacenter() { + public void shouldNotAllowNonAdminUserCreateTopicWithFallbackToRemoteDatacenterEnabled() { // given TestSecurityProvider.setUserIsAdmin(false); TopicWithSchema topic = topicWithSchema( @@ -619,11 +619,11 @@ public void shouldNotAllowNonAdminUserCreateTopicWithNonDefaultFallbackToRemoteD //then response.expectStatus().isBadRequest(); assertThat(response.expectBody(String.class).returnResult().getResponseBody()) - .contains("User is not allowed to set non-default fallback to remote datacenter"); + .contains("User is not allowed to enable fallback to remote datacenter"); } @Test - public void shouldAllowAdminUserCreateTopicWithNonDefaultFallbackToRemoteDatacenterEnabled() { + public void shouldAllowAdminUserCreateTopicWithFallbackToRemoteDatacenterEnabled() { // given TestSecurityProvider.setUserIsAdmin(true); TopicWithSchema topic = topicWithSchema( @@ -641,12 +641,11 @@ public void shouldAllowAdminUserCreateTopicWithNonDefaultFallbackToRemoteDatacen } @Test - public void shouldNotAllowNonAdminUserToChangeFallbackToRemoteDatacenter() { + public void shouldNotAllowNonAdminUserToEnableFallbackToRemoteDatacenter() { // given - Topic topic = hermes.initHelper().createTopic(topicWithRandomName() - .withFallbackToRemoteDatacenterEnabled().build()); + Topic topic = hermes.initHelper().createTopic(topicWithRandomName().build()); TestSecurityProvider.setUserIsAdmin(false); - PatchData patchData = PatchData.from(ImmutableMap.of("fallbackToRemoteDatacenterEnabled", false)); + PatchData patchData = PatchData.from(ImmutableMap.of("fallbackToRemoteDatacenterEnabled", true)); // when WebTestClient.ResponseSpec response = hermes.api().updateTopic(topic.getQualifiedName(), patchData); @@ -654,17 +653,15 @@ public void shouldNotAllowNonAdminUserToChangeFallbackToRemoteDatacenter() { //then response.expectStatus().isBadRequest(); assertThat(response.expectBody(String.class).returnResult().getResponseBody()) - .contains("User is not allowed to update fallback to remote datacenter for this topic"); + .contains("User is not allowed to enable fallback to remote datacenter"); } @Test - public void shouldAllowAdminUserToChangeFallbackToRemoteDatacenter() { + public void shouldAllowAdminUserToEnableFallbackToRemoteDatacenter() { // given - Topic topic = hermes.initHelper().createTopic(topicWithRandomName() - .withFallbackToRemoteDatacenterEnabled() - .build()); + Topic topic = hermes.initHelper().createTopic(topicWithRandomName().build()); TestSecurityProvider.setUserIsAdmin(true); - PatchData patchData = PatchData.from(ImmutableMap.of("fallbackToRemoteDatacenterEnabled", false)); + PatchData patchData = PatchData.from(ImmutableMap.of("fallbackToRemoteDatacenterEnabled", true)); // when WebTestClient.ResponseSpec response = hermes.api().updateTopic(topic.getQualifiedName(), patchData); From 3e3e5843414d2d4b098ad91710caf372d2a6608b Mon Sep 17 00:00:00 2001 From: Maciej Moscicki Date: Wed, 3 Jul 2024 16:18:49 +0200 Subject: [PATCH 70/87] Improved topic validation (#1878) * Revert "Revert accidental "improve fallback to remote dc validation" (#1877)" This reverts commit 19beb5856ad4184fdf98057f5bd61aa8cc2d7a0e. * adjust tests --- .../topic/validator/TopicValidator.java | 14 +++++++---- .../topic/validator/TopicValidatorTest.groovy | 3 ++- ...lidatorWithRealApiPreconditionsTest.groovy | 4 +++- .../management/TopicManagementTest.java | 23 +++++++++++-------- 4 files changed, 27 insertions(+), 17 deletions(-) diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/validator/TopicValidator.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/validator/TopicValidator.java index eaa69ef2cd..fd1e22d8da 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/validator/TopicValidator.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/validator/TopicValidator.java @@ -9,6 +9,7 @@ import pl.allegro.tech.hermes.api.RetentionTime; import pl.allegro.tech.hermes.api.Topic; import pl.allegro.tech.hermes.management.api.validator.ApiPreconditions; +import pl.allegro.tech.hermes.management.config.TopicProperties; import pl.allegro.tech.hermes.management.domain.auth.RequestUser; import pl.allegro.tech.hermes.management.domain.owner.validator.OwnerIdValidator; import pl.allegro.tech.hermes.management.domain.topic.CreatorRights; @@ -27,18 +28,21 @@ public class TopicValidator { private final TopicLabelsValidator topicLabelsValidator; private final SchemaRepository schemaRepository; private final ApiPreconditions apiPreconditions; + private final TopicProperties topicProperties; @Autowired public TopicValidator(OwnerIdValidator ownerIdValidator, ContentTypeValidator contentTypeValidator, TopicLabelsValidator topicLabelsValidator, SchemaRepository schemaRepository, - ApiPreconditions apiPreconditions) { + ApiPreconditions apiPreconditions, + TopicProperties topicProperties) { this.ownerIdValidator = ownerIdValidator; this.contentTypeValidator = contentTypeValidator; this.topicLabelsValidator = topicLabelsValidator; this.schemaRepository = schemaRepository; this.apiPreconditions = apiPreconditions; + this.topicProperties = topicProperties; } public void ensureCreatedTopicIsValid(Topic created, RequestUser createdBy, CreatorRights creatorRights) { @@ -47,8 +51,8 @@ public void ensureCreatedTopicIsValid(Topic created, RequestUser createdBy, Crea checkContentType(created); checkTopicLabels(created); - if (created.isFallbackToRemoteDatacenterEnabled() && !createdBy.isAdmin()) { - throw new TopicValidationException("User is not allowed to enable fallback to remote datacenter"); + if ((created.isFallbackToRemoteDatacenterEnabled() != topicProperties.isDefaultFallbackToRemoteDatacenterEnabled()) && !createdBy.isAdmin()) { + throw new TopicValidationException("User is not allowed to set non-default fallback to remote datacenter for this topic"); } if (created.getChaos().mode() != ChaosMode.DISABLED && !createdBy.isAdmin()) { @@ -72,8 +76,8 @@ public void ensureUpdatedTopicIsValid(Topic updated, Topic previous, RequestUser checkOwner(updated); checkTopicLabels(updated); - if (!previous.isFallbackToRemoteDatacenterEnabled() && updated.isFallbackToRemoteDatacenterEnabled() && !modifiedBy.isAdmin()) { - throw new TopicValidationException("User is not allowed to enable fallback to remote datacenter"); + if ((previous.isFallbackToRemoteDatacenterEnabled() != updated.isFallbackToRemoteDatacenterEnabled()) && !modifiedBy.isAdmin()) { + throw new TopicValidationException("User is not allowed to update fallback to remote datacenter for this topic"); } if (!previous.getChaos().equals(updated.getChaos()) && !modifiedBy.isAdmin()) { diff --git a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/domain/topic/validator/TopicValidatorTest.groovy b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/domain/topic/validator/TopicValidatorTest.groovy index 42a18fbc38..f92a84181a 100644 --- a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/domain/topic/validator/TopicValidatorTest.groovy +++ b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/domain/topic/validator/TopicValidatorTest.groovy @@ -32,6 +32,7 @@ class TopicValidatorTest extends Specification { def contentTypeWhitelistValidator = Stub(ContentTypeValidator) def apiPreconditions = Stub(ApiPreconditions) def topicLabelsValidator + def topicProperties = new TopicProperties() @Subject TopicValidator topicValidator @@ -42,7 +43,7 @@ class TopicValidatorTest extends Specification { topicProperties.setAllowedTopicLabels(allowedLabels) topicLabelsValidator = new TopicLabelsValidator(topicProperties) - topicValidator = new TopicValidator(ownerDescriptorValidator, contentTypeWhitelistValidator, topicLabelsValidator, schemaRepository, apiPreconditions) + topicValidator = new TopicValidator(ownerDescriptorValidator, contentTypeWhitelistValidator, topicLabelsValidator, schemaRepository, apiPreconditions, topicProperties) } def "topic with basic properties when creating should be valid"() { diff --git a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/domain/topic/validator/TopicValidatorWithRealApiPreconditionsTest.groovy b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/domain/topic/validator/TopicValidatorWithRealApiPreconditionsTest.groovy index 14aa7ec92e..132be2c38a 100644 --- a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/domain/topic/validator/TopicValidatorWithRealApiPreconditionsTest.groovy +++ b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/domain/topic/validator/TopicValidatorWithRealApiPreconditionsTest.groovy @@ -2,6 +2,7 @@ package pl.allegro.tech.hermes.management.domain.topic.validator import pl.allegro.tech.hermes.api.RetentionTime import pl.allegro.tech.hermes.management.api.validator.ApiPreconditions +import pl.allegro.tech.hermes.management.config.TopicProperties import pl.allegro.tech.hermes.management.domain.auth.TestRequestUser import pl.allegro.tech.hermes.management.domain.owner.validator.OwnerIdValidator import pl.allegro.tech.hermes.schema.SchemaRepository @@ -27,9 +28,10 @@ class TopicValidatorWithRealApiPreconditionsTest extends Specification { def ownerDescriptorValidator = Stub(OwnerIdValidator) def contentTypeWhitelistValidator = Stub(ContentTypeValidator) def topicLabelsValidator = Stub(TopicLabelsValidator) + def topicProperties = new TopicProperties() @Subject - def topicValidator = new TopicValidator(ownerDescriptorValidator, contentTypeWhitelistValidator, topicLabelsValidator, schemaRepository, new ApiPreconditions()) + def topicValidator = new TopicValidator(ownerDescriptorValidator, contentTypeWhitelistValidator, topicLabelsValidator, schemaRepository, new ApiPreconditions(), topicProperties) @Unroll def "creating and updating topic with up to 7 days retention time should be valid"() { diff --git a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/TopicManagementTest.java b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/TopicManagementTest.java index d097604b40..461492ccd9 100644 --- a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/TopicManagementTest.java +++ b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/TopicManagementTest.java @@ -603,7 +603,7 @@ public void shouldCreateTopicEvenIfExistsInBrokers() { } @Test - public void shouldNotAllowNonAdminUserCreateTopicWithFallbackToRemoteDatacenterEnabled() { + public void shouldNotAllowNonAdminUserCreateTopicWithNonDefaultFallbackToRemoteDatacenter() { // given TestSecurityProvider.setUserIsAdmin(false); TopicWithSchema topic = topicWithSchema( @@ -619,11 +619,11 @@ public void shouldNotAllowNonAdminUserCreateTopicWithFallbackToRemoteDatacenterE //then response.expectStatus().isBadRequest(); assertThat(response.expectBody(String.class).returnResult().getResponseBody()) - .contains("User is not allowed to enable fallback to remote datacenter"); + .contains("User is not allowed to set non-default fallback to remote datacenter"); } @Test - public void shouldAllowAdminUserCreateTopicWithFallbackToRemoteDatacenterEnabled() { + public void shouldAllowAdminUserCreateTopicWithNonDefaultFallbackToRemoteDatacenterEnabled() { // given TestSecurityProvider.setUserIsAdmin(true); TopicWithSchema topic = topicWithSchema( @@ -641,11 +641,12 @@ public void shouldAllowAdminUserCreateTopicWithFallbackToRemoteDatacenterEnabled } @Test - public void shouldNotAllowNonAdminUserToEnableFallbackToRemoteDatacenter() { + public void shouldNotAllowNonAdminUserToChangeFallbackToRemoteDatacenter() { // given - Topic topic = hermes.initHelper().createTopic(topicWithRandomName().build()); + Topic topic = hermes.initHelper().createTopic(topicWithRandomName() + .withFallbackToRemoteDatacenterEnabled().build()); TestSecurityProvider.setUserIsAdmin(false); - PatchData patchData = PatchData.from(ImmutableMap.of("fallbackToRemoteDatacenterEnabled", true)); + PatchData patchData = PatchData.from(ImmutableMap.of("fallbackToRemoteDatacenterEnabled", false)); // when WebTestClient.ResponseSpec response = hermes.api().updateTopic(topic.getQualifiedName(), patchData); @@ -653,15 +654,17 @@ public void shouldNotAllowNonAdminUserToEnableFallbackToRemoteDatacenter() { //then response.expectStatus().isBadRequest(); assertThat(response.expectBody(String.class).returnResult().getResponseBody()) - .contains("User is not allowed to enable fallback to remote datacenter"); + .contains("User is not allowed to update fallback to remote datacenter for this topic"); } @Test - public void shouldAllowAdminUserToEnableFallbackToRemoteDatacenter() { + public void shouldAllowAdminUserToChangeFallbackToRemoteDatacenter() { // given - Topic topic = hermes.initHelper().createTopic(topicWithRandomName().build()); + Topic topic = hermes.initHelper().createTopic(topicWithRandomName() + .withFallbackToRemoteDatacenterEnabled() + .build()); TestSecurityProvider.setUserIsAdmin(true); - PatchData patchData = PatchData.from(ImmutableMap.of("fallbackToRemoteDatacenterEnabled", true)); + PatchData patchData = PatchData.from(ImmutableMap.of("fallbackToRemoteDatacenterEnabled", false)); // when WebTestClient.ResponseSpec response = hermes.api().updateTopic(topic.getQualifiedName(), patchData); From e3874ca39218a4dd4900cdc3a28dc23556b6e240 Mon Sep 17 00:00:00 2001 From: Maciej Moscicki Date: Tue, 23 Jul 2024 12:25:51 +0200 Subject: [PATCH 71/87] add docs about hermes message duplication (#1880) --- docs/docs/user/publishing.md | 4 ++-- docs/docs/user/subscribing.md | 18 ++++++++++++++++++ 2 files changed, 20 insertions(+), 2 deletions(-) diff --git a/docs/docs/user/publishing.md b/docs/docs/user/publishing.md index 4889a49fcb..044535aa61 100644 --- a/docs/docs/user/publishing.md +++ b/docs/docs/user/publishing.md @@ -181,11 +181,11 @@ Hermes supports a remote datacenter fallback mechanism for [multi datacenter dep Fallback is configured on per topic basis, using a `fallbackToRemoteDatacenterEnabled` property: -```http request +```http PUT /topics/my.group.my-topic { - "fallbackToRemoteDatacenterEnabled": true, + "fallbackToRemoteDatacenterEnabled": true } ``` diff --git a/docs/docs/user/subscribing.md b/docs/docs/user/subscribing.md index 15ce53350e..8112b6a809 100644 --- a/docs/docs/user/subscribing.md +++ b/docs/docs/user/subscribing.md @@ -742,3 +742,21 @@ Note that by default Hermes does not give any guarantees about assigning message When messages are published with `parition-key` and consumed with `BATCH` mode (or `SERIAL` with `inflightSize=1`) they will be ordered as long as they were published to one DC. Publishing messages with same `parition-key` to multiple DCs does not guarantee ordering because messages are stored in separate kafka clusters. + +## Message duplication + +Hermes messages can be duplicated on different levels. Publishers are advised to include an idempotency key in message schema to allow +consumers to process messages idempotently, relying on Hermes `messageId` is not sufficient. + +Scenarios in which subscribes may receive logically duplicated messages: + +- Publisher sends a message to Hermes, the request is timed out and retried. In this case both the first and the second message may end up on Kafka and +they will have different Hermes `messageId`. Both messages are later sent to subscriber. +- Once message is received by Hermes it is relayed to Kafka. Message may then be duplicated because of Kafka producer level retries. Duplicated messages will +have the same Hermes `messageId`. +- When using [remote DC fallback](https://hermes-pubsub.readthedocs.io/en/latest/user/publishing/#remote-dc-fallback), slow messages are speculatively +sent to remote DC. This may result in a message being duplicated across DCs if both the local and remote message are saved, both copies will have the +same `messageId`. +- Even if there is one instance of a given message on a given topic, the message may still be delivered to subscriber multiple times. +When message is acknowledged by a subscriber its offset is not commited to Kafka synchronously. If Hermes instance is restarted +after receiving an ack from subscriber but before commiting the message offset to Kafka, the message will be redelivered. From d9d92e9a23e652d9d0d23453edb82f10db34633a Mon Sep 17 00:00:00 2001 From: Mateusz <76775507+szczygiel-m@users.noreply.github.com> Date: Thu, 25 Jul 2024 17:53:44 +0200 Subject: [PATCH 72/87] New offset committing (#1862) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit New offsets committing mechanism --------- Co-authored-by: Daniel Fąderski Co-authored-by: Piotr Rzysko --- .../tech/hermes/common/metric/Gauges.java | 1 - .../common/metric/SubscriptionMetrics.java | 5 + .../config/ConsumerConfiguration.java | 7 - .../config/ConsumerReceiverConfiguration.java | 3 - .../config/SupervisorConfiguration.java | 20 +- .../consumers/consumer/BatchConsumer.java | 58 ++++- .../consumer/ConsumerMessageSender.java | 15 +- .../ConsumerMessageSenderFactory.java | 11 +- .../consumers/consumer/SerialConsumer.java | 69 ++++-- .../filtering/FilteredMessageHandler.java | 12 +- .../consumer/offset/MessageState.java | 6 + .../consumer/offset/OffsetCommitter.java | 111 ++++----- .../consumer/offset/OffsetQueue.java | 38 ---- .../consumer/offset/OffsetsToCommit.java | 31 --- .../consumer/offset/PendingOffsets.java | 85 +++++++ .../offset/PendingOffsetsAppender.java | 6 + .../offset/SubscriptionPartitionOffset.java | 4 + .../consumer/receiver/MessageCommitter.java | 7 - .../consumer/receiver/ReceiverFactory.java | 4 +- .../kafka/KafkaMessageReceiverFactory.java | 15 +- .../consumer/result/DefaultErrorHandler.java | 10 +- .../result/DefaultSuccessHandler.java | 9 +- .../consumers/supervisor/ConsumerFactory.java | 27 ++- .../NonblockingConsumersSupervisor.java | 19 +- .../supervisor/process/ConsumerProcess.java | 3 - .../process/ConsumerProcessSupervisor.java | 4 +- .../consumers/supervisor/process/Signal.java | 7 +- .../offset/OffsetCommitterTest.groovy | 210 +++++++++--------- ...roovy => OffsetCommitterTestHelper.groovy} | 16 +- .../consumer/offset/PendingOffsetsTest.groovy | 113 ++++++++++ .../result/DefaultErrorHandlerTest.groovy | 12 +- .../result/DefaultSuccessHandlerTest.groovy | 8 +- .../ConsumerProcessSupervisorTest.groovy | 7 +- .../process/SignalsFilterTest.groovy | 2 - .../consumer/ConsumerMessageSenderTest.java | 35 ++- .../ConsumerTestRuntimeEnvironment.java | 5 +- .../client/integration/HermesTestClient.java | 29 ++- .../KafkaRetransmissionServiceTest.java | 32 +-- .../integrationtests/PublishingAvroTest.java | 18 +- 39 files changed, 604 insertions(+), 470 deletions(-) create mode 100644 hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/offset/MessageState.java delete mode 100644 hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/offset/OffsetQueue.java delete mode 100644 hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/offset/OffsetsToCommit.java create mode 100644 hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/offset/PendingOffsets.java create mode 100644 hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/offset/PendingOffsetsAppender.java delete mode 100644 hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/receiver/MessageCommitter.java rename hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/offset/{MockMessageCommitter.groovy => OffsetCommitterTestHelper.groovy} (51%) create mode 100644 hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/offset/PendingOffsetsTest.groovy diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/Gauges.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/Gauges.java index 5202b7355d..5990e03c95 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/Gauges.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/Gauges.java @@ -13,5 +13,4 @@ public class Gauges { public static final String CONSUMER_SENDER_HTTP_2_SERIAL_CLIENT_CONNECTIONS = "http-clients.serial.http2.connections"; public static final String CONSUMER_SENDER_HTTP_2_SERIAL_CLIENT_PENDING_CONNECTIONS = "http-clients.serial.http2.pending-connections"; - } diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/SubscriptionMetrics.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/SubscriptionMetrics.java index 95a1b85369..ee381390d1 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/SubscriptionMetrics.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/SubscriptionMetrics.java @@ -61,6 +61,10 @@ public void registerInflightGauge(SubscriptionName subscription, T obj, ToDo meterRegistry.gauge(SubscriptionMetricsNames.SUBSCRIPTION_INFLIGHT, subscriptionTags(subscription), obj, f); } + public void registerPendingOffsetsGauge(SubscriptionName subscription, T obj, ToDoubleFunction f) { + meterRegistry.gauge(SubscriptionMetricsNames.SUBSCRIPTION_PENDING_OFFSETS, subscriptionTags(subscription), obj, f); + } + public HermesTimer consumerIdleTimer(SubscriptionName subscription) { return HermesTimer.from( meterRegistry.timer(SubscriptionMetricsNames.SUBSCRIPTION_IDLE_DURATION, subscriptionTags(subscription)) @@ -118,6 +122,7 @@ public static class SubscriptionMetricsNames { public static final String SUBSCRIPTION_LATENCY = "subscription.latency"; public static final String SUBSCRIPTION_RATE_LIMITER_ACQUIRE = "subscription.rate-limiter-acquire"; public static final String SUBSCRIPTION_INFLIGHT = "subscription.inflight"; + public static final String SUBSCRIPTION_PENDING_OFFSETS = "subscription.pending-offsets"; public static final String SUBSCRIPTION_IDLE_DURATION = "subscription.idle-duration"; public static final String SUBSCRIPTION_FILTERED_OUT = "subscription.filtered-out"; public static final String SUBSCRIPTION_HTTP_STATUS_CODES = "subscription.http-status-codes"; diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/ConsumerConfiguration.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/ConsumerConfiguration.java index 5d912ef791..28c8b0548f 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/ConsumerConfiguration.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/ConsumerConfiguration.java @@ -18,7 +18,6 @@ import pl.allegro.tech.hermes.consumers.consumer.interpolation.MessageBodyInterpolator; import pl.allegro.tech.hermes.consumers.consumer.interpolation.UriInterpolator; import pl.allegro.tech.hermes.consumers.consumer.offset.ConsumerPartitionAssignmentState; -import pl.allegro.tech.hermes.consumers.consumer.offset.OffsetQueue; import pl.allegro.tech.hermes.consumers.consumer.rate.ConsumerRateLimitSupervisor; import pl.allegro.tech.hermes.consumers.consumer.rate.calculator.OutputRateCalculatorFactory; import pl.allegro.tech.hermes.consumers.consumer.rate.maxrate.MaxRatePathSerializer; @@ -110,12 +109,6 @@ public MaxRateSupervisor maxRateSupervisor(MaxRateProperties maxRateProperties, ); } - @Bean - public OffsetQueue offsetQueue(MetricsFacade metrics, - CommitOffsetProperties commitOffsetProperties) { - return new OffsetQueue(metrics, commitOffsetProperties.getQueuesSize()); - } - @Bean public ConsumerRateLimitSupervisor consumerRateLimitSupervisor(RateProperties rateProperties) { return new ConsumerRateLimitSupervisor(rateProperties.getLimiterSupervisorPeriod()); diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/ConsumerReceiverConfiguration.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/ConsumerReceiverConfiguration.java index 0593c487c1..09fa41cf9b 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/ConsumerReceiverConfiguration.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/ConsumerReceiverConfiguration.java @@ -8,7 +8,6 @@ import pl.allegro.tech.hermes.common.message.wrapper.CompositeMessageContentWrapper; import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.consumers.consumer.offset.ConsumerPartitionAssignmentState; -import pl.allegro.tech.hermes.consumers.consumer.offset.OffsetQueue; import pl.allegro.tech.hermes.consumers.consumer.receiver.ReceiverFactory; import pl.allegro.tech.hermes.consumers.consumer.receiver.kafka.BasicMessageContentReaderFactory; import pl.allegro.tech.hermes.consumers.consumer.receiver.kafka.KafkaConsumerRecordToMessageConverterFactory; @@ -40,7 +39,6 @@ public ReceiverFactory kafkaMessageReceiverFactory(CommonConsumerProperties comm KafkaClustersProperties kafkaClustersProperties, KafkaConsumerRecordToMessageConverterFactory messageConverterFactory, MetricsFacade metrics, - OffsetQueue offsetQueue, KafkaNamesMapper kafkaNamesMapper, FilterChainFactory filterChainFactory, Trackers trackers, @@ -56,7 +54,6 @@ public ReceiverFactory kafkaMessageReceiverFactory(CommonConsumerProperties comm kafkaProperties, messageConverterFactory, metrics, - offsetQueue, kafkaNamesMapper, filterChainFactory, trackers, diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/SupervisorConfiguration.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/SupervisorConfiguration.java index 055f3d34d3..0be54f5b08 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/SupervisorConfiguration.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/SupervisorConfiguration.java @@ -20,7 +20,6 @@ import pl.allegro.tech.hermes.consumers.consumer.converter.MessageConverterResolver; import pl.allegro.tech.hermes.consumers.consumer.load.SubscriptionLoadRecordersRegistry; import pl.allegro.tech.hermes.consumers.consumer.offset.ConsumerPartitionAssignmentState; -import pl.allegro.tech.hermes.consumers.consumer.offset.OffsetQueue; import pl.allegro.tech.hermes.consumers.consumer.rate.ConsumerRateLimitSupervisor; import pl.allegro.tech.hermes.consumers.consumer.rate.calculator.OutputRateCalculatorFactory; import pl.allegro.tech.hermes.consumers.consumer.receiver.ReceiverFactory; @@ -264,7 +263,6 @@ public ConsumerFactory consumerFactory(ReceiverFactory messageReceiverFactory, ConsumerRateLimitSupervisor consumerRateLimitSupervisor, OutputRateCalculatorFactory outputRateCalculatorFactory, Trackers trackers, - OffsetQueue offsetQueue, ConsumerMessageSenderFactory consumerMessageSenderFactory, TopicRepository topicRepository, MessageConverterResolver messageConverterResolver, @@ -273,7 +271,9 @@ public ConsumerFactory consumerFactory(ReceiverFactory messageReceiverFactory, MessageBatchSenderFactory batchSenderFactory, ConsumerAuthorizationHandler consumerAuthorizationHandler, Clock clock, - SubscriptionLoadRecordersRegistry subscriptionLoadRecordersRegistry) { + SubscriptionLoadRecordersRegistry subscriptionLoadRecordersRegistry, + ConsumerPartitionAssignmentState consumerPartitionAssignmentState, + CommitOffsetProperties commitOffsetProperties) { return new ConsumerFactory( messageReceiverFactory, metrics, @@ -281,7 +281,6 @@ public ConsumerFactory consumerFactory(ReceiverFactory messageReceiverFactory, consumerRateLimitSupervisor, outputRateCalculatorFactory, trackers, - offsetQueue, consumerMessageSenderFactory, topicRepository, messageConverterResolver, @@ -290,7 +289,10 @@ public ConsumerFactory consumerFactory(ReceiverFactory messageReceiverFactory, batchSenderFactory, consumerAuthorizationHandler, clock, - subscriptionLoadRecordersRegistry + subscriptionLoadRecordersRegistry, + consumerPartitionAssignmentState, + commitOffsetProperties.getPeriod(), + commitOffsetProperties.getQueuesSize() ); } @@ -304,18 +306,16 @@ public ConsumersExecutorService consumersExecutorService(CommonConsumerPropertie public ConsumersSupervisor nonblockingConsumersSupervisor(CommonConsumerProperties commonConsumerProperties, ConsumersExecutorService executor, ConsumerFactory consumerFactory, - OffsetQueue offsetQueue, ConsumerPartitionAssignmentState consumerPartitionAssignmentState, Retransmitter retransmitter, UndeliveredMessageLogPersister undeliveredMessageLogPersister, SubscriptionRepository subscriptionRepository, MetricsFacade metrics, ConsumerMonitor monitor, - Clock clock, - CommitOffsetProperties commitOffsetProperties) { - return new NonblockingConsumersSupervisor(commonConsumerProperties, executor, consumerFactory, offsetQueue, + Clock clock) { + return new NonblockingConsumersSupervisor(commonConsumerProperties, executor, consumerFactory, consumerPartitionAssignmentState, retransmitter, undeliveredMessageLogPersister, - subscriptionRepository, metrics, monitor, clock, commitOffsetProperties.getPeriod()); + subscriptionRepository, metrics, monitor, clock); } @Bean(initMethod = "start", destroyMethod = "shutdown") diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/BatchConsumer.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/BatchConsumer.java index 9ef73b023b..8ae220fa94 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/BatchConsumer.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/BatchConsumer.java @@ -18,7 +18,7 @@ import pl.allegro.tech.hermes.consumers.consumer.batch.MessageBatchingResult; import pl.allegro.tech.hermes.consumers.consumer.converter.MessageConverterResolver; import pl.allegro.tech.hermes.consumers.consumer.load.SubscriptionLoadRecorder; -import pl.allegro.tech.hermes.consumers.consumer.offset.OffsetQueue; +import pl.allegro.tech.hermes.consumers.consumer.offset.SubscriptionPartition; import pl.allegro.tech.hermes.consumers.consumer.offset.SubscriptionPartitionOffset; import pl.allegro.tech.hermes.consumers.consumer.rate.BatchConsumerRateLimiter; import pl.allegro.tech.hermes.consumers.consumer.receiver.MessageReceiver; @@ -29,6 +29,11 @@ import pl.allegro.tech.hermes.tracker.consumers.Trackers; import java.io.IOException; +import java.time.Duration; +import java.time.Instant; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; import java.util.Optional; import java.util.Set; @@ -49,9 +54,9 @@ public class BatchConsumer implements Consumer { private final CompositeMessageContentWrapper compositeMessageContentWrapper; private final Trackers trackers; private final SubscriptionLoadRecorder loadRecorder; + private final Duration commitPeriod; private Topic topic; - private final OffsetQueue offsetQueue; private Subscription subscription; private volatile boolean consuming = true; @@ -60,10 +65,13 @@ public class BatchConsumer implements Consumer { private final BatchConsumerMetrics metrics; private MessageBatchReceiver receiver; + private final Map maxPendingOffsets = new HashMap<>(); + + private Instant lastCommitTime; + public BatchConsumer(ReceiverFactory messageReceiverFactory, MessageBatchSender sender, MessageBatchFactory batchFactory, - OffsetQueue offsetQueue, MessageConverterResolver messageConverterResolver, CompositeMessageContentWrapper compositeMessageContentWrapper, MetricsFacade metricsFacade, @@ -71,11 +79,11 @@ public BatchConsumer(ReceiverFactory messageReceiverFactory, Subscription subscription, Topic topic, boolean useTopicMessageSize, - SubscriptionLoadRecorder loadRecorder) { + SubscriptionLoadRecorder loadRecorder, + Duration commitPeriod) { this.messageReceiverFactory = messageReceiverFactory; this.sender = sender; this.batchFactory = batchFactory; - this.offsetQueue = offsetQueue; this.subscription = subscription; this.useTopicMessageSize = useTopicMessageSize; this.loadRecorder = loadRecorder; @@ -85,6 +93,8 @@ public BatchConsumer(ReceiverFactory messageReceiverFactory, this.compositeMessageContentWrapper = compositeMessageContentWrapper; this.topic = topic; this.trackers = trackers; + this.commitPeriod = commitPeriod; + this.lastCommitTime = Instant.now(); } @Override @@ -94,17 +104,17 @@ public void consume(Runnable signalsInterrupt) { logger.debug("Trying to create new batch [subscription={}].", subscription.getQualifiedName()); signalsInterrupt.run(); + commitIfReady(); MessageBatchingResult result = receiver.next(subscription, signalsInterrupt); inflight = of(result.getBatch()); inflight.ifPresent(batch -> { logger.debug("Delivering batch [subscription={}].", subscription.getQualifiedName()); - offerInflightOffsets(batch); deliver(signalsInterrupt, batch, createRetryer(batch, subscription.getBatchSubscriptionPolicy())); - offerCommittedOffsets(batch); + offerProcessedOffsets(batch); logger.debug("Finished delivering batch [subscription={}]", subscription.getQualifiedName()); }); @@ -118,12 +128,35 @@ public void consume(Runnable signalsInterrupt) { } } - private void offerInflightOffsets(MessageBatch batch) { - batch.getPartitionOffsets().forEach(offsetQueue::offerInflightOffset); + private void commitIfReady() { + if (isReadyToCommit()) { + Set offsetsToCommit = new HashSet<>(); + + for (Map.Entry entry : maxPendingOffsets.entrySet()) { + offsetsToCommit.add(new SubscriptionPartitionOffset(entry.getKey(), entry.getValue())); + } + + if (!offsetsToCommit.isEmpty()) { + commit(offsetsToCommit); + } + lastCommitTime = Instant.now(); + } } - private void offerCommittedOffsets(MessageBatch batch) { - batch.getPartitionOffsets().forEach(offsetQueue::offerCommittedOffset); + private boolean isReadyToCommit() { + return Duration.between(lastCommitTime, Instant.now()).toMillis() > commitPeriod.toMillis(); + } + + private void offerProcessedOffsets(MessageBatch batch) { + for (SubscriptionPartitionOffset offset : batch.getPartitionOffsets()) { + putOffset(offset); + } + } + + private void putOffset(SubscriptionPartitionOffset offset) { + maxPendingOffsets.compute(offset.getSubscriptionPartition(), (subscriptionPartition, maxOffset) -> + maxOffset == null ? offset.getOffset() : Math.max(maxOffset, offset.getOffset()) + ); } @Override @@ -135,7 +168,8 @@ public void initialize() { subscription, new BatchConsumerRateLimiter(), loadRecorder, - metricsFacade + metricsFacade, + this::putOffset ); logger.debug("Consumer: preparing batch receiver for subscription {}", subscription.getQualifiedName()); diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSender.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSender.java index 151235730d..14fc29a8bf 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSender.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSender.java @@ -7,12 +7,12 @@ import pl.allegro.tech.hermes.api.Subscription; import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.consumers.consumer.load.SubscriptionLoadRecorder; +import pl.allegro.tech.hermes.consumers.consumer.offset.PendingOffsets; import pl.allegro.tech.hermes.consumers.consumer.profiling.ConsumerProfiler; import pl.allegro.tech.hermes.consumers.consumer.profiling.ConsumerRun; import pl.allegro.tech.hermes.consumers.consumer.profiling.DefaultConsumerProfiler; import pl.allegro.tech.hermes.consumers.consumer.profiling.Measurement; import pl.allegro.tech.hermes.consumers.consumer.profiling.NoOpConsumerProfiler; -import pl.allegro.tech.hermes.consumers.consumer.rate.InflightsPool; import pl.allegro.tech.hermes.consumers.consumer.rate.SerialConsumerRateLimiter; import pl.allegro.tech.hermes.consumers.consumer.result.ErrorHandler; import pl.allegro.tech.hermes.consumers.consumer.result.SuccessHandler; @@ -39,6 +39,7 @@ import static java.lang.String.format; import static org.apache.commons.lang3.math.NumberUtils.INTEGER_ZERO; +import static pl.allegro.tech.hermes.consumers.consumer.offset.SubscriptionPartitionOffset.subscriptionPartitionOffset; public class ConsumerMessageSender { @@ -48,7 +49,7 @@ public class ConsumerMessageSender { private final List errorHandlers; private final MessageSenderFactory messageSenderFactory; private final Clock clock; - private final InflightsPool inflight; + private final PendingOffsets pendingOffsets; private final SubscriptionLoadRecorder loadRecorder; private final HermesTimer consumerLatencyTimer; private final HermesCounter retries; @@ -70,7 +71,7 @@ public ConsumerMessageSender(Subscription subscription, List errorHandlers, SerialConsumerRateLimiter rateLimiter, ExecutorService deliveryReportingExecutor, - InflightsPool inflight, + PendingOffsets pendingOffsets, MetricsFacade metrics, int asyncTimeoutMs, FutureAsyncTimeout futureAsyncTimeout, @@ -87,7 +88,7 @@ public ConsumerMessageSender(Subscription subscription, this.asyncTimeoutMs = asyncTimeoutMs; this.messageSender = messageSender(subscription); this.subscription = subscription; - this.inflight = inflight; + this.pendingOffsets = pendingOffsets; this.consumerLatencyTimer = metrics.subscriptions().latency(subscription.getQualifiedName()); metrics.subscriptions().registerInflightGauge(subscription.getQualifiedName(), this, sender -> sender.inflightCount.doubleValue()); this.retries = metrics.subscriptions().retries(subscription.getQualifiedName()); @@ -258,14 +259,16 @@ private void logResultInfo(Message message, MessageSendingResultLogInfo logInfo) } private void handleMessageDiscarding(Message message, MessageSendingResult result, ConsumerProfiler profiler) { - inflight.release(); + pendingOffsets.markAsProcessed(subscriptionPartitionOffset(subscription.getQualifiedName(), + message.getPartitionOffset(), message.getPartitionAssignmentTerm())); inflightCount.decrement(); errorHandlers.forEach(h -> h.handleDiscarded(message, subscription, result)); profiler.flushMeasurements(ConsumerRun.DISCARDED); } private void handleMessageSendingSuccess(Message message, MessageSendingResult result, ConsumerProfiler profiler) { - inflight.release(); + pendingOffsets.markAsProcessed(subscriptionPartitionOffset(subscription.getQualifiedName(), + message.getPartitionOffset(), message.getPartitionAssignmentTerm())); inflightCount.decrement(); successHandlers.forEach(h -> h.handleSuccess(message, subscription, result)); profiler.flushMeasurements(ConsumerRun.DELIVERED); diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSenderFactory.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSenderFactory.java index c47557bcf7..066670df9b 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSenderFactory.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSenderFactory.java @@ -5,8 +5,7 @@ import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.common.metric.executor.InstrumentedExecutorServiceFactory; import pl.allegro.tech.hermes.consumers.consumer.load.SubscriptionLoadRecorder; -import pl.allegro.tech.hermes.consumers.consumer.offset.OffsetQueue; -import pl.allegro.tech.hermes.consumers.consumer.rate.InflightsPool; +import pl.allegro.tech.hermes.consumers.consumer.offset.PendingOffsets; import pl.allegro.tech.hermes.consumers.consumer.rate.SerialConsumerRateLimiter; import pl.allegro.tech.hermes.consumers.consumer.result.DefaultErrorHandler; import pl.allegro.tech.hermes.consumers.consumer.result.DefaultSuccessHandler; @@ -57,19 +56,17 @@ public ConsumerMessageSenderFactory(String kafkaClusterName, MessageSenderFactor public ConsumerMessageSender create(Subscription subscription, SerialConsumerRateLimiter consumerRateLimiter, - OffsetQueue offsetQueue, - InflightsPool inflight, + PendingOffsets pendingOffsets, SubscriptionLoadRecorder subscriptionLoadRecorder, MetricsFacade metrics) { List successHandlers = Arrays.asList( consumerAuthorizationHandler, - new DefaultSuccessHandler(offsetQueue, metrics, trackers, subscription.getQualifiedName())); + new DefaultSuccessHandler(metrics, trackers, subscription.getQualifiedName())); List errorHandlers = Arrays.asList( consumerAuthorizationHandler, new DefaultErrorHandler( - offsetQueue, metrics, undeliveredMessageLog, clock, @@ -85,7 +82,7 @@ public ConsumerMessageSender create(Subscription subscription, errorHandlers, consumerRateLimiter, rateLimiterReportingExecutor, - inflight, + pendingOffsets, metrics, senderAsyncTimeoutMs, futureAsyncTimeout, diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/SerialConsumer.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/SerialConsumer.java index 3a65ed3ca4..aa4bc636e2 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/SerialConsumer.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/SerialConsumer.java @@ -9,14 +9,15 @@ import pl.allegro.tech.hermes.consumers.CommonConsumerParameters; import pl.allegro.tech.hermes.consumers.consumer.converter.MessageConverterResolver; import pl.allegro.tech.hermes.consumers.consumer.load.SubscriptionLoadRecorder; -import pl.allegro.tech.hermes.consumers.consumer.offset.OffsetQueue; +import pl.allegro.tech.hermes.consumers.consumer.offset.ConsumerPartitionAssignmentState; +import pl.allegro.tech.hermes.consumers.consumer.offset.OffsetCommitter; +import pl.allegro.tech.hermes.consumers.consumer.offset.PendingOffsets; import pl.allegro.tech.hermes.consumers.consumer.offset.SubscriptionPartitionOffset; import pl.allegro.tech.hermes.consumers.consumer.profiling.ConsumerProfiler; import pl.allegro.tech.hermes.consumers.consumer.profiling.ConsumerRun; import pl.allegro.tech.hermes.consumers.consumer.profiling.DefaultConsumerProfiler; import pl.allegro.tech.hermes.consumers.consumer.profiling.Measurement; import pl.allegro.tech.hermes.consumers.consumer.profiling.NoOpConsumerProfiler; -import pl.allegro.tech.hermes.consumers.consumer.rate.AdjustableSemaphore; import pl.allegro.tech.hermes.consumers.consumer.rate.SerialConsumerRateLimiter; import pl.allegro.tech.hermes.consumers.consumer.receiver.MessageReceiver; import pl.allegro.tech.hermes.consumers.consumer.receiver.ReceiverFactory; @@ -24,9 +25,9 @@ import pl.allegro.tech.hermes.tracker.consumers.Trackers; import java.time.Duration; +import java.time.Instant; import java.util.Optional; import java.util.Set; -import java.util.concurrent.TimeUnit; import static pl.allegro.tech.hermes.consumers.consumer.message.MessageConverter.toMessageMetadata; import static pl.allegro.tech.hermes.consumers.consumer.offset.SubscriptionPartitionOffset.subscriptionPartitionOffset; @@ -42,10 +43,11 @@ public class SerialConsumer implements Consumer { private final MessageConverterResolver messageConverterResolver; private final ConsumerMessageSender sender; private final boolean useTopicMessageSizeEnabled; - private final OffsetQueue offsetQueue; + private final PendingOffsets pendingOffsets; private final ConsumerAuthorizationHandler consumerAuthorizationHandler; - private final AdjustableSemaphore inflightSemaphore; private final SubscriptionLoadRecorder loadRecorder; + private final OffsetCommitter offsetCommitter; + private final Duration commitPeriod; private final int defaultInflight; private final Duration signalProcessingInterval; @@ -55,6 +57,8 @@ public class SerialConsumer implements Consumer { private MessageReceiver messageReceiver; + private Instant lastCommitTime; + public SerialConsumer(ReceiverFactory messageReceiverFactory, MetricsFacade metrics, Subscription subscription, @@ -64,32 +68,35 @@ public SerialConsumer(ReceiverFactory messageReceiverFactory, MessageConverterResolver messageConverterResolver, Topic topic, CommonConsumerParameters commonConsumerParameters, - OffsetQueue offsetQueue, ConsumerAuthorizationHandler consumerAuthorizationHandler, - SubscriptionLoadRecorder loadRecorder) { + SubscriptionLoadRecorder loadRecorder, + ConsumerPartitionAssignmentState consumerPartitionAssignmentState, + Duration commitPeriod, + int offsetQueueSize) { this.defaultInflight = commonConsumerParameters.getSerialConsumer().getInflightSize(); this.signalProcessingInterval = commonConsumerParameters.getSerialConsumer().getSignalProcessingInterval(); - this.inflightSemaphore = new AdjustableSemaphore(calculateInflightSize(subscription)); this.messageReceiverFactory = messageReceiverFactory; this.metrics = metrics; this.subscription = subscription; this.rateLimiter = rateLimiter; this.useTopicMessageSizeEnabled = commonConsumerParameters.isUseTopicMessageSizeEnabled(); - this.offsetQueue = offsetQueue; + this.pendingOffsets = new PendingOffsets(subscription.getQualifiedName(), metrics, calculateInflightSize(subscription), offsetQueueSize); this.consumerAuthorizationHandler = consumerAuthorizationHandler; this.trackers = trackers; this.messageConverterResolver = messageConverterResolver; this.loadRecorder = loadRecorder; this.messageReceiver = new UninitializedMessageReceiver(); this.topic = topic; + this.offsetCommitter = new OffsetCommitter(consumerPartitionAssignmentState, metrics); this.sender = consumerMessageSenderFactory.create( subscription, rateLimiter, - offsetQueue, - inflightSemaphore::release, + pendingOffsets, loadRecorder, metrics ); + this.commitPeriod = commitPeriod; + this.lastCommitTime = Instant.now(); } private int calculateInflightSize(Subscription subscription) { @@ -106,8 +113,9 @@ public void consume(Runnable signalsInterrupt) { loadRecorder.recordSingleOperation(); profiler.startPartialMeasurement(Measurement.SIGNALS_INTERRUPT_RUN); signalsInterrupt.run(); + commitIfReady(); profiler.stopPartialMeasurement(); - } while (!inflightSemaphore.tryAcquire(signalProcessingInterval.toMillis(), TimeUnit.MILLISECONDS)); + } while (!pendingOffsets.tryAcquireSlot(signalProcessingInterval)); profiler.measure(Measurement.MESSAGE_RECEIVER_NEXT); Optional maybeMessage = messageReceiver.next(); @@ -126,7 +134,7 @@ public void consume(Runnable signalsInterrupt) { Message convertedMessage = messageConverterResolver.converterFor(message, subscription).convert(message, topic); sendMessage(convertedMessage, profiler); } else { - inflightSemaphore.release(); + pendingOffsets.releaseSlot(); profiler.flushMeasurements(ConsumerRun.EMPTY); } } catch (InterruptedException e) { @@ -137,13 +145,27 @@ public void consume(Runnable signalsInterrupt) { } } - private void sendMessage(Message message, ConsumerProfiler profiler) { + private void commitIfReady() { + if (isReadyToCommit()) { + Set offsetsToCommit = offsetCommitter.calculateOffsetsToBeCommitted(pendingOffsets.getOffsetsSnapshotAndReleaseProcessedSlots()); + if (!offsetsToCommit.isEmpty()) { + commit(offsetsToCommit); + } + lastCommitTime = Instant.now(); + } + } + + private boolean isReadyToCommit() { + return Duration.between(lastCommitTime, Instant.now()).toMillis() > commitPeriod.toMillis(); + } + + private void sendMessage(Message message, ConsumerProfiler profiler) throws InterruptedException { profiler.measure(Measurement.OFFER_INFLIGHT_OFFSET); - offsetQueue.offerInflightOffset( + pendingOffsets.markAsInflight( subscriptionPartitionOffset(subscription.getQualifiedName(), - message.getPartitionOffset(), - message.getPartitionAssignmentTerm()) - ); + message.getPartitionOffset(), + message.getPartitionAssignmentTerm() + )); profiler.measure(Measurement.TRACKERS_LOG_INFLIGHT); trackers.get(subscription).logInflight(toMessageMetadata(message, subscription)); @@ -162,7 +184,14 @@ public void initialize() { } private void initializeMessageReceiver() { - this.messageReceiver = messageReceiverFactory.createMessageReceiver(topic, subscription, rateLimiter, loadRecorder, metrics); + this.messageReceiver = messageReceiverFactory.createMessageReceiver( + topic, + subscription, + rateLimiter, + loadRecorder, + metrics, + pendingOffsets::markAsProcessed + ); } /** @@ -181,7 +210,7 @@ public void tearDown() { @Override public void updateSubscription(Subscription newSubscription) { logger.info("Updating consumer for subscription {}", subscription.getQualifiedName()); - inflightSemaphore.setMaxPermits(calculateInflightSize(newSubscription)); + pendingOffsets.setInflightSize(calculateInflightSize(newSubscription)); rateLimiter.updateSubscription(newSubscription); sender.updateSubscription(newSubscription); messageReceiver.update(newSubscription); diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/filtering/FilteredMessageHandler.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/filtering/FilteredMessageHandler.java index 12063eb085..58af3a135e 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/filtering/FilteredMessageHandler.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/filtering/FilteredMessageHandler.java @@ -6,7 +6,7 @@ import pl.allegro.tech.hermes.api.SubscriptionName; import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.consumers.consumer.Message; -import pl.allegro.tech.hermes.consumers.consumer.offset.OffsetQueue; +import pl.allegro.tech.hermes.consumers.consumer.offset.PendingOffsetsAppender; import pl.allegro.tech.hermes.consumers.consumer.rate.ConsumerRateLimiter; import pl.allegro.tech.hermes.domain.filtering.chain.FilterResult; import pl.allegro.tech.hermes.metrics.HermesCounter; @@ -19,20 +19,20 @@ public class FilteredMessageHandler { - private final OffsetQueue offsetQueue; + private final PendingOffsetsAppender pendingOffsets; private final Optional consumerRateLimiter; private final Trackers trackers; private final HermesCounter filteredOutCounter; private static final Logger logger = LoggerFactory.getLogger(FilteredMessageHandler.class); - public FilteredMessageHandler(OffsetQueue offsetQueue, - ConsumerRateLimiter consumerRateLimiter, + public FilteredMessageHandler(ConsumerRateLimiter consumerRateLimiter, + PendingOffsetsAppender pendingOffsetsAppender, Trackers trackers, MetricsFacade metrics, SubscriptionName subscriptionName) { - this.offsetQueue = offsetQueue; this.consumerRateLimiter = Optional.ofNullable(consumerRateLimiter); + this.pendingOffsets = pendingOffsetsAppender; this.trackers = trackers; this.filteredOutCounter = metrics.subscriptions().filteredOutCounter(subscriptionName); } @@ -43,7 +43,7 @@ public void handle(FilterResult result, Message message, Subscription subscripti logger.debug("Message filtered for subscription {} {}", subscription.getQualifiedName(), result); } - offsetQueue.offerCommittedOffset(subscriptionPartitionOffset(subscription.getQualifiedName(), + pendingOffsets.markAsProcessed(subscriptionPartitionOffset(subscription.getQualifiedName(), message.getPartitionOffset(), message.getPartitionAssignmentTerm())); filteredOutCounter.increment(); diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/offset/MessageState.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/offset/MessageState.java new file mode 100644 index 0000000000..db120576cc --- /dev/null +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/offset/MessageState.java @@ -0,0 +1,6 @@ +package pl.allegro.tech.hermes.consumers.consumer.offset; + +public enum MessageState { + INFLIGHT, + PROCESSED +} diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/offset/OffsetCommitter.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/offset/OffsetCommitter.java index ad96812df3..4f6e539a93 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/offset/OffsetCommitter.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/offset/OffsetCommitter.java @@ -1,24 +1,21 @@ package pl.allegro.tech.hermes.consumers.consumer.offset; import com.google.common.collect.Sets; -import com.google.common.util.concurrent.ThreadFactoryBuilder; -import org.jctools.queues.MessagePassingQueue; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import pl.allegro.tech.hermes.common.metric.MetricsFacade; -import pl.allegro.tech.hermes.consumers.consumer.receiver.MessageCommitter; import pl.allegro.tech.hermes.metrics.HermesCounter; import pl.allegro.tech.hermes.metrics.HermesTimer; import pl.allegro.tech.hermes.metrics.HermesTimerContext; +import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Set; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; import java.util.function.BiFunction; +import java.util.function.Consumer; import java.util.function.Function; /** @@ -70,20 +67,12 @@ * *

This algorithm is very simple, memory efficient, can be performed in single thread and introduces no locks.

*/ -public class OffsetCommitter implements Runnable { +public class OffsetCommitter { private static final Logger logger = LoggerFactory.getLogger(OffsetCommitter.class); - private final ScheduledExecutorService scheduledExecutor = Executors.newSingleThreadScheduledExecutor( - new ThreadFactoryBuilder().setNameFormat("offset-committer-%d").build()); - - private final int offsetCommitPeriodSeconds; - - private final OffsetQueue offsetQueue; - private final ConsumerPartitionAssignmentState partitionAssignmentState; - private final MessageCommitter messageCommitter; private final HermesCounter obsoleteCounter; private final HermesCounter committedCounter; @@ -93,41 +82,44 @@ public class OffsetCommitter implements Runnable { private final Map maxCommittedOffsets = new HashMap<>(); public OffsetCommitter( - OffsetQueue offsetQueue, ConsumerPartitionAssignmentState partitionAssignmentState, - MessageCommitter messageCommitter, - int offsetCommitPeriodSeconds, MetricsFacade metrics ) { - this.offsetQueue = offsetQueue; this.partitionAssignmentState = partitionAssignmentState; - this.messageCommitter = messageCommitter; - this.offsetCommitPeriodSeconds = offsetCommitPeriodSeconds; this.obsoleteCounter = metrics.offsetCommits().obsoleteCounter(); this.committedCounter = metrics.offsetCommits().committedCounter(); this.timer = metrics.offsetCommits().duration(); } - @Override - public void run() { + public Set calculateOffsetsToBeCommitted(Map offsets) { try (HermesTimerContext ignored = timer.time()) { - // committed offsets need to be drained first so that there is no possibility of new committed offsets - // showing up after inflight queue is drained - this would lead to stall in committing offsets - ReducingConsumer committedOffsetsReducer = processCommittedOffsets(); + List processedOffsets = new ArrayList<>(); + for (Map.Entry entry : offsets.entrySet()) { + if (entry.getValue() == MessageState.PROCESSED) { + processedOffsets.add(entry.getKey()); + } + } + + List allOffsets = new ArrayList<>(); + for (Map.Entry entry : offsets.entrySet()) { + allOffsets.add(entry.getKey()); + } + + ReducingConsumer processedOffsetsReducer = prepareProcessedOffsets(processedOffsets); // update stored max committed offsets with offsets drained from queue - Map maxDrainedCommittedOffsets = committedOffsetsReducer.reduced; - updateMaxCommittedOffsets(maxDrainedCommittedOffsets); + Map maxDrainedProcessedOffsets = processedOffsetsReducer.reduced; + updateMaxProcessedOffsets(maxDrainedProcessedOffsets); - ReducingConsumer inflightOffsetReducer = processInflightOffsets(committedOffsetsReducer.all); + ReducingConsumer inflightOffsetReducer = prepareInflightOffsets(processedOffsetsReducer.all, allOffsets); Map minInflightOffsets = inflightOffsetReducer.reduced; int scheduledToCommitCount = 0; int obsoleteCount = 0; - Set committedOffsetToBeRemoved = new HashSet<>(); + Set processedOffsetToBeRemoved = new HashSet<>(); - OffsetsToCommit offsetsToCommit = new OffsetsToCommit(); + Set offsetsToCommit = new HashSet<>(); for (SubscriptionPartition partition : Sets.union(minInflightOffsets.keySet(), maxCommittedOffsets.keySet())) { if (partitionAssignmentState.isAssignedPartitionAtCurrentTerm(partition)) { long minInflight = minInflightOffsets.getOrDefault(partition, Long.MAX_VALUE); @@ -140,7 +132,7 @@ public void run() { // if we just committed the maximum possible offset for partition, we can safely forget about it if (maxCommitted == offsetToBeCommitted) { - committedOffsetToBeRemoved.add(partition); + processedOffsetToBeRemoved.add(partition); } } else { logger.warn("Skipping offset out of bounds for subscription {}: partition={}, offset={}", @@ -150,36 +142,44 @@ public void run() { obsoleteCount++; } } - committedOffsetToBeRemoved.forEach(maxCommittedOffsets::remove); - messageCommitter.commitOffsets(offsetsToCommit); + processedOffsetToBeRemoved.forEach(maxCommittedOffsets::remove); obsoleteCounter.increment(obsoleteCount); committedCounter.increment(scheduledToCommitCount); cleanupStoredOffsetsWithObsoleteTerms(); + + return offsetsToCommit; } catch (Exception exception) { logger.error("Failed to run offset committer: {}", exception.getMessage(), exception); } + return Set.of(); } - private ReducingConsumer processCommittedOffsets() { - ReducingConsumer committedOffsetsReducer = new ReducingConsumer(Math::max, c -> c + 1); - offsetQueue.drainCommittedOffsets(committedOffsetsReducer); - committedOffsetsReducer.resetModifierFunction(); - return committedOffsetsReducer; + private ReducingConsumer prepareProcessedOffsets(List processedOffsets) { + ReducingConsumer processedOffsetsReducer = new ReducingConsumer(Math::max, c -> c + 1); + drain(processedOffsets, processedOffsetsReducer); + processedOffsetsReducer.resetModifierFunction(); + return processedOffsetsReducer; } - private void updateMaxCommittedOffsets(Map maxDrainedCommittedOffsets) { + private void updateMaxProcessedOffsets(Map maxDrainedCommittedOffsets) { maxDrainedCommittedOffsets.forEach((partition, drainedOffset) -> maxCommittedOffsets.compute(partition, (p, storedOffset) -> storedOffset == null || storedOffset < drainedOffset ? drainedOffset : storedOffset) ); } - private ReducingConsumer processInflightOffsets(Set committedOffsets) { + private ReducingConsumer prepareInflightOffsets(Set processedOffsets, + List inflightOffsetsQueue) { + // smallest undelivered message ReducingConsumer inflightOffsetReducer = new ReducingConsumer(Math::min); - offsetQueue.drainInflightOffsets(o -> reduceIfNotCommitted(o, inflightOffsetReducer, committedOffsets)); - inflightOffsets.forEach(o -> reduceIfNotCommitted(o, inflightOffsetReducer, committedOffsets)); + + // process inflights from the current iteration + drain(inflightOffsetsQueue, o -> reduceIfNotDelivered(o, inflightOffsetReducer, processedOffsets)); + + // process inflights from the previous iteration + inflightOffsets.forEach(o -> reduceIfNotDelivered(o, inflightOffsetReducer, processedOffsets)); inflightOffsets.clear(); inflightOffsets.addAll(inflightOffsetReducer.all); @@ -187,7 +187,7 @@ private ReducingConsumer processInflightOffsets(Set return inflightOffsetReducer; } - private void reduceIfNotCommitted(SubscriptionPartitionOffset offset, + private void reduceIfNotDelivered(SubscriptionPartitionOffset offset, ReducingConsumer inflightOffsetReducer, Set committedOffsets) { if (!committedOffsets.contains(offset)) { @@ -200,19 +200,20 @@ private void cleanupStoredOffsetsWithObsoleteTerms() { maxCommittedOffsets.entrySet().removeIf(entry -> !partitionAssignmentState.isAssignedPartitionAtCurrentTerm(entry.getKey())); } - public void start() { - scheduledExecutor.scheduleWithFixedDelay(this, - offsetCommitPeriodSeconds, - offsetCommitPeriodSeconds, - TimeUnit.SECONDS - ); - } - - public void shutdown() { - scheduledExecutor.shutdown(); + private void drain(List subscriptionPartitionOffsets, Consumer consumer) { + int size = subscriptionPartitionOffsets.size(); + for (int i = 0; i < size; i++) { + SubscriptionPartitionOffset element = subscriptionPartitionOffsets.get(i); + if (element != null) { + consumer.accept(element); + } else { + logger.warn("Unexpected null value while draining queue [idx={}, size={}]", i, size); + break; + } + } } - private static final class ReducingConsumer implements MessagePassingQueue.Consumer { + private static final class ReducingConsumer implements Consumer { private final BiFunction reductor; private Function modifier; private final Map reduced = new HashMap<>(); diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/offset/OffsetQueue.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/offset/OffsetQueue.java deleted file mode 100644 index aee72eb210..0000000000 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/offset/OffsetQueue.java +++ /dev/null @@ -1,38 +0,0 @@ -package pl.allegro.tech.hermes.consumers.consumer.offset; - -import org.jctools.queues.MessagePassingQueue; -import pl.allegro.tech.hermes.common.metric.MetricsFacade; -import pl.allegro.tech.hermes.consumers.queue.FullDrainMpscQueue; -import pl.allegro.tech.hermes.consumers.queue.MonitoredMpscQueue; -import pl.allegro.tech.hermes.consumers.queue.MpscQueue; -import pl.allegro.tech.hermes.consumers.queue.WaitFreeDrainMpscQueue; - -public class OffsetQueue { - - private final MpscQueue inflightOffsetsQueue; - - private final MpscQueue commitOffsetsQueue; - - public OffsetQueue(MetricsFacade metrics, int commitOffsetQueuesSize) { - this.inflightOffsetsQueue = - new MonitoredMpscQueue<>(new FullDrainMpscQueue<>(commitOffsetQueuesSize), metrics, "inflightOffsets"); - this.commitOffsetsQueue = - new MonitoredMpscQueue<>(new WaitFreeDrainMpscQueue<>(commitOffsetQueuesSize), metrics, "committedOffsets"); - } - - public void offerInflightOffset(SubscriptionPartitionOffset offset) { - inflightOffsetsQueue.offer(offset); - } - - public void offerCommittedOffset(SubscriptionPartitionOffset offset) { - commitOffsetsQueue.offer(offset); - } - - public void drainInflightOffsets(MessagePassingQueue.Consumer consumer) { - inflightOffsetsQueue.drain(consumer); - } - - public void drainCommittedOffsets(MessagePassingQueue.Consumer consumer) { - commitOffsetsQueue.drain(consumer); - } -} diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/offset/OffsetsToCommit.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/offset/OffsetsToCommit.java deleted file mode 100644 index 8da371d7f4..0000000000 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/offset/OffsetsToCommit.java +++ /dev/null @@ -1,31 +0,0 @@ -package pl.allegro.tech.hermes.consumers.consumer.offset; - -import pl.allegro.tech.hermes.api.SubscriptionName; - -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; - -public class OffsetsToCommit { - - private final Map> offsets = new HashMap<>(); - - public OffsetsToCommit add(SubscriptionPartitionOffset offset) { - Set subscriptionOffsets = offsets.get(offset.getSubscriptionName()); - if (subscriptionOffsets == null) { - subscriptionOffsets = new HashSet<>(); - offsets.put(offset.getSubscriptionName(), subscriptionOffsets); - } - subscriptionOffsets.add(offset); - return this; - } - - public Set subscriptionNames() { - return offsets.keySet(); - } - - public Set batchFor(SubscriptionName subscriptionName) { - return offsets.get(subscriptionName); - } -} diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/offset/PendingOffsets.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/offset/PendingOffsets.java new file mode 100644 index 0000000000..1abfac203e --- /dev/null +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/offset/PendingOffsets.java @@ -0,0 +1,85 @@ +package pl.allegro.tech.hermes.consumers.consumer.offset; + +import pl.allegro.tech.hermes.api.SubscriptionName; +import pl.allegro.tech.hermes.common.metric.MetricsFacade; +import pl.allegro.tech.hermes.consumers.consumer.rate.AdjustableSemaphore; + +import java.time.Duration; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; + +/** + * This class manages pending offsets for message consumption in a thread-safe manner. + * It ensures that the number of pending offsets does not exceed a specified maximum limit. + * + *

The {@code slots} map is effectively a bounded map, guarded by the {@code maxPendingOffsetsSemaphore}. + * This semaphore ensures that the number of entries in the {@code slots} map does not exceed {@code maxPendingOffsets} and prevents running out of memory. + * The semaphore is used to acquire permits before adding entries to the map and to release permits when entries are removed. + *

+ * + *

The {@code inflightSemaphore} is used to limit the number of messages that are currently being processed (inflight). + * It helps control the concurrency level of message processing. + *

+ * + *

Note: Methods that modify the state of the {@code slots} map, such as {@code markAsProcessed} and {@code markAsInflight}, + * must only be called after successfully acquiring a permit using the {@code tryAcquireSlot} method. + *

+ */ +public class PendingOffsets { + + private final ConcurrentHashMap slots = new ConcurrentHashMap<>(); + private final AdjustableSemaphore inflightSemaphore; + private final Semaphore maxPendingOffsetsSemaphore; + + public PendingOffsets(SubscriptionName subscriptionName, MetricsFacade metrics, int inflightQueueSize, int maxPendingOffsets) { + this.maxPendingOffsetsSemaphore = new Semaphore(maxPendingOffsets); + this.inflightSemaphore = new AdjustableSemaphore(inflightQueueSize); + metrics.subscriptions().registerPendingOffsetsGauge(subscriptionName, maxPendingOffsetsSemaphore, slots -> (maxPendingOffsets - (double) slots.availablePermits()) / maxPendingOffsets); + } + + public void setInflightSize(int inflightQueueSize) { + this.inflightSemaphore.setMaxPermits(inflightQueueSize); + } + + public void markAsProcessed(SubscriptionPartitionOffset subscriptionPartitionOffset) { + inflightSemaphore.release(); + slots.put(subscriptionPartitionOffset, MessageState.PROCESSED); + } + + public boolean tryAcquireSlot(Duration processingInterval) throws InterruptedException { + if (inflightSemaphore.tryAcquire(processingInterval.toMillis(), TimeUnit.MILLISECONDS)) { + if (maxPendingOffsetsSemaphore.tryAcquire(processingInterval.toMillis(), TimeUnit.MILLISECONDS)) { + return true; + } + inflightSemaphore.release(); + } + return false; + } + + public void markAsInflight(SubscriptionPartitionOffset subscriptionPartitionOffset) { + slots.put(subscriptionPartitionOffset, MessageState.INFLIGHT); + } + + public Map getOffsetsSnapshotAndReleaseProcessedSlots() { + int permitsReleased = 0; + Map offsetSnapshot = new HashMap<>(); + + for (Map.Entry entry : slots.entrySet()) { + offsetSnapshot.put(entry.getKey(), entry.getValue()); + if (entry.getValue() == MessageState.PROCESSED) { + slots.remove(entry.getKey()); + permitsReleased++; + } + } + maxPendingOffsetsSemaphore.release(permitsReleased); + return offsetSnapshot; + } + + public void releaseSlot() { + inflightSemaphore.release(); + maxPendingOffsetsSemaphore.release(); + } +} diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/offset/PendingOffsetsAppender.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/offset/PendingOffsetsAppender.java new file mode 100644 index 0000000000..a2405cf46d --- /dev/null +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/offset/PendingOffsetsAppender.java @@ -0,0 +1,6 @@ +package pl.allegro.tech.hermes.consumers.consumer.offset; + +public interface PendingOffsetsAppender { + + void markAsProcessed(SubscriptionPartitionOffset subscriptionPartitionOffset); +} diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/offset/SubscriptionPartitionOffset.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/offset/SubscriptionPartitionOffset.java index b5ef044c8c..120b8ad766 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/offset/SubscriptionPartitionOffset.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/offset/SubscriptionPartitionOffset.java @@ -46,6 +46,10 @@ public long getOffset() { return offset; } + public PartitionOffset getPartitionOffset() { + return new PartitionOffset(subscriptionPartition.getKafkaTopicName(), offset, subscriptionPartition.getPartition()); + } + public long getPartitionAssignmentTerm() { return subscriptionPartition.getPartitionAssignmentTerm(); } diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/receiver/MessageCommitter.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/receiver/MessageCommitter.java deleted file mode 100644 index ed1fa97fc4..0000000000 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/receiver/MessageCommitter.java +++ /dev/null @@ -1,7 +0,0 @@ -package pl.allegro.tech.hermes.consumers.consumer.receiver; - -import pl.allegro.tech.hermes.consumers.consumer.offset.OffsetsToCommit; - -public interface MessageCommitter { - void commitOffsets(OffsetsToCommit offsetsToCommit); -} diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/receiver/ReceiverFactory.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/receiver/ReceiverFactory.java index 4077ad22bf..c3dce357a3 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/receiver/ReceiverFactory.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/receiver/ReceiverFactory.java @@ -4,6 +4,7 @@ import pl.allegro.tech.hermes.api.Topic; import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.consumers.consumer.load.SubscriptionLoadRecorder; +import pl.allegro.tech.hermes.consumers.consumer.offset.PendingOffsetsAppender; import pl.allegro.tech.hermes.consumers.consumer.rate.ConsumerRateLimiter; public interface ReceiverFactory { @@ -12,6 +13,7 @@ MessageReceiver createMessageReceiver(Topic receivingTopic, Subscription subscription, ConsumerRateLimiter consumerRateLimiter, SubscriptionLoadRecorder subscriptionLoadRecorder, - MetricsFacade metrics); + MetricsFacade metrics, + PendingOffsetsAppender pendingOffsetsAppender); } diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/receiver/kafka/KafkaMessageReceiverFactory.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/receiver/kafka/KafkaMessageReceiverFactory.java index 816c5b12d0..26a58cfd4a 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/receiver/kafka/KafkaMessageReceiverFactory.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/receiver/kafka/KafkaMessageReceiverFactory.java @@ -13,7 +13,7 @@ import pl.allegro.tech.hermes.consumers.consumer.idletime.IdleTimeCalculator; import pl.allegro.tech.hermes.consumers.consumer.load.SubscriptionLoadRecorder; import pl.allegro.tech.hermes.consumers.consumer.offset.ConsumerPartitionAssignmentState; -import pl.allegro.tech.hermes.consumers.consumer.offset.OffsetQueue; +import pl.allegro.tech.hermes.consumers.consumer.offset.PendingOffsetsAppender; import pl.allegro.tech.hermes.consumers.consumer.rate.ConsumerRateLimiter; import pl.allegro.tech.hermes.consumers.consumer.receiver.MessageReceiver; import pl.allegro.tech.hermes.consumers.consumer.receiver.ReceiverFactory; @@ -60,7 +60,6 @@ public class KafkaMessageReceiverFactory implements ReceiverFactory { private final KafkaConsumerParameters kafkaConsumerParameters; private final KafkaConsumerRecordToMessageConverterFactory messageConverterFactory; private final MetricsFacade metricsFacade; - private final OffsetQueue offsetQueue; private final KafkaNamesMapper kafkaNamesMapper; private final FilterChainFactory filterChainFactory; private final Trackers trackers; @@ -72,7 +71,6 @@ public KafkaMessageReceiverFactory(CommonConsumerParameters commonConsumerParame KafkaParameters kafkaParameters, KafkaConsumerRecordToMessageConverterFactory messageConverterFactory, MetricsFacade metricsFacade, - OffsetQueue offsetQueue, KafkaNamesMapper kafkaNamesMapper, FilterChainFactory filterChainFactory, Trackers trackers, @@ -83,7 +81,6 @@ public KafkaMessageReceiverFactory(CommonConsumerParameters commonConsumerParame this.kafkaParameters = kafkaParameters; this.messageConverterFactory = messageConverterFactory; this.metricsFacade = metricsFacade; - this.offsetQueue = offsetQueue; this.kafkaNamesMapper = kafkaNamesMapper; this.filterChainFactory = filterChainFactory; this.trackers = trackers; @@ -95,7 +92,8 @@ public MessageReceiver createMessageReceiver(Topic topic, Subscription subscription, ConsumerRateLimiter consumerRateLimiter, SubscriptionLoadRecorder loadReporter, - MetricsFacade metrics) { + MetricsFacade metrics, + PendingOffsetsAppender pendingOffsetsAppender) { MessageReceiver receiver = createKafkaSingleThreadedMessageReceiver(topic, subscription, loadReporter); @@ -104,7 +102,7 @@ public MessageReceiver createMessageReceiver(Topic topic, } if (consumerReceiverParameters.isFilteringEnabled()) { - receiver = createFilteringMessageReceiver(receiver, consumerRateLimiter, subscription, metrics); + receiver = createFilteringMessageReceiver(receiver, consumerRateLimiter, subscription, metrics, pendingOffsetsAppender); } return receiver; @@ -140,11 +138,12 @@ private MessageReceiver createThrottlingMessageReceiver(MessageReceiver receiver private MessageReceiver createFilteringMessageReceiver(MessageReceiver receiver, ConsumerRateLimiter consumerRateLimiter, Subscription subscription, - MetricsFacade metrics) { + MetricsFacade metrics, + PendingOffsetsAppender pendingOffsetsAppender) { boolean filteringRateLimitEnabled = consumerReceiverParameters.isFilteringRateLimiterEnabled(); FilteredMessageHandler filteredMessageHandler = new FilteredMessageHandler( - offsetQueue, filteringRateLimitEnabled ? consumerRateLimiter : null, + pendingOffsetsAppender, trackers, metrics, subscription.getQualifiedName() diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/result/DefaultErrorHandler.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/result/DefaultErrorHandler.java index ea7747d698..945cd8df01 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/result/DefaultErrorHandler.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/result/DefaultErrorHandler.java @@ -7,7 +7,6 @@ import pl.allegro.tech.hermes.common.message.undelivered.UndeliveredMessageLog; import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.consumers.consumer.Message; -import pl.allegro.tech.hermes.consumers.consumer.offset.OffsetQueue; import pl.allegro.tech.hermes.consumers.consumer.sender.MessageSendingResult; import pl.allegro.tech.hermes.metrics.HermesCounter; import pl.allegro.tech.hermes.metrics.HermesHistogram; @@ -19,13 +18,11 @@ import static pl.allegro.tech.hermes.api.SentMessageTrace.Builder.undeliveredMessage; import static pl.allegro.tech.hermes.consumers.consumer.message.MessageConverter.toMessageMetadata; -import static pl.allegro.tech.hermes.consumers.consumer.offset.SubscriptionPartitionOffset.subscriptionPartitionOffset; public class DefaultErrorHandler implements ErrorHandler { private static final Logger logger = LoggerFactory.getLogger(DefaultErrorHandler.class); - private final OffsetQueue offsetQueue; private final MetricsFacade metrics; private final UndeliveredMessageLog undeliveredMessageLog; private final Clock clock; @@ -41,14 +38,12 @@ public class DefaultErrorHandler implements ErrorHandler { private final Map httpStatusCodes = new ConcurrentHashMap<>(); - public DefaultErrorHandler(OffsetQueue offsetQueue, - MetricsFacade metrics, + public DefaultErrorHandler(MetricsFacade metrics, UndeliveredMessageLog undeliveredMessageLog, Clock clock, Trackers trackers, String cluster, SubscriptionName subscriptionName) { - this.offsetQueue = offsetQueue; this.metrics = metrics; this.undeliveredMessageLog = undeliveredMessageLog; this.clock = clock; @@ -67,9 +62,6 @@ public DefaultErrorHandler(OffsetQueue offsetQueue, public void handleDiscarded(Message message, Subscription subscription, MessageSendingResult result) { logResult(message, subscription, result); - offsetQueue.offerCommittedOffset(subscriptionPartitionOffset(subscription.getQualifiedName(), - message.getPartitionOffset(), message.getPartitionAssignmentTerm())); - discarded.increment(); inflightTime.record(System.currentTimeMillis() - message.getReadingTimestamp()); diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/result/DefaultSuccessHandler.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/result/DefaultSuccessHandler.java index acd52b773e..2d74f9cd2f 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/result/DefaultSuccessHandler.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/result/DefaultSuccessHandler.java @@ -4,7 +4,6 @@ import pl.allegro.tech.hermes.api.SubscriptionName; import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.consumers.consumer.Message; -import pl.allegro.tech.hermes.consumers.consumer.offset.OffsetQueue; import pl.allegro.tech.hermes.consumers.consumer.sender.MessageSendingResult; import pl.allegro.tech.hermes.metrics.HermesCounter; import pl.allegro.tech.hermes.metrics.HermesHistogram; @@ -14,24 +13,20 @@ import java.util.concurrent.ConcurrentHashMap; import static pl.allegro.tech.hermes.consumers.consumer.message.MessageConverter.toMessageMetadata; -import static pl.allegro.tech.hermes.consumers.consumer.offset.SubscriptionPartitionOffset.subscriptionPartitionOffset; public class DefaultSuccessHandler implements SuccessHandler { private final Trackers trackers; private final SubscriptionName subscriptionName; - private final OffsetQueue offsetQueue; private final MetricsFacade metrics; private final Map httpStatusCodes = new ConcurrentHashMap<>(); private final HermesCounter throughputInBytes; private final HermesCounter successes; private final HermesHistogram inflightTime; - public DefaultSuccessHandler(OffsetQueue offsetQueue, - MetricsFacade metrics, + public DefaultSuccessHandler(MetricsFacade metrics, Trackers trackers, SubscriptionName subscriptionName) { - this.offsetQueue = offsetQueue; this.metrics = metrics; this.trackers = trackers; this.subscriptionName = subscriptionName; @@ -42,8 +37,6 @@ public DefaultSuccessHandler(OffsetQueue offsetQueue, @Override public void handleSuccess(Message message, Subscription subscription, MessageSendingResult result) { - offsetQueue.offerCommittedOffset(subscriptionPartitionOffset(subscription.getQualifiedName(), - message.getPartitionOffset(), message.getPartitionAssignmentTerm())); markSuccess(message, result); trackers.get(subscription).logSent(toMessageMetadata(message, subscription), result.getHostname()); } diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/ConsumerFactory.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/ConsumerFactory.java index fc4c0334f1..f0108ccd3b 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/ConsumerFactory.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/ConsumerFactory.java @@ -14,7 +14,7 @@ import pl.allegro.tech.hermes.consumers.consumer.converter.MessageConverterResolver; import pl.allegro.tech.hermes.consumers.consumer.load.SubscriptionLoadRecorder; import pl.allegro.tech.hermes.consumers.consumer.load.SubscriptionLoadRecordersRegistry; -import pl.allegro.tech.hermes.consumers.consumer.offset.OffsetQueue; +import pl.allegro.tech.hermes.consumers.consumer.offset.ConsumerPartitionAssignmentState; import pl.allegro.tech.hermes.consumers.consumer.rate.ConsumerRateLimitSupervisor; import pl.allegro.tech.hermes.consumers.consumer.rate.SerialConsumerRateLimiter; import pl.allegro.tech.hermes.consumers.consumer.rate.calculator.OutputRateCalculatorFactory; @@ -24,6 +24,7 @@ import pl.allegro.tech.hermes.tracker.consumers.Trackers; import java.time.Clock; +import java.time.Duration; public class ConsumerFactory { @@ -33,7 +34,6 @@ public class ConsumerFactory { private final MetricsFacade metrics; private final CommonConsumerParameters commonConsumerParameters; private final Trackers trackers; - private final OffsetQueue offsetQueue; private final ConsumerMessageSenderFactory consumerMessageSenderFactory; private final TopicRepository topicRepository; private final MessageConverterResolver messageConverterResolver; @@ -43,6 +43,9 @@ public class ConsumerFactory { private final ConsumerAuthorizationHandler consumerAuthorizationHandler; private final Clock clock; private final SubscriptionLoadRecordersRegistry subscriptionLoadRecordersRegistry; + private final ConsumerPartitionAssignmentState consumerPartitionAssignmentState; + private final Duration commitPeriod; + private final int offsetQueueSize; public ConsumerFactory(ReceiverFactory messageReceiverFactory, MetricsFacade metrics, @@ -50,7 +53,6 @@ public ConsumerFactory(ReceiverFactory messageReceiverFactory, ConsumerRateLimitSupervisor consumerRateLimitSupervisor, OutputRateCalculatorFactory outputRateCalculatorFactory, Trackers trackers, - OffsetQueue offsetQueue, ConsumerMessageSenderFactory consumerMessageSenderFactory, TopicRepository topicRepository, MessageConverterResolver messageConverterResolver, @@ -59,14 +61,16 @@ public ConsumerFactory(ReceiverFactory messageReceiverFactory, MessageBatchSenderFactory batchSenderFactory, ConsumerAuthorizationHandler consumerAuthorizationHandler, Clock clock, - SubscriptionLoadRecordersRegistry subscriptionLoadRecordersRegistry) { + SubscriptionLoadRecordersRegistry subscriptionLoadRecordersRegistry, + ConsumerPartitionAssignmentState consumerPartitionAssignmentState, + Duration commitPeriod, + int offsetQueueSize) { this.messageReceiverFactory = messageReceiverFactory; this.metrics = metrics; this.commonConsumerParameters = commonConsumerParameters; this.consumerRateLimitSupervisor = consumerRateLimitSupervisor; this.outputRateCalculatorFactory = outputRateCalculatorFactory; this.trackers = trackers; - this.offsetQueue = offsetQueue; this.consumerMessageSenderFactory = consumerMessageSenderFactory; this.topicRepository = topicRepository; this.messageConverterResolver = messageConverterResolver; @@ -76,6 +80,9 @@ public ConsumerFactory(ReceiverFactory messageReceiverFactory, this.consumerAuthorizationHandler = consumerAuthorizationHandler; this.clock = clock; this.subscriptionLoadRecordersRegistry = subscriptionLoadRecordersRegistry; + this.consumerPartitionAssignmentState = consumerPartitionAssignmentState; + this.commitPeriod = commitPeriod; + this.offsetQueueSize = offsetQueueSize; } public Consumer createConsumer(Subscription subscription) { @@ -85,7 +92,6 @@ public Consumer createConsumer(Subscription subscription) { return new BatchConsumer(messageReceiverFactory, batchSenderFactory.create(subscription), batchFactory, - offsetQueue, messageConverterResolver, compositeMessageContentWrapper, metrics, @@ -93,7 +99,8 @@ public Consumer createConsumer(Subscription subscription) { subscription, topic, commonConsumerParameters.isUseTopicMessageSizeEnabled(), - loadRecorder + loadRecorder, + commitPeriod ); } else { SerialConsumerRateLimiter consumerRateLimiter = new SerialConsumerRateLimiter(subscription, @@ -109,9 +116,11 @@ public Consumer createConsumer(Subscription subscription) { messageConverterResolver, topic, commonConsumerParameters, - offsetQueue, consumerAuthorizationHandler, - loadRecorder + loadRecorder, + consumerPartitionAssignmentState, + commitPeriod, + offsetQueueSize ); } } diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/NonblockingConsumersSupervisor.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/NonblockingConsumersSupervisor.java index c5dbbf8fbe..0911f05fea 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/NonblockingConsumersSupervisor.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/NonblockingConsumersSupervisor.java @@ -9,8 +9,6 @@ import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.consumers.CommonConsumerParameters; import pl.allegro.tech.hermes.consumers.consumer.offset.ConsumerPartitionAssignmentState; -import pl.allegro.tech.hermes.consumers.consumer.offset.OffsetCommitter; -import pl.allegro.tech.hermes.consumers.consumer.offset.OffsetQueue; import pl.allegro.tech.hermes.consumers.health.ConsumerMonitor; import pl.allegro.tech.hermes.consumers.message.undelivered.UndeliveredMessageLogPersister; import pl.allegro.tech.hermes.consumers.supervisor.process.ConsumerProcessFactory; @@ -31,7 +29,6 @@ import static pl.allegro.tech.hermes.api.Subscription.State.PENDING; import static pl.allegro.tech.hermes.consumers.health.Checks.SUBSCRIPTIONS; import static pl.allegro.tech.hermes.consumers.health.Checks.SUBSCRIPTIONS_COUNT; -import static pl.allegro.tech.hermes.consumers.supervisor.process.Signal.SignalType.COMMIT; public class NonblockingConsumersSupervisor implements ConsumersSupervisor { private static final Logger logger = LoggerFactory.getLogger(NonblockingConsumersSupervisor.class); @@ -39,7 +36,6 @@ public class NonblockingConsumersSupervisor implements ConsumersSupervisor { private final ConsumerProcessSupervisor backgroundProcess; private final UndeliveredMessageLogPersister undeliveredMessageLogPersister; private final Duration backgroundSupervisorInterval; - private final OffsetCommitter offsetCommitter; private final SubscriptionRepository subscriptionRepository; private final ScheduledExecutorService scheduledExecutor; @@ -47,15 +43,13 @@ public class NonblockingConsumersSupervisor implements ConsumersSupervisor { public NonblockingConsumersSupervisor(CommonConsumerParameters commonConsumerParameters, ConsumersExecutorService executor, ConsumerFactory consumerFactory, - OffsetQueue offsetQueue, ConsumerPartitionAssignmentState consumerPartitionAssignmentState, Retransmitter retransmitter, UndeliveredMessageLogPersister undeliveredMessageLogPersister, SubscriptionRepository subscriptionRepository, MetricsFacade metrics, ConsumerMonitor monitor, - Clock clock, - Duration commitOffsetPeriod) { + Clock clock) { this.undeliveredMessageLogPersister = undeliveredMessageLogPersister; this.subscriptionRepository = subscriptionRepository; this.backgroundSupervisorInterval = commonConsumerParameters.getBackgroundSupervisor().getInterval(); @@ -68,15 +62,6 @@ public NonblockingConsumersSupervisor(CommonConsumerParameters commonConsumerPar commonConsumerParameters.getSignalProcessingQueueSize(), commonConsumerParameters.getBackgroundSupervisor().getKillAfter()); this.scheduledExecutor = createExecutorForSupervision(); - this.offsetCommitter = new OffsetCommitter( - offsetQueue, - consumerPartitionAssignmentState, - (offsets) -> offsets.subscriptionNames().forEach(subscription -> - backgroundProcess.accept(Signal.of(COMMIT, subscription, offsets.batchFor(subscription))) - ), - (int) commitOffsetPeriod.toSeconds(), - metrics - ); monitor.register(SUBSCRIPTIONS, backgroundProcess::runningSubscriptionsStatus); monitor.register(SUBSCRIPTIONS_COUNT, backgroundProcess::countRunningProcesses); } @@ -137,7 +122,6 @@ public void start() { backgroundSupervisorInterval.toMillis(), backgroundSupervisorInterval.toMillis(), TimeUnit.MILLISECONDS); - offsetCommitter.start(); undeliveredMessageLogPersister.start(); } @@ -145,7 +129,6 @@ public void start() { public void shutdown() { backgroundProcess.shutdown(); scheduledExecutor.shutdown(); - offsetCommitter.shutdown(); undeliveredMessageLogPersister.shutdown(); } } diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/process/ConsumerProcess.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/process/ConsumerProcess.java index 576ff01b9e..c9d5983646 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/process/ConsumerProcess.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/process/ConsumerProcess.java @@ -126,9 +126,6 @@ private void process(Signal signal) { case UPDATE_TOPIC: consumer.updateTopic(signal.getPayload()); break; - case COMMIT: - consumer.commit(signal.getPayload()); - break; default: logger.warn("Unhandled signal found {}", signal); break; diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/process/ConsumerProcessSupervisor.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/process/ConsumerProcessSupervisor.java index b62825530a..458ba8f0c0 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/process/ConsumerProcessSupervisor.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/process/ConsumerProcessSupervisor.java @@ -20,7 +20,6 @@ import java.util.Set; import java.util.concurrent.Future; -import static java.util.stream.Collectors.toList; import static pl.allegro.tech.hermes.consumers.supervisor.process.Signal.SignalType.START; import static pl.allegro.tech.hermes.consumers.supervisor.process.Signal.SignalType.STOP; @@ -110,7 +109,7 @@ public Integer countRunningProcesses() { private void restartUnhealthy() { runningConsumerProcesses.stream() .filter(process -> !process.getConsumerProcess().isHealthy()) - .collect(toList()) + .toList() .forEach(process -> { logger.info("Lost contact with consumer {} (last seen {}ms ago). Attempting to kill this process and spawn new one.", process.getConsumerProcess(), process.getConsumerProcess().lastSeen()); @@ -144,7 +143,6 @@ private void processSignal(Signal signal) { break; case UPDATE_TOPIC: case RETRANSMIT: - case COMMIT: forRunningConsumerProcess(signal, runningProcess -> runningProcess.getConsumerProcess().accept(signal)); break; case STOP: diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/process/Signal.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/process/Signal.java index 41d4c6b94b..57a17b7bc0 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/process/Signal.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/process/Signal.java @@ -17,10 +17,10 @@ public class Signal { private final long id; public enum SignalType { - START, STOP, RETRANSMIT, UPDATE_SUBSCRIPTION, UPDATE_TOPIC, COMMIT + START, STOP, RETRANSMIT, UPDATE_SUBSCRIPTION, UPDATE_TOPIC } - private static AtomicLong SIGNALS_COUNTER = new AtomicLong(); + private static final AtomicLong SIGNALS_COUNTER = new AtomicLong(); private Signal(SignalType type, SubscriptionName target, Object payload, long executeAfterTimestamp, long id) { this.type = type; @@ -77,10 +77,9 @@ public boolean equals(Object o) { if (this == o) { return true; } - if (!(o instanceof Signal)) { + if (!(o instanceof Signal signal)) { return false; } - Signal signal = (Signal) o; return type == signal.type && Objects.equals(target, signal.target); } diff --git a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/offset/OffsetCommitterTest.groovy b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/offset/OffsetCommitterTest.groovy index a668980afb..6c763c4b71 100644 --- a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/offset/OffsetCommitterTest.groovy +++ b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/offset/OffsetCommitterTest.groovy @@ -14,9 +14,9 @@ class OffsetCommitterTest extends Specification { @Shared KafkaTopicName KAFKA_TOPIC_NAME = KafkaTopicName.valueOf("group_topic") - private OffsetQueue queue = new OffsetQueue(TestMetricsFacadeFactory.create(), 200_000) + private PendingOffsets offsetsSlots = new PendingOffsets(SUBSCRIPTION_NAME, TestMetricsFacadeFactory.create(), 50, 2000) - private MockMessageCommitter messageCommitter = new MockMessageCommitter() + private OffsetCommitterTestHelper offsetCommitterTestHelper = new OffsetCommitterTestHelper() private ConsumerPartitionAssignmentState state @@ -24,318 +24,318 @@ class OffsetCommitterTest extends Specification { def setup() { state = new ConsumerPartitionAssignmentState() - def commitInterval = 10 - committer = new OffsetCommitter(queue, state, messageCommitter, commitInterval, TestMetricsFacadeFactory.create()) + committer = new OffsetCommitter(state, TestMetricsFacadeFactory.create()) } def "should not commit offsets with negative values"() { given: assignPartitions(1) - queue.offerInflightOffset(offset(1, -123)) - queue.offerCommittedOffset(offset(1, -123)) + offsetsSlots.markAsInflight(offset(1, -123)) + offsetsSlots.markAsProcessed(offset(1, -123)) when: - committer.run() + offsetCommitterTestHelper.markCommittedOffsets(committer.calculateOffsetsToBeCommitted(offsetsSlots.getOffsetsSnapshotAndReleaseProcessedSlots())) then: - messageCommitter.nothingCommitted(1) + offsetCommitterTestHelper.nothingCommitted(1) } def "should not commit offset with long max value"() { given: assignPartitions(1) def offsetTooLarge = Long.MAX_VALUE - 1 // we actually commit the offset we want to read next, so it'll be +1 - queue.offerInflightOffset(offset(1, offsetTooLarge)) - queue.offerCommittedOffset(offset(1, offsetTooLarge)) + offsetsSlots.markAsInflight(offset(1, offsetTooLarge)) + offsetsSlots.markAsProcessed(offset(1, offsetTooLarge)) when: - committer.run() + offsetCommitterTestHelper.markCommittedOffsets(committer.calculateOffsetsToBeCommitted(offsetsSlots.getOffsetsSnapshotAndReleaseProcessedSlots())) then: - messageCommitter.nothingCommitted(1) + offsetCommitterTestHelper.nothingCommitted(1) } def "should commit smallest offset of uncommitted message"() { given: assignPartitions(1) - queue.offerInflightOffset(offset(1, 1)) - queue.offerInflightOffset(offset(1, 2)) - queue.offerInflightOffset(offset(1, 3)) - queue.offerInflightOffset(offset(1, 4)) + offsetsSlots.markAsInflight(offset(1, 1)) + offsetsSlots.markAsInflight(offset(1, 2)) + offsetsSlots.markAsInflight(offset(1, 3)) + offsetsSlots.markAsInflight(offset(1, 4)) - queue.offerCommittedOffset(offset(1, 1)) - queue.offerCommittedOffset(offset(1, 4)) + offsetsSlots.markAsProcessed(offset(1, 1)) + offsetsSlots.markAsProcessed(offset(1, 4)) when: - committer.run() + offsetCommitterTestHelper.markCommittedOffsets(committer.calculateOffsetsToBeCommitted(offsetsSlots.getOffsetsSnapshotAndReleaseProcessedSlots())) then: - messageCommitter.wereCommitted(1, offset(1, 2)) + offsetCommitterTestHelper.wereCommitted(1, offset(1, 2)) } def "should increment offset by 1 only if it comes from committed offsets to match Kafka offset definition"() { given: assignPartitions(1, 2) - queue.offerInflightOffset(offset(1, 1)) - queue.offerCommittedOffset(offset(1, 1)) + offsetsSlots.markAsInflight(offset(1, 1)) + offsetsSlots.markAsProcessed(offset(1, 1)) - queue.offerInflightOffset(offset(2, 1)) + offsetsSlots.markAsInflight(offset(2, 1)) when: - committer.run() + offsetCommitterTestHelper.markCommittedOffsets(committer.calculateOffsetsToBeCommitted(offsetsSlots.getOffsetsSnapshotAndReleaseProcessedSlots())) then: - messageCommitter.wereCommitted(1, offset(1, 2), offset(2, 1)) + offsetCommitterTestHelper.wereCommitted(1, offset(1, 2), offset(2, 1)) } def "should commit max offset of committed offsets when no smaller inflights exist"() { given: assignPartitions(1) - queue.offerInflightOffset(offset(1, 3)) - queue.offerInflightOffset(offset(1, 4)) + offsetsSlots.markAsInflight(offset(1, 3)) + offsetsSlots.markAsInflight(offset(1, 4)) - queue.offerCommittedOffset(offset(1, 3)) - queue.offerCommittedOffset(offset(1, 4)) + offsetsSlots.markAsProcessed(offset(1, 3)) + offsetsSlots.markAsProcessed(offset(1, 4)) when: - committer.run() + offsetCommitterTestHelper.markCommittedOffsets(committer.calculateOffsetsToBeCommitted(offsetsSlots.getOffsetsSnapshotAndReleaseProcessedSlots())) then: - messageCommitter.wereCommitted(1, offset(1, 5)) + offsetCommitterTestHelper.wereCommitted(1, offset(1, 5)) } def "should commit same offset twice when there are no new offsets to commit"() { given: assignPartitions(1) - queue.offerInflightOffset(offset(1, 5)) + offsetsSlots.markAsInflight(offset(1, 5)) when: - committer.run() + offsetCommitterTestHelper.markCommittedOffsets(committer.calculateOffsetsToBeCommitted(offsetsSlots.getOffsetsSnapshotAndReleaseProcessedSlots())) then: - messageCommitter.wereCommitted(1, offset(1, 5)) + offsetCommitterTestHelper.wereCommitted(1, offset(1, 5)) when: - committer.run() + offsetCommitterTestHelper.markCommittedOffsets(committer.calculateOffsetsToBeCommitted(offsetsSlots.getOffsetsSnapshotAndReleaseProcessedSlots())) then: - messageCommitter.wereCommitted(2, offset(1, 5)) + offsetCommitterTestHelper.wereCommitted(2, offset(1, 5)) } def "should not mix offsets from different partitions and topics"() { given: assignPartitions(1, 2) - queue.offerInflightOffset(offset(1, 3)) - queue.offerInflightOffset(offset(1, 4)) + offsetsSlots.markAsInflight(offset(1, 3)) + offsetsSlots.markAsInflight(offset(1, 4)) - queue.offerInflightOffset(offset(2, 10)) - queue.offerInflightOffset(offset(2, 11)) + offsetsSlots.markAsInflight(offset(2, 10)) + offsetsSlots.markAsInflight(offset(2, 11)) - queue.offerCommittedOffset(offset(1, 3)) - queue.offerCommittedOffset(offset(1, 4)) - queue.offerCommittedOffset(offset(2, 11)) + offsetsSlots.markAsProcessed(offset(1, 3)) + offsetsSlots.markAsProcessed(offset(1, 4)) + offsetsSlots.markAsProcessed(offset(2, 11)) when: - committer.run() + offsetCommitterTestHelper.markCommittedOffsets(committer.calculateOffsetsToBeCommitted(offsetsSlots.getOffsetsSnapshotAndReleaseProcessedSlots())) then: - messageCommitter.wereCommitted(1, offset(1, 5), offset(2, 10)) + offsetCommitterTestHelper.wereCommitted(1, offset(1, 5), offset(2, 10)) } def "should get rid of leftover inflight offsets when revoked from topic partitions"() { given: assignPartitions(1) - queue.offerInflightOffset(offset(1, 3)) + offsetsSlots.markAsInflight(offset(1, 3)) when: revokeAllPartitions() - committer.run() + offsetCommitterTestHelper.markCommittedOffsets(committer.calculateOffsetsToBeCommitted(offsetsSlots.getOffsetsSnapshotAndReleaseProcessedSlots())) then: - messageCommitter.nothingCommitted(1) + offsetCommitterTestHelper.nothingCommitted(1) } def "should get rid of inflight offsets from revoked partitions"() { given: assignPartitions(1, 2) - queue.offerInflightOffset(offset(1, 3)) - queue.offerInflightOffset(offset(2, 3)) + offsetsSlots.markAsInflight(offset(1, 3)) + offsetsSlots.markAsInflight(offset(2, 3)) when: revokePartitions(1) - committer.run() + offsetCommitterTestHelper.markCommittedOffsets(committer.calculateOffsetsToBeCommitted(offsetsSlots.getOffsetsSnapshotAndReleaseProcessedSlots())) then: - messageCommitter.wereCommitted(1, offset(2, 3)) + offsetCommitterTestHelper.wereCommitted(1, offset(2, 3)) } def "should get rid of committed offsets from revoked partitions"() { given: assignPartitions(1) - queue.offerCommittedOffset(offset(1, 3)) + offsetsSlots.markAsInflight(offset(1, 3)) + offsetsSlots.markAsProcessed(offset(1, 3)) when: revokePartitions(1) - committer.run() + offsetCommitterTestHelper.markCommittedOffsets(committer.calculateOffsetsToBeCommitted(offsetsSlots.getOffsetsSnapshotAndReleaseProcessedSlots())) then: - messageCommitter.nothingCommitted(1) + offsetCommitterTestHelper.nothingCommitted(1) } def "should get rid of leftover committed offsets when revoked from topic partitions"() { given: assignPartitions(1) - queue.offerInflightOffset(offset(1, 3)) - queue.offerCommittedOffset(offset(1, 3)) + offsetsSlots.markAsInflight(offset(1, 3)) + offsetsSlots.markAsProcessed(offset(1, 3)) when: - committer.run() + offsetCommitterTestHelper.markCommittedOffsets(committer.calculateOffsetsToBeCommitted(offsetsSlots.getOffsetsSnapshotAndReleaseProcessedSlots())) then: - messageCommitter.wereCommitted(1, offset(1, 4)) + offsetCommitterTestHelper.wereCommitted(1, offset(1, 4)) when: - queue.offerInflightOffset(offset(1, 4)) - queue.offerCommittedOffset(offset(1, 4)) + offsetsSlots.markAsInflight(offset(1, 4)) + offsetsSlots.markAsProcessed(offset(1, 4)) and: revokeAllPartitions() and: - committer.run() + offsetCommitterTestHelper.markCommittedOffsets(committer.calculateOffsetsToBeCommitted(offsetsSlots.getOffsetsSnapshotAndReleaseProcessedSlots())) then: - messageCommitter.nothingCommitted(2) + offsetCommitterTestHelper.nothingCommitted(2) } def "should not commit offsets in next iteration after reassigning partition"() { given: assignPartitions(1) - queue.offerInflightOffset(offset(1, 3)) + offsetsSlots.markAsInflight(offset(1, 3)) when: - committer.run() + offsetCommitterTestHelper.markCommittedOffsets(committer.calculateOffsetsToBeCommitted(offsetsSlots.getOffsetsSnapshotAndReleaseProcessedSlots())) then: - messageCommitter.wereCommitted(1, offset(1, 3)) + offsetCommitterTestHelper.wereCommitted(1, offset(1, 3)) when: revokePartitions(1) - committer.run() + offsetCommitterTestHelper.markCommittedOffsets(committer.calculateOffsetsToBeCommitted(offsetsSlots.getOffsetsSnapshotAndReleaseProcessedSlots())) then: - messageCommitter.nothingCommitted(2) + offsetCommitterTestHelper.nothingCommitted(2) when: assignPartitions(1) - committer.run() + offsetCommitterTestHelper.markCommittedOffsets(committer.calculateOffsetsToBeCommitted(offsetsSlots.getOffsetsSnapshotAndReleaseProcessedSlots())) then: - messageCommitter.nothingCommitted(3) + offsetCommitterTestHelper.nothingCommitted(3) } def "should commit only offsets from current term"() { given: assignPartitions(1) - queue.offerInflightOffset(offset(1, 3)) + offsetsSlots.markAsInflight(offset(1, 3)) when: - committer.run() + offsetCommitterTestHelper.markCommittedOffsets(committer.calculateOffsetsToBeCommitted(offsetsSlots.getOffsetsSnapshotAndReleaseProcessedSlots())) then: - messageCommitter.wereCommitted(1, offset(1, 3)) + offsetCommitterTestHelper.wereCommitted(1, offset(1, 3)) when: - queue.offerCommittedOffset(offset(1, 3)) - queue.offerInflightOffset(offset(1, 4)) - queue.offerCommittedOffset(offset(1, 4)) + offsetsSlots.markAsProcessed(offset(1, 3)) + offsetsSlots.markAsInflight(offset(1, 4)) + offsetsSlots.markAsProcessed(offset(1, 4)) revokePartitions(1) assignPartitions(1) and: - committer.run() + offsetCommitterTestHelper.markCommittedOffsets(committer.calculateOffsetsToBeCommitted(offsetsSlots.getOffsetsSnapshotAndReleaseProcessedSlots())) then: - messageCommitter.nothingCommitted(2) + offsetCommitterTestHelper.nothingCommitted(2) when: - queue.offerCommittedOffset(offsetFromTerm(1, 4, 0)) // message from previous term=0 + offsetsSlots.markAsProcessed(offsetFromTerm(1, 4, 0)) // message from previous term=0 and: - committer.run() + offsetCommitterTestHelper.markCommittedOffsets(committer.calculateOffsetsToBeCommitted(offsetsSlots.getOffsetsSnapshotAndReleaseProcessedSlots())) then: - messageCommitter.nothingCommitted(3) + offsetCommitterTestHelper.nothingCommitted(3) } def "should commit maximum commited offset no matter what order committed offset return"() { given: assignPartitions(1) - queue.offerInflightOffset(offset(1, 3)) - queue.offerInflightOffset(offset(1, 4)) - queue.offerInflightOffset(offset(1, 5)) + offsetsSlots.markAsInflight(offset(1, 3)) + offsetsSlots.markAsInflight(offset(1, 4)) + offsetsSlots.markAsInflight(offset(1, 5)) when: - committer.run() + offsetCommitterTestHelper.markCommittedOffsets(committer.calculateOffsetsToBeCommitted(offsetsSlots.getOffsetsSnapshotAndReleaseProcessedSlots())) then: - messageCommitter.wereCommitted(1, offset(1, 3)) + offsetCommitterTestHelper.wereCommitted(1, offset(1, 3)) when: - queue.offerCommittedOffset(offset(1, 4)) - queue.offerCommittedOffset(offset(1, 5)) + offsetsSlots.markAsProcessed(offset(1, 4)) + offsetsSlots.markAsProcessed(offset(1, 5)) and: - committer.run() + offsetCommitterTestHelper.markCommittedOffsets(committer.calculateOffsetsToBeCommitted(offsetsSlots.getOffsetsSnapshotAndReleaseProcessedSlots())) then: - messageCommitter.wereCommitted(2, offset(1, 3)) + offsetCommitterTestHelper.wereCommitted(2, offset(1, 3)) when: - queue.offerCommittedOffset(offset(1, 3)) + offsetsSlots.markAsProcessed(offset(1, 3)) and: - committer.run() + offsetCommitterTestHelper.markCommittedOffsets(committer.calculateOffsetsToBeCommitted(offsetsSlots.getOffsetsSnapshotAndReleaseProcessedSlots())) then: - messageCommitter.wereCommitted(3, offset(1, 6)) + offsetCommitterTestHelper.wereCommitted(3, offset(1, 6)) } def "should drop maximum committed offset when lost partition assignment"() { given: assignPartitions(1) - queue.offerInflightOffset(offset(1, 3)) - queue.offerInflightOffset(offset(1, 4)) - queue.offerInflightOffset(offset(1, 5)) + offsetsSlots.markAsInflight(offset(1, 3)) + offsetsSlots.markAsInflight(offset(1, 4)) + offsetsSlots.markAsInflight(offset(1, 5)) when: - committer.run() + offsetCommitterTestHelper.markCommittedOffsets(committer.calculateOffsetsToBeCommitted(offsetsSlots.getOffsetsSnapshotAndReleaseProcessedSlots())) then: - messageCommitter.wereCommitted(1, offset(1, 3)) + offsetCommitterTestHelper.wereCommitted(1, offset(1, 3)) when: - queue.offerCommittedOffset(offset(1, 4)) - queue.offerCommittedOffset(offset(1, 5)) + offsetsSlots.markAsProcessed(offset(1, 4)) + offsetsSlots.markAsProcessed(offset(1, 5)) and: - committer.run() + offsetCommitterTestHelper.markCommittedOffsets(committer.calculateOffsetsToBeCommitted(offsetsSlots.getOffsetsSnapshotAndReleaseProcessedSlots())) then: - messageCommitter.wereCommitted(2, offset(1, 3)) + offsetCommitterTestHelper.wereCommitted(2, offset(1, 3)) when: - queue.offerCommittedOffset(offset(1, 3)) + offsetsSlots.markAsProcessed(offset(1, 3)) revokePartitions(1) assignPartitions(1) and: - committer.run() + offsetCommitterTestHelper.markCommittedOffsets(committer.calculateOffsetsToBeCommitted(offsetsSlots.getOffsetsSnapshotAndReleaseProcessedSlots())) then: - messageCommitter.nothingCommitted(3) + offsetCommitterTestHelper.nothingCommitted(3) } private SubscriptionPartitionOffset offset(int partition, long offset) { diff --git a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/offset/MockMessageCommitter.groovy b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/offset/OffsetCommitterTestHelper.groovy similarity index 51% rename from hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/offset/MockMessageCommitter.groovy rename to hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/offset/OffsetCommitterTestHelper.groovy index 3b68cc5731..0a93df3fd6 100644 --- a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/offset/MockMessageCommitter.groovy +++ b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/offset/OffsetCommitterTestHelper.groovy @@ -1,10 +1,8 @@ package pl.allegro.tech.hermes.consumers.consumer.offset -import pl.allegro.tech.hermes.consumers.consumer.receiver.MessageCommitter +class OffsetCommitterTestHelper { -class MockMessageCommitter implements MessageCommitter { - - private final List recordedValues = [] + private final List> recordedValues = [] private int iteration = -1 @@ -13,11 +11,8 @@ class MockMessageCommitter implements MessageCommitter { } boolean wereCommitted(int iteration, SubscriptionPartitionOffset... expectedOffsets) { - OffsetsToCommit offsetsToCommit = recordedValues[iteration - 1] Set allOffsets = [] as Set - offsetsToCommit.subscriptionNames().each { subscription -> - allOffsets.addAll(offsetsToCommit.batchFor(subscription)) - } + recordedValues[iteration - 1].each {allOffsets.add(it)} Set expectedOffsetsSet = [] as Set expectedOffsets.each { expectedOffsetsSet.add(it) } @@ -25,9 +20,8 @@ class MockMessageCommitter implements MessageCommitter { allOffsets == expectedOffsetsSet } - @Override - void commitOffsets(OffsetsToCommit offsetsToCommit) { - recordedValues.add(offsetsToCommit) + void markCommittedOffsets(Set subscriptionPartitionOffsets) { + recordedValues.add(subscriptionPartitionOffsets) iteration++ } } diff --git a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/offset/PendingOffsetsTest.groovy b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/offset/PendingOffsetsTest.groovy new file mode 100644 index 0000000000..9effb38965 --- /dev/null +++ b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/offset/PendingOffsetsTest.groovy @@ -0,0 +1,113 @@ +package pl.allegro.tech.hermes.consumers.consumer.offset + +import pl.allegro.tech.hermes.api.SubscriptionName +import pl.allegro.tech.hermes.common.kafka.KafkaTopicName +import pl.allegro.tech.hermes.common.metric.MetricsFacade +import pl.allegro.tech.hermes.test.helper.metrics.TestMetricsFacadeFactory +import spock.lang.Shared +import spock.lang.Specification + +import java.time.Duration + +class PendingOffsetsTest extends Specification { + + @Shared + SubscriptionName SUBSCRIPTION_NAME = SubscriptionName.fromString('group.topic$sub') + + @Shared + KafkaTopicName KAFKA_TOPIC_NAME = KafkaTopicName.valueOf("group_topic") + + @Shared + Duration ACQUIRE_DURATION = Duration.ofMillis(100) + + private MetricsFacade testMetricsFacade = TestMetricsFacadeFactory.create() + + private ConsumerPartitionAssignmentState state = new ConsumerPartitionAssignmentState() + + def "should not allow for more inflight offsets than allowed"() { + given: + PendingOffsets offsetsSlots = createOffsetsSlots(2, 10) + 2.times {offsetsSlots.tryAcquireSlot(ACQUIRE_DURATION)} + + when: + boolean isAcquired = offsetsSlots.tryAcquireSlot(ACQUIRE_DURATION) + + then: + !isAcquired + } + + def "should not allow for more total offsets than allowed"() { + given: + PendingOffsets offsetsSlots = createOffsetsSlots(2, 3) + 2.times {offsetsSlots.tryAcquireSlot(ACQUIRE_DURATION)} + offsetsSlots.markAsInflight(offset(1, 1)) + offsetsSlots.markAsInflight(offset(1, 2)) + offsetsSlots.markAsProcessed(offset(1, 1)) + offsetsSlots.markAsProcessed(offset(1, 2)) + offsetsSlots.tryAcquireSlot(ACQUIRE_DURATION) + + when: + boolean isAcquired = offsetsSlots.tryAcquireSlot(ACQUIRE_DURATION) + + then: + !isAcquired + } + + def "should free inflight offsets"() { + given: + PendingOffsets offsetsSlots = createOffsetsSlots(1, 10) + offsetsSlots.tryAcquireSlot(ACQUIRE_DURATION) + offsetsSlots.markAsInflight(offset(1, 1)) + + when: + boolean isAcquired = offsetsSlots.tryAcquireSlot(ACQUIRE_DURATION) + + then: + !isAcquired + + when: + offsetsSlots.markAsProcessed(offset(1, 1)) + boolean isAcquiredAfterRelease = offsetsSlots.tryAcquireSlot(ACQUIRE_DURATION) + + then: + isAcquiredAfterRelease + } + + def "should free offsetQueue and return offset snapshot"() { + given: + Map expectedOffsetSnapshot = Map.of(offset(1, 1), MessageState.PROCESSED, offset(1, 2), MessageState.PROCESSED) + PendingOffsets offsetsSlots = createOffsetsSlots(2, 2) + 2.times{offsetsSlots.tryAcquireSlot(ACQUIRE_DURATION)} + offsetsSlots.markAsInflight(offset(1, 1)) + offsetsSlots.markAsInflight(offset(1, 2)) + offsetsSlots.markAsProcessed(offset(1, 1)) + offsetsSlots.markAsProcessed(offset(1, 2)) + + when: + boolean isAcquired = offsetsSlots.tryAcquireSlot(ACQUIRE_DURATION) + + then: + !isAcquired + + when: + Map offsetSnapshot = offsetsSlots.getOffsetsSnapshotAndReleaseProcessedSlots() + boolean isAcquiredAfterRelease = offsetsSlots.tryAcquireSlot(ACQUIRE_DURATION) + + then: + offsetSnapshot == expectedOffsetSnapshot + isAcquiredAfterRelease + } + + private PendingOffsets createOffsetsSlots(int inflightSize, int offsetQueueSize) { + return new PendingOffsets(SUBSCRIPTION_NAME, testMetricsFacade, inflightSize, offsetQueueSize) + } + + private SubscriptionPartitionOffset offset(int partition, long offset) { + offsetFromTerm(partition, offset, state.currentTerm(SUBSCRIPTION_NAME)) + } + + private SubscriptionPartitionOffset offsetFromTerm(int partition, long offset, long term) { + def partitionOffset = new SubscriptionPartition(KAFKA_TOPIC_NAME, SUBSCRIPTION_NAME, partition, term) + return new SubscriptionPartitionOffset(partitionOffset, offset) + } +} diff --git a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/result/DefaultErrorHandlerTest.groovy b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/result/DefaultErrorHandlerTest.groovy index ab5bc5225f..78cc25efd7 100644 --- a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/result/DefaultErrorHandlerTest.groovy +++ b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/result/DefaultErrorHandlerTest.groovy @@ -5,10 +5,8 @@ import pl.allegro.tech.hermes.api.TrackingMode import pl.allegro.tech.hermes.common.message.undelivered.UndeliveredMessageLog import pl.allegro.tech.hermes.common.metric.MetricsFacade import pl.allegro.tech.hermes.consumers.consumer.Message -import pl.allegro.tech.hermes.consumers.consumer.offset.OffsetQueue import pl.allegro.tech.hermes.consumers.consumer.sender.MessageSendingResult import pl.allegro.tech.hermes.consumers.test.MessageBuilder -import pl.allegro.tech.hermes.test.helper.metrics.TestMetricsFacadeFactory import pl.allegro.tech.hermes.tracker.consumers.Trackers import spock.lang.Specification @@ -18,8 +16,6 @@ import static pl.allegro.tech.hermes.test.helper.builder.SubscriptionBuilder.sub class DefaultErrorHandlerTest extends Specification { - private OffsetQueue offsetQueue = new OffsetQueue(TestMetricsFacadeFactory.create(), 200_000) - private UndeliveredMessageLog undeliveredLog = Mock(UndeliveredMessageLog) private InMemoryLogRepository sendingTracker = new InMemoryLogRepository() @@ -30,9 +26,9 @@ class DefaultErrorHandlerTest extends Specification { .withTrackingMode(TrackingMode.TRACK_ALL).build() private DefaultErrorHandler handler = new DefaultErrorHandler( - offsetQueue, Stub(MetricsFacade), undeliveredLog, Clock.systemUTC(), trackers, "cluster", subscription.qualifiedName) + Stub(MetricsFacade), undeliveredLog, Clock.systemUTC(), trackers, "cluster", subscription.qualifiedName) - def "should save tracking information on message failure but not commit message"() { + def "should save tracking information on message failure"() { given: Message message = MessageBuilder.withTestMessage().withPartitionOffset('kafka_topic', 0, 123L).build() MessageSendingResult result = MessageSendingResult.failedResult(500) @@ -42,10 +38,9 @@ class DefaultErrorHandlerTest extends Specification { then: sendingTracker.hasFailedLog('kafka_topic', 0, 123L) - offsetQueue.drainCommittedOffsets({ o -> assert !(o.partition == 0 && o.offset == 123L) }) } - def "should commit message and save tracking information on message discard"() { + def "should save tracking information on message discard"() { given: Message message = MessageBuilder.withTestMessage().withPartitionOffset('kafka_topic', 0, 123L).build() MessageSendingResult result = MessageSendingResult.failedResult(500) @@ -55,6 +50,5 @@ class DefaultErrorHandlerTest extends Specification { then: sendingTracker.hasDiscardedLog('kafka_topic', 0, 123L) - offsetQueue.drainCommittedOffsets({ o -> assert o.partition == 0 && o.offset == 123L }) } } diff --git a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/result/DefaultSuccessHandlerTest.groovy b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/result/DefaultSuccessHandlerTest.groovy index 68241e20f0..7a6166cbb9 100644 --- a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/result/DefaultSuccessHandlerTest.groovy +++ b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/result/DefaultSuccessHandlerTest.groovy @@ -4,10 +4,8 @@ import pl.allegro.tech.hermes.api.Subscription import pl.allegro.tech.hermes.api.TrackingMode import pl.allegro.tech.hermes.common.metric.MetricsFacade import pl.allegro.tech.hermes.consumers.consumer.Message -import pl.allegro.tech.hermes.consumers.consumer.offset.OffsetQueue import pl.allegro.tech.hermes.consumers.consumer.sender.MessageSendingResult import pl.allegro.tech.hermes.consumers.test.MessageBuilder -import pl.allegro.tech.hermes.test.helper.metrics.TestMetricsFacadeFactory import pl.allegro.tech.hermes.tracker.consumers.Trackers import spock.lang.Specification @@ -15,7 +13,6 @@ import static pl.allegro.tech.hermes.test.helper.builder.SubscriptionBuilder.sub class DefaultSuccessHandlerTest extends Specification { - private OffsetQueue offsetQueue = new OffsetQueue(TestMetricsFacadeFactory.create(), 200_000) private InMemoryLogRepository sendingTracker = new InMemoryLogRepository() @@ -24,9 +21,9 @@ class DefaultSuccessHandlerTest extends Specification { private Subscription subscription = subscription('group.topic', 'subscription') .withTrackingMode(TrackingMode.TRACK_ALL).build() - private DefaultSuccessHandler handler = new DefaultSuccessHandler(offsetQueue, Stub(MetricsFacade), trackers, subscription.qualifiedName) + private DefaultSuccessHandler handler = new DefaultSuccessHandler(Stub(MetricsFacade), trackers, subscription.qualifiedName) - def "should commit message and save tracking information on message success"() { + def "should save tracking information on message success"() { given: Message message = MessageBuilder.withTestMessage().withPartitionOffset('kafka_topic', 0, 123L).build() MessageSendingResult result = MessageSendingResult.failedResult(500) @@ -36,6 +33,5 @@ class DefaultSuccessHandlerTest extends Specification { then: sendingTracker.hasSuccessfulLog('kafka_topic', 0, 123L) - offsetQueue.drainCommittedOffsets({ o -> assert o.partition == 0 && o.offset == 123L }) } } diff --git a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/supervisor/process/ConsumerProcessSupervisorTest.groovy b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/supervisor/process/ConsumerProcessSupervisorTest.groovy index a40fff24f1..70f41b34e5 100644 --- a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/supervisor/process/ConsumerProcessSupervisorTest.groovy +++ b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/supervisor/process/ConsumerProcessSupervisorTest.groovy @@ -23,7 +23,11 @@ import java.time.ZoneId import java.util.function.Consumer import static java.util.concurrent.TimeUnit.MILLISECONDS -import static pl.allegro.tech.hermes.consumers.supervisor.process.Signal.SignalType.* +import static pl.allegro.tech.hermes.consumers.supervisor.process.Signal.SignalType.RETRANSMIT +import static pl.allegro.tech.hermes.consumers.supervisor.process.Signal.SignalType.START +import static pl.allegro.tech.hermes.consumers.supervisor.process.Signal.SignalType.STOP +import static pl.allegro.tech.hermes.consumers.supervisor.process.Signal.SignalType.UPDATE_SUBSCRIPTION +import static pl.allegro.tech.hermes.consumers.supervisor.process.Signal.SignalType.UPDATE_TOPIC import static pl.allegro.tech.hermes.test.helper.builder.SubscriptionBuilder.subscription import static pl.allegro.tech.hermes.test.helper.endpoint.TimeoutAdjuster.adjust @@ -39,7 +43,6 @@ class ConsumerProcessSupervisorTest extends Specification { retransmit : Signal.of(RETRANSMIT, subscriptionName), updateSubscription: Signal.of(UPDATE_SUBSCRIPTION, subscriptionName, subscription1), updateTopic : Signal.of(UPDATE_TOPIC, subscriptionName, topic1), - commit : Signal.of(COMMIT, subscriptionName) ] ConsumerProcessSupervisor supervisor diff --git a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/supervisor/process/SignalsFilterTest.groovy b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/supervisor/process/SignalsFilterTest.groovy index 71df100400..256cd3a7c7 100644 --- a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/supervisor/process/SignalsFilterTest.groovy +++ b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/supervisor/process/SignalsFilterTest.groovy @@ -15,7 +15,6 @@ import java.time.ZoneId import static pl.allegro.tech.hermes.api.SubscriptionName.fromString import static pl.allegro.tech.hermes.consumers.consumer.offset.SubscriptionPartitionOffset.subscriptionPartitionOffset -import static pl.allegro.tech.hermes.consumers.supervisor.process.Signal.SignalType.COMMIT import static pl.allegro.tech.hermes.consumers.supervisor.process.Signal.SignalType.RETRANSMIT import static pl.allegro.tech.hermes.consumers.supervisor.process.Signal.SignalType.START import static pl.allegro.tech.hermes.consumers.supervisor.process.Signal.SignalType.STOP @@ -91,7 +90,6 @@ class SignalsFilterTest extends Specification { signalType | firstSignalPayload | secondSignalPayload UPDATE_TOPIC | 'first-update' | 'second-update' UPDATE_SUBSCRIPTION | 'first-update' | 'second-update' - COMMIT | [offset(1, 10), offset(2, 11)] | [offset(1, 11)] RETRANSMIT | null | null } diff --git a/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSenderTest.java b/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSenderTest.java index 31cd6539b1..67791d41d5 100644 --- a/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSenderTest.java +++ b/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSenderTest.java @@ -8,9 +8,10 @@ import pl.allegro.tech.hermes.api.Subscription; import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.common.metric.SubscriptionMetrics; +import pl.allegro.tech.hermes.consumers.consumer.offset.PendingOffsets; +import pl.allegro.tech.hermes.consumers.consumer.offset.SubscriptionPartitionOffset; import pl.allegro.tech.hermes.consumers.consumer.profiling.ConsumerProfiler; import pl.allegro.tech.hermes.consumers.consumer.profiling.NoOpConsumerProfiler; -import pl.allegro.tech.hermes.consumers.consumer.rate.AdjustableSemaphore; import pl.allegro.tech.hermes.consumers.consumer.rate.SerialConsumerRateLimiter; import pl.allegro.tech.hermes.consumers.consumer.result.ErrorHandler; import pl.allegro.tech.hermes.consumers.consumer.result.SuccessHandler; @@ -68,6 +69,9 @@ public class ConsumerMessageSenderTest { @Mock private SerialConsumerRateLimiter rateLimiter; + @Mock + private PendingOffsets pendingOffsets; + @Mock private HermesTimer consumerLatencyTimer; @@ -89,8 +93,6 @@ public class ConsumerMessageSenderTest { @Mock private HermesCounter errors; - private AdjustableSemaphore inflightSemaphore; - private final ConsumerProfiler profiler = new NoOpConsumerProfiler(); private ConsumerMessageSender sender; @@ -106,7 +108,6 @@ public void setUp() { when(metricsFacade.subscriptions()).thenReturn(subscriptionMetrics); setUpMetrics(subscription); setUpMetrics(subscriptionWith4xxRetry); - inflightSemaphore = new AdjustableSemaphore(0); sender = consumerMessageSender(subscription); } @@ -131,7 +132,7 @@ public void shouldHandleSuccessfulSending() { verify(successHandler, timeout(1000)).handleSuccess(eq(message), eq(subscription), any(MessageSendingResult.class)); // then - verifySemaphoreReleased(); + verify(pendingOffsets).markAsProcessed(any(SubscriptionPartitionOffset.class)); verifyLatencyTimersCountedTimes(subscription, 1, 1); verifyRateLimiterAcquireTimersCountedTimes(subscription, 1, 1); verifyNoInteractions(errorHandler); @@ -151,7 +152,7 @@ public void shouldKeepTryingToSendMessageFailedSending() { verify(successHandler, timeout(1000)).handleSuccess(eq(message), eq(subscription), any(MessageSendingResult.class)); // then - verifySemaphoreReleased(); + verify(pendingOffsets).markAsProcessed(any(SubscriptionPartitionOffset.class)); verifyLatencyTimersCountedTimes(subscription, 3, 3); verifyRateLimiterAcquireTimersCountedTimes(subscription, 3, 3); verifyErrorHandlerHandleFailed(message, subscription, 2); @@ -170,7 +171,7 @@ public void shouldDiscardMessageWhenTTLIsExceeded() { // then verify(errorHandler, timeout(1000)).handleDiscarded(eq(message), eq(subscription), any(MessageSendingResult.class)); - verifySemaphoreReleased(); + verify(pendingOffsets).markAsProcessed(any(SubscriptionPartitionOffset.class)); verifyNoInteractions(successHandler); verifyLatencyTimersCountedTimes(subscription, 1, 1); verifyRateLimiterAcquireTimersCountedTimes(subscription, 1, 1); @@ -189,7 +190,7 @@ public void shouldNotKeepTryingToSendMessageFailedWithStatusCode4xx() { // then verify(errorHandler, timeout(1000)).handleDiscarded(eq(message), eq(subscription), any(MessageSendingResult.class)); - verifySemaphoreReleased(); + verify(pendingOffsets).markAsProcessed(any(SubscriptionPartitionOffset.class)); verifyNoInteractions(successHandler); verifyLatencyTimersCountedTimes(subscription, 1, 1); verifyRateLimiterAcquireTimersCountedTimes(subscription, 1, 1); @@ -210,11 +211,11 @@ public void shouldKeepTryingToSendMessageFailedWithStatusCode4xxForSubscriptionW verify(successHandler, timeout(1000)).handleSuccess(eq(message), eq(subscriptionWith4xxRetry), any(MessageSendingResult.class)); // then - verifySemaphoreReleased(); + verify(pendingOffsets).markAsProcessed(any(SubscriptionPartitionOffset.class)); verify(errorHandler, - timeout(1000).times(expectedNumbersOfFailures)).handleFailed(eq(message), - eq(subscriptionWith4xxRetry), - any(MessageSendingResult.class) + timeout(1000).times(expectedNumbersOfFailures)).handleFailed(eq(message), + eq(subscriptionWith4xxRetry), + any(MessageSendingResult.class) ); verifyRateLimiterAcquired(expectedNumbersOfFailures + 1); verifyRetryCounted(expectedNumbersOfFailures); @@ -275,7 +276,7 @@ public void shouldNotRetryOnRetryAfterAboveTtl() { // then verify(errorHandler, timeout(1000)).handleDiscarded(eq(message), eq(subscription), any(MessageSendingResult.class)); - verifySemaphoreReleased(); + verify(pendingOffsets).markAsProcessed(any(SubscriptionPartitionOffset.class)); verifyNoInteractions(successHandler); verifyLatencyTimersCountedTimes(subscription, 1, 1); verifyRateLimiterAcquireTimersCountedTimes(subscription, 1, 1); @@ -417,7 +418,7 @@ public void shouldIgnoreExponentialRetryBackoffWithRetryAfter() { //then verify(successHandler, timeout(retrySeconds * 1000 * 2 + 500)) - .handleSuccess(eq(message), eq(subscription), any(MessageSendingResult.class)); + .handleSuccess(eq(message), eq(subscription), any(MessageSendingResult.class)); verifyRateLimiterAcquired(expectedNumberOfRetries + 1); verifyRetryCounted(expectedNumberOfRetries); } @@ -452,7 +453,7 @@ private ConsumerMessageSender consumerMessageSender(Subscription subscription) { List.of(errorHandler), rateLimiter, Executors.newSingleThreadExecutor(), - () -> inflightSemaphore.release(), + pendingOffsets, metricsFacade, ASYNC_TIMEOUT_MS, new FutureAsyncTimeout(Executors.newSingleThreadScheduledExecutor()), @@ -564,10 +565,6 @@ private CompletableFuture backoff(int seconds) { return CompletableFuture.completedFuture(MessageSendingResult.retryAfter(seconds)); } - private void verifySemaphoreReleased() { - assertThat(inflightSemaphore.availablePermits()).isEqualTo(1); - } - private void verifyRateLimiterAcquired() { verifyRateLimiterAcquired(1); } diff --git a/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/supervisor/workload/ConsumerTestRuntimeEnvironment.java b/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/supervisor/workload/ConsumerTestRuntimeEnvironment.java index 4018066ac8..878522aa0b 100644 --- a/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/supervisor/workload/ConsumerTestRuntimeEnvironment.java +++ b/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/supervisor/workload/ConsumerTestRuntimeEnvironment.java @@ -16,7 +16,6 @@ import pl.allegro.tech.hermes.consumers.config.WorkloadProperties; import pl.allegro.tech.hermes.consumers.config.ZookeeperProperties; import pl.allegro.tech.hermes.consumers.consumer.offset.ConsumerPartitionAssignmentState; -import pl.allegro.tech.hermes.consumers.consumer.offset.OffsetQueue; import pl.allegro.tech.hermes.consumers.health.ConsumerMonitor; import pl.allegro.tech.hermes.consumers.message.undelivered.UndeliveredMessageLogPersister; import pl.allegro.tech.hermes.consumers.registry.ConsumerNodesRegistry; @@ -222,15 +221,13 @@ ConsumersSupervisor consumersSupervisor(ConsumerFactory consumerFactory) { return new NonblockingConsumersSupervisor(commonConsumerProperties, new ConsumersExecutorService(new CommonConsumerProperties().getThreadPoolSize(), metrics), consumerFactory, - mock(OffsetQueue.class), partitionAssignmentState, mock(Retransmitter.class), mock(UndeliveredMessageLogPersister.class), subscriptionRepository, metrics, mock(ConsumerMonitor.class), - Clock.systemDefaultZone(), - Duration.ofSeconds(60)); + Clock.systemDefaultZone()); } ConsumersRuntimeMonitor monitor(String consumerId, diff --git a/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/client/integration/HermesTestClient.java b/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/client/integration/HermesTestClient.java index 7ac2fcd6df..1f646f1e76 100644 --- a/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/client/integration/HermesTestClient.java +++ b/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/client/integration/HermesTestClient.java @@ -2,7 +2,6 @@ import jakarta.ws.rs.core.Response; import java.time.Duration; -import org.assertj.core.api.Assertions; import org.springframework.http.HttpHeaders; import org.springframework.test.web.reactive.server.WebTestClient; import org.springframework.util.MultiValueMap; @@ -21,6 +20,8 @@ import java.io.IOException; import java.util.List; +import java.util.Objects; +import java.util.concurrent.atomic.AtomicLong; import static org.assertj.core.api.Assertions.assertThat; import static org.awaitility.Awaitility.waitAtMost; @@ -143,6 +144,28 @@ public void waitUntilSubscriptionSuspended(String topicQualifiedName, String sub ); } + public void waitUntilConsumerCommitsOffset(String topicQualifiedName, String subscriptionName) { + long committedMessagesCount = calculateCommittedMessages(topicQualifiedName, subscriptionName); + waitAtMost(adjust(Duration.ofMinutes(1))).untilAsserted(() -> { + long currentCommittedMessagesCount = calculateCommittedMessages(topicQualifiedName, subscriptionName); + assertThat(currentCommittedMessagesCount).isGreaterThan(committedMessagesCount); + } + ); + } + + private long calculateCommittedMessages(String topicQualifiedName, String subscription) { + AtomicLong messagesCommittedCount = new AtomicLong(0); + List consumerGroups = getConsumerGroupsDescription(topicQualifiedName, subscription) + .expectBodyList(ConsumerGroup.class) + .returnResult().getResponseBody(); + Objects.requireNonNull(consumerGroups).forEach(consumerGroup -> + consumerGroup.getMembers().forEach(member -> + member.getPartitions().forEach(partition -> + messagesCommittedCount.addAndGet(partition.getCurrentOffset()) + ))); + return messagesCommittedCount.get(); + } + public int publishUntilSuccess(String topicQualifiedName, String body) { return frontendTestClient.publishUntilSuccess(topicQualifiedName, body); } @@ -400,10 +423,6 @@ public WebTestClient.ResponseSpec updateOAuthProvider(String name, PatchData pat return managementTestClient.updateOAuthProvider(name, patch); } - public WebTestClient.ResponseSpec searchOwners(String source, String searchString) { - return managementTestClient.searchOwners(source, searchString); - } - public WebTestClient.ResponseSpec setMode(String mode) { return managementTestClient.setMode(mode); } diff --git a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/KafkaRetransmissionServiceTest.java b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/KafkaRetransmissionServiceTest.java index 513d3e39c2..7bc280419d 100644 --- a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/KafkaRetransmissionServiceTest.java +++ b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/KafkaRetransmissionServiceTest.java @@ -11,7 +11,6 @@ import pl.allegro.tech.hermes.api.Subscription; import pl.allegro.tech.hermes.api.Topic; import pl.allegro.tech.hermes.common.kafka.offset.PartitionOffset; -import pl.allegro.tech.hermes.consumers.supervisor.process.RunningSubscriptionStatus; import pl.allegro.tech.hermes.integrationtests.setup.HermesExtension; import pl.allegro.tech.hermes.integrationtests.subscriber.TestSubscriber; import pl.allegro.tech.hermes.integrationtests.subscriber.TestSubscribersExtension; @@ -19,22 +18,16 @@ import pl.allegro.tech.hermes.test.helper.avro.AvroUser; import pl.allegro.tech.hermes.test.helper.message.TestMessage; -import java.time.Clock; -import java.time.Duration; import java.time.OffsetDateTime; import java.util.ArrayList; import java.util.List; -import java.util.function.Predicate; import java.util.stream.Collectors; import static java.util.stream.IntStream.range; import static org.assertj.core.api.Assertions.assertThat; -import static org.awaitility.Awaitility.waitAtMost; import static pl.allegro.tech.hermes.api.PatchData.patchData; -import static pl.allegro.tech.hermes.consumers.supervisor.process.Signal.SignalType.COMMIT; import static pl.allegro.tech.hermes.test.helper.builder.SubscriptionBuilder.subscriptionWithRandomName; import static pl.allegro.tech.hermes.test.helper.builder.TopicBuilder.topicWithRandomName; -import static pl.allegro.tech.hermes.test.helper.endpoint.TimeoutAdjuster.adjust; public class KafkaRetransmissionServiceTest { @@ -42,8 +35,6 @@ public class KafkaRetransmissionServiceTest { private final AvroUser avroUser = new AvroUser(); - private final Clock clock = Clock.systemDefaultZone(); - private final List messages = new ArrayList<>() {{ range(0, 4).forEach(i -> add(TestMessage.random().body())); }}; @@ -69,7 +60,7 @@ public void shouldMoveOffsetNearGivenTimestamp() throws InterruptedException { final OffsetRetransmissionDate retransmissionDate = new OffsetRetransmissionDate(OffsetDateTime.now()); Thread.sleep(1000); publishAndConsumeMessages(messages2, topic, subscriber); - waitUntilConsumerCommitsOffset(topic.getQualifiedName(), subscription.getName()); + hermes.api().waitUntilConsumerCommitsOffset(topic.getQualifiedName(), subscription.getName()); // when WebTestClient.ResponseSpec response = hermes.api().retransmit(topic.getQualifiedName(), subscription.getName(), retransmissionDate, false); @@ -90,7 +81,7 @@ public void shouldMoveOffsetInDryRunMode() throws InterruptedException { Thread.sleep(2000); final OffsetRetransmissionDate retransmissionDate = new OffsetRetransmissionDate(OffsetDateTime.now()); publishAndConsumeMessages(messages2, topic, subscriber); - waitUntilConsumerCommitsOffset(topic.getQualifiedName(), subscription.getName()); + hermes.api().waitUntilConsumerCommitsOffset(topic.getQualifiedName(), subscription.getName()); subscriber.reset(); // when @@ -115,13 +106,13 @@ public void shouldMoveOffsetInDryRunModeForTopicsMigratedToAvro() throws Interru Subscription subscription = hermes.initHelper().createSubscription(subscriptionWithRandomName(topic.getName(), subscriber.getEndpoint()).build()); hermes.api().publish(topic.getQualifiedName(), TestMessage.simple().body()); - waitUntilConsumerCommitsOffset(topic.getQualifiedName(), subscription.getName()); + hermes.api().waitUntilConsumerCommitsOffset(topic.getQualifiedName(), subscription.getName()); Thread.sleep(1000); //wait 1s because our date time format has seconds precision final OffsetRetransmissionDate retransmissionDate = new OffsetRetransmissionDate(OffsetDateTime.now()); hermes.api().publish(topic.getQualifiedName(), TestMessage.simple().body()); - waitUntilConsumerCommitsOffset(topic.getQualifiedName(), subscription.getName()); + hermes.api().waitUntilConsumerCommitsOffset(topic.getQualifiedName(), subscription.getName()); PatchData patch = patchData() .set("contentType", ContentType.AVRO) @@ -132,7 +123,7 @@ public void shouldMoveOffsetInDryRunModeForTopicsMigratedToAvro() throws Interru hermes.api().updateTopic(topic.getQualifiedName(), patch).expectStatus().isOk(); hermes.api().publishAvroUntilSuccess(topic.getQualifiedName(), avroUser.asBytes()); - waitUntilConsumerCommitsOffset(topic.getQualifiedName(), subscription.getName()); + hermes.api().waitUntilConsumerCommitsOffset(topic.getQualifiedName(), subscription.getName()); // when WebTestClient.ResponseSpec response = hermes.api().retransmit(topic.getQualifiedName(), subscription.getName(), retransmissionDate, true); @@ -153,19 +144,6 @@ private void publishAndConsumeMessages(List messages, Topic topic, TestS messages.forEach(subscriber::waitUntilReceived); } - private void waitUntilConsumerCommitsOffset(String topicQualifiedName, String subscription) { - long currentTime = clock.millis(); - until(Duration.ofMinutes(1), topicQualifiedName, subscription, sub -> - sub.getSignalTimesheet().getOrDefault(COMMIT, 0L) > currentTime); - } - - private void until(Duration duration, String topicQualifiedName, String subscription, Predicate predicate) { - waitAtMost(adjust(duration)).until(() -> - hermes.api().getRunningSubscriptionsStatus().stream() - .filter(sub -> sub.getQualifiedName().equals(topicQualifiedName + "$" + subscription)) - .anyMatch(predicate)); - } - private record PartitionOffsetsPerKafkaTopic(List avroPartitionOffsets, List jsonPartitionOffsets) { diff --git a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/PublishingAvroTest.java b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/PublishingAvroTest.java index dcf6cb2a05..706572a6f4 100644 --- a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/PublishingAvroTest.java +++ b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/PublishingAvroTest.java @@ -36,7 +36,6 @@ import static pl.allegro.tech.hermes.api.PatchData.patchData; import static pl.allegro.tech.hermes.api.TopicWithSchema.topicWithSchema; import static pl.allegro.tech.hermes.client.HermesMessage.hermesMessage; -import static pl.allegro.tech.hermes.consumers.supervisor.process.Signal.SignalType.COMMIT; import static pl.allegro.tech.hermes.consumers.supervisor.process.Signal.SignalType.UPDATE_SUBSCRIPTION; import static pl.allegro.tech.hermes.test.helper.avro.AvroUserSchemaLoader.load; import static pl.allegro.tech.hermes.test.helper.builder.SubscriptionBuilder.subscription; @@ -63,8 +62,8 @@ public class PublishingAvroTest { public void shouldPublishAvroAndConsumeJsonMessage() { // given TopicWithSchema topicWithSchema = topicWithSchema(topicWithRandomName() - .withContentType(AVRO) - .build(), user.getSchemaAsString()); + .withContentType(AVRO) + .build(), user.getSchemaAsString()); Topic topic = hermes.initHelper().createTopicWithSchema(topicWithSchema); TestSubscriber subscriber = subscribers.createSubscriber(); @@ -378,7 +377,7 @@ public void shouldUseExplicitSchemaVersionWhenPublishingAndConsuming() { TestSubscriber subscriber = subscribers.createSubscriber(); hermes.initHelper().createSubscription(subscription(topic.getQualifiedName(), "subscription", subscriber.getEndpoint()) - .withContentType(AVRO) + .withContentType(AVRO) .build()); hermes.api().ensureSchemaSaved(topic.getQualifiedName(), false, load("/schema/user_v2.avsc").toString()); @@ -489,7 +488,7 @@ public void shouldPublishAndConsumeJsonMessageAfterMigrationFromJsonToAvro() { subscriber.waitUntilReceived(beforeMigrationMessage.body()); subscriber.reset(); - waitUntilConsumerCommitsOffset(topic, "subscription"); + hermes.api().waitUntilConsumerCommitsOffset(topic.getQualifiedName(), "subscription"); PatchData patch = patchData() .set("contentType", ContentType.AVRO) @@ -546,15 +545,6 @@ private void waitUntilSubscriptionContentTypeChanged(Topic topic, String subscri }); } - private void waitUntilConsumerCommitsOffset(Topic topic, String subscription) { - long currentTime = clock.millis(); - waitAtMost(adjust(Duration.ofMinutes(1))).until(() -> - hermes.api().getRunningSubscriptionsStatus().stream() - .filter(sub -> sub.getQualifiedName().equals(topic.getQualifiedName() + "$" + subscription)) - .anyMatch(sub -> sub.getSignalTimesheet().getOrDefault(COMMIT, 0L) > currentTime)); - - } - private void waitUntilConsumersUpdateSubscription(final long currentTime, Topic topic, String subscription) { waitAtMost(adjust(Duration.ofSeconds(10))).until(() -> hermes.api().getRunningSubscriptionsStatus().stream() From 4292174088b4c1f5bd317c6a5c5a1be6f60eef9c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20F=C4=85derski?= Date: Thu, 1 Aug 2024 16:59:05 +0200 Subject: [PATCH 73/87] Rewrite victoriametrics client to pure promql (#1882) * Initial rewriting of victoriametrics client to the pure PromQl * Get rid of hacky code related to handling status codes query in one requests * Add few fixes * Add more tests for caching unavailable metrics * Debug reason of wiremock server timestous * Fix failing prometheus metrics tests * Add code refactor * Add metrics for prometheus requests * Improve creating prometheus client dependencies * Improve creating prometheus client dependencies * Fix handling errors when one of the requests fails * Remove unnecessary log * CR fixes * CR fixes * CR fixes --- build.gradle | 2 +- docs/docs/configuration/metrics.md | 4 +- .../tech/hermes/api/MetricDecimalValue.java | 13 ++ .../ExternalMonitoringClientProperties.java | 22 +- .../ExternalMonitoringConfiguration.java | 38 +++- .../subscription/SubscriptionService.java | 2 + .../metrics/MonitoringMetricsContainer.java | 12 +- .../prometheus/CachingPrometheusClient.java | 30 ++- .../prometheus/PrometheusClient.java | 39 +++- ...er.java => PrometheusMetricsProvider.java} | 90 ++++---- .../prometheus/PrometheusResponse.java | 26 --- .../RestTemplatePrometheusClient.java | 143 +++++++------ ...edSubscriptionMetricsRepositoryTest.groovy | 47 ++-- ...heusBasedTopicMetricsRepositoryTest.groovy | 19 +- .../CachingPrometheusClientTest.groovy | 40 +++- .../RestTemplatePrometheusClientTest.groovy | 200 ++++++++++++++---- .../__files/full_response.json | 156 -------------- .../__files/partial_response.json | 115 ---------- .../__files/prometheus_empty_response.json | 7 + ...scription_2xx_http_status_codes_total.json | 19 ++ ...scription_4xx_http_status_codes_total.json | 19 ++ ...scription_5xx_http_status_codes_total.json | 19 ++ .../__files/subscription_batches_total.json | 19 ++ .../__files/subscription_delivered_total.json | 19 ++ .../subscription_other_errors_total.json | 19 ++ .../__files/subscription_retries_total.json | 19 ++ .../subscription_throughput_bytes_total.json | 19 ++ .../__files/subscription_timeouts_total.json | 19 ++ .../prometheus/PrometheusExtension.java | 73 ++++--- .../prometheus/PrometheusResponse.java | 9 +- .../prometheus/SubscriptionMetrics.java | 32 ++- .../prometheus/TopicMetrics.java | 34 ++- .../management/QueryEndpointTest.java | 2 +- .../SubscriptionManagementTest.java | 5 +- 34 files changed, 741 insertions(+), 590 deletions(-) rename hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/{VictoriaMetricsMetricsProvider.java => PrometheusMetricsProvider.java} (50%) delete mode 100644 hermes-management/src/test/resources/prometheus-stubs/__files/full_response.json delete mode 100644 hermes-management/src/test/resources/prometheus-stubs/__files/partial_response.json create mode 100644 hermes-management/src/test/resources/prometheus-stubs/__files/prometheus_empty_response.json create mode 100644 hermes-management/src/test/resources/prometheus-stubs/__files/subscription_2xx_http_status_codes_total.json create mode 100644 hermes-management/src/test/resources/prometheus-stubs/__files/subscription_4xx_http_status_codes_total.json create mode 100644 hermes-management/src/test/resources/prometheus-stubs/__files/subscription_5xx_http_status_codes_total.json create mode 100644 hermes-management/src/test/resources/prometheus-stubs/__files/subscription_batches_total.json create mode 100644 hermes-management/src/test/resources/prometheus-stubs/__files/subscription_delivered_total.json create mode 100644 hermes-management/src/test/resources/prometheus-stubs/__files/subscription_other_errors_total.json create mode 100644 hermes-management/src/test/resources/prometheus-stubs/__files/subscription_retries_total.json create mode 100644 hermes-management/src/test/resources/prometheus-stubs/__files/subscription_throughput_bytes_total.json create mode 100644 hermes-management/src/test/resources/prometheus-stubs/__files/subscription_timeouts_total.json diff --git a/build.gradle b/build.gradle index 0c855d7a33..3d68efc4d1 100644 --- a/build.gradle +++ b/build.gradle @@ -56,7 +56,7 @@ allprojects { curator : '5.4.0', dropwizard_metrics: '4.2.25', micrometer_metrics: '1.12.5', - wiremock : '3.5.2', + wiremock : '3.9.0', spock : '2.4-M4-groovy-4.0', groovy : '4.0.21', avro : '1.11.3', diff --git a/docs/docs/configuration/metrics.md b/docs/docs/configuration/metrics.md index 86f78fe134..5e172d4ebf 100644 --- a/docs/docs/configuration/metrics.md +++ b/docs/docs/configuration/metrics.md @@ -12,9 +12,9 @@ Option | Description {modulePrefix}.metrics.prometheus.step | The step size to use in computing windowed statistics | 60s {modulePrefix}.metrics.prometheus.descriptions | If meter descriptions should be sent to Prometheus | true -In order to be able to access basic metrics via Management API, it needs to be configured to reach VictoriaMetrics API: +In order to be able to access basic metrics via Management API, it needs to be configured to reach Prometheus API: Option | Description | Default value ------------------------------------------|-----------------------------------------------| ------------- prometheus.client.enabled | Should fetch external metrics from Prometheus | true -prometheus.client.externalMonitoringUrl | URI to VictoriaMetrics HTTP API | http://localhost:18090 +prometheus.client.externalMonitoringUrl | URI to Prometheus HTTP API | http://localhost:18090 diff --git a/hermes-api/src/main/java/pl/allegro/tech/hermes/api/MetricDecimalValue.java b/hermes-api/src/main/java/pl/allegro/tech/hermes/api/MetricDecimalValue.java index ff033e775a..c412b1c3a4 100644 --- a/hermes-api/src/main/java/pl/allegro/tech/hermes/api/MetricDecimalValue.java +++ b/hermes-api/src/main/java/pl/allegro/tech/hermes/api/MetricDecimalValue.java @@ -10,6 +10,7 @@ public class MetricDecimalValue { private static final String UNAVAILABLE_STRING = "unavailable"; private static final MetricDecimalValue UNAVAILABLE = new MetricDecimalValue(false, "-1.0"); + private static final MetricDecimalValue DEFAULT_VALUE = new MetricDecimalValue(true, "0.0"); private final boolean available; private final String value; @@ -23,6 +24,10 @@ public static MetricDecimalValue unavailable() { return UNAVAILABLE; } + public static MetricDecimalValue defaultValue() { + return DEFAULT_VALUE; + } + public static MetricDecimalValue of(String value) { return new MetricDecimalValue(true, value); } @@ -65,4 +70,12 @@ public boolean equals(Object o) { public int hashCode() { return Objects.hash(available, value); } + + @Override + public String toString() { + return "MetricDecimalValue{" + + "available=" + available + + ", value='" + value + '\'' + + '}'; + } } diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ExternalMonitoringClientProperties.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ExternalMonitoringClientProperties.java index 92f20ea63c..4b50c67e32 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ExternalMonitoringClientProperties.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ExternalMonitoringClientProperties.java @@ -8,14 +8,18 @@ public class ExternalMonitoringClientProperties { private int maxConnections = 100; - private int maxConnectionsPerRoute = 10; + private int maxConnectionsPerRoute = 100; private int cacheTtlSeconds = 55; private int cacheSize = 100_000; + private int fetchingTimeoutMillis = 5000; + private int fetchingThreads = 30; + private String externalMonitoringUrl = "http://localhost:18090"; + public int getConnectionTimeoutMillis() { return connectionTimeoutMillis; } @@ -71,4 +75,20 @@ public String getExternalMonitoringUrl() { public void setExternalMonitoringUrl(String externalMonitoringUrl) { this.externalMonitoringUrl = externalMonitoringUrl; } + + public int getFetchingThreads() { + return fetchingThreads; + } + + public void setFetchingThreads(int fetchingThreads) { + this.fetchingThreads = fetchingThreads; + } + + public int getFetchingTimeoutMillis() { + return fetchingTimeoutMillis; + } + + public void setFetchingTimeoutMillis(int fetchingTimeoutMillis) { + this.fetchingTimeoutMillis = fetchingTimeoutMillis; + } } diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ExternalMonitoringConfiguration.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ExternalMonitoringConfiguration.java index ed3148d5b3..96149ec69f 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ExternalMonitoringConfiguration.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ExternalMonitoringConfiguration.java @@ -1,5 +1,7 @@ package pl.allegro.tech.hermes.management.config; +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import io.micrometer.core.instrument.MeterRegistry; import org.apache.hc.client5.http.classic.HttpClient; import org.apache.hc.client5.http.config.RequestConfig; import org.apache.hc.client5.http.impl.classic.HttpClientBuilder; @@ -7,6 +9,7 @@ import org.apache.hc.client5.http.impl.io.PoolingHttpClientConnectionManagerBuilder; import org.apache.hc.core5.util.Timeout; import org.springframework.beans.factory.annotation.Qualifier; +import org.springframework.boot.autoconfigure.condition.ConditionalOnMissingBean; import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty; import org.springframework.context.annotation.Bean; import org.springframework.context.annotation.Configuration; @@ -15,31 +18,40 @@ import org.springframework.web.client.RestTemplate; import pl.allegro.tech.hermes.management.infrastructure.prometheus.CachingPrometheusClient; import pl.allegro.tech.hermes.management.infrastructure.prometheus.PrometheusClient; +import pl.allegro.tech.hermes.management.infrastructure.prometheus.PrometheusMetricsProvider; import pl.allegro.tech.hermes.management.infrastructure.prometheus.RestTemplatePrometheusClient; -import pl.allegro.tech.hermes.management.infrastructure.prometheus.VictoriaMetricsMetricsProvider; import java.net.URI; +import java.time.Duration; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import static com.google.common.base.Ticker.systemTicker; @Configuration +@ConditionalOnProperty(value = "prometheus.client.enabled", havingValue = "true") public class ExternalMonitoringConfiguration { @Bean - @ConditionalOnProperty(value = "prometheus.client.enabled", havingValue = "true") - public VictoriaMetricsMetricsProvider prometheusMetricsProvider(PrometheusClient prometheusClient, - PrometheusMonitoringClientProperties properties) { - return new VictoriaMetricsMetricsProvider(prometheusClient, + public PrometheusMetricsProvider prometheusMetricsProvider(PrometheusClient prometheusClient, + PrometheusMonitoringClientProperties properties) { + return new PrometheusMetricsProvider(prometheusClient, properties.getConsumersMetricsPrefix(), properties.getFrontendMetricsPrefix(), properties.getAdditionalFilters()); } @Bean - @ConditionalOnProperty(value = "prometheus.client.enabled", havingValue = "true") public PrometheusClient prometheusClient(@Qualifier("monitoringRestTemplate") RestTemplate monitoringRestTemplate, - PrometheusMonitoringClientProperties clientProperties) { + PrometheusMonitoringClientProperties clientProperties, + @Qualifier("prometheusFetcherExecutorService") ExecutorService executorService, + MeterRegistry meterRegistry) { RestTemplatePrometheusClient underlyingPrometheusClient = - new RestTemplatePrometheusClient(monitoringRestTemplate, URI.create(clientProperties.getExternalMonitoringUrl())); + new RestTemplatePrometheusClient( + monitoringRestTemplate, + URI.create(clientProperties.getExternalMonitoringUrl()), + executorService, + Duration.ofMillis(clientProperties.getFetchingTimeoutMillis()), + meterRegistry); return new CachingPrometheusClient( underlyingPrometheusClient, systemTicker(), @@ -49,6 +61,7 @@ public PrometheusClient prometheusClient(@Qualifier("monitoringRestTemplate") Re } @Bean("monitoringRestTemplate") + @ConditionalOnMissingBean(name = "monitoringRestTemplate") public RestTemplate restTemplate(ExternalMonitoringClientProperties clientProperties) { PoolingHttpClientConnectionManager connectionManager = PoolingHttpClientConnectionManagerBuilder.create() .setMaxConnTotal(clientProperties.getMaxConnections()) @@ -66,7 +79,14 @@ public RestTemplate restTemplate(ExternalMonitoringClientProperties clientProper .build(); ClientHttpRequestFactory clientHttpRequestFactory = new HttpComponentsClientHttpRequestFactory(client); - return new RestTemplate(clientHttpRequestFactory); } + + @Bean("prometheusFetcherExecutorService") + @ConditionalOnMissingBean(name = "prometheusFetcherExecutorService") + public ExecutorService executorService(ExternalMonitoringClientProperties clientProperties) { + return Executors.newFixedThreadPool(clientProperties.getFetchingThreads(), + new ThreadFactoryBuilder().setNameFormat("prometheus-metrics-fetcher-%d").build() + ); + } } diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/subscription/SubscriptionService.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/subscription/SubscriptionService.java index 82b65febd6..4cbfeae6ec 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/subscription/SubscriptionService.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/subscription/SubscriptionService.java @@ -344,8 +344,10 @@ private List getUnhealthyList(Collection()); } - public MonitoringMetricsContainer addMetricValue(String metricPath, MetricDecimalValue value) { + public MonitoringMetricsContainer addMetricValue(String query, MetricDecimalValue value) { if (!isAvailable) { throw new IllegalStateException("Adding value to unavailable metrics container"); } - this.metrics.put(metricPath, value); + this.metrics.put(query, value); return this; } - public MetricDecimalValue metricValue(String metricPath) { + public MetricDecimalValue metricValue(String query) { if (!isAvailable) { return MetricDecimalValue.unavailable(); } - return metrics.getOrDefault(metricPath, DEFAULT_VALUE); + return metrics.getOrDefault(query, DEFAULT_VALUE); + } + + public boolean hasUnavailableMetrics() { + return !isAvailable || metrics.entrySet().stream().anyMatch(e -> !e.getValue().isAvailable()); } } diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/CachingPrometheusClient.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/CachingPrometheusClient.java index 942beb67a7..36675b8b6f 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/CachingPrometheusClient.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/CachingPrometheusClient.java @@ -6,14 +6,25 @@ import com.google.common.cache.LoadingCache; import pl.allegro.tech.hermes.management.infrastructure.metrics.MonitoringMetricsContainer; +import java.util.List; import java.util.concurrent.ExecutionException; import static java.util.concurrent.TimeUnit.SECONDS; + public class CachingPrometheusClient implements PrometheusClient { private final PrometheusClient underlyingPrometheusClient; - private final LoadingCache prometheusMetricsCache; + /* + Metrics will always be requested in the context of a single subscription/topic. The single sub/topic will + always result in the same list of metrics queries. There is no overlapping between metrics used in the context of + topic or subscriptions. That's why it is safe to use a list of queries as a caching key. + + Maybe it will be worth to cache it per query except of queries when there will be too much overhead + of refreshing all sub/topic metrics if the single fetch fails (currently we invalidate whole metrics container + when one of the sub metric is unavailable) + */ + private final LoadingCache, MonitoringMetricsContainer> prometheusMetricsCache; public CachingPrometheusClient(PrometheusClient underlyingPrometheusClient, Ticker ticker, long cacheTtlInSeconds, long cacheSize) { @@ -26,19 +37,24 @@ public CachingPrometheusClient(PrometheusClient underlyingPrometheusClient, Tick } @Override - public MonitoringMetricsContainer readMetrics(String query) { + public MonitoringMetricsContainer readMetrics(List queries) { try { - return prometheusMetricsCache.get(query); + MonitoringMetricsContainer monitoringMetricsContainer = prometheusMetricsCache.get(List.copyOf(queries)); + if (monitoringMetricsContainer.hasUnavailableMetrics()) { + // try to reload the on the next fetch + prometheusMetricsCache.invalidate(queries); + } + return monitoringMetricsContainer; } catch (ExecutionException e) { - // should never happen because the loader does not throw any checked exceptions + // should never happen because the loader does not throw any exceptions throw new RuntimeException(e); } } - private class PrometheusMetricsCacheLoader extends CacheLoader { + private class PrometheusMetricsCacheLoader extends CacheLoader, MonitoringMetricsContainer> { @Override - public MonitoringMetricsContainer load(String query) { - return underlyingPrometheusClient.readMetrics(query); + public MonitoringMetricsContainer load(List queries) { + return underlyingPrometheusClient.readMetrics(queries); } } } \ No newline at end of file diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/PrometheusClient.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/PrometheusClient.java index 6c5c4a5493..33170fb9ee 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/PrometheusClient.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/PrometheusClient.java @@ -1,7 +1,44 @@ package pl.allegro.tech.hermes.management.infrastructure.prometheus; +import pl.allegro.tech.hermes.api.SubscriptionName; +import pl.allegro.tech.hermes.api.TopicName; import pl.allegro.tech.hermes.management.infrastructure.metrics.MonitoringMetricsContainer; +import java.util.List; + + public interface PrometheusClient { - MonitoringMetricsContainer readMetrics(String query); + String SUBSCRIPTION_QUERY_FORMAT = "sum by (group, topic, subscription)" + + " (irate({__name__='%s', group='%s', topic='%s', subscription='%s', %s}[1m]))"; + + String SUBSCRIPTION_QUERY_FORMAT_STATUS_CODE = "sum by (group, topic, subscription)" + + " (irate({__name__='%s', group='%s', topic='%s', subscription='%s', status_code=~'%s', %s}[1m]))"; + + String TOPIC_QUERY_FORMAT = "sum by (group, topic) (irate({__name__='%s', group='%s', " + + "topic='%s', %s}[1m]))"; + + default MonitoringMetricsContainer readMetrics(String... query) { + return readMetrics(List.of(query)); + } + + MonitoringMetricsContainer readMetrics(List queries); + + static String forSubscription(String name, SubscriptionName subscriptionName, String additionalFilters) { + return String.format(SUBSCRIPTION_QUERY_FORMAT, name, + subscriptionName.getTopicName().getGroupName(), subscriptionName.getTopicName().getName(), + subscriptionName.getName(), additionalFilters); + } + + static String forSubscriptionStatusCode(String name, SubscriptionName subscriptionName, + String regex, String additionalFilters) { + return String.format(SUBSCRIPTION_QUERY_FORMAT_STATUS_CODE, name, + subscriptionName.getTopicName().getGroupName(), subscriptionName.getTopicName().getName(), + subscriptionName.getName(), regex, additionalFilters); + } + + + static String forTopic(String name, TopicName topicName, String additionalFilters) { + return String.format(TOPIC_QUERY_FORMAT, name, + topicName.getGroupName(), topicName.getName(), additionalFilters); + } } diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/VictoriaMetricsMetricsProvider.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/PrometheusMetricsProvider.java similarity index 50% rename from hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/VictoriaMetricsMetricsProvider.java rename to hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/PrometheusMetricsProvider.java index 48bcd67600..7121281b5a 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/VictoriaMetricsMetricsProvider.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/PrometheusMetricsProvider.java @@ -6,11 +6,12 @@ import pl.allegro.tech.hermes.management.infrastructure.metrics.MonitoringSubscriptionMetricsProvider; import pl.allegro.tech.hermes.management.infrastructure.metrics.MonitoringTopicMetricsProvider; -import java.util.List; -import java.util.stream.Collectors; -import java.util.stream.Stream; +import static pl.allegro.tech.hermes.management.infrastructure.prometheus.PrometheusClient.forSubscription; +import static pl.allegro.tech.hermes.management.infrastructure.prometheus.PrometheusClient.forSubscriptionStatusCode; +import static pl.allegro.tech.hermes.management.infrastructure.prometheus.PrometheusClient.forTopic; -public class VictoriaMetricsMetricsProvider implements MonitoringSubscriptionMetricsProvider, MonitoringTopicMetricsProvider { + +public class PrometheusMetricsProvider implements MonitoringSubscriptionMetricsProvider, MonitoringTopicMetricsProvider { private static final String SUBSCRIPTION_DELIVERED = "subscription_delivered_total"; private static final String SUBSCRIPTION_TIMEOUTS = "subscription_timeouts_total"; @@ -18,9 +19,6 @@ public class VictoriaMetricsMetricsProvider implements MonitoringSubscriptionMet private static final String SUBSCRIPTION_OTHER_ERRORS = "subscription_other_errors_total"; private static final String SUBSCRIPTION_BATCHES = "subscription_batches_total"; private static final String SUBSCRIPTION_STATUS_CODES = "subscription_http_status_codes_total"; - private static final String SUBSCRIPTION_STATUS_CODES_2XX = SUBSCRIPTION_STATUS_CODES + "_2xx"; - private static final String SUBSCRIPTION_STATUS_CODES_4XX = SUBSCRIPTION_STATUS_CODES + "_4xx"; - private static final String SUBSCRIPTION_STATUS_CODES_5XX = SUBSCRIPTION_STATUS_CODES + "_5xx"; private static final String SUBSCRIPTION_RETRIES = "subscription_retries_total"; private static final String TOPIC_RATE = "topic_requests_total"; @@ -30,72 +28,62 @@ public class VictoriaMetricsMetricsProvider implements MonitoringSubscriptionMet private final String consumersMetricsPrefix; private final String frontendMetricsPrefix; private final String additionalFilters; - private final String subscriptionMetricsToQuery; - private final String topicMetricsToQuery; private final PrometheusClient prometheusClient; - public VictoriaMetricsMetricsProvider(PrometheusClient prometheusClient, String consumersMetricsPrefix, - String frontendMetricsPrefix, String additionalFilters) { + public PrometheusMetricsProvider(PrometheusClient prometheusClient, String consumersMetricsPrefix, + String frontendMetricsPrefix, String additionalFilters) { this.prometheusClient = prometheusClient; this.consumersMetricsPrefix = consumersMetricsPrefix.isEmpty() ? "" : consumersMetricsPrefix + "_"; this.frontendMetricsPrefix = frontendMetricsPrefix.isEmpty() ? "" : frontendMetricsPrefix + "_"; this.additionalFilters = additionalFilters; - this.subscriptionMetricsToQuery = Stream.of(SUBSCRIPTION_DELIVERED, SUBSCRIPTION_TIMEOUTS, SUBSCRIPTION_RETRIES, - SUBSCRIPTION_THROUGHPUT, SUBSCRIPTION_OTHER_ERRORS, SUBSCRIPTION_BATCHES, SUBSCRIPTION_STATUS_CODES) - .map(this::consumerMetricName) - .collect(Collectors.joining("|")); - this.topicMetricsToQuery = String.join("|", List.of( - frontendMetricName(TOPIC_RATE), - consumerMetricName(TOPIC_DELIVERY_RATE), - frontendMetricName(TOPIC_THROUGHPUT_RATE) - )); } @Override public MonitoringSubscriptionMetrics subscriptionMetrics(SubscriptionName subscriptionName) { - /* - The query is based on MetricsQL, available only in VictoriaMetrics - https://docs.victoriametrics.com/MetricsQL.html. Basic PromQL does not support `keep_metric_names` param. - */ - String queryFormat = "sum by (__name__, group, topic, subscription, status_code)" - + " (irate({__name__=~'%s', group='%s', topic='%s', subscription='%s', %s}[1m]) keep_metric_names)"; - String query = String.format(queryFormat, subscriptionMetricsToQuery, subscriptionName.getTopicName().getGroupName(), - subscriptionName.getTopicName().getName(), subscriptionName.getName(), additionalFilters); - MonitoringMetricsContainer prometheusMetricsContainer = prometheusClient.readMetrics(query); + String subscriptionDeliveredQuery = forSubscription(consumerMetricName(SUBSCRIPTION_DELIVERED), subscriptionName, additionalFilters); + String subscriptionTimeoutsQuery = forSubscription(consumerMetricName(SUBSCRIPTION_TIMEOUTS), subscriptionName, additionalFilters); + String subscriptionThroughputQuery = forSubscription(consumerMetricName(SUBSCRIPTION_THROUGHPUT), subscriptionName, additionalFilters); + String subscriptionOtherErrorsQuery = forSubscription(consumerMetricName(SUBSCRIPTION_OTHER_ERRORS), subscriptionName, additionalFilters); + String subscriptionBatchesQuery = forSubscription(consumerMetricName(SUBSCRIPTION_BATCHES), subscriptionName, additionalFilters); + String subscriptionRetriesQuery = forSubscription(consumerMetricName(SUBSCRIPTION_RETRIES), subscriptionName, additionalFilters); + String subscription2xx = forSubscriptionStatusCode(consumerMetricName(SUBSCRIPTION_STATUS_CODES), subscriptionName, "2.*", additionalFilters); + String subscription4xx = forSubscriptionStatusCode(consumerMetricName(SUBSCRIPTION_STATUS_CODES), subscriptionName, "4.*", additionalFilters); + String subscription5xx = forSubscriptionStatusCode(consumerMetricName(SUBSCRIPTION_STATUS_CODES), subscriptionName, "5.*", additionalFilters); + + MonitoringMetricsContainer prometheusMetricsContainer = prometheusClient.readMetrics( + subscriptionDeliveredQuery, subscriptionTimeoutsQuery, subscriptionRetriesQuery, subscriptionThroughputQuery, + subscriptionOtherErrorsQuery, subscriptionBatchesQuery, subscription2xx, subscription4xx, subscription5xx + ); return MonitoringSubscriptionMetricsProvider .metricsBuilder() - .withRate(prometheusMetricsContainer.metricValue(consumerMetricName(SUBSCRIPTION_DELIVERED))) - .withTimeouts(prometheusMetricsContainer.metricValue(consumerMetricName(SUBSCRIPTION_TIMEOUTS))) - .withThroughput(prometheusMetricsContainer.metricValue(consumerMetricName(SUBSCRIPTION_THROUGHPUT))) - .withOtherErrors(prometheusMetricsContainer.metricValue(consumerMetricName(SUBSCRIPTION_OTHER_ERRORS))) - .withMetricPathBatchRate(prometheusMetricsContainer.metricValue(consumerMetricName(SUBSCRIPTION_BATCHES))) - .withCodes2xx(prometheusMetricsContainer.metricValue(consumerMetricName(SUBSCRIPTION_STATUS_CODES_2XX))) - .withCode4xx(prometheusMetricsContainer.metricValue(consumerMetricName(SUBSCRIPTION_STATUS_CODES_4XX))) - .withCode5xx(prometheusMetricsContainer.metricValue(consumerMetricName(SUBSCRIPTION_STATUS_CODES_5XX))) - .withRetries(prometheusMetricsContainer.metricValue(consumerMetricName(SUBSCRIPTION_RETRIES))) + .withRate(prometheusMetricsContainer.metricValue(subscriptionDeliveredQuery)) + .withTimeouts(prometheusMetricsContainer.metricValue(subscriptionTimeoutsQuery)) + .withThroughput(prometheusMetricsContainer.metricValue(subscriptionThroughputQuery)) + .withOtherErrors(prometheusMetricsContainer.metricValue(subscriptionOtherErrorsQuery)) + .withMetricPathBatchRate(prometheusMetricsContainer.metricValue(subscriptionBatchesQuery)) + .withCodes2xx(prometheusMetricsContainer.metricValue(subscription2xx)) + .withCode4xx(prometheusMetricsContainer.metricValue(subscription4xx)) + .withCode5xx(prometheusMetricsContainer.metricValue(subscription5xx)) + .withRetries(prometheusMetricsContainer.metricValue(subscriptionRetriesQuery)) .build(); } @Override public MonitoringTopicMetrics topicMetrics(TopicName topicName) { - /* - The query is based on MetricsQL, available only in VictoriaMetrics - https://docs.victoriametrics.com/MetricsQL.html. Basic PromQL does not support `keep_metric_names` param. - */ - String queryFormat = "sum by (__name__, group, topic) (irate({__name__=~'%s', group='%s', " - + "topic='%s', %s}[1m]) keep_metric_names)"; - String query = String.format(queryFormat, topicMetricsToQuery, topicName.getGroupName(), topicName.getName(), - additionalFilters); - MonitoringMetricsContainer prometheusMetricsContainer = prometheusClient.readMetrics(query); + String topicRateQuery = forTopic(frontendMetricName(TOPIC_RATE), topicName, additionalFilters); + String topicDeliveryRateQuery = forTopic(consumerMetricName(TOPIC_DELIVERY_RATE), topicName, additionalFilters); + String topicThroughputQuery = forTopic(frontendMetricName(TOPIC_THROUGHPUT_RATE), topicName, additionalFilters); + + MonitoringMetricsContainer prometheusMetricsContainer = prometheusClient.readMetrics( + topicRateQuery, topicDeliveryRateQuery, topicThroughputQuery); return MonitoringTopicMetricsProvider .metricsBuilder() - .withRate(prometheusMetricsContainer.metricValue(frontendMetricName(TOPIC_RATE))) - .withDeliveryRate(prometheusMetricsContainer.metricValue(consumerMetricName(TOPIC_DELIVERY_RATE))) - .withThroughput(prometheusMetricsContainer.metricValue(frontendMetricName(TOPIC_THROUGHPUT_RATE))) + .withRate(prometheusMetricsContainer.metricValue(topicRateQuery)) + .withDeliveryRate(prometheusMetricsContainer.metricValue(topicDeliveryRateQuery)) + .withThroughput(prometheusMetricsContainer.metricValue(topicThroughputQuery)) .build(); } - private String consumerMetricName(String name) { return consumersMetricsPrefix + name; } diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/PrometheusResponse.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/PrometheusResponse.java index bcb3156dc7..74c1c29c9e 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/PrometheusResponse.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/PrometheusResponse.java @@ -22,7 +22,6 @@ boolean isVector() { @JsonIgnoreProperties(ignoreUnknown = true) record VectorResult( - @JsonProperty("metric") MetricName metricName, @JsonProperty("value") List vector) { private static final int VALID_VECTOR_LENGTH = 2; @@ -34,31 +33,6 @@ Optional getValue() { } return Optional.of(Double.parseDouble(vector.get(SCALAR_INDEX_VALUE))); } - - VectorResult renameMetric(String newMetricName) { - return new VectorResult(new MetricName(newMetricName, metricName.statusCode), vector); - } - } - - @JsonIgnoreProperties(ignoreUnknown = true) - record MetricName( - @JsonProperty(value = "__name__") String name, - @JsonProperty(value = "status_code") Optional statusCode) { - boolean is2xxStatusCode() { - return hasStatusCode() && statusCode.get().startsWith("2"); - } - - boolean is4xxStatusCode() { - return hasStatusCode() && statusCode.get().startsWith("4"); - } - - boolean is5xxStatusCode() { - return hasStatusCode() && statusCode.get().startsWith("5"); - } - - private boolean hasStatusCode() { - return statusCode.isPresent(); - } } } diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/RestTemplatePrometheusClient.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/RestTemplatePrometheusClient.java index 3d5139e17a..d10e09e425 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/RestTemplatePrometheusClient.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/RestTemplatePrometheusClient.java @@ -1,22 +1,25 @@ package pl.allegro.tech.hermes.management.infrastructure.prometheus; import com.google.common.base.Preconditions; +import io.micrometer.core.instrument.MeterRegistry; import org.apache.commons.lang3.tuple.Pair; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.http.HttpEntity; import org.springframework.http.HttpMethod; -import org.springframework.http.ResponseEntity; +import org.springframework.web.client.HttpStatusCodeException; import org.springframework.web.client.RestTemplate; import pl.allegro.tech.hermes.api.MetricDecimalValue; import pl.allegro.tech.hermes.management.infrastructure.metrics.MonitoringMetricsContainer; import java.net.URI; +import java.time.Duration; import java.util.List; import java.util.Map; -import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; -import java.util.stream.Stream; import static java.net.URLEncoder.encode; import static java.nio.charset.StandardCharsets.UTF_8; @@ -28,94 +31,90 @@ public class RestTemplatePrometheusClient implements PrometheusClient { private final URI prometheusUri; private final RestTemplate restTemplate; + private final ExecutorService executorService; + private final Duration fetchingTimeout; + private final MeterRegistry meterRegistry; - public RestTemplatePrometheusClient(RestTemplate restTemplate, URI prometheusUri) { + + public RestTemplatePrometheusClient(RestTemplate restTemplate, + URI prometheusUri, + ExecutorService executorService, + Duration fetchingTimeoutMillis, + MeterRegistry meterRegistry) { this.restTemplate = restTemplate; this.prometheusUri = prometheusUri; + this.executorService = executorService; + this.fetchingTimeout = fetchingTimeoutMillis; + this.meterRegistry = meterRegistry; } @Override - public MonitoringMetricsContainer readMetrics(String query) { - try { - PrometheusResponse response = queryPrometheus(query); - Preconditions.checkNotNull(response, "Prometheus response is null"); - Preconditions.checkState(response.isSuccess(), "Prometheus response does not contain valid data"); + public MonitoringMetricsContainer readMetrics(List queries) { + return fetchInParallelFromPrometheus(queries); + } + + private MonitoringMetricsContainer fetchInParallelFromPrometheus(List queries) { + CompletableFuture> aggregatedFuture = getAggregatedCompletableFuture(queries); - Map> metricsGroupedByName = groupMetricsByName(response); - return produceMetricsContainer(metricsGroupedByName); - } catch (Exception exception) { - logger.warn("Unable to read from Prometheus...", exception); + try { + Map metrics = aggregatedFuture.get(fetchingTimeout.toMillis(), TimeUnit.MILLISECONDS); + return MonitoringMetricsContainer.initialized(metrics); + } catch (InterruptedException e) { + // possibly let know the caller that the thread was interrupted + Thread.currentThread().interrupt(); + logger.warn("Prometheus fetching thread was interrupted...", e); + return MonitoringMetricsContainer.unavailable(); + } catch (Exception ex) { + logger.warn("Unexpected exception during fetching metrics from prometheus...", ex); return MonitoringMetricsContainer.unavailable(); } } - private PrometheusResponse queryPrometheus(String query) { - URI queryUri = URI.create(prometheusUri.toString() + "/api/v1/query?query=" + encode(query, UTF_8)); - - ResponseEntity response = restTemplate.exchange(queryUri, - HttpMethod.GET, HttpEntity.EMPTY, PrometheusResponse.class); - return response.getBody(); + private CompletableFuture> getAggregatedCompletableFuture(List queries) { + // has to be collected to run in parallel + List>> futures = queries.stream() + .map(this::readSingleMetric) + .toList(); + + return CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])) + .thenApply( + v -> futures.stream().map(CompletableFuture::join) + .collect(Collectors.toMap(Pair::getKey, Pair::getValue)) + ); } - private static Map> groupMetricsByName(PrometheusResponse response) { - return response.data().results().stream() - .map(RestTemplatePrometheusClient::renameStatusCodesMetricsNames) - .collect(Collectors.groupingBy(r -> r.metricName().name())); + private CompletableFuture> readSingleMetric(String query) { + return CompletableFuture.supplyAsync(() -> queryPrometheus(query), executorService); } - private static MonitoringMetricsContainer produceMetricsContainer( - Map> metricsGroupedByName) { - MonitoringMetricsContainer metricsContainer = MonitoringMetricsContainer.createEmpty(); - - Stream> metricsSummedByStatusCodeFamily = metricsGroupedByName.entrySet().stream() - .map(RestTemplatePrometheusClient::sumMetricsWithTheSameName); + private Pair queryPrometheus(String query) { + try { + URI queryUri = URI.create(prometheusUri.toString() + "/api/v1/query?query=" + encode(query, UTF_8)); + PrometheusResponse response = restTemplate.exchange(queryUri, + HttpMethod.GET, HttpEntity.EMPTY, PrometheusResponse.class).getBody(); - metricsSummedByStatusCodeFamily.forEach(pair -> metricsContainer.addMetricValue( - pair.getKey(), - MetricDecimalValue.of(pair.getValue().toString()))); - return metricsContainer; - } + Preconditions.checkNotNull(response, "Prometheus response is null"); + Preconditions.checkState(response.isSuccess(), "Prometheus response does not contain valid data"); - private static PrometheusResponse.VectorResult renameStatusCodesMetricsNames(PrometheusResponse.VectorResult r) { - /* - Renames any metric containing status_code tag to the _2xx/3xx/4xx/5xx> metric name. For example: - VectorResult( - metricName=MetricName( - name=hermes_consumers_subscription_http_status_codes_total, - statusCode=Optional[200]), - vector=[...] - ) - ----> - VectorResult( - metricName=MetricName( - name=hermes_consumers_subscription_http_status_codes_total_2xx, - statusCode=Optional[200]), - vector=[...] - ) - It allows then to sum metrics accordingly to the status code family. - */ - String suffix = ""; - if (r.metricName().is2xxStatusCode()) { - suffix = "_2xx"; - } else if (r.metricName().is4xxStatusCode()) { - suffix = "_4xx"; - } else if (r.metricName().is5xxStatusCode()) { - suffix = "_5xx"; + MetricDecimalValue result = parseResponse(response); + meterRegistry.counter("read-metric-from-prometheus.success").increment(); + return Pair.of(query, result); + } catch (HttpStatusCodeException ex) { + logger.warn("Unable to read from Prometheus. Query: {}, Status code: {}. Response body: {}", + query, ex.getStatusCode(), ex.getResponseBodyAsString(), ex); + return Pair.of(query, MetricDecimalValue.unavailable()); + } catch (Exception ex) { + logger.warn("Unable to read from Prometheus. Query: {}", query, ex); + meterRegistry.counter("read-metric-from-prometheus.error").increment(); + return Pair.of(query, MetricDecimalValue.unavailable()); } - return r.renameMetric(r.metricName().name() + suffix); } - /* - We have to sum some metrics on the client side because Prometheus does not support this kind of aggregation when using - query for multiple __name__ metrics. - */ - private static Pair sumMetricsWithTheSameName(Map.Entry> e) { - return Pair.of( - e.getKey(), - e.getValue().stream() - .map(PrometheusResponse.VectorResult::getValue) - .filter(Optional::isPresent) - .map(Optional::get) - .mapToDouble(d -> d).sum()); + private MetricDecimalValue parseResponse(PrometheusResponse response) { + return response.data().results().stream() + .findFirst() + .flatMap(PrometheusResponse.VectorResult::getValue) + .map(value -> MetricDecimalValue.of(value.toString())) + .orElse(MetricDecimalValue.defaultValue()); } } diff --git a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridPrometheusBasedSubscriptionMetricsRepositoryTest.groovy b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridPrometheusBasedSubscriptionMetricsRepositoryTest.groovy index 6c18b04c48..8ed2088bf3 100644 --- a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridPrometheusBasedSubscriptionMetricsRepositoryTest.groovy +++ b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridPrometheusBasedSubscriptionMetricsRepositoryTest.groovy @@ -7,7 +7,7 @@ import pl.allegro.tech.hermes.api.TopicName import pl.allegro.tech.hermes.infrastructure.zookeeper.ZookeeperPaths import pl.allegro.tech.hermes.management.domain.subscription.SubscriptionLagSource import pl.allegro.tech.hermes.management.infrastructure.prometheus.PrometheusClient -import pl.allegro.tech.hermes.management.infrastructure.prometheus.VictoriaMetricsMetricsProvider +import pl.allegro.tech.hermes.management.infrastructure.prometheus.PrometheusMetricsProvider import spock.lang.Specification import static pl.allegro.tech.hermes.api.MetricDecimalValue.of @@ -22,29 +22,36 @@ class HybridPrometheusBasedSubscriptionMetricsRepositoryTest extends Specificati private SubscriptionLagSource lagSource = new NoOpSubscriptionLagSource() - private VictoriaMetricsMetricsProvider prometheusMetricsProvider = new VictoriaMetricsMetricsProvider( + private PrometheusMetricsProvider prometheusMetricsProvider = new PrometheusMetricsProvider( client, "hermes_consumers", "hermes_frontend", "service=~'hermes'") private HybridSubscriptionMetricsRepository repository = new HybridSubscriptionMetricsRepository(prometheusMetricsProvider, summedSharedCounter, zookeeperPaths, lagSource) - private static final String query = "sum by (__name__, group, topic, subscription, status_code) " + - "(irate({__name__=~'hermes_consumers_subscription_delivered_total" + - "|hermes_consumers_subscription_timeouts_total" + - "|hermes_consumers_subscription_retries_total" + - "|hermes_consumers_subscription_throughput_bytes_total" + - "|hermes_consumers_subscription_other_errors_total" + - "|hermes_consumers_subscription_batches_total" + - "|hermes_consumers_subscription_http_status_codes_total', " + - "group='group', topic='topic', subscription='subscription', service=~'hermes'}[1m]) keep_metric_names)" + private final static String subscriptionQuery = "sum by (group, topic, subscription) (irate({__name__='hermes_consumers_subscription_%s_total', group='group', topic='topic', subscription='subscription', service=~'hermes'}[1m]))"; + private final static String deliveredQuery = String.format(subscriptionQuery, "delivered") + private final static String timeoutsQuery = String.format(subscriptionQuery, "timeouts") + private final static String retriesQuery = String.format(subscriptionQuery, "retries") + private final static String throughputQuery = String.format(subscriptionQuery, "throughput_bytes") + private final static String otherErrorsQuery = String.format(subscriptionQuery, "other_errors") + private final static String batchesQuery = String.format(subscriptionQuery, "batches") + // these queries are different as they contains additional status code filters + private final static String status2xxQuery = "sum by (group, topic, subscription) (irate({__name__='hermes_consumers_subscription_http_status_codes_total', group='group', topic='topic', subscription='subscription', status_code=~'2.*', service=~'hermes'}[1m]))" + private final static String status4xxQuery = "sum by (group, topic, subscription) (irate({__name__='hermes_consumers_subscription_http_status_codes_total', group='group', topic='topic', subscription='subscription', status_code=~'4.*', service=~'hermes'}[1m]))" + private final static String status5xxQuery = "sum by (group, topic, subscription) (irate({__name__='hermes_consumers_subscription_http_status_codes_total', group='group', topic='topic', subscription='subscription', status_code=~'5.*', service=~'hermes'}[1m]))" + + private static final List queries = List.of( + deliveredQuery, timeoutsQuery, retriesQuery, throughputQuery, otherErrorsQuery, batchesQuery, + status2xxQuery, status4xxQuery, status5xxQuery + ) def "should read subscription metrics from multiple places"() { given: - client.readMetrics(query) >> MonitoringMetricsContainer.createEmpty() - .addMetricValue("hermes_consumers_subscription_delivered_total", of('10')) - .addMetricValue("hermes_consumers_subscription_timeouts_total", of('100')) - .addMetricValue("hermes_consumers_subscription_retries_total", of('20')) - .addMetricValue("hermes_consumers_subscription_other_errors_total", of('1000')) + client.readMetrics(queries) >> MonitoringMetricsContainer.createEmpty() + .addMetricValue(deliveredQuery, of('10')) + .addMetricValue(timeoutsQuery, of('100')) + .addMetricValue(retriesQuery, of('20')) + .addMetricValue(otherErrorsQuery, of('1000')) summedSharedCounter.getValue('/hermes/groups/group/topics/topic/subscriptions/subscription/metrics/delivered') >> 100 summedSharedCounter.getValue('/hermes/groups/group/topics/topic/subscriptions/subscription/metrics/discarded') >> 1 summedSharedCounter.getValue('/hermes/groups/group/topics/topic/subscriptions/subscription/metrics/volume') >> 16 @@ -66,10 +73,10 @@ class HybridPrometheusBasedSubscriptionMetricsRepositoryTest extends Specificati def "should read subscription metrics for all http status codes"() { given: - client.readMetrics(query) >> MonitoringMetricsContainer.createEmpty() - .addMetricValue("hermes_consumers_subscription_http_status_codes_total_2xx", of('2')) - .addMetricValue("hermes_consumers_subscription_http_status_codes_total_4xx", of('4')) - .addMetricValue("hermes_consumers_subscription_http_status_codes_total_5xx", of('5')) + client.readMetrics(queries) >> MonitoringMetricsContainer.createEmpty() + .addMetricValue(status2xxQuery, of('2')) + .addMetricValue(status4xxQuery, of('4')) + .addMetricValue(status5xxQuery, of('5')) when: SubscriptionMetrics metrics = repository.loadMetrics(new TopicName('group', 'topic'), 'subscription') diff --git a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridPrometheusBasedTopicMetricsRepositoryTest.groovy b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridPrometheusBasedTopicMetricsRepositoryTest.groovy index c3dcad72ad..c1dcd3cc38 100644 --- a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridPrometheusBasedTopicMetricsRepositoryTest.groovy +++ b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridPrometheusBasedTopicMetricsRepositoryTest.groovy @@ -5,7 +5,7 @@ import pl.allegro.tech.hermes.api.TopicName import pl.allegro.tech.hermes.domain.subscription.SubscriptionRepository import pl.allegro.tech.hermes.infrastructure.zookeeper.ZookeeperPaths import pl.allegro.tech.hermes.management.infrastructure.prometheus.PrometheusClient -import pl.allegro.tech.hermes.management.infrastructure.prometheus.VictoriaMetricsMetricsProvider +import pl.allegro.tech.hermes.management.infrastructure.prometheus.PrometheusMetricsProvider import spock.lang.Specification import static pl.allegro.tech.hermes.api.MetricDecimalValue.of @@ -20,23 +20,24 @@ class HybridPrometheusBasedTopicMetricsRepositoryTest extends Specification { private SubscriptionRepository subscriptionRepository = Mock(SubscriptionRepository) - private VictoriaMetricsMetricsProvider prometheusMetricsProvider = new VictoriaMetricsMetricsProvider(client, + private PrometheusMetricsProvider prometheusMetricsProvider = new PrometheusMetricsProvider(client, "hermes_consumers", "hermes_frontend", "service='hermes'") private HybridTopicMetricsRepository repository = new HybridTopicMetricsRepository(prometheusMetricsProvider, summedSharedCounter, zookeeperPaths, subscriptionRepository) + private String topicRequestsQuery = "sum by (group, topic) (irate({__name__='hermes_frontend_topic_requests_total', group='group', topic='topic', service='hermes'}[1m]))" + private String topicDeliveredQuery = "sum by (group, topic) (irate({__name__='hermes_consumers_subscription_delivered_total', group='group', topic='topic', service='hermes'}[1m]))" + private String topicThroughputQuery = "sum by (group, topic) (irate({__name__='hermes_frontend_topic_throughput_bytes_total', group='group', topic='topic', service='hermes'}[1m]))" + def "should load metrics from graphite and zookeeper"() { given: - String query = "sum by (__name__, group, topic) (irate({__name__=~'hermes_frontend_topic_requests_total" + - "|hermes_consumers_subscription_delivered_total" + - "|hermes_frontend_topic_throughput_bytes_total', group='group', " + - "topic='topic', service='hermes'}[1m]) keep_metric_names)" + List queries = List.of(topicRequestsQuery, topicDeliveredQuery, topicThroughputQuery) TopicName topic = new TopicName('group', 'topic') - client.readMetrics(query) >> MonitoringMetricsContainer.createEmpty() - .addMetricValue("hermes_frontend_topic_requests_total", of('10')) - .addMetricValue("hermes_consumers_subscription_delivered_total", of('20')) + client.readMetrics(queries) >> MonitoringMetricsContainer.createEmpty() + .addMetricValue(topicRequestsQuery, of('10')) + .addMetricValue(topicDeliveredQuery, of('20')) summedSharedCounter.getValue('/hermes/groups/group/topics/topic/metrics/published') >> 100 summedSharedCounter.getValue('/hermes/groups/group/topics/topic/metrics/volume') >> 1024 subscriptionRepository.listSubscriptionNames(topic) >> ["subscription1", "subscription2"] diff --git a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/prometheus/CachingPrometheusClientTest.groovy b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/prometheus/CachingPrometheusClientTest.groovy index 1345607d0f..07d52f64a5 100644 --- a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/prometheus/CachingPrometheusClientTest.groovy +++ b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/prometheus/CachingPrometheusClientTest.groovy @@ -8,6 +8,7 @@ import spock.lang.Subject import java.time.Duration import static pl.allegro.tech.hermes.api.MetricDecimalValue.of +import static pl.allegro.tech.hermes.api.MetricDecimalValue.unavailable class CachingPrometheusClientTest extends Specification { static final CACHE_TTL_IN_SECONDS = 30 @@ -16,16 +17,18 @@ class CachingPrometheusClientTest extends Specification { def underlyingClient = Mock(PrometheusClient) def ticker = new FakeTicker() + def queries = List.of("query") @Subject def cachingClient = new CachingPrometheusClient(underlyingClient, ticker, CACHE_TTL_IN_SECONDS, CACHE_SIZE) def "should return metrics from the underlying client"() { given: - underlyingClient.readMetrics("someQuery") >> MonitoringMetricsContainer.initialized([metric_1: of("1"), metric_2: of("2")]) + underlyingClient.readMetrics(queries) >> MonitoringMetricsContainer.initialized( + ["metric_1": of("1"), "metric_2": of("2")]) when: - def metrics = cachingClient.readMetrics("someQuery") + def metrics = cachingClient.readMetrics(queries) then: metrics.metricValue("metric_1") == of("1") @@ -34,21 +37,42 @@ class CachingPrometheusClientTest extends Specification { def "should return metrics from cache while TTL has not expired"() { when: - cachingClient.readMetrics("someQuery") + cachingClient.readMetrics(queries) ticker.advance(CACHE_TTL.minusSeconds(1)) - cachingClient.readMetrics("someQuery") + cachingClient.readMetrics(queries) then: - 1 * underlyingClient.readMetrics("someQuery") >> MonitoringMetricsContainer.initialized([metric_1: of("1"), metric_2: of("2")]) + 1 * underlyingClient.readMetrics(queries) >> MonitoringMetricsContainer.initialized( + ["metric_1": of("1"), "metric_2": of("2")]) } def "should get metrics from the underlying client after TTL expires"() { when: - cachingClient.readMetrics("someQuery") + cachingClient.readMetrics(queries) ticker.advance(CACHE_TTL.plusSeconds(1)) - cachingClient.readMetrics("someQuery") + cachingClient.readMetrics(queries) then: - 2 * underlyingClient.readMetrics("someQuery") >> MonitoringMetricsContainer.initialized([metric_1: of("1"), metric_2: of("2")]) + 2 * underlyingClient.readMetrics(queries) >> MonitoringMetricsContainer.initialized( + ["metric_1": of("1"), "metric_2": of("2")]) + } + + def "should invalidate partially unavailable data and retry fetch on the next client metrics read"() { + when: + cachingClient.readMetrics(queries) + cachingClient.readMetrics(queries) + + then: + 2 * underlyingClient.readMetrics(queries) >> MonitoringMetricsContainer.initialized( + ["metric_1": unavailable(), "metric_2": of("2")]) + } + + def "should invalidate completely unavailable data and retry fetch on the next client metrics read"() { + when: + cachingClient.readMetrics(queries) + cachingClient.readMetrics(queries) + + then: + 2 * underlyingClient.readMetrics(queries) >> MonitoringMetricsContainer.unavailable() } } diff --git a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/prometheus/RestTemplatePrometheusClientTest.groovy b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/prometheus/RestTemplatePrometheusClientTest.groovy index 6f68f65075..5d8c8024dd 100644 --- a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/prometheus/RestTemplatePrometheusClientTest.groovy +++ b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/prometheus/RestTemplatePrometheusClientTest.groovy @@ -1,88 +1,204 @@ package pl.allegro.tech.hermes.management.infrastructure.prometheus + import com.github.tomakehurst.wiremock.client.WireMock import com.github.tomakehurst.wiremock.junit.WireMockRule +import io.micrometer.core.instrument.simple.SimpleMeterRegistry import jakarta.ws.rs.core.MediaType +import org.apache.hc.client5.http.classic.HttpClient +import org.apache.hc.client5.http.config.RequestConfig +import org.apache.hc.client5.http.impl.classic.HttpClientBuilder +import org.apache.hc.core5.util.Timeout import org.junit.Rule +import org.springframework.http.client.ClientHttpRequestFactory +import org.springframework.http.client.HttpComponentsClientHttpRequestFactory import org.springframework.web.client.RestTemplate import pl.allegro.tech.hermes.management.infrastructure.metrics.MonitoringMetricsContainer import pl.allegro.tech.hermes.test.helper.util.Ports import spock.lang.Specification import java.nio.charset.StandardCharsets +import java.time.Duration +import java.util.concurrent.ExecutorService +import java.util.concurrent.Executors import static com.github.tomakehurst.wiremock.client.WireMock.urlEqualTo import static com.github.tomakehurst.wiremock.core.WireMockConfiguration.wireMockConfig +import static pl.allegro.tech.hermes.api.MetricDecimalValue.defaultValue import static pl.allegro.tech.hermes.api.MetricDecimalValue.of +import static pl.allegro.tech.hermes.api.MetricDecimalValue.unavailable class RestTemplatePrometheusClientTest extends Specification { private static final int PROMETHEUS_HTTP_PORT = Ports.nextAvailable() - private static final String query = "sum by (__name__,group,topic,subscription,status_code)" + - "(irate({__name__=~'hermes_consumers_subscription_delivered_total" + - "|hermes_consumers_subscription_timeouts_total" + - "|hermes_consumers_subscription_retries_total" + - "|hermes_consumers_subscription_throughput_bytes_total" + - "|hermes_consumers_subscription_other_errors_total" + - "|hermes_consumers_subscription_batches_total" + - "|hermes_consumers_subscription_http_status_codes_total'," + - "group='pl.allegro.tech.hermes',topic='Monitor',subscription='consumer1'}[1m]) keep_metric_names)" + def subscriptionDeliveredQuery = "sum by (group, topic, subscription) (irate({__name__='hermes_consumers_subscription_delivered_total', group='pl.allegro.tech.hermes', topic='Monitor', subscription='consumer1' service=~'hermes'}[1m]))" + def subscriptionTimeoutsQuery = "sum by (group, topic, subscription) (irate({__name__='hermes_consumers_subscription_timeouts_total', group='pl.allegro.tech.hermes', topic='Monitor', subscription='consumer1', service=~'hermes'}[1m]))" + def subscriptionRetriesQuery = "sum by (group, topic, subscription) (irate({__name__='hermes_consumers_subscription_retries_total', group='pl.allegro.tech.hermes', topic='Monitor', subscription='consumer1', service=~'hermes'}[1m]))" + def subscriptionThroughputQuery = "sum by (group, topic, subscription) (irate({__name__='hermes_consumers_subscription_throughput_bytes_total', group='pl.allegro.tech.hermes', topic='Monitor', subscription='consumer1', service=~'hermes'}[1m]))" + def subscriptionErrorsQuery = "sum by (group, topic, subscription) (irate({__name__='hermes_consumers_subscription_other_errors_total', group='pl.allegro.tech.hermes', topic='Monitor', subscription='consumer1', service=~'hermes'}[1m]))" + def subscriptionBatchesQuery = "sum by (group, topic, subscription) (irate({__name__='hermes_consumers_subscription_batches_total', group='pl.allegro.tech.hermes', topic='Monitor', subscription='consumer1', service=~'hermes'}[1m]))" + def subscription2xxStatusCodesQuery = "sum by (group, topic, subscription) (irate({__name__='hermes_consumers_subscription_http_status_codes_total', group='pl.allegro.tech.hermes', topic='Monitor', subscription='consumer1', status_code=~'2.*', service=~'hermes'}[1m]))" + def subscription4xxStatusCodesQuery = "sum by (group, topic, subscription) (irate({__name__='hermes_consumers_subscription_http_status_codes_total', group='pl.allegro.tech.hermes', topic='Monitor', subscription='consumer1', status_code=~'4.*', service=~'hermes'}[1m]))" + def subscription5xxStatusCodesQuery = "sum by (group, topic, subscription) (irate({__name__='hermes_consumers_subscription_http_status_codes_total', group='pl.allegro.tech.hermes', topic='Monitor', subscription='consumer1', status_code=~'5.*', service=~'hermes'}[1m]))" + + def queries = List.of(subscriptionDeliveredQuery, subscriptionTimeoutsQuery, subscriptionRetriesQuery, subscriptionThroughputQuery, + subscriptionErrorsQuery, subscriptionBatchesQuery, subscription2xxStatusCodesQuery, subscription4xxStatusCodesQuery, + subscription5xxStatusCodesQuery + ) @Rule - WireMockRule wireMockRule = new WireMockRule( - wireMockConfig().port(PROMETHEUS_HTTP_PORT).usingFilesUnderClasspath("prometheus-stubs")) + WireMockRule wireMockServer = new WireMockRule( + wireMockConfig() + .port(PROMETHEUS_HTTP_PORT).usingFilesUnderClasspath("prometheus-stubs")) private RestTemplatePrometheusClient client void setup() { - RestTemplate restTemplate = new RestTemplate() - client = new RestTemplatePrometheusClient(restTemplate, URI.create("http://localhost:$PROMETHEUS_HTTP_PORT")) + ExecutorService executorService = Executors.newFixedThreadPool(10) + RestTemplate restTemplate = createRestTemplateWithTimeout(Duration.ofSeconds(1)) + client = new RestTemplatePrometheusClient(restTemplate, URI.create("http://localhost:$PROMETHEUS_HTTP_PORT"), + executorService, Duration.ofSeconds(5), new SimpleMeterRegistry()) + wireMockServer.resetAll() } def "should get metrics for path"() { given: - mockPrometheus(query, "full_response.json") + def queriesStubs = List.of( + new FileStub(subscriptionDeliveredQuery, "subscription_delivered_total.json"), + new FileStub(subscriptionTimeoutsQuery, "subscription_timeouts_total.json"), + new FileStub(subscriptionRetriesQuery, "subscription_retries_total.json"), + new FileStub(subscriptionThroughputQuery, "subscription_throughput_bytes_total.json"), + new FileStub(subscriptionErrorsQuery, "subscription_other_errors_total.json"), + new FileStub(subscriptionBatchesQuery, "subscription_batches_total.json"), + new FileStub(subscription2xxStatusCodesQuery, "subscription_2xx_http_status_codes_total.json"), + new FileStub(subscription4xxStatusCodesQuery, "subscription_4xx_http_status_codes_total.json"), + new FileStub(subscription5xxStatusCodesQuery, "subscription_5xx_http_status_codes_total.json"), + ) + mockPrometheus(queriesStubs) when: - MonitoringMetricsContainer metrics = client.readMetrics(query) + MonitoringMetricsContainer metrics = client.readMetrics(queries) then: - metrics.metricValue("hermes_consumers_subscription_delivered_total") == of("1.0") - metrics.metricValue("hermes_consumers_subscription_timeouts_total") == of("2.0") - metrics.metricValue("hermes_consumers_subscription_retries_total") == of("1.0") - metrics.metricValue("hermes_consumers_subscription_throughput_bytes_total") == of("3.0") - metrics.metricValue("hermes_consumers_subscription_other_errors_total") == of("4.0") - metrics.metricValue("hermes_consumers_subscription_batches_total") == of("5.0") - metrics.metricValue("hermes_consumers_subscription_http_status_codes_total_2xx") == of("2.0") - metrics.metricValue("hermes_consumers_subscription_http_status_codes_total_4xx") == of("1.0") - metrics.metricValue("hermes_consumers_subscription_http_status_codes_total_5xx") == of("2.0") + metrics.metricValue(subscriptionDeliveredQuery) == of("1.0") + metrics.metricValue(subscriptionTimeoutsQuery) == of("2.0") + metrics.metricValue(subscriptionRetriesQuery) == of("1.0") + metrics.metricValue(subscriptionThroughputQuery) == of("3.0") + metrics.metricValue(subscriptionErrorsQuery) == of("4.0") + metrics.metricValue(subscriptionBatchesQuery) == of("5.0") + metrics.metricValue(subscription2xxStatusCodesQuery) == of("2.0") + metrics.metricValue(subscription4xxStatusCodesQuery) == of("1.0") + metrics.metricValue(subscription5xxStatusCodesQuery) == of("2.0") } def "should return default value when metric has no value"() { given: - mockPrometheus(query, "partial_response.json") + def queriesStubs = List.of( + emptyStub(subscriptionDeliveredQuery), + new FileStub(subscriptionTimeoutsQuery, "subscription_timeouts_total.json"), + new FileStub(subscriptionRetriesQuery, "subscription_retries_total.json"), + emptyStub(subscriptionThroughputQuery), + emptyStub(subscriptionErrorsQuery), + emptyStub(subscriptionBatchesQuery), + emptyStub(subscription2xxStatusCodesQuery), + emptyStub(subscription4xxStatusCodesQuery), + emptyStub(subscription5xxStatusCodesQuery) + ) + mockPrometheus(queriesStubs) + + when: + MonitoringMetricsContainer metrics = client.readMetrics(queries) + + then: + metrics.metricValue(subscriptionDeliveredQuery) == defaultValue() + metrics.metricValue(subscriptionTimeoutsQuery) == of("2.0") + metrics.metricValue(subscriptionRetriesQuery) == of("1.0") + metrics.metricValue(subscriptionThroughputQuery) == defaultValue() + metrics.metricValue(subscriptionErrorsQuery) == defaultValue() + metrics.metricValue(subscriptionBatchesQuery) == defaultValue() + metrics.metricValue(subscription2xxStatusCodesQuery) == defaultValue() + metrics.metricValue(subscription4xxStatusCodesQuery) == defaultValue() + metrics.metricValue(subscription5xxStatusCodesQuery) == defaultValue() + } + + def "should return partial results when some of the requests fails"() { + given: + def queriesToFail = List.of( + subscriptionDeliveredQuery, + subscriptionThroughputQuery, + subscriptionErrorsQuery, + subscriptionBatchesQuery, + subscription2xxStatusCodesQuery, + subscription4xxStatusCodesQuery, + subscription5xxStatusCodesQuery, + ) + def queriesToSuccess = List.of( + new FileStub(subscriptionTimeoutsQuery, "subscription_timeouts_total.json"), + new FileStub(subscriptionRetriesQuery, "subscription_retries_total.json"), + ) + mockPrometheus(queriesToSuccess) + mockPrometheusTimeout(queriesToFail, Duration.ofSeconds(5)) when: - MonitoringMetricsContainer metrics = client.readMetrics(query) + MonitoringMetricsContainer metrics = client.readMetrics(queries) then: - metrics.metricValue("hermes_consumers_subscription_delivered_total") == of("0.0") - metrics.metricValue("hermes_consumers_subscription_timeouts_total") == of("2.0") - metrics.metricValue("hermes_consumers_subscription_retries_total") == of("1.0") - metrics.metricValue("hermes_consumers_subscription_throughput_bytes_total") == of("3.0") - metrics.metricValue("hermes_consumers_subscription_other_errors_total") == of("4.0") - metrics.metricValue("hermes_consumers_subscription_batches_total") == of("5.0") - metrics.metricValue("hermes_consumers_subscription_http_status_codes_total_2xx") == of("2.0") - metrics.metricValue("hermes_consumers_subscription_http_status_codes_total_4xx") == of("1.0") - metrics.metricValue("hermes_consumers_subscription_http_status_codes_total_5xx") == of("0.0") + metrics.metricValue(subscriptionDeliveredQuery) == unavailable() + metrics.metricValue(subscriptionTimeoutsQuery) == of("2.0") + metrics.metricValue(subscriptionRetriesQuery) == of("1.0") + metrics.metricValue(subscriptionThroughputQuery) == unavailable() + metrics.metricValue(subscriptionErrorsQuery) == unavailable() + metrics.metricValue(subscriptionBatchesQuery) == unavailable() + metrics.metricValue(subscription2xxStatusCodesQuery) == unavailable() + metrics.metricValue(subscription4xxStatusCodesQuery) == unavailable() + metrics.metricValue(subscription5xxStatusCodesQuery) == unavailable() + } + + private void mockPrometheus(List stubs) { + stubs.forEach { s -> + String encodedQuery = URLEncoder.encode(s.query, StandardCharsets.UTF_8) + wireMockServer.stubFor(WireMock.get(urlEqualTo(String.format("/api/v1/query?query=%s", encodedQuery))) + .willReturn(WireMock.aResponse() + .withStatus(200) + .withHeader("Content-Type", MediaType.APPLICATION_JSON) + .withBodyFile(s.fileName))) + } + } + + private void mockPrometheusTimeout(List queries, Duration delay) { + queries.forEach { q -> + String encodedQuery = URLEncoder.encode(q, StandardCharsets.UTF_8) + wireMockServer.stubFor(WireMock.get(urlEqualTo(String.format("/api/v1/query?query=%s", encodedQuery))) + .willReturn(WireMock.aResponse() + .withHeader("Content-Type", MediaType.APPLICATION_JSON) + .withFixedDelay(delay.toMillis() as Integer))); + } + } + + private RestTemplate createRestTemplateWithTimeout(Duration timeout) { + RequestConfig requestConfig = RequestConfig.custom() + .setConnectTimeout(Timeout.ofMilliseconds(timeout.toMillis())) + .setResponseTimeout(Timeout.ofMilliseconds(timeout.toMillis())) + .build(); + + HttpClient client = HttpClientBuilder.create() + .setDefaultRequestConfig(requestConfig) + .build(); + + ClientHttpRequestFactory clientHttpRequestFactory = new HttpComponentsClientHttpRequestFactory(client); + return new RestTemplate(clientHttpRequestFactory); + } + + static class FileStub { + FileStub(String query, String fileName) { + this.query = query + this.fileName = fileName + } + String query; + String fileName } - private void mockPrometheus(String query, String responseFile) { - String encodedQuery = URLEncoder.encode(query, StandardCharsets.UTF_8) - WireMock.stubFor(WireMock.get(urlEqualTo(String.format("/api/v1/query?query=%s", encodedQuery))) - .willReturn(WireMock.aResponse() - .withStatus(200) - .withHeader("Content-Type", MediaType.APPLICATION_JSON) - .withBodyFile(responseFile))) + FileStub emptyStub(String query) { + return new FileStub(query, "prometheus_empty_response.json") } } diff --git a/hermes-management/src/test/resources/prometheus-stubs/__files/full_response.json b/hermes-management/src/test/resources/prometheus-stubs/__files/full_response.json deleted file mode 100644 index 7f813819d0..0000000000 --- a/hermes-management/src/test/resources/prometheus-stubs/__files/full_response.json +++ /dev/null @@ -1,156 +0,0 @@ -{ - "status": "success", - "data": { - "resultType": "vector", - "result": [ - { - "metric": { - "__name__": "hermes_consumers_subscription_delivered_total", - "group": "pl.allegro.tech.hermes", - "subscription": "hermesSubscription", - "topic": "hermesTopic" - }, - "value": [ - 1692281425.609, - "1" - ], - "group": 1 - }, - { - "metric": { - "__name__": "hermes_consumers_subscription_timeouts_total", - "group": "pl.allegro.tech.hermes", - "subscription": "hermesSubscription", - "topic": "hermesTopic" - }, - "value": [ - 1692281425.609, - "2" - ], - "group": 1 - }, - { - "metric": { - "__name__": "hermes_consumers_subscription_retries_total", - "group": "pl.allegro.tech.hermes", - "subscription": "hermesSubscription", - "topic": "hermesTopic" - }, - "value": [ - 1692281425.609, - "1" - ], - "group": 1 - }, - { - "metric": { - "__name__": "hermes_consumers_subscription_throughput_bytes_total", - "group": "pl.allegro.tech.hermes", - "subscription": "hermesSubscription", - "topic": "hermesTopic" - }, - "value": [ - 1692281425.609, - "3" - ], - "group": 1 - }, - { - "metric": { - "__name__": "hermes_consumers_subscription_other_errors_total", - "group": "pl.allegro.tech.hermes", - "subscription": "hermesSubscription", - "topic": "hermesTopic" - }, - "value": [ - 1692281425.609, - "4" - ], - "group": 1 - }, - { - "metric": { - "__name__": "hermes_consumers_subscription_batches_total", - "group": "pl.allegro.tech.hermes", - "subscription": "hermesSubscription", - "topic": "hermesTopic" - }, - "value": [ - 1692281425.609, - "5" - ], - "group": 1 - }, - { - "metric": { - "__name__": "hermes_consumers_subscription_http_status_codes_total", - "status_code": "200", - "group": "pl.allegro.tech.hermes", - "subscription": "hermesSubscription", - "topic": "hermesTopic" - }, - "value": [ - 1692281425.609, - "1" - ], - "group": 1 - }, - { - "metric": { - "__name__": "hermes_consumers_subscription_http_status_codes_total", - "status_code": "201", - "group": "pl.allegro.tech.hermes", - "subscription": "hermesSubscription", - "topic": "hermesTopic" - }, - "value": [ - 1692281425.609, - "1" - ], - "group": 1 - }, - { - "metric": { - "__name__": "hermes_consumers_subscription_http_status_codes_total", - "status_code": "401", - "group": "pl.allegro.tech.hermes", - "subscription": "hermesSubscription", - "topic": "hermesTopic" - }, - "value": [ - 1692281425.609, - "1" - ], - "group": 1 - }, - { - "metric": { - "__name__": "hermes_consumers_subscription_http_status_codes_total", - "status_code": "503", - "group": "pl.allegro.tech.hermes", - "subscription": "hermesSubscription", - "topic": "hermesTopic" - }, - "value": [ - 1692281425.609, - "1" - ], - "group": 1 - }, - { - "metric": { - "__name__": "hermes_consumers_subscription_http_status_codes_total", - "status_code": "500", - "group": "pl.allegro.tech.hermes", - "subscription": "hermesSubscription", - "topic": "hermesTopic" - }, - "value": [ - 1692281425.609, - "1" - ], - "group": 1 - } - ] - } -} \ No newline at end of file diff --git a/hermes-management/src/test/resources/prometheus-stubs/__files/partial_response.json b/hermes-management/src/test/resources/prometheus-stubs/__files/partial_response.json deleted file mode 100644 index a2da55c2b5..0000000000 --- a/hermes-management/src/test/resources/prometheus-stubs/__files/partial_response.json +++ /dev/null @@ -1,115 +0,0 @@ -{ - "status": "success", - "data": { - "resultType": "vector", - "result": [ - { - "metric": { - "__name__": "hermes_consumers_subscription_timeouts_total", - "group": "pl.allegro.tech.hermes", - "subscription": "hermesSubscription", - "topic": "hermesTopic" - }, - "value": [ - 1692281425.609, - "2" - ], - "group": 1 - }, - { - "metric": { - "__name__": "hermes_consumers_subscription_retries_total", - "group": "pl.allegro.tech.hermes", - "subscription": "hermesSubscription", - "topic": "hermesTopic" - }, - "value": [ - 1692281425.609, - "1" - ], - "group": 1 - }, - { - "metric": { - "__name__": "hermes_consumers_subscription_throughput_bytes_total", - "group": "pl.allegro.tech.hermes", - "subscription": "hermesSubscription", - "topic": "hermesTopic" - }, - "value": [ - 1692281425.609, - "3" - ], - "group": 1 - }, - { - "metric": { - "__name__": "hermes_consumers_subscription_other_errors_total", - "group": "pl.allegro.tech.hermes", - "subscription": "hermesSubscription", - "topic": "hermesTopic" - }, - "value": [ - 1692281425.609, - "4" - ], - "group": 1 - }, - { - "metric": { - "__name__": "hermes_consumers_subscription_batches_total", - "group": "pl.allegro.tech.hermes", - "subscription": "hermesSubscription", - "topic": "hermesTopic" - }, - "value": [ - 1692281425.609, - "5" - ], - "group": 1 - }, - { - "metric": { - "__name__": "hermes_consumers_subscription_http_status_codes_total", - "status_code": "200", - "group": "pl.allegro.tech.hermes", - "subscription": "hermesSubscription", - "topic": "hermesTopic" - }, - "value": [ - 1692281425.609, - "1" - ], - "group": 1 - }, - { - "metric": { - "__name__": "hermes_consumers_subscription_http_status_codes_total", - "status_code": "201", - "group": "pl.allegro.tech.hermes", - "subscription": "hermesSubscription", - "topic": "hermesTopic" - }, - "value": [ - 1692281425.609, - "1" - ], - "group": 1 - }, - { - "metric": { - "__name__": "hermes_consumers_subscription_http_status_codes_total", - "status_code": "401", - "group": "pl.allegro.tech.hermes", - "subscription": "hermesSubscription", - "topic": "hermesTopic" - }, - "value": [ - 1692281425.609, - "1" - ], - "group": 1 - } - ] - } -} \ No newline at end of file diff --git a/hermes-management/src/test/resources/prometheus-stubs/__files/prometheus_empty_response.json b/hermes-management/src/test/resources/prometheus-stubs/__files/prometheus_empty_response.json new file mode 100644 index 0000000000..ef28156d55 --- /dev/null +++ b/hermes-management/src/test/resources/prometheus-stubs/__files/prometheus_empty_response.json @@ -0,0 +1,7 @@ +{ + "status": "success", + "data": { + "resultType": "vector", + "result": [] + } +} diff --git a/hermes-management/src/test/resources/prometheus-stubs/__files/subscription_2xx_http_status_codes_total.json b/hermes-management/src/test/resources/prometheus-stubs/__files/subscription_2xx_http_status_codes_total.json new file mode 100644 index 0000000000..7a0da2287c --- /dev/null +++ b/hermes-management/src/test/resources/prometheus-stubs/__files/subscription_2xx_http_status_codes_total.json @@ -0,0 +1,19 @@ +{ + "status": "success", + "data": { + "resultType": "vector", + "result": [ + { + "metric": { + "group": "pl.allegro.tech.hermes", + "subscription": "hermesSubscription", + "topic": "hermesTopic" + }, + "value": [ + 1692281425.609, + "2" + ] + } + ] + } +} diff --git a/hermes-management/src/test/resources/prometheus-stubs/__files/subscription_4xx_http_status_codes_total.json b/hermes-management/src/test/resources/prometheus-stubs/__files/subscription_4xx_http_status_codes_total.json new file mode 100644 index 0000000000..b4ae98b299 --- /dev/null +++ b/hermes-management/src/test/resources/prometheus-stubs/__files/subscription_4xx_http_status_codes_total.json @@ -0,0 +1,19 @@ +{ + "status": "success", + "data": { + "resultType": "vector", + "result": [ + { + "metric": { + "group": "pl.allegro.tech.hermes", + "subscription": "hermesSubscription", + "topic": "hermesTopic" + }, + "value": [ + 1692281425.609, + "1" + ] + } + ] + } +} diff --git a/hermes-management/src/test/resources/prometheus-stubs/__files/subscription_5xx_http_status_codes_total.json b/hermes-management/src/test/resources/prometheus-stubs/__files/subscription_5xx_http_status_codes_total.json new file mode 100644 index 0000000000..7a0da2287c --- /dev/null +++ b/hermes-management/src/test/resources/prometheus-stubs/__files/subscription_5xx_http_status_codes_total.json @@ -0,0 +1,19 @@ +{ + "status": "success", + "data": { + "resultType": "vector", + "result": [ + { + "metric": { + "group": "pl.allegro.tech.hermes", + "subscription": "hermesSubscription", + "topic": "hermesTopic" + }, + "value": [ + 1692281425.609, + "2" + ] + } + ] + } +} diff --git a/hermes-management/src/test/resources/prometheus-stubs/__files/subscription_batches_total.json b/hermes-management/src/test/resources/prometheus-stubs/__files/subscription_batches_total.json new file mode 100644 index 0000000000..5a460603df --- /dev/null +++ b/hermes-management/src/test/resources/prometheus-stubs/__files/subscription_batches_total.json @@ -0,0 +1,19 @@ +{ + "status": "success", + "data": { + "resultType": "vector", + "result": [ + { + "metric": { + "group": "pl.allegro.tech.hermes", + "subscription": "hermesSubscription", + "topic": "hermesTopic" + }, + "value": [ + 1692281425.609, + "5" + ] + } + ] + } +} diff --git a/hermes-management/src/test/resources/prometheus-stubs/__files/subscription_delivered_total.json b/hermes-management/src/test/resources/prometheus-stubs/__files/subscription_delivered_total.json new file mode 100644 index 0000000000..b4ae98b299 --- /dev/null +++ b/hermes-management/src/test/resources/prometheus-stubs/__files/subscription_delivered_total.json @@ -0,0 +1,19 @@ +{ + "status": "success", + "data": { + "resultType": "vector", + "result": [ + { + "metric": { + "group": "pl.allegro.tech.hermes", + "subscription": "hermesSubscription", + "topic": "hermesTopic" + }, + "value": [ + 1692281425.609, + "1" + ] + } + ] + } +} diff --git a/hermes-management/src/test/resources/prometheus-stubs/__files/subscription_other_errors_total.json b/hermes-management/src/test/resources/prometheus-stubs/__files/subscription_other_errors_total.json new file mode 100644 index 0000000000..ac0dc498bb --- /dev/null +++ b/hermes-management/src/test/resources/prometheus-stubs/__files/subscription_other_errors_total.json @@ -0,0 +1,19 @@ +{ + "status": "success", + "data": { + "resultType": "vector", + "result": [ + { + "metric": { + "group": "pl.allegro.tech.hermes", + "subscription": "hermesSubscription", + "topic": "hermesTopic" + }, + "value": [ + 1692281425.609, + "4" + ] + } + ] + } +} diff --git a/hermes-management/src/test/resources/prometheus-stubs/__files/subscription_retries_total.json b/hermes-management/src/test/resources/prometheus-stubs/__files/subscription_retries_total.json new file mode 100644 index 0000000000..b4ae98b299 --- /dev/null +++ b/hermes-management/src/test/resources/prometheus-stubs/__files/subscription_retries_total.json @@ -0,0 +1,19 @@ +{ + "status": "success", + "data": { + "resultType": "vector", + "result": [ + { + "metric": { + "group": "pl.allegro.tech.hermes", + "subscription": "hermesSubscription", + "topic": "hermesTopic" + }, + "value": [ + 1692281425.609, + "1" + ] + } + ] + } +} diff --git a/hermes-management/src/test/resources/prometheus-stubs/__files/subscription_throughput_bytes_total.json b/hermes-management/src/test/resources/prometheus-stubs/__files/subscription_throughput_bytes_total.json new file mode 100644 index 0000000000..1b944f3d6b --- /dev/null +++ b/hermes-management/src/test/resources/prometheus-stubs/__files/subscription_throughput_bytes_total.json @@ -0,0 +1,19 @@ +{ + "status": "success", + "data": { + "resultType": "vector", + "result": [ + { + "metric": { + "group": "pl.allegro.tech.hermes", + "subscription": "hermesSubscription", + "topic": "hermesTopic" + }, + "value": [ + 1692281425.609, + "3" + ] + } + ] + } +} diff --git a/hermes-management/src/test/resources/prometheus-stubs/__files/subscription_timeouts_total.json b/hermes-management/src/test/resources/prometheus-stubs/__files/subscription_timeouts_total.json new file mode 100644 index 0000000000..7a0da2287c --- /dev/null +++ b/hermes-management/src/test/resources/prometheus-stubs/__files/subscription_timeouts_total.json @@ -0,0 +1,19 @@ +{ + "status": "success", + "data": { + "resultType": "vector", + "result": [ + { + "metric": { + "group": "pl.allegro.tech.hermes", + "subscription": "hermesSubscription", + "topic": "hermesTopic" + }, + "value": [ + 1692281425.609, + "2" + ] + } + ] + } +} diff --git a/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/prometheus/PrometheusExtension.java b/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/prometheus/PrometheusExtension.java index 8daf23be19..b39b5c3f48 100644 --- a/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/prometheus/PrometheusExtension.java +++ b/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/prometheus/PrometheusExtension.java @@ -12,12 +12,14 @@ import java.time.Duration; import java.util.List; -import java.util.stream.Collectors; import static com.github.tomakehurst.wiremock.client.WireMock.aResponse; import static com.github.tomakehurst.wiremock.client.WireMock.equalTo; import static com.github.tomakehurst.wiremock.client.WireMock.get; import static com.github.tomakehurst.wiremock.client.WireMock.urlPathEqualTo; +import static pl.allegro.tech.hermes.management.infrastructure.prometheus.PrometheusClient.forSubscription; +import static pl.allegro.tech.hermes.management.infrastructure.prometheus.PrometheusClient.forSubscriptionStatusCode; +import static pl.allegro.tech.hermes.management.infrastructure.prometheus.PrometheusClient.forTopic; public class PrometheusExtension implements AfterEachCallback, BeforeAllCallback, ExtensionContext.Store.CloseableResource { @@ -47,19 +49,31 @@ public String getEndpoint() { return "http://localhost:" + wiremock.port(); } + @SuppressWarnings("checkstyle:VariableDeclarationUsageDistance") public void stubSubscriptionMetrics(SubscriptionMetrics metrics) { - SubscriptionName name = metrics.name(); - String query = """ - sum by (__name__, group, topic, subscription, status_code) ( - irate( - {__name__=~'hermes_consumers_subscription_delivered_total|hermes_consumers_subscription_timeouts_total|hermes_consumers_subscription_retries_total|hermes_consumers_subscription_throughput_bytes_total|hermes_consumers_subscription_other_errors_total|hermes_consumers_subscription_batches_total|hermes_consumers_subscription_http_status_codes_total', group='%s', topic='%s', subscription='%s', }[1m] - ) keep_metric_names - ) - """ - .formatted(name.getTopicName().getGroupName(), name.getTopicName().getName(), name.getName()) - .lines() - .map(String::stripLeading) - .collect(Collectors.joining()); + SubscriptionName subName = metrics.name(); + String deliveredQuery = forSubscription("hermes_consumers_subscription_delivered_total", subName, ""); + String timeoutsQuery = forSubscription("hermes_consumers_subscription_timeouts_total", subName, ""); + String retriesQuery = forSubscription("hermes_consumers_subscription_retries_total", subName, ""); + String throughputQuery = forSubscription("hermes_consumers_subscription_throughput_bytes_total", subName, ""); + String errorsQuery = forSubscription("hermes_consumers_subscription_other_errors_total", subName, ""); + String batchesQuery = forSubscription("hermes_consumers_subscription_batches_total", subName, ""); + String statusCodes2xxQuery = forSubscriptionStatusCode("hermes_consumers_subscription_http_status_codes_total", subName, "2.*", ""); + String statusCodes4xxQuery = forSubscriptionStatusCode("hermes_consumers_subscription_http_status_codes_total", subName, "4.*", ""); + String statusCodes5xxQuery = forSubscriptionStatusCode("hermes_consumers_subscription_http_status_codes_total", subName, "5.*", ""); + + stub(deliveredQuery, metrics.toPrometheusRateResponse()); + stub(timeoutsQuery, metrics.toPrometheusDefaultResponse()); + stub(retriesQuery, metrics.toPrometheusDefaultResponse()); + stub(throughputQuery, metrics.toPrometheusThroughputResponse()); + stub(errorsQuery, metrics.toPrometheusDefaultResponse()); + stub(batchesQuery, metrics.toPrometheusDefaultResponse()); + stub(statusCodes2xxQuery, metrics.toPrometheusStatusCodesResponse()); + stub(statusCodes4xxQuery, metrics.toPrometheusStatusCodesResponse()); + stub(statusCodes5xxQuery, metrics.toPrometheusStatusCodesResponse()); + } + + private void stub(String query, PrometheusResponse response) { wiremock.addStubMapping( get(urlPathEqualTo("/api/v1/query")) .withQueryParam("query", equalTo(query)) @@ -67,36 +81,21 @@ sum by (__name__, group, topic, subscription, status_code) ( aResponse() .withStatus(200) .withHeader("Content-Type", "application/json") - .withBody(writeValueAsString(metrics.toPrometheusResponse())) + .withBody(writeValueAsString(response)) ) .build() ); } public void stubTopicMetrics(TopicMetrics metrics) { - TopicName name = metrics.name(); - String query = """ - sum by (__name__, group, topic) ( - irate( - {__name__=~'hermes_frontend_topic_requests_total|hermes_consumers_subscription_delivered_total|hermes_frontend_topic_throughput_bytes_total', group='%s', topic='%s', }[1m] - ) keep_metric_names - ) - """ - .formatted(name.getGroupName(), name.getName()) - .lines() - .map(String::stripLeading) - .collect(Collectors.joining()); - wiremock.addStubMapping( - get(urlPathEqualTo("/api/v1/query")) - .withQueryParam("query", equalTo(query)) - .willReturn( - aResponse() - .withStatus(200) - .withHeader("Content-Type", "application/json") - .withBody(writeValueAsString(metrics.toPrometheusResponse())) - ) - .build() - ); + TopicName topicName = metrics.name(); + String requestsQuery = forTopic("hermes_frontend_topic_requests_total", topicName, ""); + String deliveredQuery = forTopic("hermes_consumers_subscription_delivered_total", topicName, ""); + String throughputQuery = forTopic("hermes_frontend_topic_throughput_bytes_total", topicName, ""); + + stub(requestsQuery, metrics.toPrometheusRequestsResponse()); + stub(deliveredQuery, metrics.toDeliveredResponse()); + stub(throughputQuery, metrics.toPrometheusThroughputResponse()); } public void stubDelay(Duration duration) { diff --git a/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/prometheus/PrometheusResponse.java b/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/prometheus/PrometheusResponse.java index f6aa84e197..df4bd8757d 100644 --- a/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/prometheus/PrometheusResponse.java +++ b/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/prometheus/PrometheusResponse.java @@ -1,6 +1,5 @@ package pl.allegro.tech.hermes.integrationtests.prometheus; -import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; @@ -12,12 +11,6 @@ record Data(@JsonProperty("resultType") String resultType, @JsonProperty("result") List results) { } - record Result(@JsonProperty("metric") MetricName metricName, - @JsonProperty("value") List values) { - } - - @JsonInclude(JsonInclude.Include.NON_NULL) - record MetricName(@JsonProperty(value = "__name__") String name, - @JsonProperty(value = "status_code") String statusCode) { + record Result(@JsonProperty("value") List values) { } } diff --git a/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/prometheus/SubscriptionMetrics.java b/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/prometheus/SubscriptionMetrics.java index fbe2efbd0c..438def3702 100644 --- a/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/prometheus/SubscriptionMetrics.java +++ b/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/prometheus/SubscriptionMetrics.java @@ -7,33 +7,38 @@ import java.util.List; import java.util.Map; -public record SubscriptionMetrics(SubscriptionName name, int rate, int throughput, Map ratePerStatusCode) { +public record SubscriptionMetrics(SubscriptionName name, int rate, int throughput, + Map ratePerStatusCode) { private static final String TIMESTAMP = "1396860420"; - private static final String SUBSCRIPTION_DELIVERED = "hermes_consumers_subscription_delivered_total"; - private static final String SUBSCRIPTION_THROUGHPUT = "hermes_consumers_subscription_throughput_bytes_total"; - private static final String SUBSCRIPTION_STATUS_CODES = "hermes_consumers_subscription_http_status_codes_total"; public static SubscriptionMetricsBuilder subscriptionMetrics(SubscriptionName name) { return new SubscriptionMetricsBuilder(name); } - PrometheusResponse toPrometheusResponse() { + PrometheusResponse toPrometheusRateResponse() { List results = new ArrayList<>(); results.add( new PrometheusResponse.Result( - new PrometheusResponse.MetricName(SUBSCRIPTION_DELIVERED, null), List.of(TIMESTAMP, String.valueOf(rate))) ); + return new PrometheusResponse("success", new PrometheusResponse.Data("vector", results)); + } + + PrometheusResponse toPrometheusThroughputResponse() { + List results = new ArrayList<>(); results.add( new PrometheusResponse.Result( - new PrometheusResponse.MetricName(SUBSCRIPTION_THROUGHPUT, null), List.of(TIMESTAMP, String.valueOf(throughput)) ) ); + return new PrometheusResponse("success", new PrometheusResponse.Data("vector", results)); + } + + PrometheusResponse toPrometheusStatusCodesResponse() { + List results = new ArrayList<>(); ratePerStatusCode.forEach((code, rate) -> results.add( new PrometheusResponse.Result( - new PrometheusResponse.MetricName(SUBSCRIPTION_STATUS_CODES, code), List.of(TIMESTAMP, String.valueOf(rate)) ) ) @@ -41,6 +46,17 @@ PrometheusResponse toPrometheusResponse() { return new PrometheusResponse("success", new PrometheusResponse.Data("vector", results)); } + PrometheusResponse toPrometheusDefaultResponse() { + List results = new ArrayList<>(); + ratePerStatusCode.forEach((code, rate) -> results.add( + new PrometheusResponse.Result( + List.of(TIMESTAMP, "0.0") + ) + ) + ); + return new PrometheusResponse("success", new PrometheusResponse.Data("vector", results)); + } + public static class SubscriptionMetricsBuilder { private final SubscriptionName name; private int rate = 0; diff --git a/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/prometheus/TopicMetrics.java b/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/prometheus/TopicMetrics.java index 06ec44cf6b..3481e953ea 100644 --- a/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/prometheus/TopicMetrics.java +++ b/integration-tests/src/common/java/pl/allegro/tech/hermes/integrationtests/prometheus/TopicMetrics.java @@ -7,30 +7,46 @@ public record TopicMetrics(TopicName name, int rate, int deliveryRate, int throughput) { private static final String TIMESTAMP = "1396860420"; - private static final String TOPIC_REQUESTS_TOTAL = "hermes_frontend_topic_requests_total"; - private static final String TOPIC_DELIVERED_TOTAL = "hermes_consumers_subscription_delivered_total"; - private static final String TOPIC_THROUGHPUT_TOTAL = "hermes_frontend_topic_throughput_bytes_total"; public static TopicMetricsBuilder topicMetrics(TopicName name) { return new TopicMetricsBuilder(name); } - PrometheusResponse toPrometheusResponse() { + PrometheusResponse toPrometheusRequestsResponse() { return new PrometheusResponse( "success", new PrometheusResponse.Data( "vector", List.of( new PrometheusResponse.Result( - new PrometheusResponse.MetricName(TOPIC_REQUESTS_TOTAL, null), List.of(TIMESTAMP, String.valueOf(rate)) - ), + ) + ) + ) + ); + } + + PrometheusResponse toDeliveredResponse() { + return new PrometheusResponse( + "success", + new PrometheusResponse.Data( + "vector", + List.of( new PrometheusResponse.Result( - new PrometheusResponse.MetricName(TOPIC_DELIVERED_TOTAL, null), List.of(TIMESTAMP, String.valueOf(deliveryRate)) - ), + ) + ) + ) + ); + } + + PrometheusResponse toPrometheusThroughputResponse() { + return new PrometheusResponse( + "success", + new PrometheusResponse.Data( + "vector", + List.of( new PrometheusResponse.Result( - new PrometheusResponse.MetricName(TOPIC_THROUGHPUT_TOTAL, null), List.of(TIMESTAMP, String.valueOf(throughput)) ) ) diff --git a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/QueryEndpointTest.java b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/QueryEndpointTest.java index 12f9badb8b..023c092dcb 100644 --- a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/QueryEndpointTest.java +++ b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/QueryEndpointTest.java @@ -527,7 +527,7 @@ public void shouldHandleUnavailableSubscriptionsMetrics() { ); String queryGetAllSubscriptionsMetrics = "{\"query\": {}}"; String queryGetSubscriptionsMetricsWithPositiveRate = "{\"query\": {\"rate\": {\"gt\": 0}}}"; - prometheus.stubDelay(ofMinutes(10)); + prometheus.stubDelay(Duration.ofMillis(3000)); waitAtMost(adjust(Duration.ofMinutes(1))).untilAsserted(() -> { // when diff --git a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/SubscriptionManagementTest.java b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/SubscriptionManagementTest.java index a5629282be..a1e3f1d475 100644 --- a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/SubscriptionManagementTest.java +++ b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/SubscriptionManagementTest.java @@ -332,7 +332,6 @@ public void shouldReturnHealthyStatusForAHealthySubscription() { WebTestClient.ResponseSpec response = hermes.api().getSubscriptionHealth(topic.getQualifiedName(), subscription.getName()); // then - assertThat(response.expectBody(SubscriptionHealth.class).returnResult().getResponseBody()) .isEqualTo(SubscriptionHealth.HEALTHY); } @@ -386,8 +385,8 @@ public void shouldNotAllowSubscriptionWithBatchDeliveryAndAvroContentType() { // given Topic topic = hermes.initHelper().createTopic(topicWithRandomName().build()); Subscription subscription = subscriptionWithRandomName(topic.getName()) - .withDeliveryType(DeliveryType.BATCH) - .withContentType(ContentType.AVRO) + .withDeliveryType(DeliveryType.BATCH) + .withContentType(ContentType.AVRO) .build(); // when From a3cf6fd7ed7750815a260a69fd37a0193375511d Mon Sep 17 00:00:00 2001 From: Maciej Moscicki Date: Mon, 5 Aug 2024 12:19:04 +0200 Subject: [PATCH 74/87] Faster topic removal in MultiDC setup (#1884) --- .../zookeeper/ZookeeperBasedRepository.java | 24 ++++++++ .../zookeeper/ZookeeperGroupRepository.java | 12 +++- .../ZookeeperMessagePreviewRepository.java | 4 +- .../zookeeper/ZookeeperPaths.java | 8 +++ .../zookeeper/ZookeeperTopicRepository.java | 58 ++++++++++++++++++- .../ZookeeperTopicRepositoryTest.groovy | 28 ++++++++- .../config/kafka/KafkaConfiguration.java | 2 +- ...tiDatacenterRepositoryCommandExecutor.java | 8 ++- .../subscription/SubscriptionRemover.java | 4 +- .../management/domain/topic/TopicService.java | 6 ++ .../domain/topic/schema/SchemaService.java | 7 +++ .../service/KafkaBrokerTopicManagement.java | 17 +++++- 12 files changed, 166 insertions(+), 12 deletions(-) diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/infrastructure/zookeeper/ZookeeperBasedRepository.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/infrastructure/zookeeper/ZookeeperBasedRepository.java index dcb958901c..241fcb0260 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/infrastructure/zookeeper/ZookeeperBasedRepository.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/infrastructure/zookeeper/ZookeeperBasedRepository.java @@ -5,6 +5,8 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.commons.lang3.ArrayUtils; import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.api.transaction.CuratorTransactionFinal; +import org.apache.curator.utils.ZKPaths; import org.apache.zookeeper.data.Stat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -18,6 +20,7 @@ import java.util.List; import java.util.Optional; import java.util.function.BiConsumer; +import java.util.stream.Collectors; public abstract class ZookeeperBasedRepository { @@ -75,6 +78,13 @@ protected List childrenOf(String path) { } } + protected List childrenPathsOf(String path) { + List childNodes = childrenOf(path); + return childNodes.stream() + .map(child -> ZKPaths.makePath(path, child)) + .collect(Collectors.toList()); + } + @SuppressWarnings("unchecked") protected byte[] readFrom(String path) { return readWithStatFrom(path, bytes -> bytes, (t, stat) -> {}, false).get(); @@ -156,6 +166,20 @@ protected void createInTransaction(String path, Object value, String childPath) .commit(); } + protected void deleteInTransaction(List paths) throws Exception { + if (paths.isEmpty()) { + throw new InternalProcessingException("Attempting to remove empty set of paths from ZK"); + } + ensureConnected(); + CuratorTransactionFinal transaction = zookeeper.inTransaction().delete().forPath(paths.get(0)).and(); + + for (int i = 1; i < paths.size(); i++) { + transaction = transaction.delete().forPath(paths.get(i)).and(); + } + + transaction.commit(); + } + protected void create(String path, Object value) throws Exception { ensureConnected(); zookeeper.create().forPath(path, mapper.writeValueAsBytes(value)); diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/infrastructure/zookeeper/ZookeeperGroupRepository.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/infrastructure/zookeeper/ZookeeperGroupRepository.java index d546ba6044..b9c745796d 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/infrastructure/zookeeper/ZookeeperGroupRepository.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/infrastructure/zookeeper/ZookeeperGroupRepository.java @@ -7,6 +7,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import pl.allegro.tech.hermes.api.Group; +import pl.allegro.tech.hermes.api.TopicName; import pl.allegro.tech.hermes.common.exception.InternalProcessingException; import pl.allegro.tech.hermes.domain.group.GroupAlreadyExistsException; import pl.allegro.tech.hermes.domain.group.GroupNotEmptyException; @@ -65,14 +66,23 @@ public void updateGroup(Group group) { } } + /** + * Atomic removal of group and group/topics + * nodes is required to prevent lengthy loop during removal, see: + * {@link pl.allegro.tech.hermes.infrastructure.zookeeper.ZookeeperTopicRepository#removeTopic(TopicName)}. + */ @Override public void removeGroup(String groupName) { ensureGroupExists(groupName); ensureGroupIsEmpty(groupName); logger.info("Removing group: {}", groupName); + List pathsToDelete = List.of( + paths.topicsPath(groupName), + paths.groupPath(groupName) + ); try { - remove(paths.groupPath(groupName)); + deleteInTransaction(pathsToDelete); } catch (Exception e) { throw new InternalProcessingException(e); } diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/infrastructure/zookeeper/ZookeeperMessagePreviewRepository.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/infrastructure/zookeeper/ZookeeperMessagePreviewRepository.java index 0fd938b668..e80b574f02 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/infrastructure/zookeeper/ZookeeperMessagePreviewRepository.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/infrastructure/zookeeper/ZookeeperMessagePreviewRepository.java @@ -28,7 +28,7 @@ public ZookeeperMessagePreviewRepository(CuratorFramework zookeeper, ObjectMappe @Override public List loadPreview(TopicName topicName) { try { - return Optional.of(paths.topicPath(topicName, ZookeeperPaths.PREVIEW_PATH)) + return Optional.of(paths.topicPreviewPath(topicName)) .filter(this::pathExists) .flatMap(p -> readFrom(p, new TypeReference>() {}, true)) .orElseGet(ArrayList::new); @@ -50,7 +50,7 @@ private void persistMessage(TopicName topic, List messages) { logger.debug("Persisting {} messages for preview of topic: {}", messages.size(), topic.qualifiedName()); try { if (pathExists(paths.topicPath(topic))) { - String previewPath = paths.topicPath(topic, ZookeeperPaths.PREVIEW_PATH); + String previewPath = paths.topicPreviewPath(topic); ensurePathExists(previewPath); overwrite(previewPath, messages); } diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/infrastructure/zookeeper/ZookeeperPaths.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/infrastructure/zookeeper/ZookeeperPaths.java index 9e04bcc64c..be3cb23393 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/infrastructure/zookeeper/ZookeeperPaths.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/infrastructure/zookeeper/ZookeeperPaths.java @@ -77,6 +77,14 @@ public String topicPath(TopicName topicName, String... tail) { return Joiner.on(URL_SEPARATOR).join(topicsPath(topicName.getGroupName()), topicName.getName(), (Object[]) tail); } + public String topicPreviewPath(TopicName topicName) { + return topicPath(topicName, ZookeeperPaths.PREVIEW_PATH); + } + + public String topicMetricsPath(TopicName topicName) { + return topicPath(topicName, METRICS_PATH); + } + public String subscriptionPath(TopicName topicName, String subscriptionName, String... tail) { return Joiner.on(URL_SEPARATOR).join(subscriptionsPath(topicName), subscriptionName, (Object[]) tail); } diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/infrastructure/zookeeper/ZookeeperTopicRepository.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/infrastructure/zookeeper/ZookeeperTopicRepository.java index 60dcb49bc5..3642677d37 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/infrastructure/zookeeper/ZookeeperTopicRepository.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/infrastructure/zookeeper/ZookeeperTopicRepository.java @@ -13,6 +13,7 @@ import pl.allegro.tech.hermes.domain.topic.TopicNotExistsException; import pl.allegro.tech.hermes.domain.topic.TopicRepository; +import java.util.ArrayList; import java.util.Collection; import java.util.List; import java.util.Optional; @@ -77,12 +78,67 @@ public void createTopic(Topic topic) { } } + /** + * To remove topic node, we must remove topic node and its children. The tree looks like this: + *
    + *
  • - topic + *
  • ----- /subscriptions (required) + *
  • ----- /preview (optional) + *
  • ----- /metrics (optional) + *
  • --------------- /volume + *
  • --------------- /published + *
+ * + *

One way to remove the whole tree for topic that would be to use deletingChildrenIfNeeded(): + * e.g. zookeeper.delete().deletingChildrenIfNeeded().forPath(topicPath). + * However, deletingChildrenIfNeeded is not atomic. It first tries to remove the node topic + * and upon receiving KeeperException.NotEmptyException it tries to remove children recursively + * and then retries the node removal. This means that there is a potentially large time gap between + * removal of topic/subscriptions node and topic node, especially when topic removal is being done + * in remote DC. + * + *

It turns out that PathChildrenCache used by HierarchicalCacheLevel in + * Consumers and Frontend listens for topics/subscriptions changes and recreates that node when deleted. + * If the recreation happens between the topic/subscriptions and topic node removal + * than the whole removal process must be repeated resulting in a lengthy loop that may even result in StackOverflowException. + * Example of that scenario would be + *

    + *
  1. DELETE topic - issued by management, fails with KeeperException.NotEmptyException + *
  2. DELETE topic/subscriptions - issued by management, succeeds + *
  3. CREATE topic/subscriptions - issued by frontend, succeeds + *
  4. DELETE topic - issued by management, fails with KeeperException.NotEmptyException + *
  5. [...] + *
+ * + *

To solve this we must remove topic and topic/subscriptions atomically. However, we must also remove + * other topic children. Transaction API does not allow for optional deletes so we: + *

    + *
  1. find all children paths + *
  2. delete all children in one transaction + *
+ */ @Override public void removeTopic(TopicName topicName) { ensureTopicExists(topicName); logger.info("Removing topic: " + topicName); + + List pathsForRemoval = new ArrayList<>(); + String topicMetricsPath = paths.topicMetricsPath(topicName); + if (pathExists(topicMetricsPath)) { + pathsForRemoval.addAll(childrenPathsOf(topicMetricsPath)); + pathsForRemoval.add(topicMetricsPath); + } + + String topicPreviewPath = paths.topicPreviewPath(topicName); + if (pathExists(topicPreviewPath)) { + pathsForRemoval.add(topicPreviewPath); + } + + pathsForRemoval.add(paths.subscriptionsPath(topicName)); + pathsForRemoval.add(paths.topicPath(topicName)); + try { - remove(paths.topicPath(topicName)); + deleteInTransaction(pathsForRemoval); } catch (Exception e) { throw new InternalProcessingException(e); } diff --git a/hermes-common/src/test/groovy/pl/allegro/tech/hermes/infrastructure/zookeeper/ZookeeperTopicRepositoryTest.groovy b/hermes-common/src/test/groovy/pl/allegro/tech/hermes/infrastructure/zookeeper/ZookeeperTopicRepositoryTest.groovy index 71bcd15bb4..7ac1377790 100644 --- a/hermes-common/src/test/groovy/pl/allegro/tech/hermes/infrastructure/zookeeper/ZookeeperTopicRepositoryTest.groovy +++ b/hermes-common/src/test/groovy/pl/allegro/tech/hermes/infrastructure/zookeeper/ZookeeperTopicRepositoryTest.groovy @@ -185,17 +185,16 @@ class ZookeeperTopicRepositoryTest extends IntegrationTest { !repository.topicExists(new TopicName(GROUP, 'remove')) } - def "should remove topic with metrics but without subscriptions"() { + def "should remove topic with metrics and without preview"() { given: def topicName = "topicWithMetrics" repository.createTopic(topic(GROUP, topicName).build()) wait.untilTopicCreated(GROUP, topicName) - def path = pathsCompiler.compile(BASE_ZOOKEEPER_PATH + ZookeeperCounterStorage.SUBSCRIPTION_DELIVERED, pathContext() + def path = pathsCompiler.compile(BASE_ZOOKEEPER_PATH + ZookeeperCounterStorage.TOPIC_VOLUME_COUNTER, pathContext() .withGroup(GROUP) .withTopic(topicName) - .withSubscription("sample") .build()) zookeeper().create().creatingParentsIfNeeded().forPath(path, '1'.bytes) wait.untilZookeeperPathIsCreated(path) @@ -207,6 +206,29 @@ class ZookeeperTopicRepositoryTest extends IntegrationTest { !repository.topicExists(new TopicName(GROUP, topicName)) } + def "should remove topic with metrics and preview"() { + given: "a topic" + Topic topic = topic(GROUP, "topicWithMetricsAndPreview").build() + repository.createTopic(topic) + wait.untilTopicCreated(GROUP, topic.getName().getName()) + + and: "volume metric in zk for that topic" + String metricPath = paths.topicMetricPath(topic.getName(), "volume") + zookeeper().create().creatingParentsIfNeeded().forPath(metricPath, '1'.bytes) + wait.untilZookeeperPathIsCreated(metricPath) + + and: "preview in zk for that topic" + String previewPath = paths.topicPreviewPath(topic.getName()) + zookeeper().create().creatingParentsIfNeeded().forPath(previewPath , '1'.bytes) + wait.untilZookeeperPathIsCreated(previewPath) + + when: + repository.removeTopic(topic.getName()) + + then: + !repository.topicExists(topic.getName()) + } + def "should not throw exception on malformed topic when reading list of all topics"() { given: zookeeper().create().forPath(paths.topicPath(new TopicName(GROUP, 'malformed')), ''.bytes) diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/kafka/KafkaConfiguration.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/kafka/KafkaConfiguration.java index 45652d01b3..0c50ef5970 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/kafka/KafkaConfiguration.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/kafka/KafkaConfiguration.java @@ -82,7 +82,7 @@ MultiDCAwareService multiDCAwareService(KafkaNamesMappers kafkaNamesMappers, Sch AdminClient brokerAdminClient = brokerAdminClient(kafkaProperties); BrokerStorage storage = brokersStorage(brokerAdminClient); BrokerTopicManagement brokerTopicManagement = - new KafkaBrokerTopicManagement(topicProperties, brokerAdminClient, kafkaNamesMapper); + new KafkaBrokerTopicManagement(topicProperties, brokerAdminClient, kafkaNamesMapper, kafkaProperties.getDatacenter()); KafkaConsumerPool consumerPool = kafkaConsumersPool(kafkaProperties, storage, kafkaProperties.getBrokerList()); KafkaRawMessageReader kafkaRawMessageReader = new KafkaRawMessageReader(consumerPool, kafkaProperties.getKafkaConsumer().getPollTimeoutMillis()); diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/dc/MultiDatacenterRepositoryCommandExecutor.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/dc/MultiDatacenterRepositoryCommandExecutor.java index 6e4c7cf003..e4baa9efec 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/dc/MultiDatacenterRepositoryCommandExecutor.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/dc/MultiDatacenterRepositoryCommandExecutor.java @@ -46,9 +46,12 @@ private void execute(RepositoryCommand command, boolean isRollbackEnabled List> executedRepoHolders = new ArrayList<>(); for (DatacenterBoundRepositoryHolder repoHolder : repoHolders) { + long start = System.currentTimeMillis(); try { executedRepoHolders.add(repoHolder); + logger.info("Executing repository command: {} in ZK dc: {}", command, repoHolder.getDatacenterName()); command.execute(repoHolder); + logger.info("Successfully executed repository command: {} in ZK dc: {} in: {} ms", command, repoHolder.getDatacenterName(), System.currentTimeMillis() - start); } catch (RepositoryNotAvailableException e) { logger.warn("Execute failed with an RepositoryNotAvailableException error", e); if (isRollbackEnabled) { @@ -58,7 +61,7 @@ private void execute(RepositoryCommand command, boolean isRollbackEnabled throw ExceptionWrapper.wrapInInternalProcessingExceptionIfNeeded(e, command.toString(), repoHolder.getDatacenterName()); } } catch (Exception e) { - logger.warn("Execute failed with an error", e); + logger.warn("Failed to execute repository command: {} in ZK dc: {} in: {} ms", command, repoHolder.getDatacenterName(), System.currentTimeMillis() - start, e); if (isRollbackEnabled) { rollback(executedRepoHolders, command); } @@ -68,9 +71,12 @@ private void execute(RepositoryCommand command, boolean isRollbackEnabled } private void rollback(List> repoHolders, RepositoryCommand command) { + long start = System.currentTimeMillis(); for (DatacenterBoundRepositoryHolder repoHolder : repoHolders) { + logger.info("Executing rollback of repository command: {} in ZK dc: {}", command, repoHolder.getDatacenterName()); try { command.rollback(repoHolder); + logger.info("Successfully executed rollback of repository command: {} in ZK dc: {} in: {} ms", command, repoHolder.getDatacenterName(), System.currentTimeMillis() - start); } catch (Exception e) { logger.error("Rollback procedure failed for command {} on DC {}", command, repoHolder.getDatacenterName(), e); } diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/subscription/SubscriptionRemover.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/subscription/SubscriptionRemover.java index 4e9a90edfd..016e18b35f 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/subscription/SubscriptionRemover.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/subscription/SubscriptionRemover.java @@ -43,9 +43,11 @@ public void removeSubscription(TopicName topicName, String subscriptionName, Req public void removeSubscriptionRelatedToTopic(Topic topic, RequestUser removedBy) { List subscriptions = subscriptionRepository.listSubscriptions(topic.getName()); - ensureSubscriptionsHaveAutoRemove(subscriptions, topic.getName()); + logger.info("Removing subscriptions of topic: {}, subscriptions: {}", topic.getName(), subscriptions); + long start = System.currentTimeMillis(); subscriptions.forEach(sub -> removeSubscription(topic.getName(), sub.getName(), removedBy)); + logger.info("Removed subscriptions of topic: {} in {} ms", topic.getName(), System.currentTimeMillis() - start); } diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/TopicService.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/TopicService.java index 220fbfd09b..26fe3ae834 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/TopicService.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/TopicService.java @@ -382,8 +382,14 @@ private void removeSchema(Topic topic) { } private void removeTopic(Topic topic, RequestUser removedBy) { + logger.info("Removing topic: {} from ZK clusters", topic.getQualifiedName()); + long start = System.currentTimeMillis(); multiDcExecutor.executeByUser(new RemoveTopicRepositoryCommand(topic.getName()), removedBy); + logger.info("Removed topic: {} from ZK clusters in: {} ms", topic.getQualifiedName(), System.currentTimeMillis() - start); + logger.info("Removing topic: {} from Kafka clusters", topic.getQualifiedName()); + start = System.currentTimeMillis(); multiDCAwareService.manageTopic(brokerTopicManagement -> brokerTopicManagement.removeTopic(topic)); + logger.info("Removed topic: {} from Kafka clusters in: {} ms", topic.getQualifiedName(), System.currentTimeMillis() - start); auditor.objectRemoved(removedBy.getUsername(), topic); topicOwnerCache.onRemovedTopic(topic); } diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/schema/SchemaService.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/schema/SchemaService.java index 315cdd4cb1..c0d3bcba2c 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/schema/SchemaService.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/schema/SchemaService.java @@ -1,5 +1,7 @@ package pl.allegro.tech.hermes.management.domain.topic.schema; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.springframework.beans.factory.annotation.Autowired; import org.springframework.stereotype.Component; import pl.allegro.tech.hermes.api.RawSchema; @@ -24,6 +26,8 @@ public class SchemaService { private final SchemaValidatorProvider validatorProvider; private final TopicProperties topicProperties; + private static final Logger logger = LoggerFactory.getLogger(SchemaService.class); + @Autowired public SchemaService(RawSchemaClient rawSchemaClient, SchemaValidatorProvider validatorProvider, @@ -68,7 +72,10 @@ public void deleteAllSchemaVersions(String qualifiedTopicName) { if (!topicProperties.isRemoveSchema()) { throw new SchemaRemovalDisabledException(); } + logger.info("Removing all schema versions for topic: {}", qualifiedTopicName); + long start = System.currentTimeMillis(); rawSchemaClient.deleteAllSchemaVersions(fromQualifiedName(qualifiedTopicName)); + logger.info("Removed all schema versions for topic: {} in {} ms", qualifiedTopicName, System.currentTimeMillis() - start); } public void validateSchema(Topic topic, String schema) { diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/kafka/service/KafkaBrokerTopicManagement.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/kafka/service/KafkaBrokerTopicManagement.java index 325d362f46..f799c0b03c 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/kafka/service/KafkaBrokerTopicManagement.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/kafka/service/KafkaBrokerTopicManagement.java @@ -9,6 +9,8 @@ import org.apache.kafka.common.KafkaFuture; import org.apache.kafka.common.config.ConfigResource; import org.apache.kafka.common.config.TopicConfig; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import pl.allegro.tech.hermes.api.Topic; import pl.allegro.tech.hermes.common.kafka.KafkaNamesMapper; import pl.allegro.tech.hermes.common.kafka.KafkaTopic; @@ -32,10 +34,16 @@ public class KafkaBrokerTopicManagement implements BrokerTopicManagement { private final KafkaNamesMapper kafkaNamesMapper; - public KafkaBrokerTopicManagement(TopicProperties topicProperties, AdminClient kafkaAdminClient, KafkaNamesMapper kafkaNamesMapper) { + private final String datacenterName; + + private static final Logger logger = LoggerFactory.getLogger(KafkaBrokerTopicManagement.class); + + + public KafkaBrokerTopicManagement(TopicProperties topicProperties, AdminClient kafkaAdminClient, KafkaNamesMapper kafkaNamesMapper, String datacenterName) { this.topicProperties = topicProperties; this.kafkaAdminClient = kafkaAdminClient; this.kafkaNamesMapper = kafkaNamesMapper; + this.datacenterName = datacenterName; } @Override @@ -59,7 +67,12 @@ public void removeTopic(Topic topic) { kafkaNamesMapper.toKafkaTopics(topic).stream() .map(k -> kafkaAdminClient.deleteTopics(Collections.singletonList(k.name().asString()))) .map(DeleteTopicsResult::all) - .forEach(this::waitForKafkaFuture); + .forEach(future -> { + logger.info("Removing topic: {} from Kafka dc: {}", topic, datacenterName); + long start = System.currentTimeMillis(); + waitForKafkaFuture(future); + logger.info("Removed topic: {} from Kafka dc: {} in {} ms", topic, datacenterName, System.currentTimeMillis() - start); + }); } @Override From 0bf4c0c18e6777113e9170b79cea24662b740286 Mon Sep 17 00:00:00 2001 From: Maciej Moscicki Date: Tue, 6 Aug 2024 15:36:56 +0200 Subject: [PATCH 75/87] do not remove subscription if it already existed before subscription creation (#1887) * do not remove subscription if it already existed before subscription creation * fix MultiDatacenterRepositoryCommandExecutorTest --- .../AddTopicToBlacklistRepositoryCommand.java | 2 +- ...veTopicFromBlacklistRepositoryCommand.java | 2 +- .../UpdateCredentialsRepositoryCommand.java | 2 +- ...tiDatacenterRepositoryCommandExecutor.java | 8 +-- .../domain/dc/RepositoryCommand.java | 2 +- .../CreateGroupRepositoryCommand.java | 2 +- .../RemoveGroupRepositoryCommand.java | 2 +- .../UpdateGroupRepositoryCommand.java | 2 +- .../CreateOAuthProviderRepositoryCommand.java | 2 +- .../RemoveOAuthProviderRepositoryCommand.java | 2 +- .../UpdateOAuthProviderRepositoryCommand.java | 2 +- .../domain/readiness/SetReadinessCommand.java | 2 +- ...reateOfflineRetransmissionTaskCommand.java | 2 +- ...eleteOfflineRetransmissionTaskCommand.java | 2 +- .../domain/retransmit/RetransmitCommand.java | 2 +- .../CreateSubscriptionRepositoryCommand.java | 7 ++- .../RemoveSubscriptionRepositoryCommand.java | 2 +- .../UpdateSubscriptionRepositoryCommand.java | 2 +- .../CreateTopicRepositoryCommand.java | 2 +- .../RemoveTopicRepositoryCommand.java | 2 +- .../commands/TouchTopicRepositoryCommand.java | 2 +- .../UpdateTopicRepositoryCommand.java | 2 +- ...scriptionConstraintsRepositoryCommand.java | 2 +- ...eateTopicConstraintsRepositoryCommand.java | 2 +- ...scriptionConstraintsRepositoryCommand.java | 2 +- ...leteTopicConstraintsRepositoryCommand.java | 2 +- ...scriptionConstraintsRepositoryCommand.java | 2 +- ...dateTopicConstraintsRepositoryCommand.java | 2 +- ...centerRepositoryCommandExecutorTest.groovy | 6 +-- ...teSubscriptionRepositoryCommandTest.groovy | 53 +++++++++++++++++++ 30 files changed, 92 insertions(+), 34 deletions(-) create mode 100644 hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/domain/subscription/commands/CreateSubscriptionRepositoryCommandTest.groovy diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/blacklist/commands/AddTopicToBlacklistRepositoryCommand.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/blacklist/commands/AddTopicToBlacklistRepositoryCommand.java index 2bd07569ce..d4259545d8 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/blacklist/commands/AddTopicToBlacklistRepositoryCommand.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/blacklist/commands/AddTopicToBlacklistRepositoryCommand.java @@ -21,7 +21,7 @@ public void execute(DatacenterBoundRepositoryHolder ho } @Override - public void rollback(DatacenterBoundRepositoryHolder holder) { + public void rollback(DatacenterBoundRepositoryHolder holder, Exception exception) { holder.getRepository().remove(qualifiedTopicName); } diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/blacklist/commands/RemoveTopicFromBlacklistRepositoryCommand.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/blacklist/commands/RemoveTopicFromBlacklistRepositoryCommand.java index 8515befcc1..1ca0ef3367 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/blacklist/commands/RemoveTopicFromBlacklistRepositoryCommand.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/blacklist/commands/RemoveTopicFromBlacklistRepositoryCommand.java @@ -23,7 +23,7 @@ public void execute(DatacenterBoundRepositoryHolder ho } @Override - public void rollback(DatacenterBoundRepositoryHolder holder) { + public void rollback(DatacenterBoundRepositoryHolder holder, Exception exception) { if (exists) { holder.getRepository().add(qualifiedTopicName); } diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/credentials/commands/UpdateCredentialsRepositoryCommand.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/credentials/commands/UpdateCredentialsRepositoryCommand.java index 44ea88c3c6..f8c30cc7fc 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/credentials/commands/UpdateCredentialsRepositoryCommand.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/credentials/commands/UpdateCredentialsRepositoryCommand.java @@ -24,7 +24,7 @@ public void execute(DatacenterBoundRepositoryHolder holde } @Override - public void rollback(DatacenterBoundRepositoryHolder holder) { + public void rollback(DatacenterBoundRepositoryHolder holder, Exception exception) { } @Override diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/dc/MultiDatacenterRepositoryCommandExecutor.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/dc/MultiDatacenterRepositoryCommandExecutor.java index e4baa9efec..2648407fa4 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/dc/MultiDatacenterRepositoryCommandExecutor.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/dc/MultiDatacenterRepositoryCommandExecutor.java @@ -55,7 +55,7 @@ private void execute(RepositoryCommand command, boolean isRollbackEnabled } catch (RepositoryNotAvailableException e) { logger.warn("Execute failed with an RepositoryNotAvailableException error", e); if (isRollbackEnabled) { - rollback(executedRepoHolders, command); + rollback(executedRepoHolders, command, e); } if (shouldStopExecutionOnFailure) { throw ExceptionWrapper.wrapInInternalProcessingExceptionIfNeeded(e, command.toString(), repoHolder.getDatacenterName()); @@ -63,19 +63,19 @@ private void execute(RepositoryCommand command, boolean isRollbackEnabled } catch (Exception e) { logger.warn("Failed to execute repository command: {} in ZK dc: {} in: {} ms", command, repoHolder.getDatacenterName(), System.currentTimeMillis() - start, e); if (isRollbackEnabled) { - rollback(executedRepoHolders, command); + rollback(executedRepoHolders, command, e); } throw ExceptionWrapper.wrapInInternalProcessingExceptionIfNeeded(e, command.toString(), repoHolder.getDatacenterName()); } } } - private void rollback(List> repoHolders, RepositoryCommand command) { + private void rollback(List> repoHolders, RepositoryCommand command, Exception exception) { long start = System.currentTimeMillis(); for (DatacenterBoundRepositoryHolder repoHolder : repoHolders) { logger.info("Executing rollback of repository command: {} in ZK dc: {}", command, repoHolder.getDatacenterName()); try { - command.rollback(repoHolder); + command.rollback(repoHolder, exception); logger.info("Successfully executed rollback of repository command: {} in ZK dc: {} in: {} ms", command, repoHolder.getDatacenterName(), System.currentTimeMillis() - start); } catch (Exception e) { logger.error("Rollback procedure failed for command {} on DC {}", command, repoHolder.getDatacenterName(), e); diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/dc/RepositoryCommand.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/dc/RepositoryCommand.java index 246e0e3964..1b2d899137 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/dc/RepositoryCommand.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/dc/RepositoryCommand.java @@ -6,7 +6,7 @@ public abstract class RepositoryCommand { public abstract void execute(DatacenterBoundRepositoryHolder holder); - public abstract void rollback(DatacenterBoundRepositoryHolder holder); + public abstract void rollback(DatacenterBoundRepositoryHolder holder, Exception exception); public abstract Class getRepositoryType(); diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/group/commands/CreateGroupRepositoryCommand.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/group/commands/CreateGroupRepositoryCommand.java index 3b67269d1b..cfea032d3d 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/group/commands/CreateGroupRepositoryCommand.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/group/commands/CreateGroupRepositoryCommand.java @@ -25,7 +25,7 @@ public void execute(DatacenterBoundRepositoryHolder holder) { } @Override - public void rollback(DatacenterBoundRepositoryHolder holder) { + public void rollback(DatacenterBoundRepositoryHolder holder, Exception exception) { if (!exists) { holder.getRepository().removeGroup(group.getGroupName()); } diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/group/commands/RemoveGroupRepositoryCommand.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/group/commands/RemoveGroupRepositoryCommand.java index f9b41d8e6b..c4714744db 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/group/commands/RemoveGroupRepositoryCommand.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/group/commands/RemoveGroupRepositoryCommand.java @@ -26,7 +26,7 @@ public void execute(DatacenterBoundRepositoryHolder holder) { } @Override - public void rollback(DatacenterBoundRepositoryHolder holder) { + public void rollback(DatacenterBoundRepositoryHolder holder, Exception exception) { holder.getRepository().createGroup(backup); } diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/group/commands/UpdateGroupRepositoryCommand.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/group/commands/UpdateGroupRepositoryCommand.java index c25dcd73e3..62fa38600c 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/group/commands/UpdateGroupRepositoryCommand.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/group/commands/UpdateGroupRepositoryCommand.java @@ -26,7 +26,7 @@ public void execute(DatacenterBoundRepositoryHolder holder) { } @Override - public void rollback(DatacenterBoundRepositoryHolder holder) { + public void rollback(DatacenterBoundRepositoryHolder holder, Exception exception) { holder.getRepository().updateGroup(backup); } diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/oauth/commands/CreateOAuthProviderRepositoryCommand.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/oauth/commands/CreateOAuthProviderRepositoryCommand.java index d5b641b998..be440319d3 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/oauth/commands/CreateOAuthProviderRepositoryCommand.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/oauth/commands/CreateOAuthProviderRepositoryCommand.java @@ -22,7 +22,7 @@ public void execute(DatacenterBoundRepositoryHolder hol } @Override - public void rollback(DatacenterBoundRepositoryHolder holder) { + public void rollback(DatacenterBoundRepositoryHolder holder, Exception exception) { holder.getRepository().removeOAuthProvider(provider.getName()); } diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/oauth/commands/RemoveOAuthProviderRepositoryCommand.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/oauth/commands/RemoveOAuthProviderRepositoryCommand.java index 4ff7f08f3d..a17a725779 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/oauth/commands/RemoveOAuthProviderRepositoryCommand.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/oauth/commands/RemoveOAuthProviderRepositoryCommand.java @@ -26,7 +26,7 @@ public void execute(DatacenterBoundRepositoryHolder hol } @Override - public void rollback(DatacenterBoundRepositoryHolder holder) { + public void rollback(DatacenterBoundRepositoryHolder holder, Exception exception) { holder.getRepository().createOAuthProvider(backup); } diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/oauth/commands/UpdateOAuthProviderRepositoryCommand.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/oauth/commands/UpdateOAuthProviderRepositoryCommand.java index cf6ba8bdca..beeed66941 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/oauth/commands/UpdateOAuthProviderRepositoryCommand.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/oauth/commands/UpdateOAuthProviderRepositoryCommand.java @@ -26,7 +26,7 @@ public void execute(DatacenterBoundRepositoryHolder hol } @Override - public void rollback(DatacenterBoundRepositoryHolder holder) { + public void rollback(DatacenterBoundRepositoryHolder holder, Exception exception) { holder.getRepository().updateOAuthProvider(backup); } diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/readiness/SetReadinessCommand.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/readiness/SetReadinessCommand.java index 5cc0777f49..5dac6e9174 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/readiness/SetReadinessCommand.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/readiness/SetReadinessCommand.java @@ -22,7 +22,7 @@ public void execute(DatacenterBoundRepositoryHolder holder) { } + public void rollback(DatacenterBoundRepositoryHolder holder, Exception exception) { } @Override public Class getRepositoryType() { diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/retransmit/CreateOfflineRetransmissionTaskCommand.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/retransmit/CreateOfflineRetransmissionTaskCommand.java index b6d070a7b7..cfd2e40c61 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/retransmit/CreateOfflineRetransmissionTaskCommand.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/retransmit/CreateOfflineRetransmissionTaskCommand.java @@ -21,7 +21,7 @@ public void execute(DatacenterBoundRepositoryHolder holder) { + public void rollback(DatacenterBoundRepositoryHolder holder, Exception exception) { holder.getRepository().deleteTask(task.getTaskId()); } diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/retransmit/DeleteOfflineRetransmissionTaskCommand.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/retransmit/DeleteOfflineRetransmissionTaskCommand.java index 8697d50505..14d0f2029c 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/retransmit/DeleteOfflineRetransmissionTaskCommand.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/retransmit/DeleteOfflineRetransmissionTaskCommand.java @@ -21,7 +21,7 @@ public void execute(DatacenterBoundRepositoryHolder holder) { + public void rollback(DatacenterBoundRepositoryHolder holder, Exception exception) { } diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/retransmit/RetransmitCommand.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/retransmit/RetransmitCommand.java index eccd673d2b..669f45aca0 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/retransmit/RetransmitCommand.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/retransmit/RetransmitCommand.java @@ -23,7 +23,7 @@ public void execute(DatacenterBoundRepositoryHolder holder) { } @Override - public void rollback(DatacenterBoundRepositoryHolder holder) { + public void rollback(DatacenterBoundRepositoryHolder holder, Exception exception) { } diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/subscription/commands/CreateSubscriptionRepositoryCommand.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/subscription/commands/CreateSubscriptionRepositoryCommand.java index 7004608ff8..728dda8d7c 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/subscription/commands/CreateSubscriptionRepositoryCommand.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/subscription/commands/CreateSubscriptionRepositoryCommand.java @@ -1,6 +1,7 @@ package pl.allegro.tech.hermes.management.domain.subscription.commands; import pl.allegro.tech.hermes.api.Subscription; +import pl.allegro.tech.hermes.domain.subscription.SubscriptionAlreadyExistsException; import pl.allegro.tech.hermes.domain.subscription.SubscriptionRepository; import pl.allegro.tech.hermes.management.domain.dc.DatacenterBoundRepositoryHolder; import pl.allegro.tech.hermes.management.domain.dc.RepositoryCommand; @@ -22,7 +23,11 @@ public void execute(DatacenterBoundRepositoryHolder hold } @Override - public void rollback(DatacenterBoundRepositoryHolder holder) { + public void rollback(DatacenterBoundRepositoryHolder holder, Exception exception) { + if (exception instanceof SubscriptionAlreadyExistsException) { + // prevents removal of already existing subscription + return; + } holder.getRepository().removeSubscription(subscription.getTopicName(), subscription.getName()); } diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/subscription/commands/RemoveSubscriptionRepositoryCommand.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/subscription/commands/RemoveSubscriptionRepositoryCommand.java index 9d0c2d838d..d1b346b3dc 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/subscription/commands/RemoveSubscriptionRepositoryCommand.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/subscription/commands/RemoveSubscriptionRepositoryCommand.java @@ -30,7 +30,7 @@ public void execute(DatacenterBoundRepositoryHolder hold } @Override - public void rollback(DatacenterBoundRepositoryHolder holder) { + public void rollback(DatacenterBoundRepositoryHolder holder, Exception exception) { holder.getRepository().createSubscription(backup); } diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/subscription/commands/UpdateSubscriptionRepositoryCommand.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/subscription/commands/UpdateSubscriptionRepositoryCommand.java index 3d0684f2f1..c0ac52c3c2 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/subscription/commands/UpdateSubscriptionRepositoryCommand.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/subscription/commands/UpdateSubscriptionRepositoryCommand.java @@ -25,7 +25,7 @@ public void execute(DatacenterBoundRepositoryHolder hold } @Override - public void rollback(DatacenterBoundRepositoryHolder holder) { + public void rollback(DatacenterBoundRepositoryHolder holder, Exception exception) { holder.getRepository().updateSubscription(backup); } diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/commands/CreateTopicRepositoryCommand.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/commands/CreateTopicRepositoryCommand.java index 1666c50bf8..2842b7e988 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/commands/CreateTopicRepositoryCommand.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/commands/CreateTopicRepositoryCommand.java @@ -22,7 +22,7 @@ public void execute(DatacenterBoundRepositoryHolder holder) { } @Override - public void rollback(DatacenterBoundRepositoryHolder holder) { + public void rollback(DatacenterBoundRepositoryHolder holder, Exception exception) { /* We don't want to do a rollback due to possible race conditions with creating a topic on Kafka. It increases the probability of discrepancies between Kafka and Zookeeper: topic exists in Kafka, diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/commands/RemoveTopicRepositoryCommand.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/commands/RemoveTopicRepositoryCommand.java index d378548603..397c60c295 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/commands/RemoveTopicRepositoryCommand.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/commands/RemoveTopicRepositoryCommand.java @@ -22,7 +22,7 @@ public void execute(DatacenterBoundRepositoryHolder holder) { } @Override - public void rollback(DatacenterBoundRepositoryHolder holder) { + public void rollback(DatacenterBoundRepositoryHolder holder, Exception exception) { /* We don't want to do a rollback due to possible race conditions with creating a topic on Kafka. It increases the probability of discrepancies between Kafka and Zookeeper: topic exists in Kafka, diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/commands/TouchTopicRepositoryCommand.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/commands/TouchTopicRepositoryCommand.java index b210658ca7..d4a850f681 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/commands/TouchTopicRepositoryCommand.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/commands/TouchTopicRepositoryCommand.java @@ -22,7 +22,7 @@ public void execute(DatacenterBoundRepositoryHolder holder) { } @Override - public void rollback(DatacenterBoundRepositoryHolder holder) {} + public void rollback(DatacenterBoundRepositoryHolder holder, Exception exception) {} @Override public Class getRepositoryType() { diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/commands/UpdateTopicRepositoryCommand.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/commands/UpdateTopicRepositoryCommand.java index 85fdea7c6d..0f007f5fe9 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/commands/UpdateTopicRepositoryCommand.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/topic/commands/UpdateTopicRepositoryCommand.java @@ -26,7 +26,7 @@ public void execute(DatacenterBoundRepositoryHolder holder) { } @Override - public void rollback(DatacenterBoundRepositoryHolder holder) { + public void rollback(DatacenterBoundRepositoryHolder holder, Exception exception) { holder.getRepository().updateTopic(backup); } diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/workload/constraints/command/CreateSubscriptionConstraintsRepositoryCommand.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/workload/constraints/command/CreateSubscriptionConstraintsRepositoryCommand.java index 9f064531a2..bf38652177 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/workload/constraints/command/CreateSubscriptionConstraintsRepositoryCommand.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/workload/constraints/command/CreateSubscriptionConstraintsRepositoryCommand.java @@ -28,7 +28,7 @@ public void execute(DatacenterBoundRepositoryHolder holder) { + public void rollback(DatacenterBoundRepositoryHolder holder, Exception exception) { if (!exists) { holder.getRepository().deleteConstraints(subscriptionName); } diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/workload/constraints/command/CreateTopicConstraintsRepositoryCommand.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/workload/constraints/command/CreateTopicConstraintsRepositoryCommand.java index 33e924e63e..e96ba4875d 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/workload/constraints/command/CreateTopicConstraintsRepositoryCommand.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/workload/constraints/command/CreateTopicConstraintsRepositoryCommand.java @@ -28,7 +28,7 @@ public void execute(DatacenterBoundRepositoryHolder holder) { + public void rollback(DatacenterBoundRepositoryHolder holder, Exception exception) { if (!exist) { holder.getRepository().deleteConstraints(topicName); } diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/workload/constraints/command/DeleteSubscriptionConstraintsRepositoryCommand.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/workload/constraints/command/DeleteSubscriptionConstraintsRepositoryCommand.java index 7f04122339..3f0f713367 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/workload/constraints/command/DeleteSubscriptionConstraintsRepositoryCommand.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/workload/constraints/command/DeleteSubscriptionConstraintsRepositoryCommand.java @@ -26,7 +26,7 @@ public void execute(DatacenterBoundRepositoryHolder holder) { + public void rollback(DatacenterBoundRepositoryHolder holder, Exception exception) { if (backup != null) { holder.getRepository().createConstraints(subscriptionName, backup); } diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/workload/constraints/command/DeleteTopicConstraintsRepositoryCommand.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/workload/constraints/command/DeleteTopicConstraintsRepositoryCommand.java index 4f19ea376e..981a198968 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/workload/constraints/command/DeleteTopicConstraintsRepositoryCommand.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/workload/constraints/command/DeleteTopicConstraintsRepositoryCommand.java @@ -26,7 +26,7 @@ public void execute(DatacenterBoundRepositoryHolder holder) { + public void rollback(DatacenterBoundRepositoryHolder holder, Exception exception) { if (backup != null) { holder.getRepository().createConstraints(topicName, backup); } diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/workload/constraints/command/UpdateSubscriptionConstraintsRepositoryCommand.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/workload/constraints/command/UpdateSubscriptionConstraintsRepositoryCommand.java index bb237f9e2f..ade0564ee3 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/workload/constraints/command/UpdateSubscriptionConstraintsRepositoryCommand.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/workload/constraints/command/UpdateSubscriptionConstraintsRepositoryCommand.java @@ -28,7 +28,7 @@ public void execute(DatacenterBoundRepositoryHolder holder) { + public void rollback(DatacenterBoundRepositoryHolder holder, Exception exception) { if (backup != null) { holder.getRepository().updateConstraints(subscriptionName, backup); } diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/workload/constraints/command/UpdateTopicConstraintsRepositoryCommand.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/workload/constraints/command/UpdateTopicConstraintsRepositoryCommand.java index 6bf4f8ac8c..e69d08642b 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/workload/constraints/command/UpdateTopicConstraintsRepositoryCommand.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/workload/constraints/command/UpdateTopicConstraintsRepositoryCommand.java @@ -28,7 +28,7 @@ public void execute(DatacenterBoundRepositoryHolder holder) { + public void rollback(DatacenterBoundRepositoryHolder holder, Exception exception) { if (backup != null) { holder.getRepository().updateConstraints(topicName, backup); } diff --git a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/domain/dc/MultiDatacenterRepositoryCommandExecutorTest.groovy b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/domain/dc/MultiDatacenterRepositoryCommandExecutorTest.groovy index b6f8ec4807..d6e621932f 100644 --- a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/domain/dc/MultiDatacenterRepositoryCommandExecutorTest.groovy +++ b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/domain/dc/MultiDatacenterRepositoryCommandExecutorTest.groovy @@ -66,7 +66,7 @@ class MultiDatacenterRepositoryCommandExecutorTest extends Specification { executor.execute(command) then: - 1 * command.rollback(holder1) + 1 * command.rollback(holder1, _) thrown InternalProcessingException } @@ -104,7 +104,7 @@ class MultiDatacenterRepositoryCommandExecutorTest extends Specification { executor.executeByUser(command, ADMIN) then: - 1 * command.rollback(holder1) + 1 * command.rollback(holder1, _) thrown InternalProcessingException } @@ -160,7 +160,7 @@ class MultiDatacenterRepositoryCommandExecutorTest extends Specification { executor.executeByUser(command, NON_ADMIN) then: - 1 * command.rollback(holder1) + 1 * command.rollback(holder1, _) thrown InternalProcessingException } diff --git a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/domain/subscription/commands/CreateSubscriptionRepositoryCommandTest.groovy b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/domain/subscription/commands/CreateSubscriptionRepositoryCommandTest.groovy new file mode 100644 index 0000000000..47d59dfddb --- /dev/null +++ b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/domain/subscription/commands/CreateSubscriptionRepositoryCommandTest.groovy @@ -0,0 +1,53 @@ +package pl.allegro.tech.hermes.management.domain.subscription.commands + +import pl.allegro.tech.hermes.api.Subscription +import pl.allegro.tech.hermes.api.TopicName +import pl.allegro.tech.hermes.domain.subscription.SubscriptionAlreadyExistsException +import pl.allegro.tech.hermes.domain.subscription.SubscriptionRepository +import pl.allegro.tech.hermes.management.domain.dc.DatacenterBoundRepositoryHolder +import spock.lang.Shared +import spock.lang.Specification + +import static pl.allegro.tech.hermes.test.helper.builder.SubscriptionBuilder.subscription + +class CreateSubscriptionRepositoryCommandTest extends Specification { + + @Shared + def topicName = new TopicName("group", "topic") + + @Shared + def subscriptionName = "subscription" + + @Shared + Subscription subscription = subscription(topicName, subscriptionName).build() + + def "should not remove subscription if subscription already exists during rollback"() { + given: + SubscriptionRepository subscriptionRepository = Mock(SubscriptionRepository) + DatacenterBoundRepositoryHolder repository = Mock(DatacenterBoundRepositoryHolder) { + getRepository() >> subscriptionRepository + } + def command = new CreateSubscriptionRepositoryCommand(subscription) + + when: + command.rollback(repository, new SubscriptionAlreadyExistsException(subscription)) + + then: + 0 * subscriptionRepository.removeSubscription(topicName, subscriptionName) + } + + def "should remove subscription during rollback"() { + given: + SubscriptionRepository subscriptionRepository = Mock(SubscriptionRepository) + DatacenterBoundRepositoryHolder repository = Mock(DatacenterBoundRepositoryHolder) { + getRepository() >> subscriptionRepository + } + def command = new CreateSubscriptionRepositoryCommand(subscription) + + when: + command.rollback(repository, new RuntimeException()) + + then: + 1 * subscriptionRepository.removeSubscription(topicName, subscriptionName) + } +} From 3b02b2b07ef2f5b28590d95a20f5d6273aa89e75 Mon Sep 17 00:00:00 2001 From: Maciej Moscicki Date: Thu, 8 Aug 2024 11:40:06 +0200 Subject: [PATCH 76/87] Periodic storage consistency checks (#1885) * add ability to perform periodic storage checks * add periodic consistency check spec * register gauge based on state object --- .../common/metric/ConsistencyMetrics.java | 19 +++++ .../hermes/common/metric/MetricsFacade.java | 6 ++ .../config/ConsistencyCheckerProperties.java | 31 ++++++++ .../consistency/DcConsistencyService.java | 37 +++++++++- .../DcConsistencyServiceSpec.groovy | 73 +++++++++++++++++-- 5 files changed, 158 insertions(+), 8 deletions(-) create mode 100644 hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/ConsistencyMetrics.java diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/ConsistencyMetrics.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/ConsistencyMetrics.java new file mode 100644 index 0000000000..4324ec0e68 --- /dev/null +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/ConsistencyMetrics.java @@ -0,0 +1,19 @@ +package pl.allegro.tech.hermes.common.metric; + +import io.micrometer.core.instrument.MeterRegistry; + +import java.util.function.ToDoubleFunction; + + +public class ConsistencyMetrics { + private final MeterRegistry meterRegistry; + + ConsistencyMetrics(MeterRegistry meterRegistry) { + this.meterRegistry = meterRegistry; + + } + + public void registerStorageConsistencyGauge(T stateObject, ToDoubleFunction valueFunction) { + meterRegistry.gauge("storage.consistency", stateObject, valueFunction); + } +} diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/MetricsFacade.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/MetricsFacade.java index 23f489aaf8..9608763425 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/MetricsFacade.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/MetricsFacade.java @@ -27,6 +27,7 @@ public class MetricsFacade { private final OffsetCommitsMetrics offsetCommitsMetrics; private final MaxRateMetrics maxRateMetrics; private final BrokerMetrics brokerMetrics; + private final ConsistencyMetrics consistencyMetrics; public MetricsFacade(MeterRegistry meterRegistry) { this.meterRegistry = meterRegistry; @@ -45,6 +46,7 @@ public MetricsFacade(MeterRegistry meterRegistry) { this.offsetCommitsMetrics = new OffsetCommitsMetrics(meterRegistry); this.maxRateMetrics = new MaxRateMetrics(meterRegistry); this.brokerMetrics = new BrokerMetrics(meterRegistry); + this.consistencyMetrics = new ConsistencyMetrics(meterRegistry); } public TopicMetrics topics() { @@ -107,6 +109,10 @@ public BrokerMetrics broker() { return brokerMetrics; } + public ConsistencyMetrics consistency() { + return consistencyMetrics; + } + public void unregisterAllMetricsRelatedTo(SubscriptionName subscription) { Collection meters = Search.in(meterRegistry) .tags(subscriptionTags(subscription)) diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ConsistencyCheckerProperties.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ConsistencyCheckerProperties.java index 382e2c2ca5..fba1977eb3 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ConsistencyCheckerProperties.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ConsistencyCheckerProperties.java @@ -2,10 +2,15 @@ import org.springframework.boot.context.properties.ConfigurationProperties; +import java.time.Duration; + @ConfigurationProperties(prefix = "consistency-checker") public class ConsistencyCheckerProperties { private int threadPoolSize = 2; + private boolean periodicCheckEnabled = false; + private Duration refreshInterval = Duration.ofMinutes(15); + private Duration initialRefreshDelay = Duration.ofMinutes(2); public int getThreadPoolSize() { return threadPoolSize; @@ -14,4 +19,30 @@ public int getThreadPoolSize() { public void setThreadPoolSize(int threadPoolSize) { this.threadPoolSize = threadPoolSize; } + + + public boolean isPeriodicCheckEnabled() { + return periodicCheckEnabled; + } + + public void setPeriodicCheckEnabled(boolean periodicCheckEnabled) { + this.periodicCheckEnabled = periodicCheckEnabled; + } + + + public Duration getRefreshInterval() { + return refreshInterval; + } + + public void setRefreshInterval(Duration refreshInterval) { + this.refreshInterval = refreshInterval; + } + + public Duration getInitialRefreshDelay() { + return initialRefreshDelay; + } + + public void setInitialRefreshDelay(Duration initialRefreshDelay) { + this.initialRefreshDelay = initialRefreshDelay; + } } diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/consistency/DcConsistencyService.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/consistency/DcConsistencyService.java index 2b23998d36..9218362f0c 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/consistency/DcConsistencyService.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/consistency/DcConsistencyService.java @@ -4,6 +4,8 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.util.concurrent.ThreadFactoryBuilder; import jakarta.annotation.PreDestroy; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.springframework.stereotype.Component; import pl.allegro.tech.hermes.api.Group; import pl.allegro.tech.hermes.api.InconsistentGroup; @@ -13,6 +15,7 @@ import pl.allegro.tech.hermes.api.Subscription; import pl.allegro.tech.hermes.api.Topic; import pl.allegro.tech.hermes.api.TopicName; +import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.domain.group.GroupNotExistsException; import pl.allegro.tech.hermes.domain.group.GroupRepository; import pl.allegro.tech.hermes.domain.subscription.SubscriptionRepository; @@ -31,6 +34,9 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Function; import static java.util.Collections.emptyList; @@ -39,15 +45,20 @@ @Component public class DcConsistencyService { + private static final Logger logger = LoggerFactory.getLogger(DcConsistencyService.class); + private final ExecutorService executor; + private final ScheduledExecutorService scheduler; private final List> groupRepositories; private final List> topicRepositories; private final List> subscriptionRepositories; private final ObjectMapper objectMapper; + private final AtomicBoolean isStorageConsistent = new AtomicBoolean(true); public DcConsistencyService(RepositoryManager repositoryManager, ObjectMapper objectMapper, - ConsistencyCheckerProperties properties) { + ConsistencyCheckerProperties properties, + MetricsFacade metricsFacade) { this.groupRepositories = repositoryManager.getRepositories(GroupRepository.class); this.topicRepositories = repositoryManager.getRepositories(TopicRepository.class); this.subscriptionRepositories = repositoryManager.getRepositories(SubscriptionRepository.class); @@ -58,11 +69,33 @@ public DcConsistencyService(RepositoryManager repositoryManager, .setNameFormat("consistency-checker-%d") .build() ); + this.scheduler = Executors.newSingleThreadScheduledExecutor( + new ThreadFactoryBuilder() + .setNameFormat("consistency-checker-scheduler-%d") + .build() + ); + if (properties.isPeriodicCheckEnabled()) { + scheduler.scheduleAtFixedRate(this::reportConsistency, + properties.getInitialRefreshDelay().getSeconds(), + properties.getRefreshInterval().getSeconds(), + TimeUnit.SECONDS); + metricsFacade.consistency().registerStorageConsistencyGauge(isStorageConsistent, isConsistent -> isConsistent.get() ? 1 : 0); + } } @PreDestroy public void stop() { executor.shutdown(); + scheduler.shutdown(); + } + + private void reportConsistency() { + long start = System.currentTimeMillis(); + Set groups = listAllGroupNames(); + List inconsistentGroups = listInconsistentGroups(groups); + long durationSeconds = (System.currentTimeMillis() - start) / 1000; + logger.info("Consistency check finished in {}s, number of inconsistent groups: {}", durationSeconds, inconsistentGroups.size()); + isStorageConsistent.set(inconsistentGroups.isEmpty()); } public List listInconsistentGroups(Set groupNames) { @@ -208,4 +241,6 @@ private T resolveFuture(Future future) { throw new ConsistencyCheckingException("Fetching metadata failed", e); } } + + } diff --git a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/domain/consistency/DcConsistencyServiceSpec.groovy b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/domain/consistency/DcConsistencyServiceSpec.groovy index 9fb2d7adad..f451d8c92f 100644 --- a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/domain/consistency/DcConsistencyServiceSpec.groovy +++ b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/domain/consistency/DcConsistencyServiceSpec.groovy @@ -2,11 +2,16 @@ package pl.allegro.tech.hermes.management.domain.consistency import com.fasterxml.jackson.databind.ObjectMapper import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule +import io.micrometer.core.instrument.simple.SimpleMeterRegistry import pl.allegro.tech.hermes.api.Group import pl.allegro.tech.hermes.api.Subscription import pl.allegro.tech.hermes.api.Topic +import pl.allegro.tech.hermes.common.metric.MetricsFacade import pl.allegro.tech.hermes.management.config.ConsistencyCheckerProperties import spock.lang.Specification +import spock.util.concurrent.PollingConditions + +import java.time.Duration import static pl.allegro.tech.hermes.test.helper.builder.GroupBuilder.group import static pl.allegro.tech.hermes.test.helper.builder.SubscriptionBuilder.subscription @@ -15,6 +20,8 @@ import static pl.allegro.tech.hermes.test.helper.builder.TopicBuilder.topic class DcConsistencyServiceSpec extends Specification { def objectMapper = new ObjectMapper().registerModule(new JavaTimeModule()) + def meterRegistry = new SimpleMeterRegistry() + def metricsFacade = new MetricsFacade(meterRegistry) def "should return empty list when given groups are consistent"() { given: @@ -30,8 +37,8 @@ class DcConsistencyServiceSpec extends Specification { .addGroup(group) .addTopic(topic) .addSubscription(subscription) - DcConsistencyService dcConsistencyService = new DcConsistencyService(repositoryManager, objectMapper, - new ConsistencyCheckerProperties()) + DcConsistencyService dcConsistencyService = new DcConsistencyService(repositoryManager, + objectMapper, new ConsistencyCheckerProperties(), metricsFacade) when: def inconsistentGroups = dcConsistencyService.listInconsistentGroups([group.groupName] as Set) @@ -48,8 +55,8 @@ class DcConsistencyServiceSpec extends Specification { repositoryManager.datacenter("dc2") .addGroup(group("testGroup").build()) .addGroup(group("testGroup-dc2").build()) - DcConsistencyService consistencyService = new DcConsistencyService(repositoryManager, objectMapper, - new ConsistencyCheckerProperties()) + DcConsistencyService consistencyService = new DcConsistencyService(repositoryManager, + objectMapper, new ConsistencyCheckerProperties(), metricsFacade) when: def groups = consistencyService.listInconsistentGroups(["testGroup", "testGroup-dc1", "testGroup-dc2"] as Set) @@ -68,7 +75,7 @@ class DcConsistencyServiceSpec extends Specification { .addGroup(group) .addTopic(topic(group.groupName, "testTopic").withDescription("dc2").build()) DcConsistencyService consistencyService = new DcConsistencyService(repositoryManager, objectMapper, - new ConsistencyCheckerProperties()) + new ConsistencyCheckerProperties(), metricsFacade) when: def groups = consistencyService.listInconsistentGroups(["testGroup"] as Set) @@ -90,7 +97,7 @@ class DcConsistencyServiceSpec extends Specification { .addTopic(topic) .addSubscription(subscription(topic, "testSubscription").withDescription("dc2").build()) DcConsistencyService consistencyService = new DcConsistencyService(repositoryManager, objectMapper, - new ConsistencyCheckerProperties()) + new ConsistencyCheckerProperties(), metricsFacade) when: def groups = consistencyService.listInconsistentGroups(["testGroup"] as Set) @@ -108,7 +115,7 @@ class DcConsistencyServiceSpec extends Specification { .addGroup(group("testGroup").build()) .addGroup(group("testGroup-dc2").build()) DcConsistencyService consistencyService = new DcConsistencyService(repositoryManager, objectMapper, - new ConsistencyCheckerProperties()) + new ConsistencyCheckerProperties(), metricsFacade) when: def groups = consistencyService.listAllGroupNames() @@ -116,4 +123,56 @@ class DcConsistencyServiceSpec extends Specification { then: groups == ["testGroup", "testGroup-dc1", "testGroup-dc2"] as Set } + + def "should report storage as not consistent with periodic check"() { + given: "inconsistent storage state" + MockRepositoryManager repositoryManager = new MockRepositoryManager() + repositoryManager.datacenter("dc1") + .addGroup(group("testGroup").build()) + .addGroup(group("testGroup-dc1").build()) + repositoryManager.datacenter("dc2") + .addGroup(group("testGroup").build()) + .addGroup(group("testGroup-dc2").build()) + + and: "enabled periodic consistency checks" + def properties = new ConsistencyCheckerProperties() + properties.setPeriodicCheckEnabled(true) + properties.setInitialRefreshDelay(Duration.ofMillis(0)) + + when: "consistency service is created" + DcConsistencyService consistencyService = new DcConsistencyService(repositoryManager, + objectMapper, + properties, + metricsFacade) + + then: "storage is reported as not consistent" + new PollingConditions(timeout: 10).eventually { + meterRegistry.get("storage.consistency").gauge().value() == 0.0d + } + } + + def "should report storage as consistent with periodic check"() { + given: "consistent storage state" + MockRepositoryManager repositoryManager = new MockRepositoryManager() + repositoryManager.datacenter("dc1") + .addGroup(group("testGroup").build()) + repositoryManager.datacenter("dc2") + .addGroup(group("testGroup").build()) + + and: "enabled periodic consistency checks" + def properties = new ConsistencyCheckerProperties() + properties.setPeriodicCheckEnabled(true) + properties.setInitialRefreshDelay(Duration.ofMillis(0)) + + when: "consistency service is created" + DcConsistencyService consistencyService = new DcConsistencyService(repositoryManager, + objectMapper, + properties, + metricsFacade) + + then: "storage is reported as consistent" + new PollingConditions(timeout: 10).eventually { + meterRegistry.get("storage.consistency").gauge().value() == 1.0d + } + } } From d863e8f93aa4e6a181ef84a1c7ad89e505919e1e Mon Sep 17 00:00:00 2001 From: Maciej Moscicki Date: Wed, 14 Aug 2024 11:27:59 +0200 Subject: [PATCH 77/87] update batch receiver subscription update (#1889) --- .../pl/allegro/tech/hermes/consumers/consumer/BatchConsumer.java | 1 + 1 file changed, 1 insertion(+) diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/BatchConsumer.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/BatchConsumer.java index 8ae220fa94..f83080e2a7 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/BatchConsumer.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/BatchConsumer.java @@ -200,6 +200,7 @@ public void tearDown() { @Override public void updateSubscription(Subscription subscription) { this.subscription = subscription; + receiver.updateSubscription(subscription); } @Override From af320d61d8903e9636ce37aef42dcfbc5c551c3d Mon Sep 17 00:00:00 2001 From: bartekdrobczyk <165910486+bartekdrobczyk@users.noreply.github.com> Date: Mon, 19 Aug 2024 16:28:21 +0200 Subject: [PATCH 78/87] Offline retransmission extension with view parameter (#1881) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * SPAW-942 implementation draft * SPAW-942 test coverage & cleanup * SPAW-942 test coverage & cleanup * commit msg adjustment * test coverage & cleanup * test coverage & cleanup * checkstyle * retransmission view name refactor * pr adjustments * checkstyle * jdk8Module registered in ObjectMapper * validation check fix in shouldReturnClientErrorWhenRequestingRetransmissionWithEmptyData test * validation adjustment * validation adjustment * validation testing * validation testing * validation testing * validation testing * pr adjustments * checkstyle * Update hermes-management/src/main/java/pl/allegro/tech/hermes/management/api/OfflineRetransmissionEndpoint.java Co-authored-by: Maciej Moscicki * permission method better naming --------- Co-authored-by: Adam Izydorczyk Co-authored-by: Maciej Moscicki Co-authored-by: Daniel Fąderski --- .../api/OfflineRetransmissionRequest.java | 16 +++++-- .../hermes/api/OfflineRetransmissionTask.java | 9 +++- .../pl/allegro/tech/hermes/api/TopicName.java | 2 +- .../constraints/OneSourceRetransmission.java | 22 +++++++++ .../OneSourceRetransmissionValidator.java | 24 ++++++++++ ...neSourceRetransmissionValidatorTest.groovy | 37 +++++++++++++++ .../api/OfflineRetransmissionEndpoint.java | 26 ++++++---- .../config/ManagementConfiguration.java | 3 +- .../OfflineRetransmissionService.java | 9 ++-- .../OfflineRetransmissionManagementTest.java | 47 ++++++++++++++----- 10 files changed, 164 insertions(+), 31 deletions(-) create mode 100644 hermes-api/src/main/java/pl/allegro/tech/hermes/api/constraints/OneSourceRetransmission.java create mode 100644 hermes-api/src/main/java/pl/allegro/tech/hermes/api/constraints/OneSourceRetransmissionValidator.java create mode 100644 hermes-api/src/test/groovy/pl/allegro/tech/hermes/api/constraints/OneSourceRetransmissionValidatorTest.groovy diff --git a/hermes-api/src/main/java/pl/allegro/tech/hermes/api/OfflineRetransmissionRequest.java b/hermes-api/src/main/java/pl/allegro/tech/hermes/api/OfflineRetransmissionRequest.java index e21fcc2d31..a63b80f073 100644 --- a/hermes-api/src/main/java/pl/allegro/tech/hermes/api/OfflineRetransmissionRequest.java +++ b/hermes-api/src/main/java/pl/allegro/tech/hermes/api/OfflineRetransmissionRequest.java @@ -7,6 +7,7 @@ import jakarta.validation.constraints.NotNull; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import pl.allegro.tech.hermes.api.constraints.OneSourceRetransmission; import pl.allegro.tech.hermes.api.jackson.InstantIsoSerializer; import java.time.Instant; @@ -14,7 +15,9 @@ import java.time.format.DateTimeFormatter; import java.time.format.DateTimeParseException; import java.util.List; +import java.util.Optional; +@OneSourceRetransmission public class OfflineRetransmissionRequest { private static final List formatters = List.of( @@ -24,7 +27,7 @@ public class OfflineRetransmissionRequest { ); private static final Logger logger = LoggerFactory.getLogger(OfflineRetransmissionRequest.class); - @NotEmpty + private final String sourceViewPath; private final String sourceTopic; @NotEmpty private final String targetTopic; @@ -35,10 +38,12 @@ public class OfflineRetransmissionRequest { @JsonCreator public OfflineRetransmissionRequest( + @JsonProperty("sourceViewPath") String sourceViewPath, @JsonProperty("sourceTopic") String sourceTopic, @JsonProperty("targetTopic") String targetTopic, @JsonProperty("startTimestamp") String startTimestamp, @JsonProperty("endTimestamp") String endTimestamp) { + this.sourceViewPath = sourceViewPath; this.sourceTopic = sourceTopic; this.targetTopic = targetTopic; this.startTimestamp = initializeTimestamp(startTimestamp); @@ -62,8 +67,12 @@ private Instant initializeTimestamp(String timestamp) { return null; } - public String getSourceTopic() { - return sourceTopic; + public Optional getSourceViewPath() { + return Optional.ofNullable(sourceViewPath); + } + + public Optional getSourceTopic() { + return Optional.ofNullable(sourceTopic); } public String getTargetTopic() { @@ -84,6 +93,7 @@ public Instant getEndTimestamp() { public String toString() { return "OfflineRetransmissionRequest{" + "sourceTopic='" + sourceTopic + '\'' + + ", sourceViewPath='" + sourceViewPath + '\'' + ", targetTopic='" + targetTopic + '\'' + ", startTimestamp=" + startTimestamp + ", endTimestamp=" + endTimestamp diff --git a/hermes-api/src/main/java/pl/allegro/tech/hermes/api/OfflineRetransmissionTask.java b/hermes-api/src/main/java/pl/allegro/tech/hermes/api/OfflineRetransmissionTask.java index 9f7d68d0a3..c3fb94f05c 100644 --- a/hermes-api/src/main/java/pl/allegro/tech/hermes/api/OfflineRetransmissionTask.java +++ b/hermes-api/src/main/java/pl/allegro/tech/hermes/api/OfflineRetransmissionTask.java @@ -7,6 +7,7 @@ import pl.allegro.tech.hermes.api.jackson.InstantIsoSerializer; import java.time.Instant; +import java.util.Optional; public class OfflineRetransmissionTask { private final String taskId; @@ -16,12 +17,14 @@ public class OfflineRetransmissionTask { @JsonCreator public OfflineRetransmissionTask( @JsonProperty("taskId") String taskId, + @JsonProperty("sourceViewPath") String sourceViewPath, @JsonProperty("sourceTopic") String sourceTopic, @JsonProperty("targetTopic") String targetTopic, @JsonProperty("startTimestamp") Instant startTimestamp, @JsonProperty("endTimestamp") Instant endTimestamp, @JsonProperty("createdAt") Instant createdAt) { this(taskId, new OfflineRetransmissionRequest( + sourceViewPath, sourceTopic, targetTopic, startTimestamp.toString(), @@ -39,10 +42,14 @@ public String getTaskId() { return taskId; } - public String getSourceTopic() { + public Optional getSourceTopic() { return request.getSourceTopic(); } + public Optional getSourceViewPath() { + return request.getSourceViewPath(); + } + public String getTargetTopic() { return request.getTargetTopic(); } diff --git a/hermes-api/src/main/java/pl/allegro/tech/hermes/api/TopicName.java b/hermes-api/src/main/java/pl/allegro/tech/hermes/api/TopicName.java index 32c903fdb2..3c9704f211 100644 --- a/hermes-api/src/main/java/pl/allegro/tech/hermes/api/TopicName.java +++ b/hermes-api/src/main/java/pl/allegro/tech/hermes/api/TopicName.java @@ -39,7 +39,7 @@ public static TopicName fromQualifiedName(String qualifiedName) { int index = qualifiedName.lastIndexOf(GROUP_SEPARATOR); if (index == -1) { - throw new IllegalArgumentException("Missing group"); + throw new IllegalArgumentException("Invalid qualified name " + qualifiedName); } String groupName = qualifiedName.substring(0, index); diff --git a/hermes-api/src/main/java/pl/allegro/tech/hermes/api/constraints/OneSourceRetransmission.java b/hermes-api/src/main/java/pl/allegro/tech/hermes/api/constraints/OneSourceRetransmission.java new file mode 100644 index 0000000000..9c920a322b --- /dev/null +++ b/hermes-api/src/main/java/pl/allegro/tech/hermes/api/constraints/OneSourceRetransmission.java @@ -0,0 +1,22 @@ +package pl.allegro.tech.hermes.api.constraints; + +import jakarta.validation.Constraint; + +import java.lang.annotation.Documented; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +import static java.lang.annotation.ElementType.TYPE; + +@Documented +@Retention(RetentionPolicy.RUNTIME) +@Target({TYPE}) +@Constraint(validatedBy = OneSourceRetransmissionValidator.class) +public @interface OneSourceRetransmission { + String message() default "must contain one defined source of retransmission data - source topic or source view"; + + Class[] groups() default {}; + + Class[] payload() default {}; +} diff --git a/hermes-api/src/main/java/pl/allegro/tech/hermes/api/constraints/OneSourceRetransmissionValidator.java b/hermes-api/src/main/java/pl/allegro/tech/hermes/api/constraints/OneSourceRetransmissionValidator.java new file mode 100644 index 0000000000..a311b4b263 --- /dev/null +++ b/hermes-api/src/main/java/pl/allegro/tech/hermes/api/constraints/OneSourceRetransmissionValidator.java @@ -0,0 +1,24 @@ +package pl.allegro.tech.hermes.api.constraints; + +import jakarta.validation.ConstraintValidator; +import jakarta.validation.ConstraintValidatorContext; +import pl.allegro.tech.hermes.api.OfflineRetransmissionRequest; + +public class OneSourceRetransmissionValidator implements ConstraintValidator { + + public static final String EMPTY_STRING = ""; + + @Override + public boolean isValid(OfflineRetransmissionRequest offlineRetransmissionRequest, ConstraintValidatorContext context) { + var sourceViewPath = offlineRetransmissionRequest.getSourceViewPath(); + var sourceTopic = offlineRetransmissionRequest.getSourceTopic(); + + return (nonBlank(sourceViewPath.orElse(EMPTY_STRING)) && sourceTopic.isEmpty()) + || (nonBlank(sourceTopic.orElse(EMPTY_STRING)) && sourceViewPath.isEmpty()); + } + + private static boolean nonBlank(String value) { + return value != null && !value.isBlank(); + } + +} diff --git a/hermes-api/src/test/groovy/pl/allegro/tech/hermes/api/constraints/OneSourceRetransmissionValidatorTest.groovy b/hermes-api/src/test/groovy/pl/allegro/tech/hermes/api/constraints/OneSourceRetransmissionValidatorTest.groovy new file mode 100644 index 0000000000..e7aa7a42c3 --- /dev/null +++ b/hermes-api/src/test/groovy/pl/allegro/tech/hermes/api/constraints/OneSourceRetransmissionValidatorTest.groovy @@ -0,0 +1,37 @@ +package pl.allegro.tech.hermes.api.constraints + + +import jakarta.validation.ConstraintValidatorContext +import pl.allegro.tech.hermes.api.OfflineRetransmissionRequest +import spock.lang.Specification + +class OneSourceRetransmissionValidatorTest extends Specification { + + OneSourceRetransmissionValidator validator = new OneSourceRetransmissionValidator() + ConstraintValidatorContext mockContext = Mock() + + def "Validator should validate retransmission request when sourceViewPath is '#sourceViewPath' and sourceTopic is '#sourceTopic'"() { + given: + def request = new OfflineRetransmissionRequest( + sourceViewPath, + sourceTopic, + "someTargetTopic", + "2024-07-08T12:00:00", + "2024-07-08T13:00:00" + ) + expect: + validator.isValid(request, mockContext) == isValid + + where: + sourceViewPath | sourceTopic | isValid + null | "testTopic" | true + "testView" | null | true + null | null | false + "testView" | "testTopic" | false + "" | "" | false + " " | " " | false + "" | "testTopic" | false + "testView" | " " | false + } + +} diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/api/OfflineRetransmissionEndpoint.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/api/OfflineRetransmissionEndpoint.java index 23662d469d..2cbd5ea6ed 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/api/OfflineRetransmissionEndpoint.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/api/OfflineRetransmissionEndpoint.java @@ -17,7 +17,6 @@ import org.springframework.stereotype.Component; import pl.allegro.tech.hermes.api.OfflineRetransmissionRequest; import pl.allegro.tech.hermes.api.OfflineRetransmissionTask; -import pl.allegro.tech.hermes.api.Topic; import pl.allegro.tech.hermes.api.TopicName; import pl.allegro.tech.hermes.domain.topic.TopicRepository; import pl.allegro.tech.hermes.management.api.auth.ManagementRights; @@ -25,6 +24,7 @@ import pl.allegro.tech.hermes.management.domain.retransmit.OfflineRetransmissionService; import java.util.List; +import java.util.Optional; import static jakarta.ws.rs.core.MediaType.APPLICATION_JSON; @@ -36,6 +36,7 @@ public class OfflineRetransmissionEndpoint { private final OfflineRetransmissionService retransmissionService; private final RetransmissionPermissions permissions; private final OfflineRetransmissionAuditor auditor; + private final Logger logger = LoggerFactory.getLogger(OfflineRetransmissionEndpoint.class); public OfflineRetransmissionEndpoint(OfflineRetransmissionService retransmissionService, TopicRepository topicRepository, ManagementRights managementRights) { @@ -47,9 +48,10 @@ public OfflineRetransmissionEndpoint(OfflineRetransmissionService retransmission @POST @Consumes(APPLICATION_JSON) public Response createRetransmissionTask(@Valid OfflineRetransmissionRequest request, @Context ContainerRequestContext requestContext) { + logger.info("Offline retransmission request: {}", request); retransmissionService.validateRequest(request); - permissions.ensurePermissionsToBothTopics(request, requestContext); - OfflineRetransmissionTask task = retransmissionService.createTask(request); + permissions.ensurePermissions(request, requestContext); + var task = retransmissionService.createTask(request); auditor.auditRetransmissionCreation(request, requestContext, task); return Response.status(Response.Status.CREATED).build(); } @@ -68,24 +70,30 @@ public Response deleteRetransmissionTask(@PathParam("taskId") String taskId) { } private static class RetransmissionPermissions { + private final Logger logger = LoggerFactory.getLogger(RetransmissionPermissions.class); private final TopicRepository topicRepository; private final ManagementRights managementRights; - private RetransmissionPermissions(TopicRepository topicRepository, ManagementRights managementRights) { this.topicRepository = topicRepository; this.managementRights = managementRights; } - private void ensurePermissionsToBothTopics(OfflineRetransmissionRequest request, ContainerRequestContext requestContext) { - Topic sourceTopic = topicRepository.getTopicDetails(TopicName.fromQualifiedName(request.getSourceTopic())); - Topic targetTopic = topicRepository.getTopicDetails(TopicName.fromQualifiedName(request.getTargetTopic())); - boolean hasPermissions = managementRights.isUserAllowedToManageTopic(sourceTopic, requestContext) - && managementRights.isUserAllowedToManageTopic(targetTopic, requestContext); + private void ensurePermissions(OfflineRetransmissionRequest request, ContainerRequestContext requestContext) { + var targetTopic = topicRepository.getTopicDetails(TopicName.fromQualifiedName(request.getTargetTopic())); + var hasPermissions = validateSourceTopic(request.getSourceTopic(), requestContext) && managementRights.isUserAllowedToManageTopic(targetTopic, requestContext); if (!hasPermissions) { + logger.info("User {} has no permissions to make offline retransmission {}", requestContext.getSecurityContext().getUserPrincipal(), request); throw new PermissionDeniedException("User needs permissions to source and target topics."); } } + + private boolean validateSourceTopic(Optional sourceTopic, ContainerRequestContext requestContext) { + return sourceTopic.isEmpty() || managementRights.isUserAllowedToManageTopic( + topicRepository.getTopicDetails(TopicName.fromQualifiedName(sourceTopic.get())), + requestContext + ); + } } private static class OfflineRetransmissionAuditor { diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ManagementConfiguration.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ManagementConfiguration.java index e4cc665d9a..ffaaf0ba99 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ManagementConfiguration.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ManagementConfiguration.java @@ -5,6 +5,7 @@ import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.SerializationFeature; +import com.fasterxml.jackson.datatype.jdk8.Jdk8Module; import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule; import io.micrometer.core.instrument.MeterRegistry; import org.springframework.beans.factory.annotation.Autowired; @@ -42,7 +43,7 @@ public ObjectMapper objectMapper() { mapper.setSerializationInclusion(JsonInclude.Include.NON_NULL); mapper.disable(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES); mapper.disable(SerializationFeature.WRITE_NULL_MAP_VALUES); - mapper.registerModule(new JavaTimeModule()); + mapper.registerModules(new JavaTimeModule(), new Jdk8Module()); // Jdk8Module is required for Jackson to serialize & deserialize Optional type final InjectableValues defaultSchemaIdAwareSerializationEnabled = new InjectableValues.Std().addValue( Topic.DEFAULT_SCHEMA_ID_SERIALIZATION_ENABLED_KEY, diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/retransmit/OfflineRetransmissionService.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/retransmit/OfflineRetransmissionService.java index 6d5633e25e..0c7259da15 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/retransmit/OfflineRetransmissionService.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/retransmit/OfflineRetransmissionService.java @@ -21,7 +21,7 @@ public OfflineRetransmissionService(OfflineRetransmissionRepository offlineRetra } public void validateRequest(OfflineRetransmissionRequest request) { - TopicName sourceTopicName = TopicName.fromQualifiedName(request.getSourceTopic()); + TopicName sourceTopicName = TopicName.fromQualifiedName(request.getSourceTopic().orElse(null)); TopicName targetTopicName = TopicName.fromQualifiedName(request.getTargetTopic()); ensureTopicsExist(sourceTopicName, targetTopicName); @@ -49,12 +49,11 @@ public void deleteTask(String taskId) { } private void ensureTopicsExist(TopicName sourceTopicName, TopicName targetTopicName) { - boolean sourceTopicExists = topicRepository.topicExists(sourceTopicName); - boolean targetTopicExists = topicRepository.topicExists(targetTopicName); - if (!sourceTopicExists) { + if (sourceTopicName != null && !topicRepository.topicExists(sourceTopicName)) { throw new OfflineRetransmissionValidationException("Source topic does not exist"); } - if (!targetTopicExists) { + + if (!topicRepository.topicExists(targetTopicName)) { throw new OfflineRetransmissionValidationException("Target topic does not exist"); } } diff --git a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/OfflineRetransmissionManagementTest.java b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/OfflineRetransmissionManagementTest.java index 27847ad2e6..27739c7f72 100644 --- a/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/OfflineRetransmissionManagementTest.java +++ b/integration-tests/src/integrationTest/java/pl/allegro/tech/hermes/integrationtests/management/OfflineRetransmissionManagementTest.java @@ -45,7 +45,7 @@ public void shouldCreateRetransmissionTask() { // when OfflineRetransmissionRequest request = createRequest( sourceTopic.getQualifiedName(), - targetTopic.getQualifiedName()); + targetTopic.getQualifiedName(), null); WebTestClient.ResponseSpec response = hermes.api().createOfflineRetransmissionTask(request); Instant now = Instant.now(); @@ -62,6 +62,29 @@ public void shouldCreateRetransmissionTask() { assertThat(allTasks.get(0).getCreatedAt()).isBefore(now); } + @Test + public void shouldCreateRetransmissionTaskWithViewInsteadTopic() { + // given + var targetTopic = hermes.initHelper().createTopic(topicWithRandomName().build()); + + // when + var request = createRequest(null, targetTopic.getQualifiedName(), "testViewPath"); + var response = hermes.api().createOfflineRetransmissionTask(request); + var now = Instant.now(); + + //then + response.expectStatus().isCreated(); + + // and + var allTasks = getOfflineRetransmissionTasks(); + assertThat(allTasks.size()).isEqualTo(1); + assertThat(allTasks.get(0).getStartTimestamp()).isEqualTo(request.getStartTimestamp()); + assertThat(allTasks.get(0).getEndTimestamp()).isEqualTo(request.getEndTimestamp()); + assertThat(allTasks.get(0).getSourceTopic()).isEmpty(); + assertThat(allTasks.get(0).getSourceViewPath()).hasValue("testViewPath"); + assertThat(allTasks.get(0).getTargetTopic()).isEqualTo(request.getTargetTopic()); + assertThat(allTasks.get(0).getCreatedAt()).isBefore(now); + } @Test public void shouldReturnEmptyListIfThereAreNoTasks() { @@ -73,6 +96,7 @@ public void shouldReturnEmptyListIfThereAreNoTasks() { public void shouldReturnClientErrorWhenRequestingRetransmissionWithEmptyData() { // given OfflineRetransmissionRequest request = new OfflineRetransmissionRequest( + null, "", "", null, @@ -86,10 +110,9 @@ public void shouldReturnClientErrorWhenRequestingRetransmissionWithEmptyData() { response.expectStatus().isBadRequest(); assertThat(response.expectBody(String.class).returnResult().getResponseBody()).contains( List.of( - "sourceTopic must not be empty", - "targetTopic must not be empty", - "startTimestamp must not be null", - "endTimestamp must not be null") + "must contain one defined source of retransmission data - source topic or source view", + "startTimestamp must not be null", + "endTimestamp must not be null") ); } @@ -98,7 +121,7 @@ public void shouldReturnClientErrorWhenRequestingRetransmissionWithNotExistingSo // given Topic targetTopic = hermes.initHelper().createTopic(topicWithRandomName().build()); OfflineRetransmissionRequest request = createRequest("not.existing.sourceTopic", - targetTopic.getQualifiedName()); + targetTopic.getQualifiedName(), null); // when WebTestClient.ResponseSpec response = hermes.api().createOfflineRetransmissionTask(request); @@ -114,7 +137,7 @@ public void shouldReturnClientErrorWhenRequestingRetransmissionWithNotExistingTa // given Topic sourceTopic = hermes.initHelper().createTopic(topicWithRandomName().build()); OfflineRetransmissionRequest request = createRequest( - sourceTopic.getQualifiedName(), "not.existing.targetTopic"); + sourceTopic.getQualifiedName(), "not.existing.targetTopic", null); // when WebTestClient.ResponseSpec response = hermes.api().createOfflineRetransmissionTask(request); @@ -131,6 +154,7 @@ public void shouldReturnClientErrorWhenRequestingRetransmissionWithNegativeTimeR Topic sourceTopic = hermes.initHelper().createTopic(topicWithRandomName().build()); Topic targetTopic = hermes.initHelper().createTopic(topicWithRandomName().build()); OfflineRetransmissionRequest request = new OfflineRetransmissionRequest( + null, sourceTopic.getQualifiedName(), targetTopic.getQualifiedName(), Instant.now().toString(), @@ -151,7 +175,7 @@ public void shouldReturnClientErrorWhenRequestingRetransmissionWithTargetTopicSt Topic sourceTopic = hermes.initHelper().createTopic(topicWithRandomName().build()); Topic targetTopic = hermes.initHelper().createTopic(topicWithRandomName().withOfflineStorage(1).build()); OfflineRetransmissionRequest request = createRequest( - sourceTopic.getQualifiedName(), targetTopic.getQualifiedName()); + sourceTopic.getQualifiedName(), targetTopic.getQualifiedName(), null); // when WebTestClient.ResponseSpec response = hermes.api().createOfflineRetransmissionTask(request); @@ -170,7 +194,7 @@ public void shouldDeleteRetransmissionTask() { Topic targetTopic = hermes.initHelper().createTopic(topicWithRandomName().build()); OfflineRetransmissionRequest request = createRequest( - sourceTopic.getQualifiedName(), targetTopic.getQualifiedName()); + sourceTopic.getQualifiedName(), targetTopic.getQualifiedName(), null); hermes.api().createOfflineRetransmissionTask(request); List allTasks = getOfflineRetransmissionTasks(); @@ -203,7 +227,7 @@ public void shouldThrowAccessDeniedWhenTryingToCreateTaskWithoutPermissionsToSou Topic targetTopic = hermes.initHelper().createTopic(topicWithRandomName().build()); TestSecurityProvider.setUserIsAdmin(false); OfflineRetransmissionRequest request = createRequest( - sourceTopic.getQualifiedName(), targetTopic.getQualifiedName()); + sourceTopic.getQualifiedName(), targetTopic.getQualifiedName(), null); // when WebTestClient.ResponseSpec response = hermes.api().createOfflineRetransmissionTask(request); @@ -218,8 +242,9 @@ public void shouldThrowAccessDeniedWhenTryingToCreateTaskWithoutPermissionsToSou TestSecurityProvider.reset(); } - private OfflineRetransmissionRequest createRequest(String sourceTopic, String targetTopic) { + private OfflineRetransmissionRequest createRequest(String sourceTopic, String targetTopic, String sourceViewPath) { return new OfflineRetransmissionRequest( + sourceViewPath, sourceTopic, targetTopic, Instant.now().minusSeconds(1).toString(), From 8057c701e56e392385fbc06df719f60310d0bbe8 Mon Sep 17 00:00:00 2001 From: Maciej Moscicki Date: Wed, 21 Aug 2024 13:10:32 +0200 Subject: [PATCH 79/87] Synchronize zookeeper clusters with UI (#1888) --- hermes-console/json-server/db.json | 26 ++ hermes-console/json-server/routes.json | 1 + hermes-console/json-server/server.ts | 16 + hermes-console/src/api/hermes-client/index.ts | 42 ++ .../composables/sync/use-sync/useSync.spec.ts | 175 +++++++++ .../src/composables/sync/use-sync/useSync.ts | 116 ++++++ .../src/dummy/groupInconsistency.ts | 31 ++ hermes-console/src/i18n/en-US/index.ts | 12 + hermes-console/src/mocks/handlers.ts | 44 +++ .../consistency/useConsistencyStore.spec.ts | 43 +- .../store/consistency/useConsistencyStore.ts | 23 ++ .../inconsistent-group/InconsistentGroup.vue | 10 + .../InconsistentMetadata.spec.ts | 31 +- .../InconsistentMetadata.vue | 26 ++ .../inconsistent-topic/InconsistentTopic.vue | 30 +- .../management/api/ConsistencyEndpoint.java | 36 ++ .../consistency/DcConsistencyService.java | 100 +++++ .../consistency/SynchronizationException.java | 7 + .../domain/consistency/StorageSyncSpec.groovy | 369 ++++++++++++++++++ 19 files changed, 1134 insertions(+), 4 deletions(-) create mode 100644 hermes-console/src/composables/sync/use-sync/useSync.spec.ts create mode 100644 hermes-console/src/composables/sync/use-sync/useSync.ts create mode 100644 hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/consistency/SynchronizationException.java create mode 100644 hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/domain/consistency/StorageSyncSpec.groovy diff --git a/hermes-console/json-server/db.json b/hermes-console/json-server/db.json index 8c2720872d..23dab13ee0 100644 --- a/hermes-console/json-server/db.json +++ b/hermes-console/json-server/db.json @@ -91,6 +91,32 @@ ] } ], + "inconsistentGroups3":[ + { + "name": "pl.allegro.public.group", + "inconsistentMetadata": [], + "inconsistentTopics": [ + { + "name": "pl.allegro.public.group.DummyEvent", + "inconsistentMetadata": [], + "inconsistentSubscriptions": [ + { + "name": "pl.allegro.public.group.DummyEvent$foobar-service", + "inconsistentMetadata": [ + { + "datacenter": "DC1" + }, + { + "datacenter": "DC2", + "content": "{\n \"id\": \"foobar-service\",\n \"topicName\": \"pl.allegro.public.group.DummyEvent\",\n \"name\": \"foobar-service\",\n \"endpoint\": \"service://foobar-service/events/dummy-event\",\n \"state\": \"ACTIVE\",\n \"description\": \"Test Hermes endpoint\",\n \"subscriptionPolicy\": {\n \"rate\": 10,\n \"messageTtl\": 60,\n \"messageBackoff\": 100,\n \"requestTimeout\": 1000,\n \"socketTimeout\": 0,\n \"sendingDelay\": 0,\n \"backoffMultiplier\": 1.0,\n \"backoffMaxIntervalInSec\": 600,\n \"retryClientErrors\": true,\n \"backoffMaxIntervalMillis\": 600000\n },\n \"trackingEnabled\": false,\n \"trackingMode\": \"trackingOff\",\n \"owner\": {\n \"source\": \"Service Catalog\",\n \"id\": \"42\"\n },\n \"monitoringDetails\": {\n \"severity\": \"NON_IMPORTANT\",\n \"reaction\": \"\"\n },\n \"contentType\": \"JSON\",\n \"deliveryType\": \"SERIAL\",\n \"filters\": [\n {\n \"type\": \"avropath\",\n \"path\": \"foobar\",\n \"matcher\": \"^FOO_BAR$|^BAZ_BAR$\",\n \"matchingStrategy\": \"any\"\n },\n {\n \"type\": \"avropath\",\n \"path\": \".foo.bar.baz\",\n \"matcher\": \"true\",\n \"matchingStrategy\": \"all\"\n }\n ],\n \"mode\": \"ANYCAST\",\n \"headers\": [\n {\n \"name\": \"X-My-Header\",\n \"value\": \"boobar\"\n },\n {\n \"name\": \"X-Another-Header\",\n \"value\": \"foobar\"\n }\n ],\n \"endpointAddressResolverMetadata\": {\n \"additionalMetadata\": false,\n \"nonSupportedProperty\": 2\n },\n \"http2Enabled\": false,\n \"subscriptionIdentityHeadersEnabled\": false,\n \"autoDeleteWithTopicEnabled\": false,\n \"createdAt\": 1579507131.238,\n \"modifiedAt\": 1672140855.813\n}" + } + ] + } + ] + } + ] + } + ], "topicNames": [ "pl.allegro.public.offer.product.ProductEventV1", "pl.allegro.public.offer.product.ProductEventV2", diff --git a/hermes-console/json-server/routes.json b/hermes-console/json-server/routes.json index eb0b5ae948..0a741b6490 100644 --- a/hermes-console/json-server/routes.json +++ b/hermes-console/json-server/routes.json @@ -3,6 +3,7 @@ "/consistency/groups": "/consistencyGroups", "/consistency/inconsistencies/groups?groupNames=pl.allegro.public.offer*": "/inconsistentGroups", "/consistency/inconsistencies/groups?groupNames=pl.allegro.public.group2*": "/inconsistentGroups2", + "/consistency/inconsistencies/groups?groupNames=pl.allegro.public.group": "/inconsistentGroups3", "/groups": "/groups", "/owners/sources/Service%20Catalog/:id": "/topicsOwners/:id", "/readiness/datacenters": "/readinessDatacenters", diff --git a/hermes-console/json-server/server.ts b/hermes-console/json-server/server.ts index 6c1daab3d7..0a47cb6361 100644 --- a/hermes-console/json-server/server.ts +++ b/hermes-console/json-server/server.ts @@ -87,6 +87,22 @@ server.put( }, ); +server.post( + '/consistency/sync/topics/pl.allegro.public.group.DummyEvent/subscriptions/barbaz-service*', + (req, res) => { + res.sendStatus(200); + }, +); + +server.post( + '/consistency/sync/topics/pl.allegro.public.group.DummyEvent*', + (req, res) => { + res.status(404).jsonp({ + message: 'Group pl.allegro.public.group not found', + }); + }, +); + server.post('/filters/:topic', (req, res) => { res.jsonp(filterDebug); }); diff --git a/hermes-console/src/api/hermes-client/index.ts b/hermes-console/src/api/hermes-client/index.ts index 4c4f6f89a4..23b7ca55ef 100644 --- a/hermes-console/src/api/hermes-client/index.ts +++ b/hermes-console/src/api/hermes-client/index.ts @@ -464,3 +464,45 @@ export function verifyFilters( }, ); } + +export function syncGroup( + groupName: string, + primaryDatacenter: string, +): ResponsePromise { + return axios.post(`/consistency/sync/groups/${groupName}`, null, { + params: { + primaryDatacenter: primaryDatacenter, + }, + }); +} + +export function syncTopic( + topicQualifiedName: string, + primaryDatacenter: string, +): ResponsePromise { + return axios.post( + `/consistency/sync/topics/${topicQualifiedName}`, + null, + { + params: { + primaryDatacenter: primaryDatacenter, + }, + }, + ); +} + +export function syncSubscription( + topicQualifiedName: string, + subscriptionName: string, + primaryDatacenter: string, +): ResponsePromise { + return axios.post( + `/consistency/sync/topics/${topicQualifiedName}/subscriptions/${subscriptionName}`, + null, + { + params: { + primaryDatacenter: primaryDatacenter, + }, + }, + ); +} diff --git a/hermes-console/src/composables/sync/use-sync/useSync.spec.ts b/hermes-console/src/composables/sync/use-sync/useSync.spec.ts new file mode 100644 index 0000000000..e9c57240e4 --- /dev/null +++ b/hermes-console/src/composables/sync/use-sync/useSync.spec.ts @@ -0,0 +1,175 @@ +import { afterEach, describe, expect } from 'vitest'; +import { createTestingPinia } from '@pinia/testing'; +import { + expectNotificationDispatched, + notificationStoreSpy, +} from '@/utils/test-utils'; +import { setActivePinia } from 'pinia'; +import { setupServer } from 'msw/node'; +import { + syncGroupHandler, + syncSubscriptionHandler, + syncTopicHandler, +} from '@/mocks/handlers'; +import { useSync } from '@/composables/sync/use-sync/useSync'; +import { waitFor } from '@testing-library/vue'; + +describe('useSync', () => { + const server = setupServer(); + + const pinia = createTestingPinia({ + fakeApp: true, + }); + + beforeEach(() => { + setActivePinia(pinia); + }); + + afterEach(() => { + server.resetHandlers(); + }); + + it('should show error notification when group sync fails', async () => { + // given + const groupName = 'group'; + server.use(syncGroupHandler({ groupName, statusCode: 500 })); + server.listen(); + + const notificationStore = notificationStoreSpy(); + + // when + const { syncGroup } = useSync(); + const result = await syncGroup(groupName, 'DC1'); + + // then + expect(result).toBeFalsy(); + + await waitFor(() => { + expectNotificationDispatched(notificationStore, { + type: 'error', + title: 'notifications.consistency.sync.failure', + }); + }); + }); + + it('should show error notification when topic sync fails', async () => { + // given + const topicName = 'group.topic'; + server.use(syncTopicHandler({ topicName, statusCode: 500 })); + server.listen(); + + const notificationStore = notificationStoreSpy(); + + // when + const { syncTopic } = useSync(); + const result = await syncTopic(topicName, 'DC1'); + + // then + expect(result).toBeFalsy(); + + await waitFor(() => { + expectNotificationDispatched(notificationStore, { + type: 'error', + title: 'notifications.consistency.sync.failure', + }); + }); + }); + + it('should show error notification when subscription sync fails', async () => { + // given + const topicName = 'group.topic'; + const subscriptionName = 'subscription'; + server.use( + syncSubscriptionHandler({ topicName, subscriptionName, statusCode: 500 }), + ); + server.listen(); + + const notificationStore = notificationStoreSpy(); + + // when + const { syncSubscription } = useSync(); + const result = await syncSubscription(topicName, subscriptionName, 'DC1'); + + // then + expect(result).toBeFalsy(); + + await waitFor(() => { + expectNotificationDispatched(notificationStore, { + type: 'error', + title: 'notifications.consistency.sync.failure', + }); + }); + }); + + it('should show success notification when group sync is successful', async () => { + const groupName = 'group'; + + server.use(syncGroupHandler({ groupName, statusCode: 200 })); + server.listen(); + + const notificationStore = notificationStoreSpy(); + + // when + const { syncGroup } = useSync(); + const result = await syncGroup(groupName, 'DC1'); + + // then + expect(result).toBeTruthy(); + + await waitFor(() => { + expectNotificationDispatched(notificationStore, { + type: 'success', + text: 'notifications.consistency.sync.success', + }); + }); + }); + + it('should show success notification when topic sync is successful', async () => { + // given + const topicName = 'group.topic'; + server.use(syncTopicHandler({ topicName, statusCode: 200 })); + server.listen(); + + const notificationStore = notificationStoreSpy(); + + // when + const { syncTopic } = useSync(); + const result = await syncTopic(topicName, 'DC1'); + + // then + expect(result).toBeTruthy(); + + await waitFor(() => { + expectNotificationDispatched(notificationStore, { + type: 'success', + text: 'notifications.consistency.sync.success', + }); + }); + }); + + it('should show success notification when subscription sync is successful', async () => { + // given + const topicName = 'group.topic'; + const subscriptionName = 'subscription'; + server.use( + syncSubscriptionHandler({ topicName, subscriptionName, statusCode: 200 }), + ); + server.listen(); + + const notificationStore = notificationStoreSpy(); + + // when + const { syncSubscription } = useSync(); + const result = await syncSubscription(topicName, subscriptionName, 'DC1'); + + // then + expect(result).toBeTruthy(); + + await waitFor(() => { + expectNotificationDispatched(notificationStore, { + type: 'success', + text: 'notifications.consistency.sync.success', + }); + }); + }); +}); diff --git a/hermes-console/src/composables/sync/use-sync/useSync.ts b/hermes-console/src/composables/sync/use-sync/useSync.ts new file mode 100644 index 0000000000..ff6ed46f9e --- /dev/null +++ b/hermes-console/src/composables/sync/use-sync/useSync.ts @@ -0,0 +1,116 @@ +import { dispatchErrorNotification } from '@/utils/notification-utils'; +import { + syncGroup as doSyncGroup, + syncSubscription as doSyncSubscription, + syncTopic as doSyncTopic, +} from '@/api/hermes-client'; +import { groupName } from '@/utils/topic-utils/topic-utils'; +import { useConsistencyStore } from '@/store/consistency/useConsistencyStore'; +import { useGlobalI18n } from '@/i18n'; +import { useNotificationsStore } from '@/store/app-notifications/useAppNotifications'; + +export interface UseSync { + syncGroup: (groupName: string, primaryDatacenter: string) => Promise; + syncTopic: ( + topicQualifiedName: string, + primaryDatacenter: string, + ) => Promise; + syncSubscription: ( + topicQualifiedName: string, + subscriptionName: string, + primaryDatacenter: string, + ) => Promise; +} + +export function useSync(): UseSync { + const notificationStore = useNotificationsStore(); + const consistencyStore = useConsistencyStore(); + + const syncGroup = async (groupName: string, primaryDatacenter: string) => { + try { + await doSyncGroup(groupName, primaryDatacenter); + await notificationStore.dispatchNotification({ + text: useGlobalI18n().t('notifications.consistency.sync.success', { + group: groupName, + }), + type: 'success', + }); + await consistencyStore.refresh(groupName); + return true; + } catch (e: any) { + dispatchErrorNotification( + e, + notificationStore, + useGlobalI18n().t('notifications.consistency.sync.failure', { + group: groupName, + }), + ); + return false; + } + }; + + const syncTopic = async ( + topicQualifiedName: string, + primaryDatacenter: string, + ) => { + const group = groupName(topicQualifiedName); + try { + await doSyncTopic(topicQualifiedName, primaryDatacenter); + await notificationStore.dispatchNotification({ + text: useGlobalI18n().t('notifications.consistency.sync.success', { + group, + }), + type: 'success', + }); + await consistencyStore.refresh(group); + return true; + } catch (e: any) { + dispatchErrorNotification( + e, + notificationStore, + useGlobalI18n().t('notifications.consistency.sync.failure', { + group, + }), + ); + return false; + } + }; + + const syncSubscription = async ( + topicQualifiedName: string, + subscriptionName: string, + primaryDatacenter: string, + ) => { + const group = groupName(topicQualifiedName); + try { + await doSyncSubscription( + topicQualifiedName, + subscriptionName, + primaryDatacenter, + ); + await notificationStore.dispatchNotification({ + text: useGlobalI18n().t('notifications.consistency.sync.success', { + group, + }), + type: 'success', + }); + await consistencyStore.refresh(group); + return true; + } catch (e: any) { + dispatchErrorNotification( + e, + notificationStore, + useGlobalI18n().t('notifications.consistency.sync.failure', { + group, + }), + ); + return false; + } + }; + + return { + syncGroup, + syncSubscription, + syncTopic, + }; +} diff --git a/hermes-console/src/dummy/groupInconsistency.ts b/hermes-console/src/dummy/groupInconsistency.ts index ae114b1938..2e06da4fc7 100644 --- a/hermes-console/src/dummy/groupInconsistency.ts +++ b/hermes-console/src/dummy/groupInconsistency.ts @@ -58,3 +58,34 @@ export const dummyGroupInconsistency3: InconsistentGroup[] = [ inconsistentTopics: [], }, ]; + +export const dummyGroupInconsistency4: InconsistentGroup[] = [ + { + name: 'pl.allegro.public.group', + inconsistentMetadata: [ + { + datacenter: 'DC1', + content: '{"lorem": "ipsum"}', + }, + { + datacenter: 'DC2', + content: '{"lorem": "ipsum"}', + }, + ], + inconsistentTopics: [], + }, + { + name: 'pl.allegro.public.group2', + inconsistentMetadata: [ + { + datacenter: 'DC1', + content: '{"lorem": "ipsum"}', + }, + { + datacenter: 'DC2', + content: '{"lorem": "ipsum"}', + }, + ], + inconsistentTopics: [], + }, +]; diff --git a/hermes-console/src/i18n/en-US/index.ts b/hermes-console/src/i18n/en-US/index.ts index 37ce1b5b14..01ec3f08b8 100644 --- a/hermes-console/src/i18n/en-US/index.ts +++ b/hermes-console/src/i18n/en-US/index.ts @@ -47,6 +47,12 @@ const en_US = { confirmText: "Type 'prod' to confirm action.", }, consistency: { + sync: { + header: 'Sync datacenters', + explanation: + 'Pick DC which contains correct data. Data from that DC will be propagated to other DCs.', + cta: 'Correct data is in DC:', + }, connectionError: { title: 'Connection error', text: 'Could not fetch information about consistency', @@ -708,6 +714,12 @@ const en_US = { failure: "Couldn't delete topic {topic}", }, }, + consistency: { + sync: { + success: 'Synchronization of {group} succeeded', + failure: 'Synchronization of {group} failed', + }, + }, subscription: { create: { success: 'Subscription {subscriptionName} successfully created', diff --git a/hermes-console/src/mocks/handlers.ts b/hermes-console/src/mocks/handlers.ts index d2eceac040..ebcccbb847 100644 --- a/hermes-console/src/mocks/handlers.ts +++ b/hermes-console/src/mocks/handlers.ts @@ -975,3 +975,47 @@ export const subscriptionFilterVerificationErrorHandler = ({ status: 500, }); }); + +export const syncGroupHandler = ({ + groupName, + statusCode, +}: { + groupName: string; + statusCode: number; +}) => + http.post(`${url}/consistency/sync/groups/${groupName}`, () => { + return new HttpResponse(undefined, { + status: statusCode, + }); + }); + +export const syncTopicHandler = ({ + topicName, + statusCode, +}: { + topicName: string; + statusCode: number; +}) => + http.post(`${url}/consistency/sync/topics/${topicName}`, () => { + return new HttpResponse(undefined, { + status: statusCode, + }); + }); + +export const syncSubscriptionHandler = ({ + topicName, + statusCode, + subscriptionName, +}: { + topicName: string; + subscriptionName: string; + statusCode: number; +}) => + http.post( + `${url}/consistency/sync/topics/${topicName}/subscriptions/${subscriptionName}`, + () => { + return new HttpResponse(undefined, { + status: statusCode, + }); + }, + ); diff --git a/hermes-console/src/store/consistency/useConsistencyStore.spec.ts b/hermes-console/src/store/consistency/useConsistencyStore.spec.ts index bb5e1b73d1..52dd62e082 100644 --- a/hermes-console/src/store/consistency/useConsistencyStore.spec.ts +++ b/hermes-console/src/store/consistency/useConsistencyStore.spec.ts @@ -1,6 +1,10 @@ import { beforeEach, describe, expect } from 'vitest'; import { createPinia, setActivePinia } from 'pinia'; -import { dummyGroupInconsistency } from '@/dummy/groupInconsistency'; +import { + dummyGroupInconsistency, + dummyGroupInconsistency2, + dummyGroupInconsistency4, +} from '@/dummy/groupInconsistency'; import { dummyInconsistentGroups } from '@/dummy/inconsistentGroups'; import { fetchConsistencyGroupsErrorHandler, @@ -86,4 +90,41 @@ describe('useConsistencyStore', () => { expect(consistencyStore.error.fetchError).not.toBeNull(); }); + + it('should remove group from store when refresh returns that it is consistent', async () => { + // given + server.use(fetchGroupInconsistenciesHandler({ groupsInconsistency: [] })); + server.listen(); + const consistencyStore = useConsistencyStore(); + consistencyStore.groups = dummyGroupInconsistency4; + // make a copy of initial state + const expected = JSON.parse(JSON.stringify(dummyGroupInconsistency4[1])); + + // when + await consistencyStore.refresh(dummyGroupInconsistency4[0].name); + + // then + expect(consistencyStore.groups.length).toEqual(1); + expect(consistencyStore.groups[0]).toEqual(expected); + }); + + it('should update group in store when refresh returns different value', async () => { + // given + server.use( + fetchGroupInconsistenciesHandler({ + groupsInconsistency: dummyGroupInconsistency2, + }), + ); + server.listen(); + const consistencyStore = useConsistencyStore(); + consistencyStore.groups = dummyGroupInconsistency; + // make a copy of initial state + + // when + await consistencyStore.refresh(dummyGroupInconsistency[0].name); + + // then + expect(consistencyStore.groups.length).toEqual(1); + expect(consistencyStore.groups).toEqual(dummyGroupInconsistency2); + }); }); diff --git a/hermes-console/src/store/consistency/useConsistencyStore.ts b/hermes-console/src/store/consistency/useConsistencyStore.ts index 1d18182c20..3a6bbf4a52 100644 --- a/hermes-console/src/store/consistency/useConsistencyStore.ts +++ b/hermes-console/src/store/consistency/useConsistencyStore.ts @@ -52,6 +52,29 @@ export const useConsistencyStore = defineStore('consistency', { this.fetchInProgress = false; } }, + async refresh(group: string) { + this.fetchInProgress = true; + try { + const refreshedGroup = (await fetchInconsistentGroups([group])).data; + let groupIndex = -1; + for (let i = 0; i < this.groups.length; i++) { + if (this.groups[i].name == group) { + groupIndex = i; + break; + } + } + if (groupIndex == -1) return; + if (refreshedGroup.length == 0) { + this.groups.splice(groupIndex, 1); + } else { + this.groups[groupIndex] = refreshedGroup[0]; + } + } catch (e) { + this.error.fetchError = e as Error; + } finally { + this.fetchInProgress = false; + } + }, }, getters: { group( diff --git a/hermes-console/src/views/admin/consistency/inconsistent-group/InconsistentGroup.vue b/hermes-console/src/views/admin/consistency/inconsistent-group/InconsistentGroup.vue index 8352004610..56c7b25696 100644 --- a/hermes-console/src/views/admin/consistency/inconsistent-group/InconsistentGroup.vue +++ b/hermes-console/src/views/admin/consistency/inconsistent-group/InconsistentGroup.vue @@ -2,6 +2,7 @@ import { useConsistencyStore } from '@/store/consistency/useConsistencyStore'; import { useI18n } from 'vue-i18n'; import { useRouter } from 'vue-router'; + import { useSync } from '@/composables/sync/use-sync/useSync'; import InconsistentMetadata from '@/views/admin/consistency/inconsistent-metadata/InconsistentMetadata.vue'; const router = useRouter(); @@ -13,6 +14,7 @@ >; const consistencyStore = useConsistencyStore(); + const { syncGroup } = useSync(); const group = consistencyStore.group(groupId); @@ -33,6 +35,13 @@ title: groupId, }, ]; + + async function sync(datacenter: string) { + const succeeded = await syncGroup(groupId, datacenter); + if (succeeded) { + router.push('/ui/consistency'); + } + }