diff --git a/.github/PULL_REQUEST_TEMPLATE.md b/.github/PULL_REQUEST_TEMPLATE.md index be60eedba..45cc2b6bb 100644 --- a/.github/PULL_REQUEST_TEMPLATE.md +++ b/.github/PULL_REQUEST_TEMPLATE.md @@ -26,6 +26,6 @@ - [ ] New and existing unit tests pass locally with my changes - [ ] Any dependent changes have been merged -Check out [Contributing](https://github.com/kafbat/kafka-ui/blob/main/CONTRIBUTING.md) and [Code of Conduct](https://github.com/kafbat/kafka-ui/blob/main/CODE-OF-CONDUCT.md) +Check out [Contributing](https://github.com/kafbat/kafka-ui/blob/main/.github/CONTRIBUTING.md) and [Code of Conduct](https://github.com/kafbat/kafka-ui/blob/main/.github/CODE-OF-CONDUCT.md) **A picture of a cute animal (not mandatory but encouraged)** diff --git a/.github/workflows/frontend_tests.yml b/.github/workflows/frontend_tests.yml index 4be3239d4..10d5fd4cb 100644 --- a/.github/workflows/frontend_tests.yml +++ b/.github/workflows/frontend_tests.yml @@ -13,37 +13,45 @@ jobs: NODE_ENV: dev runs-on: ubuntu-latest steps: + - uses: actions/checkout@v4 with: # Disabling shallow clone is recommended for improving relevancy of reporting fetch-depth: 0 ref: ${{ github.event.pull_request.head.sha }} token: ${{ github.token }} + - uses: pnpm/action-setup@v3.0.0 with: version: 8.6.12 + - name: Install node uses: actions/setup-node@v4.0.1 with: node-version: "18.17.1" cache: "pnpm" cache-dependency-path: "./frontend/pnpm-lock.yaml" + - name: Install Node dependencies run: | cd frontend/ pnpm install --frozen-lockfile - - name: Generate sources + + - name: Compile run: | cd frontend/ - pnpm gen:sources + pnpm compile + - name: Linter run: | cd frontend/ pnpm lint:CI + - name: Tests run: | cd frontend/ pnpm test:CI + - name: SonarCloud Scan if: false # TODO remove when public uses: sonarsource/sonarcloud-github-action@master diff --git a/.github/workflows/release-serde-api.yml b/.github/workflows/release-serde-api.yml index af5b1db0d..746769152 100644 --- a/.github/workflows/release-serde-api.yml +++ b/.github/workflows/release-serde-api.yml @@ -33,4 +33,8 @@ jobs: - name: Publish to Maven Central run: | - mvn source:jar javadoc:jar package gpg:sign -Dgpg.passphrase=${{ secrets.GPG_PASSPHRASE }} -Dserver.username=${{ secrets.NEXUS_USERNAME }} -Dserver.password=${{ secrets.NEXUS_PASSWORD }} nexus-staging:deploy -pl serde-api -s settings.xml + mvn source:jar javadoc:jar package gpg:sign \ + -Dgpg.passphrase=${{ secrets.GPG_PASSPHRASE }} \ + -Dserver.username=${{ secrets.NEXUS_USERNAME }} \ + -Dserver.password=${{ secrets.NEXUS_PASSWORD }} \ + central-publishing:publish -pl serde-api -s settings.xml diff --git a/README.md b/README.md index 47bde530a..1ef454906 100644 --- a/README.md +++ b/README.md @@ -1,32 +1,46 @@ -![logo](documentation/images/logo.png) Kafbat UI  ------------------- -#### Versatile, fast and lightweight web UI for managing Apache Kafka® clusters. Built by developers, for developers. -
+
+logo +

Kafbat UI

-[![License](https://img.shields.io/badge/License-Apache%202.0-blue.svg)](https://github.com/kafbat/kafka-ui/blob/main/LICENSE) -![Price free](documentation/images/free-open-source.svg) -[![Release version](https://img.shields.io/github/v/release/kafbat/kafka-ui)](https://github.com/kafbat/kafka-ui/releases) -[![Chat with us](https://img.shields.io/discord/897805035122077716)](https://discord.gg/4DWzD7pGE5) -[![Docker pulls](https://img.shields.io/docker/pulls/kafbat/kafka-ui)](https://hub.docker.com/r/kafbat/kafka-ui) +Versatile, fast and lightweight web UI for managing Apache Kafka® clusters. +
+ +
+License +price free +latest release version +discord online number count + +

- DOCS • - QUICK START • - COMMUNITY DISCORD + Documentation • + Quick Start • + Community
AWS MarketplaceProductHunt

- + stats

#### Kafbat UI is a free, open-source web UI to monitor and manage Apache Kafka clusters. Kafbat UI is a simple tool that makes your data flows observable, helps find and troubleshoot issues faster and deliver optimal performance. Its lightweight dashboard makes it easy to track key metrics of your Kafka clusters - Brokers, Topics, Partitions, Production, and Consumption. -![Interface](documentation/images/Interface.gif) + +Kafbat UI, developed by Kafbat*, proudly carries forward the legacy of the UI Apache Kafka project. +Our dedication is reflected in the continuous evolution of the project, ensuring adherence to its foundational vision while adapting to meet modern demands. +We extend our gratitude to Provectus for their past support in groundbreaking work, which serves as a cornerstone for our ongoing innovation and dedication. + +* - The Kafbat team comprises key contributors from the project's inception, bringing a wealth of experience and insight to this renewed endeavor. + + +# Interface + +![Interface](https://raw.githubusercontent.com/kafbat/kafka-ui/images/overview.gif) # Features * **Multi-Cluster Management** — monitor and manage all your clusters in one place @@ -108,7 +122,7 @@ services: - ~/kui/config.yml:/etc/kafkaui/dynamic_config.yaml ``` -Please refer to our [configuration](https://ui.docs.kafbat.io/configuration/quick-start) page to proceed with further app configuration. +Please refer to our [configuration](https://ui.docs.kafbat.io/configuration/configuration-file) page to proceed with further app configuration. ## Some useful configuration related links @@ -134,8 +148,13 @@ Info endpoint (build info) is located at `/actuator/info`. # Configuration options -All of the environment variables/config properties could be found [here](https://ui.docs.kafbat.io/configuration/misc-configuration-properties). +All the environment variables/config properties could be found [here](https://ui.docs.kafbat.io/configuration/misc-configuration-properties). # Contributing Please refer to [contributing guide](https://ui.docs.kafbat.io/development/contributing), we'll guide you from there. + +# Support + +As we're fully independent, team members contribute in their free time. +Your support is crucial for us, if you wish to sponsor us, take a look [here](https://github.com/sponsors/kafbat) diff --git a/api/src/main/java/io/kafbat/ui/controller/MessagesController.java b/api/src/main/java/io/kafbat/ui/controller/MessagesController.java index 978d137b3..62189b04a 100644 --- a/api/src/main/java/io/kafbat/ui/controller/MessagesController.java +++ b/api/src/main/java/io/kafbat/ui/controller/MessagesController.java @@ -1,15 +1,18 @@ package io.kafbat.ui.controller; +import static com.google.common.base.Preconditions.checkNotNull; import static io.kafbat.ui.model.rbac.permission.TopicAction.MESSAGES_DELETE; import static io.kafbat.ui.model.rbac.permission.TopicAction.MESSAGES_PRODUCE; import static io.kafbat.ui.model.rbac.permission.TopicAction.MESSAGES_READ; -import static java.util.stream.Collectors.toMap; import io.kafbat.ui.api.MessagesApi; import io.kafbat.ui.exception.ValidationException; import io.kafbat.ui.model.ConsumerPosition; import io.kafbat.ui.model.CreateTopicMessageDTO; +import io.kafbat.ui.model.MessageFilterIdDTO; +import io.kafbat.ui.model.MessageFilterRegistrationDTO; import io.kafbat.ui.model.MessageFilterTypeDTO; +import io.kafbat.ui.model.PollingModeDTO; import io.kafbat.ui.model.SeekDirectionDTO; import io.kafbat.ui.model.SeekTypeDTO; import io.kafbat.ui.model.SerdeUsageDTO; @@ -24,14 +27,10 @@ import io.kafbat.ui.service.DeserializationService; import io.kafbat.ui.service.MessagesService; import java.util.List; -import java.util.Map; import java.util.Optional; -import javax.annotation.Nullable; import javax.validation.Valid; import lombok.RequiredArgsConstructor; import lombok.extern.slf4j.Slf4j; -import org.apache.commons.lang3.tuple.Pair; -import org.apache.kafka.common.TopicPartition; import org.springframework.http.ResponseEntity; import org.springframework.web.bind.annotation.RestController; import org.springframework.web.server.ServerWebExchange; @@ -74,6 +73,7 @@ public Mono> executeSmartFilte .map(ResponseEntity::ok); } + @Deprecated @Override public Mono>> getTopicMessages(String clusterName, String topicName, @@ -86,6 +86,23 @@ public Mono>> getTopicMessages(String String keySerde, String valueSerde, ServerWebExchange exchange) { + throw new ValidationException("Not supported"); + } + + + @Override + public Mono>> getTopicMessagesV2(String clusterName, String topicName, + PollingModeDTO mode, + List partitions, + Integer limit, + String stringFilter, + String smartFilterId, + Long offset, + Long timestamp, + String keySerde, + String valueSerde, + String cursor, + ServerWebExchange exchange) { var contextBuilder = AccessContext.builder() .cluster(clusterName) .topicActions(topicName, MESSAGES_READ) @@ -95,27 +112,26 @@ public Mono>> getTopicMessages(String contextBuilder.auditActions(AuditAction.VIEW); } - seekType = seekType != null ? seekType : SeekTypeDTO.BEGINNING; - seekDirection = seekDirection != null ? seekDirection : SeekDirectionDTO.FORWARD; - filterQueryType = filterQueryType != null ? filterQueryType : MessageFilterTypeDTO.STRING_CONTAINS; - - var positions = new ConsumerPosition( - seekType, - topicName, - parseSeekTo(topicName, seekType, seekTo) - ); - Mono>> job = Mono.just( - ResponseEntity.ok( - messagesService.loadMessages( - getCluster(clusterName), topicName, positions, q, filterQueryType, - limit, seekDirection, keySerde, valueSerde) - ) - ); - - var context = contextBuilder.build(); - return validateAccess(context) - .then(job) - .doOnEach(sig -> audit(context, sig)); + var accessContext = contextBuilder.build(); + + Flux messagesFlux; + if (cursor != null) { + messagesFlux = messagesService.loadMessages(getCluster(clusterName), topicName, cursor); + } else { + messagesFlux = messagesService.loadMessages( + getCluster(clusterName), + topicName, + ConsumerPosition.create(checkNotNull(mode), checkNotNull(topicName), partitions, timestamp, offset), + stringFilter, + smartFilterId, + limit, + keySerde, + valueSerde + ); + } + return accessControlService.validateAccess(accessContext) + .then(Mono.just(ResponseEntity.ok(messagesFlux))) + .doOnEach(sig -> auditService.audit(accessContext, sig)); } @Override @@ -136,34 +152,6 @@ public Mono> sendTopicMessages( ).doOnEach(sig -> audit(context, sig)); } - /** - * The format is [partition]::[offset] for specifying offsets - * or [partition]::[timestamp in millis] for specifying timestamps. - */ - @Nullable - private Map parseSeekTo(String topic, SeekTypeDTO seekType, List seekTo) { - if (seekTo == null || seekTo.isEmpty()) { - if (seekType == SeekTypeDTO.LATEST || seekType == SeekTypeDTO.BEGINNING) { - return null; - } - throw new ValidationException("seekTo should be set if seekType is " + seekType); - } - return seekTo.stream() - .map(p -> { - String[] split = p.split("::"); - if (split.length != 2) { - throw new IllegalArgumentException( - "Wrong seekTo argument format. See API docs for details"); - } - - return Pair.of( - new TopicPartition(topic, Integer.parseInt(split[0])), - Long.parseLong(split[1]) - ); - }) - .collect(toMap(Pair::getKey, Pair::getValue)); - } - @Override public Mono> getSerdes(String clusterName, String topicName, @@ -190,7 +178,19 @@ public Mono> getSerdes(String clusterNam ); } + @Override + public Mono> registerFilter(String clusterName, + String topicName, + Mono registration, + ServerWebExchange exchange) { - + final Mono validateAccess = accessControlService.validateAccess(AccessContext.builder() + .cluster(clusterName) + .topicActions(topicName, MESSAGES_READ) + .build()); + return validateAccess.then(registration) + .map(reg -> messagesService.registerMessageFilter(reg.getFilterCode())) + .map(id -> ResponseEntity.ok(new MessageFilterIdDTO().id(id))); + } } diff --git a/api/src/main/java/io/kafbat/ui/emitter/AbstractEmitter.java b/api/src/main/java/io/kafbat/ui/emitter/AbstractEmitter.java index 75c1a1cfc..7638586a5 100644 --- a/api/src/main/java/io/kafbat/ui/emitter/AbstractEmitter.java +++ b/api/src/main/java/io/kafbat/ui/emitter/AbstractEmitter.java @@ -1,6 +1,7 @@ package io.kafbat.ui.emitter; import io.kafbat.ui.model.TopicMessageEventDTO; +import jakarta.annotation.Nullable; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.utils.Bytes; import reactor.core.publisher.FluxSink; @@ -21,12 +22,14 @@ protected PolledRecords poll(FluxSink sink, EnhancedConsum return records; } - protected boolean sendLimitReached() { + protected boolean isSendLimitReached() { return messagesProcessing.limitReached(); } - protected void send(FluxSink sink, Iterable> records) { - messagesProcessing.send(sink, records); + protected void send(FluxSink sink, + Iterable> records, + @Nullable Cursor.Tracking cursor) { + messagesProcessing.send(sink, records, cursor); } protected void sendPhase(FluxSink sink, String name) { @@ -37,8 +40,9 @@ protected void sendConsuming(FluxSink sink, PolledRecords messagesProcessing.sentConsumingInfo(sink, records); } - protected void sendFinishStatsAndCompleteSink(FluxSink sink) { - messagesProcessing.sendFinishEvent(sink); + // cursor is null if target partitions were fully polled (no, need to do paging) + protected void sendFinishStatsAndCompleteSink(FluxSink sink, @Nullable Cursor.Tracking cursor) { + messagesProcessing.sendFinishEvents(sink, cursor); sink.complete(); } } diff --git a/api/src/main/java/io/kafbat/ui/emitter/BackwardEmitter.java b/api/src/main/java/io/kafbat/ui/emitter/BackwardEmitter.java index 606bb1ef9..82bada939 100644 --- a/api/src/main/java/io/kafbat/ui/emitter/BackwardEmitter.java +++ b/api/src/main/java/io/kafbat/ui/emitter/BackwardEmitter.java @@ -18,18 +18,15 @@ public BackwardEmitter(Supplier consumerSupplier, int messagesPerPage, ConsumerRecordDeserializer deserializer, Predicate filter, - PollingSettings pollingSettings) { + PollingSettings pollingSettings, + Cursor.Tracking cursor) { super( consumerSupplier, consumerPosition, messagesPerPage, - new MessagesProcessing( - deserializer, - filter, - false, - messagesPerPage - ), - pollingSettings + new MessagesProcessing(deserializer, filter, false, messagesPerPage), + pollingSettings, + cursor ); } diff --git a/api/src/main/java/io/kafbat/ui/emitter/ConsumingStats.java b/api/src/main/java/io/kafbat/ui/emitter/ConsumingStats.java index 363502464..6287a9903 100644 --- a/api/src/main/java/io/kafbat/ui/emitter/ConsumingStats.java +++ b/api/src/main/java/io/kafbat/ui/emitter/ConsumingStats.java @@ -2,6 +2,8 @@ import io.kafbat.ui.model.TopicMessageConsumingDTO; import io.kafbat.ui.model.TopicMessageEventDTO; +import io.kafbat.ui.model.TopicMessageNextPageCursorDTO; +import javax.annotation.Nullable; import reactor.core.publisher.FluxSink; class ConsumingStats { @@ -26,10 +28,15 @@ void incFilterApplyError() { filterApplyErrors++; } - void sendFinishEvent(FluxSink sink) { + void sendFinishEvent(FluxSink sink, @Nullable Cursor.Tracking cursor) { sink.next( new TopicMessageEventDTO() .type(TopicMessageEventDTO.TypeEnum.DONE) + .cursor( + cursor != null + ? new TopicMessageNextPageCursorDTO().id(cursor.registerCursor()) + : null + ) .consuming(createConsumingStats()) ); } diff --git a/api/src/main/java/io/kafbat/ui/emitter/Cursor.java b/api/src/main/java/io/kafbat/ui/emitter/Cursor.java new file mode 100644 index 000000000..1569cf85f --- /dev/null +++ b/api/src/main/java/io/kafbat/ui/emitter/Cursor.java @@ -0,0 +1,90 @@ +package io.kafbat.ui.emitter; + +import com.google.common.collect.HashBasedTable; +import com.google.common.collect.Table; +import io.kafbat.ui.model.ConsumerPosition; +import io.kafbat.ui.model.PollingModeDTO; +import io.kafbat.ui.model.TopicMessageDTO; +import io.kafbat.ui.serdes.ConsumerRecordDeserializer; +import java.util.HashMap; +import java.util.Map; +import java.util.function.Function; +import java.util.function.Predicate; +import org.apache.kafka.common.TopicPartition; + +public record Cursor(ConsumerRecordDeserializer deserializer, + ConsumerPosition consumerPosition, + Predicate filter, + int limit) { + + public static class Tracking { + private final ConsumerRecordDeserializer deserializer; + private final ConsumerPosition originalPosition; + private final Predicate filter; + private final int limit; + private final Function registerAction; + + //topic -> partition -> offset + private final Table trackingOffsets = HashBasedTable.create(); + + public Tracking(ConsumerRecordDeserializer deserializer, + ConsumerPosition originalPosition, + Predicate filter, + int limit, + Function registerAction) { + this.deserializer = deserializer; + this.originalPosition = originalPosition; + this.filter = filter; + this.limit = limit; + this.registerAction = registerAction; + } + + void trackOffset(String topic, int partition, long offset) { + trackingOffsets.put(topic, partition, offset); + } + + void initOffsets(Map initialSeekOffsets) { + initialSeekOffsets.forEach((tp, off) -> trackOffset(tp.topic(), tp.partition(), off)); + } + + private Map getOffsetsMap(int offsetToAdd) { + Map result = new HashMap<>(); + trackingOffsets.rowMap() + .forEach((topic, partsMap) -> + partsMap.forEach((p, off) -> result.put(new TopicPartition(topic, p), off + offsetToAdd))); + return result; + } + + String registerCursor() { + return registerAction.apply( + new Cursor( + deserializer, + new ConsumerPosition( + switch (originalPosition.pollingMode()) { + case TO_OFFSET, TO_TIMESTAMP, LATEST -> PollingModeDTO.TO_OFFSET; + case FROM_OFFSET, FROM_TIMESTAMP, EARLIEST -> PollingModeDTO.FROM_OFFSET; + case TAILING -> throw new IllegalStateException(); + }, + originalPosition.topic(), + originalPosition.partitions(), + null, + new ConsumerPosition.Offsets( + null, + getOffsetsMap( + switch (originalPosition.pollingMode()) { + case TO_OFFSET, TO_TIMESTAMP, LATEST -> 0; + // when doing forward polling we need to start from latest msg's offset + 1 + case FROM_OFFSET, FROM_TIMESTAMP, EARLIEST -> 1; + case TAILING -> throw new IllegalStateException(); + } + ) + ) + ), + filter, + limit + ) + ); + } + } + +} diff --git a/api/src/main/java/io/kafbat/ui/emitter/ForwardEmitter.java b/api/src/main/java/io/kafbat/ui/emitter/ForwardEmitter.java index 8ca2b97e2..b111170e5 100644 --- a/api/src/main/java/io/kafbat/ui/emitter/ForwardEmitter.java +++ b/api/src/main/java/io/kafbat/ui/emitter/ForwardEmitter.java @@ -18,18 +18,15 @@ public ForwardEmitter(Supplier consumerSupplier, int messagesPerPage, ConsumerRecordDeserializer deserializer, Predicate filter, - PollingSettings pollingSettings) { + PollingSettings pollingSettings, + Cursor.Tracking cursor) { super( consumerSupplier, consumerPosition, messagesPerPage, - new MessagesProcessing( - deserializer, - filter, - true, - messagesPerPage - ), - pollingSettings + new MessagesProcessing(deserializer, filter, true, messagesPerPage), + pollingSettings, + cursor ); } diff --git a/api/src/main/java/io/kafbat/ui/emitter/MessageFilters.java b/api/src/main/java/io/kafbat/ui/emitter/MessageFilters.java index b2144662f..86a42aab1 100644 --- a/api/src/main/java/io/kafbat/ui/emitter/MessageFilters.java +++ b/api/src/main/java/io/kafbat/ui/emitter/MessageFilters.java @@ -47,19 +47,16 @@ public class MessageFilters { private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - public static Predicate createMsgFilter(String query, MessageFilterTypeDTO type) { - return switch (type) { - case STRING_CONTAINS -> containsStringFilter(query); - case CEL_SCRIPT -> celScriptFilter(query); - }; + public static Predicate noop() { + return e -> true; } - static Predicate containsStringFilter(String string) { + public static Predicate containsStringFilter(String string) { return msg -> StringUtils.contains(msg.getKey(), string) || StringUtils.contains(msg.getContent(), string); } - static Predicate celScriptFilter(String script) { + public static Predicate celScriptFilter(String script) { CelValidationResult celValidationResult = CEL_COMPILER.compile(script); if (celValidationResult.hasError()) { throw new CelException(script, celValidationResult.getErrorString()); diff --git a/api/src/main/java/io/kafbat/ui/emitter/MessagesProcessing.java b/api/src/main/java/io/kafbat/ui/emitter/MessagesProcessing.java index 547a77a70..16dead8f5 100644 --- a/api/src/main/java/io/kafbat/ui/emitter/MessagesProcessing.java +++ b/api/src/main/java/io/kafbat/ui/emitter/MessagesProcessing.java @@ -39,7 +39,9 @@ boolean limitReached() { return limit != null && sentMessages >= limit; } - void send(FluxSink sink, Iterable> polled) { + void send(FluxSink sink, + Iterable> polled, + @Nullable Cursor.Tracking cursor) { sortForSending(polled, ascendingSortBeforeSend) .forEach(rec -> { if (!limitReached() && !sink.isCancelled()) { @@ -53,6 +55,9 @@ void send(FluxSink sink, Iterable sink, PolledRecords polled } } - void sendFinishEvent(FluxSink sink) { + void sendFinishEvents(FluxSink sink, @Nullable Cursor.Tracking cursor) { if (!sink.isCancelled()) { - consumingStats.sendFinishEvent(sink); + consumingStats.sendFinishEvent(sink, cursor); } } diff --git a/api/src/main/java/io/kafbat/ui/emitter/OffsetsInfo.java b/api/src/main/java/io/kafbat/ui/emitter/OffsetsInfo.java index 92ebe3a6c..9fc6c27db 100644 --- a/api/src/main/java/io/kafbat/ui/emitter/OffsetsInfo.java +++ b/api/src/main/java/io/kafbat/ui/emitter/OffsetsInfo.java @@ -1,6 +1,7 @@ package io.kafbat.ui.emitter; import com.google.common.base.Preconditions; +import com.google.common.collect.Sets; import java.util.Collection; import java.util.HashSet; import java.util.Map; @@ -61,4 +62,8 @@ long summaryOffsetsRange() { return cnt.getValue(); } + public Set allTargetPartitions() { + return Sets.union(nonEmptyPartitions, emptyPartitions); + } + } diff --git a/api/src/main/java/io/kafbat/ui/emitter/PolledRecords.java b/api/src/main/java/io/kafbat/ui/emitter/PolledRecords.java index 96d56d25b..575395f05 100644 --- a/api/src/main/java/io/kafbat/ui/emitter/PolledRecords.java +++ b/api/src/main/java/io/kafbat/ui/emitter/PolledRecords.java @@ -3,6 +3,7 @@ import java.time.Duration; import java.util.Iterator; import java.util.List; +import java.util.Set; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.common.TopicPartition; @@ -32,6 +33,10 @@ public Iterator> iterator() { return records.iterator(); } + public Set partitions() { + return records.partitions(); + } + private static int calculatePolledRecSize(Iterable> recs) { int polledBytes = 0; for (ConsumerRecord rec : recs) { diff --git a/api/src/main/java/io/kafbat/ui/emitter/RangePollingEmitter.java b/api/src/main/java/io/kafbat/ui/emitter/RangePollingEmitter.java index 3933ce8d0..a5712492e 100644 --- a/api/src/main/java/io/kafbat/ui/emitter/RangePollingEmitter.java +++ b/api/src/main/java/io/kafbat/ui/emitter/RangePollingEmitter.java @@ -17,6 +17,7 @@ abstract class RangePollingEmitter extends AbstractEmitter { private final Supplier consumerSupplier; + private final Cursor.Tracking cursor; protected final ConsumerPosition consumerPosition; protected final int messagesPerPage; @@ -24,11 +25,13 @@ protected RangePollingEmitter(Supplier consumerSupplier, ConsumerPosition consumerPosition, int messagesPerPage, MessagesProcessing messagesProcessing, - PollingSettings pollingSettings) { + PollingSettings pollingSettings, + Cursor.Tracking cursor) { super(messagesProcessing, pollingSettings); this.consumerPosition = consumerPosition; this.messagesPerPage = messagesPerPage; this.consumerSupplier = consumerSupplier; + this.cursor = cursor; } protected record FromToOffset(/*inclusive*/ long from, /*exclusive*/ long to) { @@ -46,18 +49,20 @@ public void accept(FluxSink sink) { try (EnhancedConsumer consumer = consumerSupplier.get()) { sendPhase(sink, "Consumer created"); var seekOperations = SeekOperations.create(consumer, consumerPosition); + cursor.initOffsets(seekOperations.getOffsetsForSeek()); + TreeMap pollRange = nextPollingRange(new TreeMap<>(), seekOperations); log.debug("Starting from offsets {}", pollRange); - while (!sink.isCancelled() && !pollRange.isEmpty() && !sendLimitReached()) { + while (!sink.isCancelled() && !pollRange.isEmpty() && !isSendLimitReached()) { var polled = poll(consumer, sink, pollRange); - send(sink, polled); + send(sink, polled, cursor); pollRange = nextPollingRange(pollRange, seekOperations); } if (sink.isCancelled()) { log.debug("Polling finished due to sink cancellation"); } - sendFinishStatsAndCompleteSink(sink); + sendFinishStatsAndCompleteSink(sink, pollRange.isEmpty() ? null : cursor); log.debug("Polling finished"); } catch (InterruptException kafkaInterruptException) { log.debug("Polling finished due to thread interruption"); diff --git a/api/src/main/java/io/kafbat/ui/emitter/SeekOperations.java b/api/src/main/java/io/kafbat/ui/emitter/SeekOperations.java index f7a0399ec..87f29102c 100644 --- a/api/src/main/java/io/kafbat/ui/emitter/SeekOperations.java +++ b/api/src/main/java/io/kafbat/ui/emitter/SeekOperations.java @@ -1,13 +1,13 @@ package io.kafbat.ui.emitter; +import static io.kafbat.ui.model.PollingModeDTO.TO_TIMESTAMP; +import static java.util.Objects.requireNonNull; + import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; import io.kafbat.ui.model.ConsumerPosition; -import io.kafbat.ui.model.SeekTypeDTO; +import io.kafbat.ui.model.PollingModeDTO; import java.util.HashMap; import java.util.Map; -import java.util.stream.Collectors; -import javax.annotation.Nullable; import lombok.AccessLevel; import lombok.RequiredArgsConstructor; import org.apache.commons.lang3.mutable.MutableLong; @@ -22,17 +22,11 @@ public class SeekOperations { private final Map offsetsForSeek; //only contains non-empty partitions! public static SeekOperations create(Consumer consumer, ConsumerPosition consumerPosition) { - OffsetsInfo offsetsInfo; - if (consumerPosition.getSeekTo() == null) { - offsetsInfo = new OffsetsInfo(consumer, consumerPosition.getTopic()); - } else { - offsetsInfo = new OffsetsInfo(consumer, consumerPosition.getSeekTo().keySet()); - } - return new SeekOperations( - consumer, - offsetsInfo, - getOffsetsForSeek(consumer, offsetsInfo, consumerPosition.getSeekType(), consumerPosition.getSeekTo()) - ); + OffsetsInfo offsetsInfo = consumerPosition.partitions().isEmpty() + ? new OffsetsInfo(consumer, consumerPosition.topic()) + : new OffsetsInfo(consumer, consumerPosition.partitions()); + var offsetsToSeek = getOffsetsForSeek(consumer, offsetsInfo, consumerPosition); + return new SeekOperations(consumer, offsetsInfo, offsetsToSeek); } public void assignAndSeekNonEmptyPartitions() { @@ -75,27 +69,26 @@ public Map getOffsetsForSeek() { @VisibleForTesting static Map getOffsetsForSeek(Consumer consumer, OffsetsInfo offsetsInfo, - SeekTypeDTO seekType, - @Nullable Map seekTo) { - switch (seekType) { - case LATEST: - return consumer.endOffsets(offsetsInfo.getNonEmptyPartitions()); - case BEGINNING: - return consumer.beginningOffsets(offsetsInfo.getNonEmptyPartitions()); - case OFFSET: - Preconditions.checkNotNull(seekTo); - return fixOffsets(offsetsInfo, seekTo); - case TIMESTAMP: - Preconditions.checkNotNull(seekTo); - return offsetsForTimestamp(consumer, offsetsInfo, seekTo); - default: - throw new IllegalStateException(); - } + ConsumerPosition position) { + return switch (position.pollingMode()) { + case TAILING -> consumer.endOffsets(offsetsInfo.allTargetPartitions()); + case LATEST -> consumer.endOffsets(offsetsInfo.getNonEmptyPartitions()); + case EARLIEST -> consumer.beginningOffsets(offsetsInfo.getNonEmptyPartitions()); + case FROM_OFFSET, TO_OFFSET -> fixOffsets(offsetsInfo, requireNonNull(position.offsets())); + case FROM_TIMESTAMP, TO_TIMESTAMP -> + offsetsForTimestamp(consumer, position.pollingMode(), offsetsInfo, requireNonNull(position.timestamp())); + }; } - private static Map fixOffsets(OffsetsInfo offsetsInfo, Map offsets) { - offsets = new HashMap<>(offsets); - offsets.keySet().retainAll(offsetsInfo.getNonEmptyPartitions()); + private static Map fixOffsets(OffsetsInfo offsetsInfo, + ConsumerPosition.Offsets positionOffset) { + var offsets = new HashMap(); + if (positionOffset.offset() != null) { + offsetsInfo.getNonEmptyPartitions().forEach(tp -> offsets.put(tp, positionOffset.offset())); + } else { + offsets.putAll(requireNonNull(positionOffset.tpOffsets())); + offsets.keySet().retainAll(offsetsInfo.getNonEmptyPartitions()); + } Map result = new HashMap<>(); offsets.forEach((tp, targetOffset) -> { @@ -112,13 +105,25 @@ private static Map fixOffsets(OffsetsInfo offsetsInfo, Map return result; } - private static Map offsetsForTimestamp(Consumer consumer, OffsetsInfo offsetsInfo, - Map timestamps) { - timestamps = new HashMap<>(timestamps); - timestamps.keySet().retainAll(offsetsInfo.getNonEmptyPartitions()); + private static Map offsetsForTimestamp(Consumer consumer, + PollingModeDTO pollingMode, + OffsetsInfo offsetsInfo, + Long timestamp) { + Map timestamps = new HashMap<>(); + offsetsInfo.getNonEmptyPartitions().forEach(tp -> timestamps.put(tp, timestamp)); - return consumer.offsetsForTimes(timestamps).entrySet().stream() - .filter(e -> e.getValue() != null) - .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().offset())); + Map result = new HashMap<>(); + consumer.offsetsForTimes(timestamps).forEach((tp, offsetAndTimestamp) -> { + if (offsetAndTimestamp == null) { + if (pollingMode == TO_TIMESTAMP && offsetsInfo.getNonEmptyPartitions().contains(tp)) { + // if no offset was returned this means that *all* timestamps are lower + // than target timestamp. Is case of TO_OFFSET mode we need to read from the ending of tp + result.put(tp, offsetsInfo.getEndOffsets().get(tp)); + } + } else { + result.put(tp, offsetAndTimestamp.offset()); + } + }); + return result; } } diff --git a/api/src/main/java/io/kafbat/ui/emitter/TailingEmitter.java b/api/src/main/java/io/kafbat/ui/emitter/TailingEmitter.java index 7b1d20df6..fb09af28f 100644 --- a/api/src/main/java/io/kafbat/ui/emitter/TailingEmitter.java +++ b/api/src/main/java/io/kafbat/ui/emitter/TailingEmitter.java @@ -35,7 +35,7 @@ public void accept(FluxSink sink) { while (!sink.isCancelled()) { sendPhase(sink, "Polling"); var polled = poll(sink, consumer); - send(sink, polled); + send(sink, polled, null); } sink.complete(); log.debug("Tailing finished"); @@ -55,5 +55,4 @@ private void assignAndSeek(EnhancedConsumer consumer) { consumer.assign(seekOffsets.keySet()); seekOffsets.forEach(consumer::seek); } - } diff --git a/api/src/main/java/io/kafbat/ui/model/ConsumerPosition.java b/api/src/main/java/io/kafbat/ui/model/ConsumerPosition.java index 1856f02e1..0765cdc75 100644 --- a/api/src/main/java/io/kafbat/ui/model/ConsumerPosition.java +++ b/api/src/main/java/io/kafbat/ui/model/ConsumerPosition.java @@ -1,15 +1,72 @@ package io.kafbat.ui.model; -import io.kafbat.ui.model.SeekTypeDTO; +import com.google.common.base.Preconditions; +import io.kafbat.ui.exception.ValidationException; +import java.util.List; import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; import javax.annotation.Nullable; -import lombok.Value; import org.apache.kafka.common.TopicPartition; -@Value -public class ConsumerPosition { - SeekTypeDTO seekType; - String topic; - @Nullable - Map seekTo; // null if positioning should apply to all tps +public record ConsumerPosition(PollingModeDTO pollingMode, + String topic, + List partitions, //all partitions if list is empty + @Nullable Long timestamp, + @Nullable Offsets offsets) { + + public record Offsets(@Nullable Long offset, //should be applied to all partitions + @Nullable Map tpOffsets) { + public Offsets { + // only one of properties should be set + Preconditions.checkArgument((offset == null && tpOffsets != null) || (offset != null && tpOffsets == null)); + } + } + + public static ConsumerPosition create(PollingModeDTO pollingMode, + String topic, + @Nullable List partitions, + @Nullable Long timestamp, + @Nullable Long offset) { + @Nullable var offsets = parseAndValidateOffsets(pollingMode, offset); + + var topicPartitions = Optional.ofNullable(partitions).orElse(List.of()) + .stream() + .map(p -> new TopicPartition(topic, p)) + .collect(Collectors.toList()); + + // if offsets are specified - inferring partitions list from there + topicPartitions = (offsets != null && offsets.tpOffsets() != null) + ? List.copyOf(offsets.tpOffsets().keySet()) + : topicPartitions; + + return new ConsumerPosition( + pollingMode, + topic, + topicPartitions, + validateTimestamp(pollingMode, timestamp), + offsets + ); + } + + private static Long validateTimestamp(PollingModeDTO pollingMode, @Nullable Long ts) { + if (pollingMode == PollingModeDTO.FROM_TIMESTAMP || pollingMode == PollingModeDTO.TO_TIMESTAMP) { + if (ts == null) { + throw new ValidationException("timestamp not provided for " + pollingMode); + } + } + return ts; + } + + private static Offsets parseAndValidateOffsets(PollingModeDTO pollingMode, + @Nullable Long offset) { + if (pollingMode == PollingModeDTO.FROM_OFFSET || pollingMode == PollingModeDTO.TO_OFFSET) { + if (offset == null) { + throw new ValidationException("offsets not provided for " + pollingMode); + } + return new Offsets(offset, null); + } + return null; + } + } diff --git a/api/src/main/java/io/kafbat/ui/service/ApplicationInfoService.java b/api/src/main/java/io/kafbat/ui/service/ApplicationInfoService.java index 9c396e149..0a04ce9d6 100644 --- a/api/src/main/java/io/kafbat/ui/service/ApplicationInfoService.java +++ b/api/src/main/java/io/kafbat/ui/service/ApplicationInfoService.java @@ -17,6 +17,7 @@ import org.springframework.boot.info.GitProperties; import org.springframework.scheduling.annotation.Scheduled; import org.springframework.stereotype.Service; +import reactor.core.publisher.Mono; @Service public class ApplicationInfoService { @@ -70,7 +71,7 @@ private List getEnabledFeatures() { // updating on startup and every hour @Scheduled(fixedRateString = "${github-release-info-update-rate:3600000}") public void updateGithubReleaseInfo() { - githubReleaseInfo.refresh().block(); + githubReleaseInfo.refresh().subscribe(); } } diff --git a/api/src/main/java/io/kafbat/ui/service/MessagesService.java b/api/src/main/java/io/kafbat/ui/service/MessagesService.java index c27b4b5be..2f6192e11 100644 --- a/api/src/main/java/io/kafbat/ui/service/MessagesService.java +++ b/api/src/main/java/io/kafbat/ui/service/MessagesService.java @@ -1,8 +1,13 @@ package io.kafbat.ui.service; +import com.google.common.base.Charsets; +import com.google.common.cache.Cache; +import com.google.common.cache.CacheBuilder; +import com.google.common.hash.Hashing; import com.google.common.util.concurrent.RateLimiter; import io.kafbat.ui.config.ClustersProperties; import io.kafbat.ui.emitter.BackwardEmitter; +import io.kafbat.ui.emitter.Cursor; import io.kafbat.ui.emitter.ForwardEmitter; import io.kafbat.ui.emitter.MessageFilters; import io.kafbat.ui.emitter.TailingEmitter; @@ -11,12 +16,12 @@ import io.kafbat.ui.model.ConsumerPosition; import io.kafbat.ui.model.CreateTopicMessageDTO; import io.kafbat.ui.model.KafkaCluster; -import io.kafbat.ui.model.MessageFilterTypeDTO; -import io.kafbat.ui.model.SeekDirectionDTO; +import io.kafbat.ui.model.PollingModeDTO; import io.kafbat.ui.model.SmartFilterTestExecutionDTO; import io.kafbat.ui.model.SmartFilterTestExecutionResultDTO; import io.kafbat.ui.model.TopicMessageDTO; import io.kafbat.ui.model.TopicMessageEventDTO; +import io.kafbat.ui.serdes.ConsumerRecordDeserializer; import io.kafbat.ui.serdes.ProducerRecordCreator; import io.kafbat.ui.util.SslPropertiesUtil; import java.time.Instant; @@ -27,12 +32,12 @@ import java.util.Optional; import java.util.Properties; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ThreadLocalRandom; import java.util.function.Predicate; import java.util.function.UnaryOperator; import java.util.stream.Collectors; import javax.annotation.Nullable; import lombok.extern.slf4j.Slf4j; -import org.apache.commons.lang3.StringUtils; import org.apache.kafka.clients.admin.OffsetSpec; import org.apache.kafka.clients.admin.TopicDescription; import org.apache.kafka.clients.producer.KafkaProducer; @@ -50,8 +55,11 @@ @Slf4j public class MessagesService { + private static final long SALT_FOR_HASHING = ThreadLocalRandom.current().nextLong(); + private static final int DEFAULT_MAX_PAGE_SIZE = 500; private static final int DEFAULT_PAGE_SIZE = 100; + // limiting UI messages rate to 20/sec in tailing mode private static final int TAILING_UI_MESSAGE_THROTTLE_RATE = 20; @@ -61,6 +69,12 @@ public class MessagesService { private final int maxPageSize; private final int defaultPageSize; + private final Cache> registeredFilters = CacheBuilder.newBuilder() + .maximumSize(PollingCursorsStorage.MAX_SIZE) + .build(); + + private final PollingCursorsStorage cursorsStorage = new PollingCursorsStorage(); + public MessagesService(AdminClientService adminClientService, DeserializationService deserializationService, ConsumerGroupService consumerGroupService, @@ -86,10 +100,7 @@ private Mono withExistingTopic(KafkaCluster cluster, String to public static SmartFilterTestExecutionResultDTO execSmartFilterTest(SmartFilterTestExecutionDTO execData) { Predicate predicate; try { - predicate = MessageFilters.createMsgFilter( - execData.getFilterCode(), - MessageFilterTypeDTO.CEL_SCRIPT - ); + predicate = MessageFilters.celScriptFilter(execData.getFilterCode()); } catch (Exception e) { log.info("Smart filter '{}' compilation error", execData.getFilterCode(), e); return new SmartFilterTestExecutionResultDTO() @@ -197,67 +208,103 @@ public static KafkaProducer createProducer(KafkaCluster cluster, return new KafkaProducer<>(properties); } - public Flux loadMessages(KafkaCluster cluster, String topic, + public Flux loadMessages(KafkaCluster cluster, + String topic, ConsumerPosition consumerPosition, - @Nullable String query, - MessageFilterTypeDTO filterQueryType, - @Nullable Integer pageSize, - SeekDirectionDTO seekDirection, + @Nullable String containsStringFilter, + @Nullable String filterId, + @Nullable Integer limit, @Nullable String keySerde, @Nullable String valueSerde) { + return loadMessages( + cluster, + topic, + deserializationService.deserializerFor(cluster, topic, keySerde, valueSerde), + consumerPosition, + getMsgFilter(containsStringFilter, filterId), + fixPageSize(limit) + ); + } + + public Flux loadMessages(KafkaCluster cluster, String topic, String cursorId) { + Cursor cursor = cursorsStorage.getCursor(cursorId) + .orElseThrow(() -> new ValidationException("Next page cursor not found. Maybe it was evicted from cache.")); + return loadMessages( + cluster, + topic, + cursor.deserializer(), + cursor.consumerPosition(), + cursor.filter(), + cursor.limit() + ); + } + + private Flux loadMessages(KafkaCluster cluster, + String topic, + ConsumerRecordDeserializer deserializer, + ConsumerPosition consumerPosition, + Predicate filter, + int limit) { return withExistingTopic(cluster, topic) .flux() .publishOn(Schedulers.boundedElastic()) - .flatMap(td -> loadMessagesImpl(cluster, topic, consumerPosition, query, - filterQueryType, fixPageSize(pageSize), seekDirection, keySerde, valueSerde)); - } - - private int fixPageSize(@Nullable Integer pageSize) { - return Optional.ofNullable(pageSize) - .filter(ps -> ps > 0 && ps <= maxPageSize) - .orElse(defaultPageSize); + .flatMap(td -> loadMessagesImpl(cluster, deserializer, consumerPosition, filter, limit)); } private Flux loadMessagesImpl(KafkaCluster cluster, - String topic, + ConsumerRecordDeserializer deserializer, ConsumerPosition consumerPosition, - @Nullable String query, - MessageFilterTypeDTO filterQueryType, - int limit, - SeekDirectionDTO seekDirection, - @Nullable String keySerde, - @Nullable String valueSerde) { - - var deserializer = deserializationService.deserializerFor(cluster, topic, keySerde, valueSerde); - var filter = getMsgFilter(query, filterQueryType); - var emitter = switch (seekDirection) { - case FORWARD -> new ForwardEmitter( + Predicate filter, + int limit) { + var emitter = switch (consumerPosition.pollingMode()) { + case TO_OFFSET, TO_TIMESTAMP, LATEST -> new BackwardEmitter( () -> consumerGroupService.createConsumer(cluster), - consumerPosition, limit, deserializer, filter, cluster.getPollingSettings() + consumerPosition, + limit, + deserializer, + filter, + cluster.getPollingSettings(), + cursorsStorage.createNewCursor(deserializer, consumerPosition, filter, limit) ); - case BACKWARD -> new BackwardEmitter( + case FROM_OFFSET, FROM_TIMESTAMP, EARLIEST -> new ForwardEmitter( () -> consumerGroupService.createConsumer(cluster), - consumerPosition, limit, deserializer, filter, cluster.getPollingSettings() + consumerPosition, + limit, + deserializer, + filter, + cluster.getPollingSettings(), + cursorsStorage.createNewCursor(deserializer, consumerPosition, filter, limit) ); case TAILING -> new TailingEmitter( () -> consumerGroupService.createConsumer(cluster), - consumerPosition, deserializer, filter, cluster.getPollingSettings() + consumerPosition, + deserializer, + filter, + cluster.getPollingSettings() ); }; return Flux.create(emitter) - .map(throttleUiPublish(seekDirection)); + .map(throttleUiPublish(consumerPosition.pollingMode())); } - private Predicate getMsgFilter(String query, - MessageFilterTypeDTO filterQueryType) { - if (StringUtils.isEmpty(query)) { - return evt -> true; + private Predicate getMsgFilter(@Nullable String containsStrFilter, + @Nullable String smartFilterId) { + Predicate messageFilter = MessageFilters.noop(); + if (containsStrFilter != null) { + messageFilter = messageFilter.and(MessageFilters.containsStringFilter(containsStrFilter)); } - return MessageFilters.createMsgFilter(query, filterQueryType); + if (smartFilterId != null) { + var registered = registeredFilters.getIfPresent(smartFilterId); + if (registered == null) { + throw new ValidationException("No filter was registered with id " + smartFilterId); + } + messageFilter = messageFilter.and(registered); + } + return messageFilter; } - private UnaryOperator throttleUiPublish(SeekDirectionDTO seekDirection) { - if (seekDirection == SeekDirectionDTO.TAILING) { + private UnaryOperator throttleUiPublish(PollingModeDTO pollingMode) { + if (pollingMode == PollingModeDTO.TAILING) { RateLimiter rateLimiter = RateLimiter.create(TAILING_UI_MESSAGE_THROTTLE_RATE); return m -> { rateLimiter.acquire(1); @@ -269,4 +316,22 @@ private UnaryOperator throttleUiPublish(SeekDirectionDTO seekDirection) { return UnaryOperator.identity(); } + private int fixPageSize(@Nullable Integer pageSize) { + return Optional.ofNullable(pageSize) + .filter(ps -> ps > 0 && ps <= maxPageSize) + .orElse(defaultPageSize); + } + + public String registerMessageFilter(String celCode) { + String saltedCode = celCode + SALT_FOR_HASHING; + String filterId = Hashing.sha256() + .hashString(saltedCode, Charsets.UTF_8) + .toString() + .substring(0, 8); + if (registeredFilters.getIfPresent(filterId) == null) { + registeredFilters.put(filterId, MessageFilters.celScriptFilter(celCode)); + } + return filterId; + } + } diff --git a/api/src/main/java/io/kafbat/ui/service/PollingCursorsStorage.java b/api/src/main/java/io/kafbat/ui/service/PollingCursorsStorage.java new file mode 100644 index 000000000..2b760f010 --- /dev/null +++ b/api/src/main/java/io/kafbat/ui/service/PollingCursorsStorage.java @@ -0,0 +1,44 @@ +package io.kafbat.ui.service; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.cache.Cache; +import com.google.common.cache.CacheBuilder; +import io.kafbat.ui.emitter.Cursor; +import io.kafbat.ui.model.ConsumerPosition; +import io.kafbat.ui.model.TopicMessageDTO; +import io.kafbat.ui.serdes.ConsumerRecordDeserializer; +import java.util.Map; +import java.util.Optional; +import java.util.function.Predicate; +import org.apache.commons.lang3.RandomStringUtils; + +public class PollingCursorsStorage { + + public static final int MAX_SIZE = 10_000; + + private final Cache cursorsCache = CacheBuilder.newBuilder() + .maximumSize(MAX_SIZE) + .build(); + + public Cursor.Tracking createNewCursor(ConsumerRecordDeserializer deserializer, + ConsumerPosition originalPosition, + Predicate filter, + int limit) { + return new Cursor.Tracking(deserializer, originalPosition, filter, limit, this::register); + } + + public Optional getCursor(String id) { + return Optional.ofNullable(cursorsCache.getIfPresent(id)); + } + + public String register(Cursor cursor) { + var id = RandomStringUtils.random(8, true, true); + cursorsCache.put(id, cursor); + return id; + } + + @VisibleForTesting + public Map asMap() { + return cursorsCache.asMap(); + } +} diff --git a/api/src/main/java/io/kafbat/ui/service/analyze/TopicAnalysisService.java b/api/src/main/java/io/kafbat/ui/service/analyze/TopicAnalysisService.java index 14d2c2b79..a36519e46 100644 --- a/api/src/main/java/io/kafbat/ui/service/analyze/TopicAnalysisService.java +++ b/api/src/main/java/io/kafbat/ui/service/analyze/TopicAnalysisService.java @@ -1,6 +1,6 @@ package io.kafbat.ui.service.analyze; -import static io.kafbat.ui.model.SeekTypeDTO.BEGINNING; +import static io.kafbat.ui.model.PollingModeDTO.EARLIEST; import io.kafbat.ui.emitter.EnhancedConsumer; import io.kafbat.ui.emitter.SeekOperations; @@ -14,6 +14,7 @@ import java.time.Duration; import java.time.Instant; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Optional; import lombok.RequiredArgsConstructor; @@ -104,7 +105,8 @@ public void run() { consumer.partitionsFor(topicId.topicName) .forEach(tp -> partitionStats.put(tp.partition(), new TopicAnalysisStats())); - var seekOperations = SeekOperations.create(consumer, new ConsumerPosition(BEGINNING, topicId.topicName, null)); + var seekOperations = + SeekOperations.create(consumer, new ConsumerPosition(EARLIEST, topicId.topicName, List.of(), null, null)); long summaryOffsetsRange = seekOperations.summaryOffsetsRange(); seekOperations.assignAndSeekNonEmptyPartitions(); diff --git a/api/src/test/java/io/kafbat/ui/KafkaConsumerTests.java b/api/src/test/java/io/kafbat/ui/KafkaConsumerTests.java index 126a395e4..6eeada1b6 100644 --- a/api/src/test/java/io/kafbat/ui/KafkaConsumerTests.java +++ b/api/src/test/java/io/kafbat/ui/KafkaConsumerTests.java @@ -56,7 +56,7 @@ public void shouldDeleteRecords() { } long count = webTestClient.get() - .uri("/api/clusters/{clusterName}/topics/{topicName}/messages", LOCAL, topicName) + .uri("/api/clusters/{clusterName}/topics/{topicName}/messages/v2?mode=EARLIEST", LOCAL, topicName) .accept(TEXT_EVENT_STREAM) .exchange() .expectStatus() @@ -77,7 +77,7 @@ public void shouldDeleteRecords() { .isOk(); count = webTestClient.get() - .uri("/api/clusters/{clusterName}/topics/{topicName}/messages", LOCAL, topicName) + .uri("/api/clusters/{clusterName}/topics/{topicName}/messages/v2?mode=EARLIEST", LOCAL, topicName) .exchange() .expectStatus() .isOk() diff --git a/api/src/test/java/io/kafbat/ui/emitter/CursorTest.java b/api/src/test/java/io/kafbat/ui/emitter/CursorTest.java new file mode 100644 index 000000000..3dd5dadd1 --- /dev/null +++ b/api/src/test/java/io/kafbat/ui/emitter/CursorTest.java @@ -0,0 +1,193 @@ +package io.kafbat.ui.emitter; + +import static org.assertj.core.api.Assertions.assertThat; + +import io.kafbat.ui.AbstractIntegrationTest; +import io.kafbat.ui.model.ConsumerPosition; +import io.kafbat.ui.model.PollingModeDTO; +import io.kafbat.ui.model.TopicMessageEventDTO; +import io.kafbat.ui.producer.KafkaTestProducer; +import io.kafbat.ui.serde.api.Serde; +import io.kafbat.ui.serdes.ConsumerRecordDeserializer; +import io.kafbat.ui.serdes.PropertyResolverImpl; +import io.kafbat.ui.serdes.builtin.StringSerde; +import io.kafbat.ui.service.PollingCursorsStorage; +import io.kafbat.ui.util.ApplicationMetrics; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.UUID; +import java.util.function.Consumer; +import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.TopicPartition; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import reactor.core.publisher.Flux; + + +class CursorTest extends AbstractIntegrationTest { + + static final String TOPIC = CursorTest.class.getSimpleName() + "_" + UUID.randomUUID(); + static final int MSGS_IN_PARTITION = 20; + static final int PAGE_SIZE = (MSGS_IN_PARTITION / 2) + 1; //to poll fill data set in 2 iterations + + final PollingCursorsStorage cursorsStorage = new PollingCursorsStorage(); + + @BeforeAll + static void setup() { + createTopic(new NewTopic(TOPIC, 1, (short) 1)); + try (var producer = KafkaTestProducer.forKafka(kafka)) { + for (int i = 0; i < MSGS_IN_PARTITION; i++) { + producer.send(new ProducerRecord<>(TOPIC, "msg_" + i)); + } + } + } + + @AfterAll + static void cleanup() { + deleteTopic(TOPIC); + } + + @Test + void backwardEmitter() { + var consumerPosition = new ConsumerPosition(PollingModeDTO.LATEST, TOPIC, List.of(), null, null); + var emitter = createBackwardEmitter(consumerPosition); + waitMgsgEmitted(emitter, PAGE_SIZE); + var cursor = assertCursor( + PollingModeDTO.TO_OFFSET, + offsets -> assertThat(offsets) + .hasSize(1) + .containsEntry(new TopicPartition(TOPIC, 0), 9L) + ); + + // polling remaining records using registered cursor + emitter = createBackwardEmitterWithCursor(cursor); + waitMgsgEmitted(emitter, MSGS_IN_PARTITION - PAGE_SIZE); + //checking no new cursors registered + assertThat(cursorsStorage.asMap()).hasSize(1).containsValue(cursor); + } + + @Test + void forwardEmitter() { + var consumerPosition = new ConsumerPosition(PollingModeDTO.EARLIEST, TOPIC, List.of(), null, null); + var emitter = createForwardEmitter(consumerPosition); + waitMgsgEmitted(emitter, PAGE_SIZE); + var cursor = assertCursor( + PollingModeDTO.FROM_OFFSET, + offsets -> assertThat(offsets) + .hasSize(1) + .containsEntry(new TopicPartition(TOPIC, 0), 11L) + ); + + //polling remaining records using registered cursor + emitter = createForwardEmitterWithCursor(cursor); + waitMgsgEmitted(emitter, MSGS_IN_PARTITION - PAGE_SIZE); + //checking no new cursors registered + assertThat(cursorsStorage.asMap()).hasSize(1).containsValue(cursor); + } + + private Cursor assertCursor(PollingModeDTO expectedMode, + Consumer> offsetsAssert) { + Cursor registeredCursor = cursorsStorage.asMap().values().stream().findFirst().orElse(null); + assertThat(registeredCursor).isNotNull(); + assertThat(registeredCursor.limit()).isEqualTo(PAGE_SIZE); + assertThat(registeredCursor.deserializer()).isNotNull(); + assertThat(registeredCursor.filter()).isNotNull(); + + var cursorPosition = registeredCursor.consumerPosition(); + assertThat(cursorPosition).isNotNull(); + assertThat(cursorPosition.topic()).isEqualTo(TOPIC); + assertThat(cursorPosition.partitions()).isEqualTo(List.of()); + assertThat(cursorPosition.pollingMode()).isEqualTo(expectedMode); + + offsetsAssert.accept(cursorPosition.offsets().tpOffsets()); + return registeredCursor; + } + + private void waitMgsgEmitted(AbstractEmitter emitter, int expectedMsgsCnt) { + List events = Flux.create(emitter) + .collectList() + .block(); + assertThat(events.stream().filter(m -> m.getType() == TopicMessageEventDTO.TypeEnum.MESSAGE).count()) + .isEqualTo(expectedMsgsCnt); + } + + private BackwardEmitter createBackwardEmitter(ConsumerPosition position) { + return new BackwardEmitter( + this::createConsumer, + position, + PAGE_SIZE, + createRecordsDeserializer(), + m -> true, + PollingSettings.createDefault(), + createCursor(position) + ); + } + + private BackwardEmitter createBackwardEmitterWithCursor(Cursor cursor) { + return new BackwardEmitter( + this::createConsumer, + cursor.consumerPosition(), + cursor.limit(), + cursor.deserializer(), + cursor.filter(), + PollingSettings.createDefault(), + createCursor(cursor.consumerPosition()) + ); + } + + private ForwardEmitter createForwardEmitterWithCursor(Cursor cursor) { + return new ForwardEmitter( + this::createConsumer, + cursor.consumerPosition(), + cursor.limit(), + cursor.deserializer(), + cursor.filter(), + PollingSettings.createDefault(), + createCursor(cursor.consumerPosition()) + ); + } + + private ForwardEmitter createForwardEmitter(ConsumerPosition position) { + return new ForwardEmitter( + this::createConsumer, + position, + PAGE_SIZE, + createRecordsDeserializer(), + m -> true, + PollingSettings.createDefault(), + createCursor(position) + ); + } + + private Cursor.Tracking createCursor(ConsumerPosition position) { + return cursorsStorage.createNewCursor(createRecordsDeserializer(), position, m -> true, PAGE_SIZE); + } + + private EnhancedConsumer createConsumer() { + Properties props = new Properties(); + props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka.getBootstrapServers()); + props.put(ConsumerConfig.GROUP_ID_CONFIG, UUID.randomUUID().toString()); + props.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, PAGE_SIZE - 1); // to check multiple polls + return new EnhancedConsumer(props, PollingThrottler.noop(), ApplicationMetrics.noop()); + } + + private static ConsumerRecordDeserializer createRecordsDeserializer() { + Serde s = new StringSerde(); + s.configure(PropertyResolverImpl.empty(), PropertyResolverImpl.empty(), PropertyResolverImpl.empty()); + return new ConsumerRecordDeserializer( + StringSerde.name(), + s.deserializer(null, Serde.Target.KEY), + StringSerde.name(), + s.deserializer(null, Serde.Target.VALUE), + StringSerde.name(), + s.deserializer(null, Serde.Target.KEY), + s.deserializer(null, Serde.Target.VALUE), + msg -> msg + ); + } + +} diff --git a/api/src/test/java/io/kafbat/ui/emitter/SeekOperationsTest.java b/api/src/test/java/io/kafbat/ui/emitter/SeekOperationsTest.java index d6b9a528f..2fc7ec5e3 100644 --- a/api/src/test/java/io/kafbat/ui/emitter/SeekOperationsTest.java +++ b/api/src/test/java/io/kafbat/ui/emitter/SeekOperationsTest.java @@ -1,8 +1,13 @@ package io.kafbat.ui.emitter; +import static io.kafbat.ui.model.PollingModeDTO.EARLIEST; +import static io.kafbat.ui.model.PollingModeDTO.LATEST; +import static io.kafbat.ui.model.PollingModeDTO.TAILING; import static org.assertj.core.api.Assertions.assertThat; -import io.kafbat.ui.model.SeekTypeDTO; +import io.kafbat.ui.model.ConsumerPosition; +import io.kafbat.ui.model.PollingModeDTO; +import java.util.List; import java.util.Map; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -14,6 +19,8 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Nested; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; class SeekOperationsTest { @@ -40,13 +47,22 @@ void initMockConsumer() { @Nested class GetOffsetsForSeek { + @Test + void tailing() { + var offsets = SeekOperations.getOffsetsForSeek( + consumer, + new OffsetsInfo(consumer, topic), + new ConsumerPosition(TAILING, topic, List.of(), null, null) + ); + assertThat(offsets).containsExactlyInAnyOrderEntriesOf(Map.of(tp0, 0L, tp1, 10L, tp2, 20L, tp3, 30L)); + } + @Test void latest() { var offsets = SeekOperations.getOffsetsForSeek( consumer, new OffsetsInfo(consumer, topic), - SeekTypeDTO.LATEST, - null + new ConsumerPosition(LATEST, topic, List.of(), null, null) ); assertThat(offsets).containsExactlyInAnyOrderEntriesOf(Map.of(tp2, 20L, tp3, 30L)); } @@ -56,30 +72,35 @@ void beginning() { var offsets = SeekOperations.getOffsetsForSeek( consumer, new OffsetsInfo(consumer, topic), - SeekTypeDTO.BEGINNING, - null + new ConsumerPosition(EARLIEST, topic, List.of(), null, null) ); assertThat(offsets).containsExactlyInAnyOrderEntriesOf(Map.of(tp2, 0L, tp3, 25L)); } - @Test - void offsets() { + @ParameterizedTest + @CsvSource({"TO_OFFSET", "FROM_OFFSET"}) + void offsets(PollingModeDTO mode) { var offsets = SeekOperations.getOffsetsForSeek( consumer, new OffsetsInfo(consumer, topic), - SeekTypeDTO.OFFSET, - Map.of(tp1, 10L, tp2, 10L, tp3, 26L) + new ConsumerPosition( + mode, topic, List.of(tp1, tp2, tp3), null, + new ConsumerPosition.Offsets(null, Map.of(tp1, 10L, tp2, 10L, tp3, 26L)) + ) ); assertThat(offsets).containsExactlyInAnyOrderEntriesOf(Map.of(tp2, 10L, tp3, 26L)); } - @Test - void offsetsWithBoundsFixing() { + @ParameterizedTest + @CsvSource({"TO_OFFSET", "FROM_OFFSET"}) + void offsetsWithBoundsFixing(PollingModeDTO mode) { var offsets = SeekOperations.getOffsetsForSeek( consumer, new OffsetsInfo(consumer, topic), - SeekTypeDTO.OFFSET, - Map.of(tp1, 10L, tp2, 21L, tp3, 24L) + new ConsumerPosition( + mode, topic, List.of(tp1, tp2, tp3), null, + new ConsumerPosition.Offsets(null, Map.of(tp1, 10L, tp2, 21L, tp3, 24L)) + ) ); assertThat(offsets).containsExactlyInAnyOrderEntriesOf(Map.of(tp2, 20L, tp3, 25L)); } diff --git a/api/src/test/java/io/kafbat/ui/emitter/TailingEmitterTest.java b/api/src/test/java/io/kafbat/ui/emitter/TailingEmitterTest.java index dd35e71e7..4194868d3 100644 --- a/api/src/test/java/io/kafbat/ui/emitter/TailingEmitterTest.java +++ b/api/src/test/java/io/kafbat/ui/emitter/TailingEmitterTest.java @@ -4,10 +4,9 @@ import io.kafbat.ui.AbstractIntegrationTest; import io.kafbat.ui.model.ConsumerPosition; -import io.kafbat.ui.model.MessageFilterTypeDTO; -import io.kafbat.ui.model.SeekDirectionDTO; -import io.kafbat.ui.model.SeekTypeDTO; +import io.kafbat.ui.model.PollingModeDTO; import io.kafbat.ui.model.TopicMessageEventDTO; +import io.kafbat.ui.serdes.builtin.StringSerde; import io.kafbat.ui.service.ClustersStorage; import io.kafbat.ui.service.MessagesService; import java.time.Duration; @@ -111,13 +110,12 @@ private Flux createTailingFlux( return applicationContext.getBean(MessagesService.class) .loadMessages(cluster, topicName, - new ConsumerPosition(SeekTypeDTO.LATEST, topic, null), + new ConsumerPosition(PollingModeDTO.TAILING, topic, List.of(), null, null), query, - MessageFilterTypeDTO.STRING_CONTAINS, + null, 0, - SeekDirectionDTO.TAILING, - "String", - "String"); + StringSerde.name(), + StringSerde.name()); } private List startTailing(String filterQuery) { diff --git a/api/src/test/java/io/kafbat/ui/service/MessagesServiceTest.java b/api/src/test/java/io/kafbat/ui/service/MessagesServiceTest.java index f41ea2d22..8939b50c3 100644 --- a/api/src/test/java/io/kafbat/ui/service/MessagesServiceTest.java +++ b/api/src/test/java/io/kafbat/ui/service/MessagesServiceTest.java @@ -8,20 +8,25 @@ import io.kafbat.ui.model.ConsumerPosition; import io.kafbat.ui.model.CreateTopicMessageDTO; import io.kafbat.ui.model.KafkaCluster; -import io.kafbat.ui.model.SeekDirectionDTO; -import io.kafbat.ui.model.SeekTypeDTO; +import io.kafbat.ui.model.PollingModeDTO; import io.kafbat.ui.model.SmartFilterTestExecutionDTO; import io.kafbat.ui.model.SmartFilterTestExecutionResultDTO; import io.kafbat.ui.model.TopicMessageDTO; import io.kafbat.ui.model.TopicMessageEventDTO; import io.kafbat.ui.producer.KafkaTestProducer; import io.kafbat.ui.serdes.builtin.StringSerde; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.UUID; +import java.util.concurrent.atomic.AtomicReference; import org.apache.kafka.clients.admin.NewTopic; +import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; import org.springframework.beans.factory.annotation.Autowired; import reactor.core.publisher.Flux; import reactor.test.StepVerifier; @@ -36,6 +41,8 @@ class MessagesServiceTest extends AbstractIntegrationTest { KafkaCluster cluster; + Set createdTopics = new HashSet<>(); + @BeforeEach void init() { cluster = applicationContext @@ -44,6 +51,11 @@ void init() { .get(); } + @AfterEach + void deleteCreatedTopics() { + createdTopics.forEach(MessagesServiceTest::deleteTopic); + } + @Test void deleteTopicMessagesReturnsExceptionWhenTopicNotFound() { StepVerifier.create(messagesService.deleteTopicMessages(cluster, NON_EXISTING_TOPIC, List.of())) @@ -61,7 +73,9 @@ void sendMessageReturnsExceptionWhenTopicNotFound() { @Test void loadMessagesReturnsExceptionWhenTopicNotFound() { StepVerifier.create(messagesService - .loadMessages(cluster, NON_EXISTING_TOPIC, null, null, null, 1, null, "String", "String")) + .loadMessages(cluster, NON_EXISTING_TOPIC, + new ConsumerPosition(PollingModeDTO.TAILING, NON_EXISTING_TOPIC, List.of(), null, null), + null, null, 1, "String", "String")) .expectError(TopicNotFoundException.class) .verify(); } @@ -69,32 +83,84 @@ void loadMessagesReturnsExceptionWhenTopicNotFound() { @Test void maskingAppliedOnConfiguredClusters() throws Exception { String testTopic = MASKED_TOPICS_PREFIX + UUID.randomUUID(); + createTopicWithCleanup(new NewTopic(testTopic, 1, (short) 1)); + try (var producer = KafkaTestProducer.forKafka(kafka)) { - createTopic(new NewTopic(testTopic, 1, (short) 1)); producer.send(testTopic, "message1"); producer.send(testTopic, "message2").get(); + } + + Flux msgsFlux = messagesService.loadMessages( + cluster, + testTopic, + new ConsumerPosition(PollingModeDTO.EARLIEST, testTopic, List.of(), null, null), + null, + null, + 100, + StringSerde.name(), + StringSerde.name() + ).filter(evt -> evt.getType() == TopicMessageEventDTO.TypeEnum.MESSAGE) + .map(TopicMessageEventDTO::getMessage); + + // both messages should be masked + StepVerifier.create(msgsFlux) + .expectNextMatches(msg -> msg.getContent().equals("***")) + .expectNextMatches(msg -> msg.getContent().equals("***")) + .verifyComplete(); + } - Flux msgsFlux = messagesService.loadMessages( - cluster, - testTopic, - new ConsumerPosition(SeekTypeDTO.BEGINNING, testTopic, null), - null, - null, - 100, - SeekDirectionDTO.FORWARD, - StringSerde.name(), - StringSerde.name() - ).filter(evt -> evt.getType() == TopicMessageEventDTO.TypeEnum.MESSAGE) - .map(TopicMessageEventDTO::getMessage); - - // both messages should be masked - StepVerifier.create(msgsFlux) - .expectNextMatches(msg -> msg.getContent().equals("***")) - .expectNextMatches(msg -> msg.getContent().equals("***")) - .verifyComplete(); - } finally { - deleteTopic(testTopic); + @ParameterizedTest + @CsvSource({"EARLIEST", "LATEST"}) + void cursorIsRegisteredAfterPollingIsDoneAndCanBeUsedForNextPagePolling(PollingModeDTO mode) { + String testTopic = MessagesServiceTest.class.getSimpleName() + UUID.randomUUID(); + createTopicWithCleanup(new NewTopic(testTopic, 5, (short) 1)); + + int msgsToGenerate = 100; + int pageSize = (msgsToGenerate / 2) + 1; + + try (var producer = KafkaTestProducer.forKafka(kafka)) { + for (int i = 0; i < msgsToGenerate; i++) { + producer.send(testTopic, "message_" + i); + } } + + var cursorIdCatcher = new AtomicReference(); + Flux msgsFlux = messagesService.loadMessages( + cluster, testTopic, + new ConsumerPosition(mode, testTopic, List.of(), null, null), + null, null, pageSize, StringSerde.name(), StringSerde.name()) + .doOnNext(evt -> { + if (evt.getType() == TopicMessageEventDTO.TypeEnum.DONE) { + assertThat(evt.getCursor()).isNotNull(); + cursorIdCatcher.set(evt.getCursor().getId()); + } + }) + .filter(evt -> evt.getType() == TopicMessageEventDTO.TypeEnum.MESSAGE) + .map(evt -> evt.getMessage().getContent()); + + StepVerifier.create(msgsFlux) + .expectNextCount(pageSize) + .verifyComplete(); + + assertThat(cursorIdCatcher.get()).isNotNull(); + + Flux remainingMsgs = messagesService.loadMessages(cluster, testTopic, cursorIdCatcher.get()) + .doOnNext(evt -> { + if (evt.getType() == TopicMessageEventDTO.TypeEnum.DONE) { + assertThat(evt.getCursor()).isNull(); + } + }) + .filter(evt -> evt.getType() == TopicMessageEventDTO.TypeEnum.MESSAGE) + .map(evt -> evt.getMessage().getContent()); + + StepVerifier.create(remainingMsgs) + .expectNextCount(msgsToGenerate - pageSize) + .verifyComplete(); + } + + private void createTopicWithCleanup(NewTopic newTopic) { + createTopic(newTopic); + createdTopics.add(newTopic.name()); } @Test diff --git a/api/src/test/java/io/kafbat/ui/service/RecordEmitterTest.java b/api/src/test/java/io/kafbat/ui/service/RecordEmitterTest.java index 6074ca82a..65bf1f49d 100644 --- a/api/src/test/java/io/kafbat/ui/service/RecordEmitterTest.java +++ b/api/src/test/java/io/kafbat/ui/service/RecordEmitterTest.java @@ -1,13 +1,16 @@ package io.kafbat.ui.service; -import static io.kafbat.ui.model.SeekTypeDTO.BEGINNING; -import static io.kafbat.ui.model.SeekTypeDTO.LATEST; -import static io.kafbat.ui.model.SeekTypeDTO.OFFSET; -import static io.kafbat.ui.model.SeekTypeDTO.TIMESTAMP; +import static io.kafbat.ui.model.PollingModeDTO.EARLIEST; +import static io.kafbat.ui.model.PollingModeDTO.FROM_OFFSET; +import static io.kafbat.ui.model.PollingModeDTO.FROM_TIMESTAMP; +import static io.kafbat.ui.model.PollingModeDTO.LATEST; +import static io.kafbat.ui.model.PollingModeDTO.TO_OFFSET; +import static io.kafbat.ui.model.PollingModeDTO.TO_TIMESTAMP; import static org.assertj.core.api.Assertions.assertThat; import io.kafbat.ui.AbstractIntegrationTest; import io.kafbat.ui.emitter.BackwardEmitter; +import io.kafbat.ui.emitter.Cursor; import io.kafbat.ui.emitter.EnhancedConsumer; import io.kafbat.ui.emitter.ForwardEmitter; import io.kafbat.ui.emitter.PollingSettings; @@ -43,6 +46,7 @@ import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; +import org.mockito.Mockito; import reactor.core.publisher.Flux; import reactor.core.publisher.FluxSink; import reactor.test.StepVerifier; @@ -57,16 +61,18 @@ class RecordEmitterTest extends AbstractIntegrationTest { static final String EMPTY_TOPIC = TOPIC + "_empty"; static final List SENT_RECORDS = new ArrayList<>(); static final ConsumerRecordDeserializer RECORD_DESERIALIZER = createRecordsDeserializer(); + static final Cursor.Tracking CURSOR_MOCK = Mockito.mock(Cursor.Tracking.class); static final Predicate NOOP_FILTER = m -> true; @BeforeAll static void generateMsgs() throws Exception { createTopic(new NewTopic(TOPIC, PARTITIONS, (short) 1)); createTopic(new NewTopic(EMPTY_TOPIC, PARTITIONS, (short) 1)); + long startTs = System.currentTimeMillis(); try (var producer = KafkaTestProducer.forKafka(kafka)) { for (int partition = 0; partition < PARTITIONS; partition++) { for (int i = 0; i < MSGS_PER_PARTITION; i++) { - long ts = System.currentTimeMillis() + i; + long ts = (startTs += 100); var value = "msg_" + partition + "_" + i; var metadata = producer.send( new ProducerRecord<>( @@ -115,20 +121,22 @@ private static ConsumerRecordDeserializer createRecordsDeserializer() { void pollNothingOnEmptyTopic() { var forwardEmitter = new ForwardEmitter( this::createConsumer, - new ConsumerPosition(BEGINNING, EMPTY_TOPIC, null), + new ConsumerPosition(EARLIEST, EMPTY_TOPIC, List.of(), null, null), 100, RECORD_DESERIALIZER, NOOP_FILTER, - PollingSettings.createDefault() + PollingSettings.createDefault(), + CURSOR_MOCK ); var backwardEmitter = new BackwardEmitter( this::createConsumer, - new ConsumerPosition(BEGINNING, EMPTY_TOPIC, null), + new ConsumerPosition(EARLIEST, EMPTY_TOPIC, List.of(), null, null), 100, RECORD_DESERIALIZER, NOOP_FILTER, - PollingSettings.createDefault() + PollingSettings.createDefault(), + CURSOR_MOCK ); StepVerifier.create(Flux.create(forwardEmitter)) @@ -148,20 +156,22 @@ void pollNothingOnEmptyTopic() { void pollFullTopicFromBeginning() { var forwardEmitter = new ForwardEmitter( this::createConsumer, - new ConsumerPosition(BEGINNING, TOPIC, null), + new ConsumerPosition(EARLIEST, TOPIC, List.of(), null, null), PARTITIONS * MSGS_PER_PARTITION, RECORD_DESERIALIZER, NOOP_FILTER, - PollingSettings.createDefault() + PollingSettings.createDefault(), + CURSOR_MOCK ); var backwardEmitter = new BackwardEmitter( this::createConsumer, - new ConsumerPosition(LATEST, TOPIC, null), + new ConsumerPosition(LATEST, TOPIC, List.of(), null, null), PARTITIONS * MSGS_PER_PARTITION, RECORD_DESERIALIZER, NOOP_FILTER, - PollingSettings.createDefault() + PollingSettings.createDefault(), + CURSOR_MOCK ); List expectedValues = SENT_RECORDS.stream().map(Record::getValue).collect(Collectors.toList()); @@ -180,20 +190,24 @@ void pollWithOffsets() { var forwardEmitter = new ForwardEmitter( this::createConsumer, - new ConsumerPosition(OFFSET, TOPIC, targetOffsets), + new ConsumerPosition(FROM_OFFSET, TOPIC, List.copyOf(targetOffsets.keySet()), null, + new ConsumerPosition.Offsets(null, targetOffsets)), PARTITIONS * MSGS_PER_PARTITION, RECORD_DESERIALIZER, NOOP_FILTER, - PollingSettings.createDefault() + PollingSettings.createDefault(), + CURSOR_MOCK ); var backwardEmitter = new BackwardEmitter( this::createConsumer, - new ConsumerPosition(OFFSET, TOPIC, targetOffsets), + new ConsumerPosition(TO_OFFSET, TOPIC, List.copyOf(targetOffsets.keySet()), null, + new ConsumerPosition.Offsets(null, targetOffsets)), PARTITIONS * MSGS_PER_PARTITION, RECORD_DESERIALIZER, NOOP_FILTER, - PollingSettings.createDefault() + PollingSettings.createDefault(), + CURSOR_MOCK ); var expectedValues = SENT_RECORDS.stream() @@ -213,50 +227,45 @@ void pollWithOffsets() { @Test void pollWithTimestamps() { - Map targetTimestamps = new HashMap<>(); - final Map> perPartition = - SENT_RECORDS.stream().collect(Collectors.groupingBy((r) -> r.tp)); - for (int i = 0; i < PARTITIONS; i++) { - final List records = perPartition.get(new TopicPartition(TOPIC, i)); - int randRecordIdx = ThreadLocalRandom.current().nextInt(records.size()); - log.info("partition: {} position: {}", i, randRecordIdx); - targetTimestamps.put( - new TopicPartition(TOPIC, i), - records.get(randRecordIdx).getTimestamp() - ); - } + var tsStats = SENT_RECORDS.stream().mapToLong(Record::getTimestamp).summaryStatistics(); + //choosing ts in the middle + long targetTimestamp = tsStats.getMin() + ((tsStats.getMax() - tsStats.getMin()) / 2); var forwardEmitter = new ForwardEmitter( this::createConsumer, - new ConsumerPosition(TIMESTAMP, TOPIC, targetTimestamps), + new ConsumerPosition(FROM_TIMESTAMP, TOPIC, List.of(), targetTimestamp, null), PARTITIONS * MSGS_PER_PARTITION, RECORD_DESERIALIZER, NOOP_FILTER, - PollingSettings.createDefault() + PollingSettings.createDefault(), + CURSOR_MOCK + ); + + expectEmitter( + forwardEmitter, + SENT_RECORDS.stream() + .filter(r -> r.getTimestamp() >= targetTimestamp) + .map(Record::getValue) + .collect(Collectors.toList()) ); var backwardEmitter = new BackwardEmitter( this::createConsumer, - new ConsumerPosition(TIMESTAMP, TOPIC, targetTimestamps), + new ConsumerPosition(TO_TIMESTAMP, TOPIC, List.of(), targetTimestamp, null), PARTITIONS * MSGS_PER_PARTITION, RECORD_DESERIALIZER, NOOP_FILTER, - PollingSettings.createDefault() + PollingSettings.createDefault(), + CURSOR_MOCK ); - var expectedValues = SENT_RECORDS.stream() - .filter(r -> r.getTimestamp() >= targetTimestamps.get(r.getTp())) - .map(Record::getValue) - .collect(Collectors.toList()); - - expectEmitter(forwardEmitter, expectedValues); - - expectedValues = SENT_RECORDS.stream() - .filter(r -> r.getTimestamp() < targetTimestamps.get(r.getTp())) - .map(Record::getValue) - .collect(Collectors.toList()); - - expectEmitter(backwardEmitter, expectedValues); + expectEmitter( + backwardEmitter, + SENT_RECORDS.stream() + .filter(r -> r.getTimestamp() < targetTimestamp) + .map(Record::getValue) + .collect(Collectors.toList()) + ); } @Test @@ -269,11 +278,13 @@ void backwardEmitterSeekToEnd() { var backwardEmitter = new BackwardEmitter( this::createConsumer, - new ConsumerPosition(OFFSET, TOPIC, targetOffsets), + new ConsumerPosition(TO_OFFSET, TOPIC, List.copyOf(targetOffsets.keySet()), null, + new ConsumerPosition.Offsets(null, targetOffsets)), numMessages, RECORD_DESERIALIZER, NOOP_FILTER, - PollingSettings.createDefault() + PollingSettings.createDefault(), + CURSOR_MOCK ); var expectedValues = SENT_RECORDS.stream() @@ -296,11 +307,13 @@ void backwardEmitterSeekToBegin() { var backwardEmitter = new BackwardEmitter( this::createConsumer, - new ConsumerPosition(OFFSET, TOPIC, offsets), + new ConsumerPosition(TO_OFFSET, TOPIC, List.copyOf(offsets.keySet()), null, + new ConsumerPosition.Offsets(null, offsets)), 100, RECORD_DESERIALIZER, NOOP_FILTER, - PollingSettings.createDefault() + PollingSettings.createDefault(), + CURSOR_MOCK ); expectEmitter(backwardEmitter, diff --git a/api/src/test/java/io/kafbat/ui/service/SendAndReadTests.java b/api/src/test/java/io/kafbat/ui/service/SendAndReadTests.java index e2d99df0e..cdf304509 100644 --- a/api/src/test/java/io/kafbat/ui/service/SendAndReadTests.java +++ b/api/src/test/java/io/kafbat/ui/service/SendAndReadTests.java @@ -11,8 +11,7 @@ import io.kafbat.ui.model.ConsumerPosition; import io.kafbat.ui.model.CreateTopicMessageDTO; import io.kafbat.ui.model.KafkaCluster; -import io.kafbat.ui.model.SeekDirectionDTO; -import io.kafbat.ui.model.SeekTypeDTO; +import io.kafbat.ui.model.PollingModeDTO; import io.kafbat.ui.model.TopicMessageDTO; import io.kafbat.ui.model.TopicMessageEventDTO; import io.kafbat.ui.serdes.builtin.Int32Serde; @@ -20,6 +19,7 @@ import io.kafbat.ui.serdes.builtin.StringSerde; import io.kafbat.ui.serdes.builtin.sr.SchemaRegistrySerde; import java.time.Duration; +import java.util.List; import java.util.Map; import java.util.Objects; import java.util.UUID; @@ -500,15 +500,10 @@ public void doAssert(Consumer msgAssert) { TopicMessageDTO polled = messagesService.loadMessages( targetCluster, topic, - new ConsumerPosition( - SeekTypeDTO.BEGINNING, - topic, - Map.of(new TopicPartition(topic, 0), 0L) - ), + new ConsumerPosition(PollingModeDTO.EARLIEST, topic, List.of(), null, null), null, null, 1, - SeekDirectionDTO.FORWARD, msgToSend.getKeySerde().get(), msgToSend.getValueSerde().get() ).filter(e -> e.getType().equals(TopicMessageEventDTO.TypeEnum.MESSAGE)) diff --git a/contract/src/main/resources/swagger/kafbat-ui-api.yaml b/contract/src/main/resources/swagger/kafbat-ui-api.yaml index d833c1a95..8feb20c15 100644 --- a/contract/src/main/resources/swagger/kafbat-ui-api.yaml +++ b/contract/src/main/resources/swagger/kafbat-ui-api.yaml @@ -763,6 +763,119 @@ paths: 404: description: Not found + /api/clusters/{clusterName}/topics/{topicName}/smartfilters: + post: + tags: + - Messages + summary: registerFilter + operationId: registerFilter + parameters: + - name: clusterName + in: path + required: true + schema: + type: string + - name: topicName + in: path + required: true + schema: + type: string + requestBody: + content: + application/json: + schema: + $ref: '#/components/schemas/MessageFilterRegistration' + responses: + 200: + description: OK + content: + application/json: + schema: + $ref: '#/components/schemas/MessageFilterId' + + + /api/clusters/{clusterName}/topics/{topicName}/messages/v2: + get: + tags: + - Messages + summary: getTopicMessagesV2 + operationId: getTopicMessagesV2 + parameters: + - name: clusterName + in: path + required: true + schema: + type: string + - name: topicName + in: path + required: true + schema: + type: string + - name: mode + in: query + description: Messages polling mode + schema: + $ref: "#/components/schemas/PollingMode" + - name: partitions + in: query + schema: + type: array + description: List of target partitions (all partitions if not provided) + items: + type: integer + - name: limit + in: query + description: Max number of messages can be returned + schema: + type: integer + - name: stringFilter + in: query + description: query string to contains string filtration + schema: + type: string + - name: smartFilterId + in: query + description: filter id, that was registered beforehand + schema: + type: string + - name: offset + in: query + description: message offset to read from / to + schema: + type: integer + format: int64 + - name: timestamp + in: query + description: timestamp (in ms) to read from / to + schema: + type: integer + format: int64 + - name: keySerde + in: query + description: "Serde that should be used for deserialization. Will be chosen automatically if not set." + schema: + type: string + - name: valueSerde + in: query + description: "Serde that should be used for deserialization. Will be chosen automatically if not set." + schema: + type: string + - name: cursor + in: query + description: "id of the cursor for pagination, if passed - all other query params ignored" + schema: + type: string + responses: + 200: + description: OK + content: + text/event-stream: + schema: + type: array + items: + $ref: '#/components/schemas/TopicMessageEvent' + + /api/clusters/{clusterName}/topics/{topicName}/activeproducers: get: tags: @@ -790,6 +903,7 @@ paths: items: $ref: '#/components/schemas/TopicProducerState' + /api/clusters/{clusterName}/topics/{topicName}/consumer-groups: get: tags: @@ -2831,14 +2945,17 @@ components: - PHASE - MESSAGE - CONSUMING + # if event type == DONE && cursor field is null -> topic was fully polled and ui should stop calling get messages, + # otherwise ui should pass cursor param to continue polling - DONE - - EMIT_THROTTLING message: $ref: "#/components/schemas/TopicMessage" phase: $ref: "#/components/schemas/TopicMessagePhase" consuming: $ref: "#/components/schemas/TopicMessageConsuming" + cursor: + $ref: "#/components/schemas/TopicMessageNextPageCursor" TopicMessagePhase: type: object @@ -2868,6 +2985,11 @@ components: filterApplyErrors: type: integer + TopicMessageNextPageCursor: + type: object + properties: + id: + type: string TopicMessage: type: object @@ -2940,6 +3062,29 @@ components: - TIMESTAMP - LATEST + MessageFilterRegistration: + type: object + properties: + filterCode: + type: string + + MessageFilterId: + type: object + properties: + id: + type: string + + PollingMode: + type: string + enum: + - FROM_OFFSET + - TO_OFFSET + - FROM_TIMESTAMP + - TO_TIMESTAMP + - LATEST + - EARLIEST + - TAILING + MessageFilterType: type: string enum: diff --git a/documentation/images/Interface.gif b/documentation/images/Interface.gif deleted file mode 100644 index 3cb35018b..000000000 Binary files a/documentation/images/Interface.gif and /dev/null differ diff --git a/documentation/images/logo_new.png b/documentation/images/logo_new.png new file mode 100644 index 000000000..fdbca3fa1 Binary files /dev/null and b/documentation/images/logo_new.png differ diff --git a/frontend/package.json b/frontend/package.json index 540c85a88..26be7c951 100644 --- a/frontend/package.json +++ b/frontend/package.json @@ -50,9 +50,9 @@ "scripts": { "start": "vite", "dev": "vite", - "clean": "rimraf ./src/generated-sources", - "gen:sources": "pnpm clean && openapi-generator-cli generate", - "build": "pnpm gen:sources && tsc --noEmit && vite build", + "compile": "pnpm gen:sources && tsc --noEmit", + "gen:sources": "rimraf ./src/generated-sources && openapi-generator-cli generate", + "build": "rimraf ./src/generated-sources && openapi-generator-cli generate && tsc --noEmit && vite build", "preview": "vite preview", "lint": "eslint --ext .tsx,.ts src/", "lint:fix": "eslint --ext .tsx,.ts src/ --fix", diff --git a/frontend/src/components/Brokers/Broker/Configs/Configs.styled.ts b/frontend/src/components/Brokers/Broker/Configs/Configs.styled.ts index cced53373..77768dc99 100644 --- a/frontend/src/components/Brokers/Broker/Configs/Configs.styled.ts +++ b/frontend/src/components/Brokers/Broker/Configs/Configs.styled.ts @@ -1,36 +1,6 @@ import styled from 'styled-components'; -export const ValueWrapper = styled.div` - display: flex; - justify-content: space-between; - button { - margin: 0 10px; - } -`; - -export const Value = styled.span` - line-height: 24px; - margin-right: 10px; - text-overflow: ellipsis; - max-width: 400px; - overflow: hidden; - white-space: nowrap; -`; - -export const ButtonsWrapper = styled.div` - display: flex; -`; export const SearchWrapper = styled.div` margin: 10px; width: 21%; `; - -export const Source = styled.div` - display: flex; - align-content: center; - svg { - margin-left: 10px; - vertical-align: middle; - cursor: pointer; - } -`; diff --git a/frontend/src/components/Brokers/Broker/Configs/Configs.tsx b/frontend/src/components/Brokers/Broker/Configs/Configs.tsx index 5b05b7cc5..54b9bd6a9 100644 --- a/frontend/src/components/Brokers/Broker/Configs/Configs.tsx +++ b/frontend/src/components/Brokers/Broker/Configs/Configs.tsx @@ -1,5 +1,4 @@ -import React from 'react'; -import { CellContext, ColumnDef } from '@tanstack/react-table'; +import React, { type FC, useMemo, useState } from 'react'; import { ClusterBrokerParam } from 'lib/paths'; import useAppParams from 'lib/hooks/useAppParams'; import { @@ -7,91 +6,37 @@ import { useUpdateBrokerConfigByName, } from 'lib/hooks/api/brokers'; import Table from 'components/common/NewTable'; -import { BrokerConfig, ConfigSource } from 'generated-sources'; +import type { BrokerConfig } from 'generated-sources'; import Search from 'components/common/Search/Search'; -import Tooltip from 'components/common/Tooltip/Tooltip'; -import InfoIcon from 'components/common/Icons/InfoIcon'; +import { + getBrokerConfigsTableColumns, + getConfigTableData, +} from 'components/Brokers/Broker/Configs/lib/utils'; -import InputCell from './InputCell'; import * as S from './Configs.styled'; -const tooltipContent = `DYNAMIC_TOPIC_CONFIG = dynamic topic config that is configured for a specific topic -DYNAMIC_BROKER_LOGGER_CONFIG = dynamic broker logger config that is configured for a specific broker -DYNAMIC_BROKER_CONFIG = dynamic broker config that is configured for a specific broker -DYNAMIC_DEFAULT_BROKER_CONFIG = dynamic broker config that is configured as default for all brokers in the cluster -STATIC_BROKER_CONFIG = static broker config provided as broker properties at start up (e.g. server.properties file) -DEFAULT_CONFIG = built-in default configuration for configs that have a default value -UNKNOWN = source unknown e.g. in the ConfigEntry used for alter requests where source is not set`; - -const Configs: React.FC = () => { - const [keyword, setKeyword] = React.useState(''); +const Configs: FC = () => { + const [searchQuery, setSearchQuery] = useState(''); const { clusterName, brokerId } = useAppParams(); - const { data = [] } = useBrokerConfig(clusterName, Number(brokerId)); - const stateMutation = useUpdateBrokerConfigByName( + const { data: configs = [] } = useBrokerConfig(clusterName, Number(brokerId)); + const updateBrokerConfigByName = useUpdateBrokerConfigByName( clusterName, Number(brokerId) ); - const getData = () => { - return data - .filter((item) => { - const nameMatch = item.name - .toLocaleLowerCase() - .includes(keyword.toLocaleLowerCase()); - return nameMatch - ? true - : item.value && - item.value - .toLocaleLowerCase() - .includes(keyword.toLocaleLowerCase()); // try to match the keyword on any of the item.value elements when nameMatch fails but item.value exists - }) - .sort((a, b) => { - if (a.source === b.source) return 0; - return a.source === ConfigSource.DYNAMIC_BROKER_CONFIG ? -1 : 1; - }); - }; - - const dataSource = React.useMemo(() => getData(), [data, keyword]); - - const renderCell = (props: CellContext) => ( - { - stateMutation.mutateAsync({ - name, - brokerConfigItem: { - value, - }, - }); - }} - /> + const tableData = useMemo( + () => getConfigTableData(configs, searchQuery), + [configs, searchQuery] ); - const columns = React.useMemo[]>( - () => [ - { header: 'Key', accessorKey: 'name' }, - { - header: 'Value', - accessorKey: 'value', - cell: renderCell, - }, - { - // eslint-disable-next-line react/no-unstable-nested-components - header: () => { - return ( - - Source - } - content={tooltipContent} - placement="top-end" - /> - - ); - }, - accessorKey: 'source', - }, - ], + const onUpdateInputCell = async ( + name: BrokerConfig['name'], + value: BrokerConfig['value'] + ) => + updateBrokerConfigByName.mutateAsync({ name, brokerConfigItem: { value } }); + + const columns = useMemo( + () => getBrokerConfigsTableColumns(onUpdateInputCell), [] ); @@ -99,12 +44,12 @@ const Configs: React.FC = () => { <> - +
); }; diff --git a/frontend/src/components/Brokers/Broker/Configs/InputCell.tsx b/frontend/src/components/Brokers/Broker/Configs/InputCell.tsx deleted file mode 100644 index bf54c45c6..000000000 --- a/frontend/src/components/Brokers/Broker/Configs/InputCell.tsx +++ /dev/null @@ -1,91 +0,0 @@ -import React, { useEffect } from 'react'; -import { CellContext } from '@tanstack/react-table'; -import CheckmarkIcon from 'components/common/Icons/CheckmarkIcon'; -import EditIcon from 'components/common/Icons/EditIcon'; -import CancelIcon from 'components/common/Icons/CancelIcon'; -import { useConfirm } from 'lib/hooks/useConfirm'; -import { Action, BrokerConfig, ResourceType } from 'generated-sources'; -import { Button } from 'components/common/Button/Button'; -import Input from 'components/common/Input/Input'; -import { ActionButton } from 'components/common/ActionComponent'; - -import * as S from './Configs.styled'; - -interface InputCellProps extends CellContext { - onUpdate: (name: string, value?: string) => void; -} - -const InputCell: React.FC = ({ row, getValue, onUpdate }) => { - const initialValue = `${getValue()}`; - const [isEdit, setIsEdit] = React.useState(false); - const [value, setValue] = React.useState(initialValue); - - const confirm = useConfirm(); - - const onSave = () => { - if (value !== initialValue) { - confirm('Are you sure you want to change the value?', async () => { - onUpdate(row?.original?.name, value); - }); - } - setIsEdit(false); - }; - - useEffect(() => { - setValue(initialValue); - }, [initialValue]); - - return isEdit ? ( - - setValue(target?.value)} - /> - - - - - - ) : ( - - {initialValue} - setIsEdit(true)} - permission={{ - resource: ResourceType.CLUSTERCONFIG, - action: Action.EDIT, - }} - > - Edit - - - ); -}; - -export default InputCell; diff --git a/frontend/src/components/Brokers/Broker/Configs/TableComponents/ConfigSourceHeader/ConfigSourceHeader.styled.ts b/frontend/src/components/Brokers/Broker/Configs/TableComponents/ConfigSourceHeader/ConfigSourceHeader.styled.ts new file mode 100644 index 000000000..3856932b1 --- /dev/null +++ b/frontend/src/components/Brokers/Broker/Configs/TableComponents/ConfigSourceHeader/ConfigSourceHeader.styled.ts @@ -0,0 +1,12 @@ +import styled from 'styled-components'; + +export const Source = styled.div` + display: flex; + align-content: center; + + svg { + margin-left: 10px; + vertical-align: middle; + cursor: pointer; + } +`; diff --git a/frontend/src/components/Brokers/Broker/Configs/TableComponents/ConfigSourceHeader/ConfigSourceHeader.tsx b/frontend/src/components/Brokers/Broker/Configs/TableComponents/ConfigSourceHeader/ConfigSourceHeader.tsx new file mode 100644 index 000000000..e521218e8 --- /dev/null +++ b/frontend/src/components/Brokers/Broker/Configs/TableComponents/ConfigSourceHeader/ConfigSourceHeader.tsx @@ -0,0 +1,27 @@ +import React from 'react'; +import Tooltip from 'components/common/Tooltip/Tooltip'; +import InfoIcon from 'components/common/Icons/InfoIcon'; +import { CONFIG_SOURCE_NAME_MAP } from 'components/Brokers/Broker/Configs/lib/constants'; + +import * as S from './ConfigSourceHeader.styled'; + +const tooltipContent = `${CONFIG_SOURCE_NAME_MAP.DYNAMIC_TOPIC_CONFIG} = dynamic topic config that is configured for a specific topic +${CONFIG_SOURCE_NAME_MAP.DYNAMIC_BROKER_LOGGER_CONFIG} = dynamic broker logger config that is configured for a specific broker +${CONFIG_SOURCE_NAME_MAP.DYNAMIC_BROKER_CONFIG} = dynamic broker config that is configured for a specific broker +${CONFIG_SOURCE_NAME_MAP.DYNAMIC_DEFAULT_BROKER_CONFIG} = dynamic broker config that is configured as default for all brokers in the cluster +${CONFIG_SOURCE_NAME_MAP.STATIC_BROKER_CONFIG} = static broker config provided as broker properties at start up (e.g. server.properties file) +${CONFIG_SOURCE_NAME_MAP.DEFAULT_CONFIG} = built-in default configuration for configs that have a default value +${CONFIG_SOURCE_NAME_MAP.UNKNOWN} = source unknown e.g. in the ConfigEntry used for alter requests where source is not set`; + +const ConfigSourceHeader = () => ( + + Source + } + content={tooltipContent} + placement="top-end" + /> + +); + +export default ConfigSourceHeader; diff --git a/frontend/src/components/Brokers/Broker/Configs/TableComponents/InputCell/InputCellEditMode.tsx b/frontend/src/components/Brokers/Broker/Configs/TableComponents/InputCell/InputCellEditMode.tsx new file mode 100644 index 000000000..2ac00ca63 --- /dev/null +++ b/frontend/src/components/Brokers/Broker/Configs/TableComponents/InputCell/InputCellEditMode.tsx @@ -0,0 +1,53 @@ +import React, { type FC, useState } from 'react'; +import Input from 'components/common/Input/Input'; +import { Button } from 'components/common/Button/Button'; +import CheckmarkIcon from 'components/common/Icons/CheckmarkIcon'; +import CancelIcon from 'components/common/Icons/CancelIcon'; + +import * as S from './styled'; + +interface EditModeProps { + initialValue: string; + onSave: (value: string) => void; + onCancel: () => void; +} + +const InputCellEditMode: FC = ({ + initialValue, + onSave, + onCancel, +}) => { + const [value, setValue] = useState(initialValue); + + return ( + + setValue(target.value)} + /> + + + + + + ); +}; + +export default InputCellEditMode; diff --git a/frontend/src/components/Brokers/Broker/Configs/TableComponents/InputCell/InputCellViewMode.tsx b/frontend/src/components/Brokers/Broker/Configs/TableComponents/InputCell/InputCellViewMode.tsx new file mode 100644 index 000000000..1ddfa2b03 --- /dev/null +++ b/frontend/src/components/Brokers/Broker/Configs/TableComponents/InputCell/InputCellViewMode.tsx @@ -0,0 +1,56 @@ +import React, { type FC } from 'react'; +import { Button } from 'components/common/Button/Button'; +import EditIcon from 'components/common/Icons/EditIcon'; +import type { ConfigUnit } from 'components/Brokers/Broker/Configs/lib/types'; +import Tooltip from 'components/common/Tooltip/Tooltip'; +import { getConfigDisplayValue } from 'components/Brokers/Broker/Configs/lib/utils'; + +import * as S from './styled'; + +interface InputCellViewModeProps { + value: string; + unit: ConfigUnit | undefined; + onEdit: () => void; + isDynamic: boolean; + isSensitive: boolean; + isReadOnly: boolean; +} + +const InputCellViewMode: FC = ({ + value, + unit, + onEdit, + isDynamic, + isSensitive, + isReadOnly, +}) => { + const { displayValue, title } = getConfigDisplayValue( + isSensitive, + value, + unit + ); + + return ( + + {displayValue} + + Edit + + } + showTooltip={isReadOnly} + content="Property is read-only" + placement="left" + /> + + ); +}; + +export default InputCellViewMode; diff --git a/frontend/src/components/Brokers/Broker/Configs/TableComponents/InputCell/__test__/InputCell.spec.tsx b/frontend/src/components/Brokers/Broker/Configs/TableComponents/InputCell/__test__/InputCell.spec.tsx new file mode 100644 index 000000000..421b00f21 --- /dev/null +++ b/frontend/src/components/Brokers/Broker/Configs/TableComponents/InputCell/__test__/InputCell.spec.tsx @@ -0,0 +1,72 @@ +import React from 'react'; +import { screen } from '@testing-library/react'; +import userEvent from '@testing-library/user-event'; +import InputCell, { + type InputCellProps, +} from 'components/Brokers/Broker/Configs/TableComponents/InputCell/index'; +import { render } from 'lib/testHelpers'; +import { ConfigSource } from 'generated-sources'; +import { useConfirm } from 'lib/hooks/useConfirm'; +import { BrokerConfigsTableRow } from 'components/Brokers/Broker/Configs/lib/types'; +import { Row } from '@tanstack/react-table'; + +jest.mock('lib/hooks/useConfirm', () => ({ + useConfirm: jest.fn(), +})); + +describe('InputCell', () => { + const mockOnUpdate = jest.fn(); + const initialValue = 'initialValue'; + const name = 'testName'; + const original = { + name, + source: ConfigSource.DYNAMIC_BROKER_CONFIG, + value: initialValue, + isSensitive: false, + isReadOnly: false, + }; + + beforeEach(() => { + const setupWrapper = (props?: Partial) => ( + } + onUpdate={mockOnUpdate} + /> + ); + (useConfirm as jest.Mock).mockImplementation( + () => (message: string, callback: () => void) => callback() + ); + render(setupWrapper()); + }); + + it('renders InputCellViewMode by default', () => { + expect(screen.getByText(initialValue)).toBeInTheDocument(); + }); + + it('switches to InputCellEditMode upon triggering an edit action', async () => { + const user = userEvent.setup(); + await user.click(screen.getByLabelText('editAction')); + expect( + screen.getByRole('textbox', { name: /inputValue/i }) + ).toBeInTheDocument(); + }); + + it('calls onUpdate callback with the new value when saved', async () => { + const user = userEvent.setup(); + await user.click(screen.getByLabelText('editAction')); // Enter edit mode + await user.type( + screen.getByRole('textbox', { name: /inputValue/i }), + '123' + ); + await user.click(screen.getByRole('button', { name: /confirmAction/i })); + expect(mockOnUpdate).toHaveBeenCalledWith(name, 'initialValue123'); + }); + + it('returns to InputCellViewMode upon canceling an edit', async () => { + const user = userEvent.setup(); + await user.click(screen.getByLabelText('editAction')); // Enter edit mode + await user.click(screen.getByRole('button', { name: /cancelAction/i })); + expect(screen.getByText(initialValue)).toBeInTheDocument(); // Back to view mode + }); +}); diff --git a/frontend/src/components/Brokers/Broker/Configs/TableComponents/InputCell/__test__/InputCellEditMode.spec.tsx b/frontend/src/components/Brokers/Broker/Configs/TableComponents/InputCell/__test__/InputCellEditMode.spec.tsx new file mode 100644 index 000000000..91270469e --- /dev/null +++ b/frontend/src/components/Brokers/Broker/Configs/TableComponents/InputCell/__test__/InputCellEditMode.spec.tsx @@ -0,0 +1,42 @@ +import React from 'react'; +import { screen } from '@testing-library/react'; +import InputCellEditMode from 'components/Brokers/Broker/Configs/TableComponents/InputCell/InputCellEditMode'; +import { render } from 'lib/testHelpers'; +import userEvent from '@testing-library/user-event'; + +describe('InputCellEditMode', () => { + const mockOnSave = jest.fn(); + const mockOnCancel = jest.fn(); + + beforeEach(() => { + render( + + ); + }); + + it('renders with initial value', () => { + expect(screen.getByRole('textbox', { name: /inputValue/i })).toHaveValue( + 'test' + ); + }); + + it('calls onSave with new value', async () => { + const user = userEvent.setup(); + await user.type( + screen.getByRole('textbox', { name: /inputValue/i }), + '123' + ); + await user.click(screen.getByRole('button', { name: /confirmAction/i })); + expect(mockOnSave).toHaveBeenCalledWith('test123'); + }); + + it('calls onCancel', async () => { + const user = userEvent.setup(); + await user.click(screen.getByRole('button', { name: /cancelAction/i })); + expect(mockOnCancel).toHaveBeenCalled(); + }); +}); diff --git a/frontend/src/components/Brokers/Broker/Configs/TableComponents/InputCell/__test__/InputCellViewMode.spec.tsx b/frontend/src/components/Brokers/Broker/Configs/TableComponents/InputCell/__test__/InputCellViewMode.spec.tsx new file mode 100644 index 000000000..9499fa857 --- /dev/null +++ b/frontend/src/components/Brokers/Broker/Configs/TableComponents/InputCell/__test__/InputCellViewMode.spec.tsx @@ -0,0 +1,69 @@ +import React from 'react'; +import { screen } from '@testing-library/react'; +import userEvent from '@testing-library/user-event'; +import { render } from 'lib/testHelpers'; +import InputCellViewMode from 'components/Brokers/Broker/Configs/TableComponents/InputCell/InputCellViewMode'; + +describe('InputCellViewMode', () => { + const mockOnEdit = jest.fn(); + const value = 'testValue'; + + it('displays the correct value for non-sensitive data', () => { + render( + + ); + expect(screen.getByTitle(value)).toBeInTheDocument(); + }); + + it('masks sensitive data with asterisks', () => { + render( + + ); + expect(screen.getByTitle('Sensitive Value')).toBeInTheDocument(); + expect(screen.getByText('**********')).toBeInTheDocument(); + }); + + it('renders edit button and triggers onEdit callback when clicked', async () => { + const user = userEvent.setup(); + render( + + ); + await user.click(screen.getByLabelText('editAction')); + expect(mockOnEdit).toHaveBeenCalled(); + }); + + it('disables edit button for read-only properties', () => { + render( + + ); + expect(screen.getByLabelText('editAction')).toBeDisabled(); + }); +}); diff --git a/frontend/src/components/Brokers/Broker/Configs/TableComponents/InputCell/index.tsx b/frontend/src/components/Brokers/Broker/Configs/TableComponents/InputCell/index.tsx new file mode 100644 index 000000000..a55d7852c --- /dev/null +++ b/frontend/src/components/Brokers/Broker/Configs/TableComponents/InputCell/index.tsx @@ -0,0 +1,60 @@ +import React, { type FC, useState } from 'react'; +import { useConfirm } from 'lib/hooks/useConfirm'; +import { type CellContext } from '@tanstack/react-table'; +import { type BrokerConfig } from 'generated-sources'; +import { + BrokerConfigsTableRow, + UpdateBrokerConfigCallback, +} from 'components/Brokers/Broker/Configs/lib/types'; +import { getConfigUnit } from 'components/Brokers/Broker/Configs/lib/utils'; + +import InputCellViewMode from './InputCellViewMode'; +import InputCellEditMode from './InputCellEditMode'; + +export interface InputCellProps + extends CellContext { + onUpdate: UpdateBrokerConfigCallback; +} + +const InputCell: FC = ({ row, onUpdate }) => { + const [isEdit, setIsEdit] = useState(false); + const confirm = useConfirm(); + const { + name, + source, + value: initialValue, + isSensitive, + isReadOnly, + } = row.original; + + const handleSave = (newValue: string) => { + if (newValue !== initialValue) { + confirm('Are you sure you want to change the value?', () => + onUpdate(name, newValue) + ); + } + setIsEdit(false); + }; + + const isDynamic = source === 'DYNAMIC_BROKER_CONFIG'; + const configUnit = getConfigUnit(name); + + return isEdit ? ( + setIsEdit(false)} + /> + ) : ( + setIsEdit(true)} + isDynamic={isDynamic} + isSensitive={isSensitive} + isReadOnly={isReadOnly} + /> + ); +}; + +export default InputCell; diff --git a/frontend/src/components/Brokers/Broker/Configs/TableComponents/InputCell/styled.ts b/frontend/src/components/Brokers/Broker/Configs/TableComponents/InputCell/styled.ts new file mode 100644 index 000000000..859b2d5d1 --- /dev/null +++ b/frontend/src/components/Brokers/Broker/Configs/TableComponents/InputCell/styled.ts @@ -0,0 +1,24 @@ +import styled from 'styled-components'; + +export const ValueWrapper = styled.div<{ $isDynamic?: boolean }>` + display: flex; + justify-content: space-between; + font-weight: ${({ $isDynamic }) => ($isDynamic ? 600 : 400)}; + + button { + margin: 0 10px; + } +`; + +export const Value = styled.span` + line-height: 24px; + margin-right: 10px; + text-overflow: ellipsis; + max-width: 400px; + overflow: hidden; + white-space: nowrap; +`; + +export const ButtonsWrapper = styled.div` + display: flex; +`; diff --git a/frontend/src/components/Brokers/Broker/Configs/TableComponents/index.ts b/frontend/src/components/Brokers/Broker/Configs/TableComponents/index.ts new file mode 100644 index 000000000..308200222 --- /dev/null +++ b/frontend/src/components/Brokers/Broker/Configs/TableComponents/index.ts @@ -0,0 +1,2 @@ +export { default as InputCell } from './InputCell'; +export { default as ConfigSourceHeader } from './ConfigSourceHeader/ConfigSourceHeader'; diff --git a/frontend/src/components/Brokers/Broker/Configs/lib/__test__/utils.spec.tsx b/frontend/src/components/Brokers/Broker/Configs/lib/__test__/utils.spec.tsx new file mode 100644 index 000000000..c809e6ebb --- /dev/null +++ b/frontend/src/components/Brokers/Broker/Configs/lib/__test__/utils.spec.tsx @@ -0,0 +1,88 @@ +import { + getConfigDisplayValue, + getConfigTableData, + getConfigUnit, +} from 'components/Brokers/Broker/Configs/lib/utils'; +import { ConfigSource } from 'generated-sources'; +import { render } from 'lib/testHelpers'; +import { ReactElement } from 'react'; + +describe('getConfigTableData', () => { + it('filters configs by search query and sorts by source priority', () => { + const configs = [ + { + name: 'log.retention.ms', + value: '7200000', + source: ConfigSource.DEFAULT_CONFIG, + isSensitive: true, + isReadOnly: false, + }, + { + name: 'log.segment.bytes', + value: '1073741824', + source: ConfigSource.DYNAMIC_BROKER_CONFIG, + isSensitive: false, + isReadOnly: true, + }, + { + name: 'compression.type', + value: 'producer', + source: ConfigSource.DEFAULT_CONFIG, + isSensitive: true, + isReadOnly: false, + }, + ]; + const searchQuery = 'log'; + const result = getConfigTableData(configs, searchQuery); + + expect(result).toHaveLength(2); + expect(result[0].name).toBe('log.segment.bytes'); + expect(result[1].name).toBe('log.retention.ms'); + }); +}); + +describe('getConfigUnit', () => { + it('identifies the unit of a configuration name', () => { + expect(getConfigUnit('log.retention.ms')).toBe('ms'); + expect(getConfigUnit('log.segment.bytes')).toBe('bytes'); + expect(getConfigUnit('compression.type')).toBeUndefined(); + }); +}); + +describe('getConfigDisplayValue', () => { + it('masks sensitive data with asterisks', () => { + const result = getConfigDisplayValue(true, 'testValue', undefined); + expect(result).toEqual({ + displayValue: '**********', + title: 'Sensitive Value', + }); + }); + + it('returns formatted bytes when unit is "bytes" and value is positive', () => { + const { container } = render( + getConfigDisplayValue(false, '1024', 'bytes').displayValue as ReactElement + ); + expect(container).toHaveTextContent('1 KB'); + expect(getConfigDisplayValue(false, '1024', 'bytes').title).toBe( + 'Bytes: 1024' + ); + }); + + it('returns value as is when unit is "bytes" but value is non-positive', () => { + const result = getConfigDisplayValue(false, '-1', 'bytes'); + expect(result.displayValue).toBe('-1'); + expect(result.title).toBe('-1'); + }); + + it('appends unit to the value when unit is provided and is not "bytes"', () => { + const result = getConfigDisplayValue(false, '100', 'ms'); + expect(result.displayValue).toBe('100 ms'); + expect(result.title).toBe('100 ms'); + }); + + it('returns value as is when no unit is provided', () => { + const result = getConfigDisplayValue(false, 'testValue', undefined); + expect(result.displayValue).toBe('testValue'); + expect(result.title).toBe('testValue'); + }); +}); diff --git a/frontend/src/components/Brokers/Broker/Configs/lib/constants.ts b/frontend/src/components/Brokers/Broker/Configs/lib/constants.ts new file mode 100644 index 000000000..5a829323a --- /dev/null +++ b/frontend/src/components/Brokers/Broker/Configs/lib/constants.ts @@ -0,0 +1,22 @@ +import { ConfigSource } from 'generated-sources'; + +export const CONFIG_SOURCE_NAME_MAP: Record = { + [ConfigSource.DYNAMIC_TOPIC_CONFIG]: 'Dynamic topic config', + [ConfigSource.DYNAMIC_BROKER_LOGGER_CONFIG]: 'Dynamic broker logger config', + [ConfigSource.DYNAMIC_BROKER_CONFIG]: 'Dynamic broker config', + [ConfigSource.DYNAMIC_DEFAULT_BROKER_CONFIG]: 'Dynamic default broker config', + [ConfigSource.STATIC_BROKER_CONFIG]: 'Static broker config', + [ConfigSource.DEFAULT_CONFIG]: 'Default config', + [ConfigSource.UNKNOWN]: 'Unknown', +} as const; + +export const CONFIG_SOURCE_PRIORITY = { + [ConfigSource.DYNAMIC_TOPIC_CONFIG]: 1, + [ConfigSource.DYNAMIC_BROKER_LOGGER_CONFIG]: 1, + [ConfigSource.DYNAMIC_BROKER_CONFIG]: 1, + [ConfigSource.DYNAMIC_DEFAULT_BROKER_CONFIG]: 1, + [ConfigSource.STATIC_BROKER_CONFIG]: 2, + [ConfigSource.DEFAULT_CONFIG]: 3, + [ConfigSource.UNKNOWN]: 4, + UNHANDLED: 5, +} as const; diff --git a/frontend/src/components/Brokers/Broker/Configs/lib/types.ts b/frontend/src/components/Brokers/Broker/Configs/lib/types.ts new file mode 100644 index 000000000..1a726cad3 --- /dev/null +++ b/frontend/src/components/Brokers/Broker/Configs/lib/types.ts @@ -0,0 +1,13 @@ +import { type BrokerConfig } from 'generated-sources'; + +export type BrokerConfigsTableRow = Pick< + BrokerConfig, + 'name' | 'value' | 'source' | 'isReadOnly' | 'isSensitive' +>; + +export type UpdateBrokerConfigCallback = ( + name: BrokerConfig['name'], + value: BrokerConfig['value'] +) => Promise; + +export type ConfigUnit = 'ms' | 'bytes'; diff --git a/frontend/src/components/Brokers/Broker/Configs/lib/utils.tsx b/frontend/src/components/Brokers/Broker/Configs/lib/utils.tsx new file mode 100644 index 000000000..d452a1ac8 --- /dev/null +++ b/frontend/src/components/Brokers/Broker/Configs/lib/utils.tsx @@ -0,0 +1,101 @@ +import React from 'react'; +import { type BrokerConfig, ConfigSource } from 'generated-sources'; +import { createColumnHelper } from '@tanstack/react-table'; +import * as BrokerConfigTableComponents from 'components/Brokers/Broker/Configs/TableComponents/index'; +import BytesFormatted from 'components/common/BytesFormatted/BytesFormatted'; + +import type { + BrokerConfigsTableRow, + ConfigUnit, + UpdateBrokerConfigCallback, +} from './types'; +import { CONFIG_SOURCE_NAME_MAP, CONFIG_SOURCE_PRIORITY } from './constants'; + +const getConfigFieldMatch = (field: string, query: string) => + field.toLocaleLowerCase().includes(query.toLocaleLowerCase()); + +const filterConfigsBySearchQuery = + (searchQuery: string) => (config: BrokerConfig) => { + const nameMatch = getConfigFieldMatch(config.name, searchQuery); + const valueMatch = + config.value && getConfigFieldMatch(config.value, searchQuery); + + return nameMatch ? true : valueMatch; + }; + +const getConfigSourcePriority = (source: ConfigSource): number => + CONFIG_SOURCE_PRIORITY[source]; + +const sortBrokersBySource = (a: BrokerConfig, b: BrokerConfig) => { + const priorityA = getConfigSourcePriority(a.source); + const priorityB = getConfigSourcePriority(b.source); + + return priorityA - priorityB; +}; + +export const getConfigTableData = ( + configs: BrokerConfig[], + searchQuery: string +) => + configs + .filter(filterConfigsBySearchQuery(searchQuery)) + .sort(sortBrokersBySource); + +export const getBrokerConfigsTableColumns = ( + onUpdateInputCell: UpdateBrokerConfigCallback +) => { + const columnHelper = createColumnHelper(); + + return [ + columnHelper.accessor('name', { header: 'Key' }), + columnHelper.accessor('value', { + header: 'Value', + cell: (props) => ( + + ), + }), + columnHelper.accessor('source', { + header: BrokerConfigTableComponents.ConfigSourceHeader, + cell: ({ getValue }) => CONFIG_SOURCE_NAME_MAP[getValue()], + }), + ]; +}; + +const unitPatterns = { + ms: /\.ms$/, + bytes: /\.bytes$/, +}; + +export const getConfigUnit = (configName: string): ConfigUnit | undefined => { + const found = Object.entries(unitPatterns).find(([, pattern]) => + pattern.test(configName) + ); + + return found ? (found[0] as ConfigUnit) : undefined; +}; + +export const getConfigDisplayValue = ( + isSensitive: boolean, + value: string, + unit: ConfigUnit | undefined +) => { + if (isSensitive) { + return { displayValue: '**********', title: 'Sensitive Value' }; + } + + if (unit === 'bytes') { + const intValue = parseInt(value, 10); + return { + displayValue: intValue > 0 ? : value, + title: intValue > 0 ? `Bytes: ${value}` : value.toString(), + }; + } + + return { + displayValue: unit ? `${value} ${unit}` : value, + title: unit ? `${value} ${unit}` : value, + }; +}; diff --git a/frontend/src/components/Connect/List/ListPage.tsx b/frontend/src/components/Connect/List/ListPage.tsx index 94ec8354c..5d61ff3c4 100644 --- a/frontend/src/components/Connect/List/ListPage.tsx +++ b/frontend/src/components/Connect/List/ListPage.tsx @@ -1,21 +1,23 @@ import React, { Suspense } from 'react'; import useAppParams from 'lib/hooks/useAppParams'; -import { clusterConnectorNewRelativePath, ClusterNameRoute } from 'lib/paths'; +import { ClusterNameRoute, clusterConnectorNewRelativePath } from 'lib/paths'; import ClusterContext from 'components/contexts/ClusterContext'; import Search from 'components/common/Search/Search'; import * as Metrics from 'components/common/Metrics'; import PageHeading from 'components/common/PageHeading/PageHeading'; -import { ActionButton } from 'components/common/ActionComponent'; +import Tooltip from 'components/common/Tooltip/Tooltip'; import { ControlPanelWrapper } from 'components/common/ControlPanel/ControlPanel.styled'; import PageLoader from 'components/common/PageLoader/PageLoader'; -import { Action, ConnectorState, ResourceType } from 'generated-sources'; -import { useConnectors } from 'lib/hooks/api/kafkaConnect'; +import { ConnectorState, Action, ResourceType } from 'generated-sources'; +import { useConnectors, useConnects } from 'lib/hooks/api/kafkaConnect'; +import { ActionButton } from 'components/common/ActionComponent'; import List from './List'; const ListPage: React.FC = () => { const { isReadOnly } = React.useContext(ClusterContext); const { clusterName } = useAppParams(); + const { data: connects = [] } = useConnects(clusterName); // Fetches all connectors from the API, without search criteria. Used to display general metrics. const { data: connectorsMetrics, isLoading } = useConnectors(clusterName); @@ -33,17 +35,25 @@ const ListPage: React.FC = () => { <> {!isReadOnly && ( - - Create Connector - + + Create Connector + + } + showTooltip={!connects.length} + content="No Connects available" + placement="left" + /> )} diff --git a/frontend/src/components/Connect/List/__tests__/ListPage.spec.tsx b/frontend/src/components/Connect/List/__tests__/ListPage.spec.tsx index 7da8e47e8..0cbe4dd78 100644 --- a/frontend/src/components/Connect/List/__tests__/ListPage.spec.tsx +++ b/frontend/src/components/Connect/List/__tests__/ListPage.spec.tsx @@ -1,5 +1,5 @@ import React from 'react'; -import { connectors } from 'lib/fixtures/kafkaConnect'; +import { connectors, connects } from 'lib/fixtures/kafkaConnect'; import ClusterContext, { ContextProps, initialValue, @@ -8,7 +8,7 @@ import ListPage from 'components/Connect/List/ListPage'; import { screen, within } from '@testing-library/react'; import { render, WithRoute } from 'lib/testHelpers'; import { clusterConnectorsPath } from 'lib/paths'; -import { useConnectors } from 'lib/hooks/api/kafkaConnect'; +import { useConnectors, useConnects } from 'lib/hooks/api/kafkaConnect'; jest.mock('components/Connect/List/List', () => () => (
Connectors List
@@ -16,6 +16,7 @@ jest.mock('components/Connect/List/List', () => () => ( jest.mock('lib/hooks/api/kafkaConnect', () => ({ useConnectors: jest.fn(), + useConnects: jest.fn(), })); jest.mock('components/common/Icons/SpinnerIcon', () => () => 'progressbar'); @@ -28,6 +29,10 @@ describe('Connectors List Page', () => { isLoading: false, data: [], })); + + (useConnects as jest.Mock).mockImplementation(() => ({ + data: connects, + })); }); const renderComponent = async (contextValue: ContextProps = initialValue) => @@ -178,4 +183,22 @@ describe('Connectors List Page', () => { expect(failedTasksIndicator).toHaveTextContent('Failed Tasks 1'); }); }); + + describe('Create new connector', () => { + it('Create new connector button is enabled when connects list is not empty', async () => { + await renderComponent(); + + expect(screen.getByText('Create Connector')).toBeEnabled(); + }); + + it('Create new connector button is disabled when connects list is empty', async () => { + (useConnects as jest.Mock).mockImplementation(() => ({ + data: [], + })); + + await renderComponent(); + + expect(screen.getByText('Create Connector')).toBeDisabled(); + }); + }); }); diff --git a/frontend/src/components/KsqlDb/Query/QueryForm/QueryForm.tsx b/frontend/src/components/KsqlDb/Query/QueryForm/QueryForm.tsx index 7456b5d2d..900b11e1e 100644 --- a/frontend/src/components/KsqlDb/Query/QueryForm/QueryForm.tsx +++ b/frontend/src/components/KsqlDb/Query/QueryForm/QueryForm.tsx @@ -2,10 +2,10 @@ import React from 'react'; import { FormError } from 'components/common/Input/Input.styled'; import { ErrorMessage } from '@hookform/error-message'; import { - useForm, Controller, - useFieldArray, FormProvider, + useFieldArray, + useForm, } from 'react-hook-form'; import { Button } from 'components/common/Button/Button'; import IconButtonWrapper from 'components/common/Icons/IconButtonWrapper'; @@ -24,6 +24,7 @@ interface QueryFormProps { resetResults: () => void; submitHandler: (values: FormValues) => void; } + type StreamsPropertiesType = { key: string; value: string; @@ -118,7 +119,6 @@ const QueryForm: React.FC = ({ onClick={() => setValue('ksql', '')} buttonType="primary" buttonSize="S" - isInverted > Clear diff --git a/frontend/src/components/NavBar/NavBar.styled.ts b/frontend/src/components/NavBar/NavBar.styled.ts index d9013dcdd..6f47f7223 100644 --- a/frontend/src/components/NavBar/NavBar.styled.ts +++ b/frontend/src/components/NavBar/NavBar.styled.ts @@ -1,7 +1,8 @@ import styled, { css } from 'styled-components'; import { Link } from 'react-router-dom'; import DiscordIcon from 'components/common/Icons/DiscordIcon'; -import GitIcon from 'components/common/Icons/GitIcon'; +import GitHubIcon from 'components/common/Icons/GitHubIcon'; +import ProductHuntIcon from 'components/common/Icons/ProductHuntIcon'; export const Navbar = styled.nav( ({ theme }) => css` @@ -39,8 +40,12 @@ export const SocialLink = styled.a( fill: ${icons.discord.hover}; } - ${GitIcon} { - fill: ${icons.git.hover}; + ${GitHubIcon} { + fill: ${icons.github.hover}; + } + + ${ProductHuntIcon} { + fill: ${icons.producthunt.hover}; } } @@ -49,8 +54,12 @@ export const SocialLink = styled.a( fill: ${icons.discord.active}; } - ${GitIcon} { - fill: ${icons.git.active}; + ${GitHubIcon} { + fill: ${icons.github.active}; + } + + ${ProductHuntIcon} { + fill: ${icons.producthunt.active}; } } ` diff --git a/frontend/src/components/NavBar/NavBar.tsx b/frontend/src/components/NavBar/NavBar.tsx index 1a58c7e7b..a55be1dfa 100644 --- a/frontend/src/components/NavBar/NavBar.tsx +++ b/frontend/src/components/NavBar/NavBar.tsx @@ -2,12 +2,13 @@ import React, { useContext } from 'react'; import Select from 'components/common/Select/Select'; import Logo from 'components/common/Logo/Logo'; import Version from 'components/Version/Version'; -import GitIcon from 'components/common/Icons/GitIcon'; +import GitHubIcon from 'components/common/Icons/GitHubIcon'; import DiscordIcon from 'components/common/Icons/DiscordIcon'; import AutoIcon from 'components/common/Icons/AutoIcon'; import SunIcon from 'components/common/Icons/SunIcon'; import MoonIcon from 'components/common/Icons/MoonIcon'; import { ThemeModeContext } from 'components/contexts/ThemeModeContext'; +import ProductHuntIcon from 'components/common/Icons/ProductHuntIcon'; import UserInfo from './UserInfo/UserInfo'; import * as S from './NavBar.styled'; @@ -85,7 +86,7 @@ const NavBar: React.FC = ({ onBurgerClick }) => { isThemeMode /> - + = ({ onBurgerClick }) => { > + + + diff --git a/frontend/src/components/NavBar/UserInfo/UserInfo.styled.ts b/frontend/src/components/NavBar/UserInfo/UserInfo.styled.ts index 85154d767..2f99fb293 100644 --- a/frontend/src/components/NavBar/UserInfo/UserInfo.styled.ts +++ b/frontend/src/components/NavBar/UserInfo/UserInfo.styled.ts @@ -1,19 +1,23 @@ -import styled, { css } from 'styled-components'; +import styled from 'styled-components'; export const Wrapper = styled.div` display: flex; justify-content: center; align-items: center; gap: 5px; + svg { position: relative; } `; -export const Text = styled.div( - ({ theme }) => css` - color: ${theme.button.primary.invertedColors.normal}; - ` -); +export const Text = styled.div` + color: ${({ theme }) => theme.user.color}; + + &:hover { + color: ${({ theme }) => theme.user.hoverColor}; + } +} +`; export const LogoutLink = styled.a``; diff --git a/frontend/src/components/Schemas/New/New.tsx b/frontend/src/components/Schemas/New/New.tsx index 3e5d39c9d..81c987ccd 100644 --- a/frontend/src/components/Schemas/New/New.tsx +++ b/frontend/src/components/Schemas/New/New.tsx @@ -13,7 +13,7 @@ import { useNavigate } from 'react-router-dom'; import { InputLabel } from 'components/common/Input/InputLabel.styled'; import Input from 'components/common/Input/Input'; import { FormError } from 'components/common/Input/Input.styled'; -import Select, { SelectOption } from 'components/common/Select/Select'; +import Select from 'components/common/Select/Select'; import { Button } from 'components/common/Button/Button'; import { Textarea } from 'components/common/Textbox/Textarea.styled'; import PageHeading from 'components/common/PageHeading/PageHeading'; @@ -27,7 +27,7 @@ import { yupResolver } from '@hookform/resolvers/yup'; import * as S from './New.styled'; -const SchemaTypeOptions: Array = [ +const SchemaTypeOptions = [ { value: SchemaType.AVRO, label: 'AVRO' }, { value: SchemaType.JSON, label: 'JSON' }, { value: SchemaType.PROTOBUF, label: 'PROTOBUF' }, @@ -131,7 +131,7 @@ const New: React.FC = () => { ( Save this filter @@ -100,32 +187,34 @@ const AddEditFilterContainer: React.FC = ({ inputSize="M" placeholder="Enter Name" autoComplete="off" - name="name" - defaultValue={inputDisplayNameDefaultValue} + name="id" />
- +
- - - + + {!isEdit && } + + + + diff --git a/frontend/src/components/Topics/Topic/Messages/Filters/AddFilter.tsx b/frontend/src/components/Topics/Topic/Messages/Filters/AddFilter.tsx deleted file mode 100644 index 035d98c3a..000000000 --- a/frontend/src/components/Topics/Topic/Messages/Filters/AddFilter.tsx +++ /dev/null @@ -1,122 +0,0 @@ -import React from 'react'; -import * as S from 'components/Topics/Topic/Messages/Filters/Filters.styled'; -import { MessageFilters } from 'components/Topics/Topic/Messages/Filters/Filters'; -import { FilterEdit } from 'components/Topics/Topic/Messages/Filters/FilterModal'; -import SavedFilters from 'components/Topics/Topic/Messages/Filters/SavedFilters'; -import SavedIcon from 'components/common/Icons/SavedIcon'; -import QuestionIcon from 'components/common/Icons/QuestionIcon'; -import useBoolean from 'lib/hooks/useBoolean'; -import { showAlert } from 'lib/errorHandling'; - -import AddEditFilterContainer from './AddEditFilterContainer'; -import InfoModal from './InfoModal'; - -export interface FilterModalProps { - toggleIsOpen(): void; - filters: MessageFilters[]; - addFilter(values: MessageFilters): void; - deleteFilter(index: number): void; - activeFilterHandler(activeFilter: MessageFilters, index: number): void; - toggleEditModal(): void; - editFilter(value: FilterEdit): void; - isSavedFiltersOpen: boolean; - onClickSavedFilters(newValue: boolean): void; - activeFilter?: MessageFilters; -} - -export interface AddMessageFilters extends MessageFilters { - saveFilter: boolean; -} - -const AddFilter: React.FC = ({ - toggleIsOpen, - filters, - addFilter, - deleteFilter, - activeFilterHandler, - toggleEditModal, - editFilter, - isSavedFiltersOpen, - onClickSavedFilters, - activeFilter, -}) => { - const { value: isOpen, toggle } = useBoolean(); - - const onSubmit = React.useCallback( - async (values: AddMessageFilters) => { - const isFilterExists = filters.some( - (filter) => filter.name === values.name - ); - - if (isFilterExists) { - showAlert('error', { - id: '', - title: 'Validation Error', - message: 'Filter with the same name already exists', - }); - return; - } - - const data = { ...values }; - if (data.saveFilter) { - addFilter(data); - } else { - // other case is not applying the filter - const dataCodeLabel = - data.code.length > 16 ? `${data.code.slice(0, 16)}...` : data.code; - data.name = data.name || dataCodeLabel; - - activeFilterHandler(data, -1); - toggleIsOpen(); - } - }, - [activeFilterHandler, addFilter, toggleIsOpen] - ); - return ( - <> - - Add filter -
- - - - {isOpen && } -
-
- {isSavedFiltersOpen ? ( - onClickSavedFilters(!onClickSavedFilters)} - filters={filters} - onEdit={(index: number, filter: MessageFilters) => { - toggleEditModal(); - editFilter({ index, filter }); - }} - activeFilter={activeFilter} - /> - ) : ( - <> - onClickSavedFilters(!isSavedFiltersOpen)} - > - Saved Filters - - - - )} - - ); -}; - -export default AddFilter; diff --git a/frontend/src/components/Topics/Topic/Messages/Filters/EditFilter.tsx b/frontend/src/components/Topics/Topic/Messages/Filters/EditFilter.tsx deleted file mode 100644 index 04f9c47ee..000000000 --- a/frontend/src/components/Topics/Topic/Messages/Filters/EditFilter.tsx +++ /dev/null @@ -1,37 +0,0 @@ -import React from 'react'; -import { MessageFilters } from 'components/Topics/Topic/Messages/Filters/Filters'; -import { FilterEdit } from 'components/Topics/Topic/Messages/Filters/FilterModal'; - -import AddEditFilterContainer from './AddEditFilterContainer'; -import * as S from './Filters.styled'; - -export interface EditFilterProps { - editFilter: FilterEdit; - toggleEditModal(): void; - editSavedFilter(filter: FilterEdit): void; -} - -const EditFilter: React.FC = ({ - editFilter, - toggleEditModal, - editSavedFilter, -}) => { - const onSubmit = (values: MessageFilters) => { - editSavedFilter({ index: editFilter.index, filter: values }); - toggleEditModal(); - }; - return ( - <> - Edit filter - toggleEditModal()} - submitBtnText="Save" - inputDisplayNameDefaultValue={editFilter.filter.name} - inputCodeDefaultValue={editFilter.filter.code} - submitCallback={onSubmit} - /> - - ); -}; - -export default EditFilter; diff --git a/frontend/src/components/Topics/Topic/Messages/Filters/FilterModal.tsx b/frontend/src/components/Topics/Topic/Messages/Filters/FilterModal.tsx deleted file mode 100644 index 0ada15878..000000000 --- a/frontend/src/components/Topics/Topic/Messages/Filters/FilterModal.tsx +++ /dev/null @@ -1,84 +0,0 @@ -import React from 'react'; -import * as S from 'components/Topics/Topic/Messages/Filters/Filters.styled'; -import { - ActiveMessageFilter, - MessageFilters, -} from 'components/Topics/Topic/Messages/Filters/Filters'; -import AddFilter from 'components/Topics/Topic/Messages/Filters/AddFilter'; -import EditFilter from 'components/Topics/Topic/Messages/Filters/EditFilter'; - -export interface FilterModalProps { - toggleIsOpen(): void; - filters: MessageFilters[]; - addFilter(values: MessageFilters): void; - deleteFilter(index: number): void; - activeFilterHandler(activeFilter: MessageFilters, index: number): void; - editSavedFilter(filter: FilterEdit): void; - activeFilter: ActiveMessageFilter; - quickEditMode?: boolean; -} - -export interface FilterEdit { - index: number; - filter: MessageFilters; -} - -const FilterModal: React.FC = ({ - toggleIsOpen, - filters, - addFilter, - deleteFilter, - activeFilterHandler, - editSavedFilter, - activeFilter, - quickEditMode = false, -}) => { - const [isInEditMode, setIsInEditMode] = - React.useState(quickEditMode); - const [isSavedFiltersOpen, setIsSavedFiltersOpen] = - React.useState(false); - - const toggleEditModal = () => { - setIsInEditMode(!isInEditMode); - }; - - const [editFilter, setEditFilter] = React.useState(() => { - const { index, name, code } = activeFilter; - return quickEditMode - ? { index, filter: { name, code } } - : { index: -1, filter: { name: '', code: '' } }; - }); - const editFilterHandler = (value: FilterEdit) => { - setEditFilter(value); - setIsInEditMode(!isInEditMode); - }; - - const toggleEditModalHandler = quickEditMode ? toggleIsOpen : toggleEditModal; - - return ( - - {isInEditMode ? ( - - ) : ( - setIsSavedFiltersOpen(!isSavedFiltersOpen)} - activeFilter={activeFilter} - /> - )} - - ); -}; - -export default FilterModal; diff --git a/frontend/src/components/Topics/Topic/Messages/Filters/Filters.styled.ts b/frontend/src/components/Topics/Topic/Messages/Filters/Filters.styled.ts index 7ec8fbde0..3e609fb03 100644 --- a/frontend/src/components/Topics/Topic/Messages/Filters/Filters.styled.ts +++ b/frontend/src/components/Topics/Topic/Messages/Filters/Filters.styled.ts @@ -4,6 +4,9 @@ import styled, { css } from 'styled-components'; import DatePicker from 'react-datepicker'; import EditIcon from 'components/common/Icons/EditIcon'; import closeIcon from 'components/common/Icons/CloseIcon'; +import { PollingMode } from 'generated-sources'; + +import { isModeOptionWithInput } from './utils'; interface SavedFilterProps { selected: boolean; @@ -16,32 +19,9 @@ interface MessageLoadingSpinnerProps { isFetching: boolean; } -export const FiltersWrapper = styled.div` - display: flex; - flex-direction: column; - padding-left: 16px; - padding-right: 16px; - - & > div:first-child { - display: flex; - justify-content: space-between; - padding-top: 2px; - align-items: flex-end; - } -`; - -export const FilterInputs = styled.div` - display: flex; - gap: 8px; - align-items: flex-end; - width: 90%; - flex-wrap: wrap; -`; - -export const SeekTypeSelectorWrapper = styled.div` +export const FilterModeTypeSelectorWrapper = styled.div` display: flex; & .select-wrapper { - width: 40% !important; & > select { border-radius: 4px 0 0 4px !important; } @@ -83,14 +63,6 @@ export const DatePickerInput = styled(DatePicker)` } `; -export const FiltersMetrics = styled.div` - display: flex; - justify-content: flex-end; - align-items: center; - gap: 22px; - padding-top: 16px; - padding-bottom: 16px; -`; export const Message = styled.div` font-size: 14px; color: ${({ theme }) => theme.metrics.filters.color.normal}; @@ -106,22 +78,6 @@ export const MetricsIcon = styled.div` padding-right: 6px; height: 12px; `; - -export const ClearAll = styled.div` - color: ${({ theme }) => theme.metrics.filters.color.normal}; - font-size: 12px; - cursor: pointer; - line-height: 32px; - margin-left: 8px; -`; - -export const ButtonContainer = styled.div` - width: 100%; - display: flex; - justify-content: center; - margin-top: 20px; -`; - export const ListItem = styled.li` font-size: 12px; font-weight: 400; @@ -138,19 +94,6 @@ export const InfoParagraph = styled.div` color: ${({ theme }) => theme.table.td.color.normal}; `; -export const MessageFilterModal = styled.div` - height: auto; - width: 560px; - border-radius: 8px; - background: ${({ theme }) => theme.modal.backgroundColor}; - position: absolute; - left: 25%; - border: 1px solid ${({ theme }) => theme.modal.border.contrast}; - box-shadow: ${({ theme }) => theme.modal.shadow}; - padding: 16px; - z-index: 1; -`; - export const InfoModal = styled.div` height: auto; width: 560px; @@ -171,42 +114,16 @@ export const QuestionIconContainer = styled.button` border: none; `; -export const FilterTitle = styled.h3` - line-height: 32px; - font-size: 20px; - margin-bottom: 40px; - position: relative; - display: flex; - align-items: center; - justify-content: space-between; - color: ${({ theme }) => theme.modal.color}; - &:after { - content: ''; - width: calc(100% + 32px); - height: 1px; - position: absolute; - top: 40px; - left: -16px; - display: inline-block; - background-color: ${({ theme }) => theme.modal.border.top}; - } -`; - -export const CreatedFilter = styled.p` - margin: 25px 0 10px; - font-size: 14px; - line-height: 20px; - color: ${({ theme }) => theme.savedFilter.color}; -`; - export const NoSavedFilter = styled.p` - color: ${({ theme }) => theme.savedFilter.color}; + color: ${({ theme }) => theme.default.color.normal}; + font-size: 16px; + margin-top: 10px; `; export const SavedFiltersContainer = styled.div` overflow-y: auto; height: 195px; - justify-content: space-around; - padding-left: 10px; + display: flex; + flex-direction: column; `; export const SavedFilterName = styled.div` @@ -215,9 +132,9 @@ export const SavedFilterName = styled.div` color: ${({ theme }) => theme.savedFilter.filterName}; `; -export const FilterButtonWrapper = styled.div` +export const FilterButtonWrapper = styled.div<{ isEdit: boolean }>` display: flex; - justify-content: flex-end; + justify-content: ${(props) => (props.isEdit ? 'flex-end' : 'space-between')}; margin-top: 10px; gap: 10px; padding-top: 16px; @@ -234,24 +151,20 @@ export const FilterButtonWrapper = styled.div` } `; -export const ActiveSmartFilterWrapper = styled.div` - padding: 8px 0 5px; - display: flex; - gap: 10px; - align-items: center; - justify-content: flex-start; -`; - -export const DeleteSavedFilter = styled.div.attrs({ role: 'deleteIcon' })` - margin-top: 2px; +export const DeleteSavedFilter = styled.button` cursor: pointer; color: ${({ theme }) => theme.icons.deleteIcon}; + background-color: transparent; + border: none; `; -export const FilterEdit = styled.div` +export const FilterEdit = styled.button` font-weight: 500; font-size: 14px; line-height: 20px; + background-color: transparent; + border: none; + cursor: pointer; `; export const FilterOptions = styled.div` @@ -266,19 +179,20 @@ export const SavedFilter = styled.div.attrs({ })` display: flex; justify-content: space-between; - padding-right: 5px; + padding: 5px; height: 32px; + border-radius: 4px; align-items: center; cursor: pointer; - border-top: 1px solid ${({ theme }) => theme.panelColor.borderTop}; &:hover ${FilterOptions} { display: flex; } &:hover { - background: ${({ theme }) => theme.layout.stuffColor}; + background-color: ${({ theme }) => theme.layout.stuffColor}; } - background: ${({ selected, theme }) => - selected ? theme.layout.stuffColor : theme.modal.backgroundColor}; + + background-color: ${({ selected, theme }) => + selected ? theme.layout.stuffColor : 'transparent'}; `; export const ActiveSmartFilter = styled.div` @@ -293,7 +207,7 @@ export const ActiveSmartFilter = styled.div` line-height: 20px; `; -export const EditSmartFilterIcon = styled.div( +export const EditSmartFilterIcon = styled.button( ({ theme: { icons } }) => css` color: ${icons.editIcon.normal}; display: flex; @@ -302,19 +216,26 @@ export const EditSmartFilterIcon = styled.div( height: 32px; width: 32px; cursor: pointer; + background-color: transparent; + border: none; border-left: 1px solid ${icons.editIcon.border}; - &:hover { + &:hover:not(:disabled) { ${EditIcon} { fill: ${icons.editIcon.hover}; } } - &:active { + &:active:not(:disabled) { ${EditIcon} { fill: ${icons.editIcon.active}; } } + + &:disabled { + cursor: not-allowed; + opacity: 0.5; + } ` ); @@ -323,7 +244,7 @@ export const SmartFilterName = styled.div` min-width: 32px; `; -export const DeleteSmartFilterIcon = styled.div( +export const DeleteSmartFilterIcon = styled.button( ({ theme: { icons } }) => css` color: ${icons.closeIcon.normal}; display: flex; @@ -332,6 +253,8 @@ export const DeleteSmartFilterIcon = styled.div( height: 32px; width: 32px; cursor: pointer; + background-color: transparent; + border: none; border-left: 1px solid ${icons.closeIcon.border}; svg { @@ -339,17 +262,22 @@ export const DeleteSmartFilterIcon = styled.div( width: 14px; } - &:hover { + &:hover:not(:disabled) { ${closeIcon} { fill: ${icons.closeIcon.hover}; } } - &:active { + &:active:not(:disabled) { ${closeIcon} { fill: ${icons.closeIcon.active}; } } + + &:disabled { + cursor: not-allowed; + opacity: 0.5; + } ` ); @@ -360,13 +288,15 @@ export const MessageLoading = styled.div.attrs({ font-size: ${({ theme }) => theme.heading.h3.fontSize}; display: ${({ isLive }) => (isLive ? 'flex' : 'none')}; justify-content: space-around; - width: 250px; + width: 260px; `; -export const StopLoading = styled.div` - color: ${({ theme }) => theme.pageLoader.borderColor}; +export const StopLoading = styled.button` + color: ${({ theme }) => theme.heading.base.color}; font-size: ${({ theme }) => theme.heading.h3.fontSize}; cursor: pointer; + background-color: transparent; + border: none; `; export const MessageLoadingSpinner = styled.div` @@ -388,39 +318,30 @@ export const MessageLoadingSpinner = styled.div` } `; -export const SavedFiltersTextContainer = styled.div.attrs({ - role: 'savedFilterText', -})` - display: flex; - align-items: center; - cursor: pointer; - margin-bottom: 15px; -`; - -const textStyle = css` - font-size: 14px; - color: ${({ theme }) => theme.editFilter.textColor}; - font-weight: 500; +// styled component lib bug it does not pick up the generic +export const FilterModeTypeSelect = styled(Select)` + border-top-right-radius: ${(props) => + !props.value || !isModeOptionWithInput(props.value) ? '4px' : '0'}; + border-bottom-right-radius: ${(props) => + !props.value || !isModeOptionWithInput(props.value) ? '4px' : '0'}; + user-select: none; `; -export const SavedFiltersText = styled.div` - ${textStyle}; - margin-left: 7px; +export const SavedFilterText = styled.div` + font-weight: 600; + color: ${({ theme }) => theme.default.color.normal}; `; -export const BackToCustomText = styled.div` - ${textStyle}; +export const SavedFilterClearAll = styled.button` + font-weight: 500; + color: ${({ theme }) => theme.link.color}; + background-color: transparent; + border: none; cursor: pointer; -`; + font-size: 16px; -export const SeekTypeSelect = styled(Select)` - border-top-right-radius: 0; - border-bottom-right-radius: 0; - user-select: none; -`; - -export const Serdes = styled.div` - display: flex; - gap: 24px; - padding: 8px 0; + &:disabled { + opacity: 0.5; + cursor: not-allowed; + } `; diff --git a/frontend/src/components/Topics/Topic/Messages/Filters/Filters.tsx b/frontend/src/components/Topics/Topic/Messages/Filters/Filters.tsx index 99eb6bac4..b8fc307ed 100644 --- a/frontend/src/components/Topics/Topic/Messages/Filters/Filters.tsx +++ b/frontend/src/components/Topics/Topic/Messages/Filters/Filters.tsx @@ -1,645 +1,239 @@ import 'react-datepicker/dist/react-datepicker.css'; -import { - MessageFilterType, - Partition, - SeekDirection, - SeekType, - SerdeUsage, - TopicMessage, - TopicMessageConsuming, - TopicMessageEvent, - TopicMessageEventTypeEnum, -} from 'generated-sources'; -import React, { useContext } from 'react'; -import omitBy from 'lodash/omitBy'; -import { useLocation, useNavigate, useSearchParams } from 'react-router-dom'; +import { SerdeUsage, TopicMessageConsuming } from 'generated-sources'; +import React, { useMemo, useState } from 'react'; import MultiSelect from 'components/common/MultiSelect/MultiSelect.styled'; -import { Option } from 'react-multi-select-component'; -import BytesFormatted from 'components/common/BytesFormatted/BytesFormatted'; -import { BASE_PARAMS } from 'lib/constants'; import Select from 'components/common/Select/Select'; import { Button } from 'components/common/Button/Button'; import Search from 'components/common/Search/Search'; -import FilterModal, { - FilterEdit, -} from 'components/Topics/Topic/Messages/Filters/FilterModal'; -import { SeekDirectionOptions } from 'components/Topics/Topic/Messages/Messages'; -import TopicMessagesContext from 'components/contexts/TopicMessagesContext'; -import useBoolean from 'lib/hooks/useBoolean'; -import { RouteParamsClusterTopic } from 'lib/paths'; -import useAppParams from 'lib/hooks/useAppParams'; import PlusIcon from 'components/common/Icons/PlusIcon'; -import EditIcon from 'components/common/Icons/EditIcon'; -import CloseIcon from 'components/common/Icons/CloseIcon'; -import ClockIcon from 'components/common/Icons/ClockIcon'; -import ArrowDownIcon from 'components/common/Icons/ArrowDownIcon'; -import FileIcon from 'components/common/Icons/FileIcon'; -import { useTopicDetails } from 'lib/hooks/api/topics'; -import { InputLabel } from 'components/common/Input/InputLabel.styled'; import { getSerdeOptions } from 'components/Topics/Topic/SendMessage/utils'; import { useSerdes } from 'lib/hooks/api/topicMessages'; +import useAppParams from 'lib/hooks/useAppParams'; +import { RouteParamsClusterTopic } from 'lib/paths'; +import { useMessagesFilters } from 'lib/hooks/useMessagesFilters'; +import { ModeOptions } from 'lib/hooks/filterUtils'; +import { useTopicDetails } from 'lib/hooks/api/topics'; +import EditIcon from 'components/common/Icons/EditIcon'; +import CloseIcon from 'components/common/Icons/CloseIcon'; +import FlexBox from 'components/common/FlexBox/FlexBox'; import * as S from './Filters.styled'; import { + ADD_FILTER_ID, filterOptions, - getOffsetFromSeekToParam, - getSelectedPartitionsFromSeekToParam, - getTimestampFromSeekToParam, + isLiveMode, + isModeOffsetSelector, + isModeOptionWithInput, } from './utils'; - -type Query = Record; +import FiltersSideBar from './FiltersSideBar'; +import FiltersMetrics from './FiltersMetrics'; export interface FiltersProps { phaseMessage?: string; - meta: TopicMessageConsuming; + consumptionStats?: TopicMessageConsuming; isFetching: boolean; - messageEventType?: string; - - addMessage(content: { message: TopicMessage; prepend: boolean }): void; - - resetMessages(): void; - - updatePhase(phase: string): void; - - updateMeta(meta: TopicMessageConsuming): void; - - setIsFetching(status: boolean): void; - - setMessageType(messageType: string): void; -} - -export interface MessageFilters { - name: string; - code: string; -} - -export interface ActiveMessageFilter { - index: number; - name: string; - code: string; + abortFetchData: () => void; } -const PER_PAGE = 100; - -export const SeekTypeOptions = [ - { value: SeekType.OFFSET, label: 'Offset' }, - { value: SeekType.TIMESTAMP, label: 'Timestamp' }, -]; - const Filters: React.FC = ({ - phaseMessage, - meta: { elapsedMs, bytesConsumed, messagesConsumed, filterApplyErrors }, + consumptionStats, isFetching, - addMessage, - resetMessages, - updatePhase, - updateMeta, - setIsFetching, - setMessageType, - messageEventType, + abortFetchData, + phaseMessage, }) => { const { clusterName, topicName } = useAppParams(); - const location = useLocation(); - const navigate = useNavigate(); - const [searchParams] = useSearchParams(); - const page = searchParams.get('page'); + const { + mode, + setMode, + date, + setTimeStamp, + keySerde, + setKeySerde, + valueSerde, + setValueSerde, + offset, + setOffsetValue, + search, + setSearch, + partitions: p, + setPartition, + smartFilter, + setSmartFilter, + refreshData, + } = useMessagesFilters(); const { data: topic } = useTopicDetails({ clusterName, topicName }); - - const partitions = topic?.partitions || []; - - const { seekDirection, isLive, changeSeekDirection } = - useContext(TopicMessagesContext); - - const { value: isOpen, toggle } = useBoolean(); - - const { value: isQuickEditOpen, toggle: toggleQuickEdit } = useBoolean(); - - const source = React.useRef(null); - - const [selectedPartitions, setSelectedPartitions] = React.useState( - getSelectedPartitionsFromSeekToParam(searchParams, partitions) - ); - - const [currentSeekType, setCurrentSeekType] = React.useState( - SeekTypeOptions.find( - (ele) => ele.value === (searchParams.get('seekType') as SeekType) - ) !== undefined - ? (searchParams.get('seekType') as SeekType) - : SeekType.OFFSET - ); - const [offset, setOffset] = React.useState( - getOffsetFromSeekToParam(searchParams) - ); - - const [timestamp, setTimestamp] = React.useState( - getTimestampFromSeekToParam(searchParams) - ); - const [keySerde, setKeySerde] = React.useState( - searchParams.get('keySerde') || '' - ); - const [valueSerde, setValueSerde] = React.useState( - searchParams.get('valueSerde') || '' - ); - - const [savedFilters, setSavedFilters] = React.useState( - JSON.parse(localStorage.getItem('savedFilters') ?? '[]') - ); - - let storageActiveFilter = localStorage.getItem('activeFilter'); - storageActiveFilter = - storageActiveFilter ?? JSON.stringify({ name: '', code: '', index: -1 }); - - const [activeFilter, setActiveFilter] = React.useState( - JSON.parse(storageActiveFilter) - ); - - const [queryType, setQueryType] = React.useState( - activeFilter.name - ? MessageFilterType.CEL_SCRIPT - : MessageFilterType.STRING_CONTAINS - ); - const [query, setQuery] = React.useState(searchParams.get('q') || ''); - const [isTailing, setIsTailing] = React.useState(isLive); - - const isSeekTypeControlVisible = React.useMemo( - () => selectedPartitions.length > 0, - [selectedPartitions] - ); - - const isSubmitDisabled = React.useMemo(() => { - if (isSeekTypeControlVisible) { - return ( - (currentSeekType === SeekType.TIMESTAMP && !timestamp) || isTailing - ); - } - - return false; - }, [isSeekTypeControlVisible, currentSeekType, timestamp, isTailing]); - - const partitionMap = React.useMemo( - () => - partitions.reduce>( - (acc, partition) => ({ - ...acc, - [partition.partition]: partition, - }), - {} - ), - [partitions] - ); - - const handleClearAllFilters = () => { - setCurrentSeekType(SeekType.OFFSET); - setOffset(''); - setTimestamp(null); - setQuery(''); - changeSeekDirection(SeekDirection.FORWARD); - getSelectedPartitionsFromSeekToParam(searchParams, partitions); - setSelectedPartitions( - partitions.map((partition: Partition) => { - return { - value: partition.partition, - label: `Partition #${partition.partition.toString()}`, + const [createdEditedSmartId, setCreatedEditedSmartId] = useState(); + + const partitions = useMemo(() => { + return (topic?.partitions || []).reduce<{ + dict: Record; + list: { label: string; value: number }[]; + }>( + (acc, currentValue) => { + const label = { + label: `Partition #${currentValue.partition.toString()}`, + value: currentValue.partition, }; - }) - ); - }; - - const handleFiltersSubmit = (currentOffset: string) => { - const nextAttempt = Number(searchParams.get('attempt') || 0) + 1; - const props: Query = { - q: queryType === MessageFilterType.CEL_SCRIPT ? activeFilter.code : query, - filterQueryType: queryType, - attempt: nextAttempt, - limit: PER_PAGE, - page: page || 0, - seekDirection, - keySerde: keySerde || searchParams.get('keySerde') || '', - valueSerde: valueSerde || searchParams.get('valueSerde') || '', - }; - - if (isSeekTypeControlVisible) { - switch (seekDirection) { - case SeekDirection.FORWARD: - props.seekType = SeekType.BEGINNING; - break; - case SeekDirection.BACKWARD: - case SeekDirection.TAILING: - props.seekType = SeekType.LATEST; - break; - default: - props.seekType = currentSeekType; - } - - if (offset && currentSeekType === SeekType.OFFSET) { - props.seekType = SeekType.OFFSET; - } - - if (timestamp && currentSeekType === SeekType.TIMESTAMP) { - props.seekType = SeekType.TIMESTAMP; - } - - const isSeekTypeWithSeekTo = - props.seekType === SeekType.TIMESTAMP || - props.seekType === SeekType.OFFSET; - - if ( - selectedPartitions.length !== partitions.length || - isSeekTypeWithSeekTo - ) { - // not everything in the partition is selected - props.seekTo = selectedPartitions.map(({ value }) => { - const offsetProperty = - seekDirection === SeekDirection.FORWARD ? 'offsetMin' : 'offsetMax'; - const offsetBasedSeekTo = - currentOffset || partitionMap[value][offsetProperty]; - const seekToOffset = - currentSeekType === SeekType.OFFSET - ? offsetBasedSeekTo - : timestamp?.getTime(); - return `${value}::${seekToOffset || '0'}`; - }); - } - } - - const newProps = omitBy(props, (v) => v === undefined || v === ''); - const qs = Object.keys(newProps) - .map((key) => `${key}=${encodeURIComponent(newProps[key] as string)}`) - .join('&'); - navigate({ - search: `?${qs}`, - }); - }; - - const handleSSECancel = () => { - if (!source.current) return; - setIsFetching(false); - source.current.close(); - }; - - const addFilter = (newFilter: MessageFilters) => { - const filters = [...savedFilters]; - filters.push(newFilter); - setSavedFilters(filters); - localStorage.setItem('savedFilters', JSON.stringify(filters)); - }; - const deleteFilter = (index: number) => { - const filters = [...savedFilters]; - if (activeFilter.name && activeFilter.index === index) { - localStorage.removeItem('activeFilter'); - setActiveFilter({ name: '', code: '', index: -1 }); - setQueryType(MessageFilterType.STRING_CONTAINS); - } - filters.splice(index, 1); - localStorage.setItem('savedFilters', JSON.stringify(filters)); - setSavedFilters(filters); - }; - const deleteActiveFilter = () => { - setActiveFilter({ name: '', code: '', index: -1 }); - localStorage.removeItem('activeFilter'); - setQueryType(MessageFilterType.STRING_CONTAINS); - }; - const activeFilterHandler = ( - newActiveFilter: MessageFilters, - index: number - ) => { - localStorage.setItem( - 'activeFilter', - JSON.stringify({ index, ...newActiveFilter }) + // eslint-disable-next-line no-param-reassign + acc.dict[label.value] = label; + acc.list.push(label); + return acc; + }, + { dict: {}, list: [] } ); - setActiveFilter({ index, ...newActiveFilter }); - setQueryType(MessageFilterType.CEL_SCRIPT); - }; - - const composeMessageFilter = (filter: FilterEdit): ActiveMessageFilter => ({ - index: filter.index, - name: filter.filter.name, - code: filter.filter.code, - }); - - const storeAsActiveFilter = (filter: FilterEdit) => { - const messageFilter = JSON.stringify(composeMessageFilter(filter)); - localStorage.setItem('activeFilter', messageFilter); - }; - - const editSavedFilter = (filter: FilterEdit) => { - const filters = [...savedFilters]; - filters[filter.index] = filter.filter; - if (activeFilter.name && activeFilter.index === filter.index) { - setActiveFilter(composeMessageFilter(filter)); - storeAsActiveFilter(filter); - } - localStorage.setItem('savedFilters', JSON.stringify(filters)); - setSavedFilters(filters); - }; - - const editCurrentFilter = (filter: FilterEdit) => { - if (filter.index < 0) { - setActiveFilter(composeMessageFilter(filter)); - storeAsActiveFilter(filter); - } else { - editSavedFilter(filter); - } - }; - // eslint-disable-next-line consistent-return - React.useEffect(() => { - if (location.search?.length !== 0) { - const url = `${BASE_PARAMS.basePath}/api/clusters/${encodeURIComponent( - clusterName - )}/topics/${topicName}/messages${location.search}`; - const sse = new EventSource(url); + }, [topic?.partitions]); - source.current = sse; - setIsFetching(true); + const partitionValue = useMemo(() => { + return p.map((value) => partitions.dict[value]); + }, [p, partitions]); - sse.onopen = () => { - resetMessages(); - setIsFetching(true); - }; - sse.onmessage = ({ data }) => { - const { type, message, phase, consuming }: TopicMessageEvent = - JSON.parse(data); - switch (type) { - case TopicMessageEventTypeEnum.MESSAGE: - if (message) { - addMessage({ - message, - prepend: isLive, - }); - } - break; - case TopicMessageEventTypeEnum.PHASE: - if (phase?.name) { - updatePhase(phase.name); - } - break; - case TopicMessageEventTypeEnum.CONSUMING: - if (consuming) updateMeta(consuming); - break; - case TopicMessageEventTypeEnum.DONE: - if (consuming && type) { - setMessageType(type); - updateMeta(consuming); - } - break; - default: - } - }; - - sse.onerror = () => { - setIsFetching(false); - sse.close(); - }; - - return () => { - setIsFetching(false); - sse.close(); - }; - } - }, [ - clusterName, - topicName, - seekDirection, - location, - addMessage, - resetMessages, - setIsFetching, - updateMeta, - updatePhase, - ]); - React.useEffect(() => { - if (location.search?.length === 0) { - handleFiltersSubmit(offset); - } - }, [ - seekDirection, - queryType, - activeFilter, - currentSeekType, - timestamp, - query, - location, - ]); - React.useEffect(() => { - handleFiltersSubmit(offset); - }, [ - seekDirection, - queryType, - activeFilter, - currentSeekType, - timestamp, - query, - seekDirection, - page, - ]); - - React.useEffect(() => { - setIsTailing(isLive); - }, [isLive]); - - const { data: serdes = {} } = useSerdes({ + const { data: serdes = {}, isLoading } = useSerdes({ clusterName, topicName, use: SerdeUsage.DESERIALIZE, }); + const handleRefresh = () => { + if (isLiveMode(mode) && isFetching) { + abortFetchData(); + } + refreshData(); + }; + return ( - -
- -
- Seek Type - - setCurrentSeekType(option as SeekType)} - value={currentSeekType} - selectSize="M" - minWidth="100px" - options={SeekTypeOptions} - disabled={isTailing} - /> + + + + + - {currentSeekType === SeekType.OFFSET ? ( + {isModeOptionWithInput(mode) && + (isModeOffsetSelector(mode) ? ( setOffset(value)} - disabled={isTailing} + onChange={({ target: { value } }) => { + setOffsetValue(value); + }} /> ) : ( setTimestamp(date)} + selected={date} + onChange={setTimeStamp} showTimeInput timeInputLabel="Time:" - dateFormat="MMM d, yyyy HH:mm" + dateFormat="MMM d, yyyy" placeholderText="Select timestamp" - disabled={isTailing} /> - )} - -
-
- Partitions - ({ - label: `Partition #${p.partition.toString()}`, - value: p.partition, - }))} - filterOptions={filterOptions} - value={selectedPartitions} - onChange={setSelectedPartitions} - labelledBy="Select partitions" - disabled={isTailing} - /> -
-
- Key Serde - setValueSerde(option as string)} - options={getSerdeOptions(serdes.value || [])} - value={searchParams.get('valueSerde') as string} - minWidth="170px" - selectSize="M" - disabled={isTailing} - /> -
- Clear all + ))} + + + -
-
@@ -121,18 +98,10 @@ const MessagesTable: React.FC = () => { - diff --git a/frontend/src/components/Topics/Topic/Messages/__test__/FiltersContainer.spec.tsx b/frontend/src/components/Topics/Topic/Messages/__test__/FiltersContainer.spec.tsx deleted file mode 100644 index 127a602f8..000000000 --- a/frontend/src/components/Topics/Topic/Messages/__test__/FiltersContainer.spec.tsx +++ /dev/null @@ -1,15 +0,0 @@ -import React from 'react'; -import FiltersContainer from 'components/Topics/Topic/Messages/Filters/FiltersContainer'; -import { screen } from '@testing-library/react'; -import { render } from 'lib/testHelpers'; - -jest.mock('components/Topics/Topic/Messages/Filters/Filters', () => () => ( -
mock-Filters
-)); - -describe('FiltersContainer', () => { - it('renders Filters component', () => { - render(); - expect(screen.getByText('mock-Filters')).toBeInTheDocument(); - }); -}); diff --git a/frontend/src/components/Topics/Topic/Messages/__test__/Messages.spec.tsx b/frontend/src/components/Topics/Topic/Messages/__test__/Messages.spec.tsx index 172dc8101..02c00f765 100644 --- a/frontend/src/components/Topics/Topic/Messages/__test__/Messages.spec.tsx +++ b/frontend/src/components/Topics/Topic/Messages/__test__/Messages.spec.tsx @@ -1,107 +1,46 @@ import React from 'react'; -import { screen, waitFor } from '@testing-library/react'; -import { render, EventSourceMock, WithRoute } from 'lib/testHelpers'; -import Messages, { - SeekDirectionOptions, - SeekDirectionOptionsObj, -} from 'components/Topics/Topic/Messages/Messages'; -import { SeekDirection, SeekType } from 'generated-sources'; -import userEvent from '@testing-library/user-event'; -import { clusterTopicMessagesPath } from 'lib/paths'; -import { useSerdes } from 'lib/hooks/api/topicMessages'; -import { serdesPayload } from 'lib/fixtures/topicMessages'; -import { useTopicDetails } from 'lib/hooks/api/topics'; -import { externalTopicPayload } from 'lib/fixtures/topics'; +import { render } from 'lib/testHelpers'; +import Messages from 'components/Topics/Topic/Messages/Messages'; +import { useTopicMessages } from 'lib/hooks/api/topicMessages'; +import { screen } from '@testing-library/react'; + +const mockFilterComponents = 'mockFilterComponents'; +const mockMessagesTable = 'mockMessagesTable'; jest.mock('lib/hooks/api/topicMessages', () => ({ - useSerdes: jest.fn(), + useTopicMessages: jest.fn(), })); -jest.mock('lib/hooks/api/topics', () => ({ - useTopicDetails: jest.fn(), -})); +jest.mock('components/Topics/Topic/Messages/MessagesTable', () => () => ( +
{mockMessagesTable}
+)); + +jest.mock('components/Topics/Topic/Messages/Filters/Filters', () => () => ( +
{mockFilterComponents}
+)); describe('Messages', () => { - const searchParams = `?filterQueryType=STRING_CONTAINS&attempt=0&limit=100&seekDirection=${SeekDirection.FORWARD}&seekType=${SeekType.OFFSET}&seekTo=0::9`; - const renderComponent = (param: string = searchParams) => { - const query = new URLSearchParams(param).toString(); - const path = `${clusterTopicMessagesPath()}?${query}`; - return render( - - - , - { - initialEntries: [path], - } - ); + const renderComponent = () => { + return render(); }; beforeEach(() => { - Object.defineProperty(window, 'EventSource', { - value: EventSourceMock, - }); - (useSerdes as jest.Mock).mockImplementation(() => ({ - data: serdesPayload, - })); - (useTopicDetails as jest.Mock).mockImplementation(() => ({ - data: externalTopicPayload, + (useTopicMessages as jest.Mock).mockImplementation(() => ({ + data: { messages: [], isFetching: false }, })); }); + describe('component rendering default behavior with the search params', () => { beforeEach(() => { renderComponent(); }); - it('should check default seekDirection if it actually take the value from the url', () => { - expect(screen.getAllByRole('listbox')[3]).toHaveTextContent( - SeekDirectionOptionsObj[SeekDirection.FORWARD].label - ); - }); - it('should check the SeekDirection select changes with live option', async () => { - const seekDirectionSelect = screen.getAllByRole('listbox')[3]; - const seekDirectionOption = screen.getAllByRole('option')[3]; - - expect(seekDirectionOption).toHaveTextContent( - SeekDirectionOptionsObj[SeekDirection.FORWARD].label - ); - - const labelValue1 = SeekDirectionOptions[1].label; - await userEvent.click(seekDirectionSelect); - await userEvent.selectOptions(seekDirectionSelect, [labelValue1]); - expect(seekDirectionOption).toHaveTextContent(labelValue1); - - const labelValue0 = SeekDirectionOptions[0].label; - await userEvent.click(seekDirectionSelect); - await userEvent.selectOptions(seekDirectionSelect, [labelValue0]); - expect(seekDirectionOption).toHaveTextContent(labelValue0); - - const liveOptionConf = SeekDirectionOptions[2]; - const labelValue2 = liveOptionConf.label; - await userEvent.click(seekDirectionSelect); - - const options = screen.getAllByRole('option'); - const liveModeLi = options.find( - (option) => option.getAttribute('value') === liveOptionConf.value - ); - expect(liveModeLi).toBeInTheDocument(); - if (!liveModeLi) return; // to make TS happy - await userEvent.selectOptions(seekDirectionSelect, [liveModeLi]); - expect(seekDirectionOption).toHaveTextContent(labelValue2); - - await waitFor(() => { - expect(screen.getByRole('contentLoader')).toBeInTheDocument(); - }); + it('should check if the filters are shown in the messages', () => { + expect(screen.getByText(mockFilterComponents)).toBeInTheDocument(); }); - }); - describe('Component rendering with custom Url search params', () => { - it('reacts to a change of seekDirection in the url which make the select pick up different value', () => { - renderComponent( - searchParams.replace(SeekDirection.FORWARD, SeekDirection.BACKWARD) - ); - expect(screen.getAllByRole('listbox')[3]).toHaveTextContent( - SeekDirectionOptionsObj[SeekDirection.BACKWARD].label - ); + it('should check if the table of messages are shown in the messages', () => { + expect(screen.getByText(mockMessagesTable)).toBeInTheDocument(); }); }); }); diff --git a/frontend/src/components/Topics/Topic/Messages/__test__/MessagesTable.spec.tsx b/frontend/src/components/Topics/Topic/Messages/__test__/MessagesTable.spec.tsx index 7b1e80f8c..d179a2100 100644 --- a/frontend/src/components/Topics/Topic/Messages/__test__/MessagesTable.spec.tsx +++ b/frontend/src/components/Topics/Topic/Messages/__test__/MessagesTable.spec.tsx @@ -2,63 +2,41 @@ import React from 'react'; import { screen } from '@testing-library/react'; import userEvent from '@testing-library/user-event'; import { render } from 'lib/testHelpers'; -import MessagesTable from 'components/Topics/Topic/Messages/MessagesTable'; -import { SeekDirection, SeekType, TopicMessage } from 'generated-sources'; -import TopicMessagesContext, { - ContextProps, -} from 'components/contexts/TopicMessagesContext'; -import { - topicMessagePayload, - topicMessagesMetaPayload, -} from 'redux/reducers/topicMessages/__test__/fixtures'; - -const mockTopicsMessages: TopicMessage[] = [{ ...topicMessagePayload }]; +import MessagesTable, { + MessagesTableProps, +} from 'components/Topics/Topic/Messages/MessagesTable'; +import { TopicMessage, TopicMessageTimestampTypeEnum } from 'generated-sources'; +import { useIsLiveMode } from 'lib/hooks/useMessagesFilters'; + +export const topicMessagePayload: TopicMessage = { + partition: 29, + offset: 14, + timestamp: new Date('2021-07-21T23:25:14.865Z'), + timestampType: TopicMessageTimestampTypeEnum.CREATE_TIME, + key: 'schema-registry', + headers: {}, + content: + '{"host":"schemaregistry1","port":8085,"master_eligibility":true,"scheme":"http","version":1}', +}; + +const mockTopicsMessages = [{ ...topicMessagePayload }]; const mockNavigate = jest.fn(); + jest.mock('react-router-dom', () => ({ ...jest.requireActual('react-router-dom'), useNavigate: () => mockNavigate, })); -describe('MessagesTable', () => { - const searchParams = new URLSearchParams({ - filterQueryType: 'STRING_CONTAINS', - attempt: '0', - limit: '100', - seekDirection: SeekDirection.FORWARD, - seekType: SeekType.OFFSET, - seekTo: '0::9', - }); - const contextValue: ContextProps = { - isLive: false, - seekDirection: SeekDirection.FORWARD, - changeSeekDirection: jest.fn(), - }; +jest.mock('lib/hooks/useMessagesFilters', () => ({ + useIsLiveMode: jest.fn(), + useRefreshData: jest.fn(), +})); - const renderComponent = ( - params: URLSearchParams = searchParams, - ctx: ContextProps = contextValue, - messages: TopicMessage[] = [], - isFetching?: boolean, - path?: string - ) => { - const customPath = path || params.toString(); +describe('MessagesTable', () => { + const renderComponent = (props?: Partial) => { return render( - - - , - { - initialEntries: [`/messages?${customPath}`], - preloadedState: { - topicMessages: { - messages, - meta: { - ...topicMessagesMetaPayload, - }, - isFetching: !!isFetching, - }, - }, - } + ); }; @@ -90,33 +68,35 @@ describe('MessagesTable', () => { }); describe('Custom Setup with different props value', () => { - it('should check if next button and previous is disabled isLive Param', () => { - renderComponent(searchParams, { ...contextValue, isLive: true }); + it('should check if next button is disabled isLive Param', () => { + renderComponent({ isFetching: true }); + expect(screen.queryByText(/next/i)).toBeDisabled(); + }); + + it('should check if next button is disabled if there is no nextCursor', () => { + (useIsLiveMode as jest.Mock).mockImplementation(() => false); + renderComponent({ isFetching: false }); expect(screen.queryByText(/next/i)).toBeDisabled(); - expect(screen.queryByText(/back/i)).toBeDisabled(); }); - it('should check the display of the loader element', () => { - renderComponent( - searchParams, - { ...contextValue, isLive: true }, - [], - true - ); + it('should check the display of the loader element during loader', () => { + renderComponent({ isFetching: true }); expect(screen.getByRole('progressbar')).toBeInTheDocument(); }); }); describe('should render Messages table with data', () => { beforeEach(() => { - renderComponent(searchParams, { ...contextValue }, mockTopicsMessages); + renderComponent({ messages: mockTopicsMessages, isFetching: false }); }); it('should check the rendering of the messages', () => { expect(screen.queryByText(/No messages found/i)).not.toBeInTheDocument(); - expect( - screen.getByText(mockTopicsMessages[0].content as string) - ).toBeInTheDocument(); + if (mockTopicsMessages[0].content) { + expect( + screen.getByText(mockTopicsMessages[0].content) + ).toBeInTheDocument(); + } }); }); }); diff --git a/frontend/src/components/Topics/Topic/Messages/__test__/utils.spec.ts b/frontend/src/components/Topics/Topic/Messages/__test__/utils.spec.ts index 97dd0ec7b..d0f00967d 100644 --- a/frontend/src/components/Topics/Topic/Messages/__test__/utils.spec.ts +++ b/frontend/src/components/Topics/Topic/Messages/__test__/utils.spec.ts @@ -1,11 +1,16 @@ import { Option } from 'react-multi-select-component'; import { + ADD_FILTER_ID, filterOptions, getOffsetFromSeekToParam, - getTimestampFromSeekToParam, getSelectedPartitionsFromSeekToParam, + getTimestampFromSeekToParam, + isEditingFilterMode, + isLiveMode, + isModeOffsetSelector, + isModeOptionWithInput, } from 'components/Topics/Topic/Messages/Filters/utils'; -import { SeekType, Partition } from 'generated-sources'; +import { Partition, PollingMode, SeekType } from 'generated-sources'; const options: Option[] = [ { @@ -117,4 +122,47 @@ describe('utils', () => { ]); }); }); + + describe('isModeOptionWithInput', () => { + describe('check the validity if Mode offset Selector only during', () => { + expect(isModeOptionWithInput(PollingMode.TAILING)).toBeFalsy(); + expect(isModeOptionWithInput(PollingMode.LATEST)).toBeFalsy(); + expect(isModeOptionWithInput(PollingMode.EARLIEST)).toBeFalsy(); + expect(isModeOptionWithInput(PollingMode.FROM_TIMESTAMP)).toBeTruthy(); + expect(isModeOptionWithInput(PollingMode.TO_TIMESTAMP)).toBeTruthy(); + expect(isModeOptionWithInput(PollingMode.FROM_OFFSET)).toBeTruthy(); + expect(isModeOptionWithInput(PollingMode.TO_OFFSET)).toBeTruthy(); + }); + }); + + describe('isModeOffsetSelector', () => { + it('check the validity if Mode offset Selector only during', () => { + expect(isModeOffsetSelector(PollingMode.TAILING)).toBeFalsy(); + expect(isModeOffsetSelector(PollingMode.LATEST)).toBeFalsy(); + expect(isModeOffsetSelector(PollingMode.EARLIEST)).toBeFalsy(); + expect(isModeOffsetSelector(PollingMode.FROM_TIMESTAMP)).toBeFalsy(); + expect(isModeOffsetSelector(PollingMode.TO_TIMESTAMP)).toBeFalsy(); + expect(isModeOffsetSelector(PollingMode.FROM_OFFSET)).toBeTruthy(); + expect(isModeOffsetSelector(PollingMode.TO_OFFSET)).toBeTruthy(); + }); + }); + + describe('isLiveMode', () => { + it('should check the validity of data on;y during tailing mode', () => { + expect(isLiveMode(PollingMode.TAILING)).toBeTruthy(); + expect(isLiveMode(PollingMode.LATEST)).toBeFalsy(); + expect(isLiveMode(PollingMode.EARLIEST)).toBeFalsy(); + expect(isLiveMode(PollingMode.FROM_TIMESTAMP)).toBeFalsy(); + expect(isLiveMode(PollingMode.TO_TIMESTAMP)).toBeFalsy(); + expect(isLiveMode(PollingMode.FROM_OFFSET)).toBeFalsy(); + expect(isLiveMode(PollingMode.TO_OFFSET)).toBeFalsy(); + }); + }); + + describe('isEditingFilterMode', () => { + it('should editing value', () => { + expect(isEditingFilterMode('testing')).toBeTruthy(); + expect(isEditingFilterMode(ADD_FILTER_ID)).toBeFalsy(); + }); + }); }); diff --git a/frontend/src/components/Topics/Topic/SendMessage/SendMessage.tsx b/frontend/src/components/Topics/Topic/SendMessage/SendMessage.tsx index bef7a4ddd..4bdb981f8 100644 --- a/frontend/src/components/Topics/Topic/SendMessage/SendMessage.tsx +++ b/frontend/src/components/Topics/Topic/SendMessage/SendMessage.tsx @@ -3,7 +3,7 @@ import { useForm, Controller } from 'react-hook-form'; import { RouteParamsClusterTopic } from 'lib/paths'; import { Button } from 'components/common/Button/Button'; import Editor from 'components/common/Editor/Editor'; -import Select, { SelectOption } from 'components/common/Select/Select'; +import Select from 'components/common/Select/Select'; import Switch from 'components/common/Switch/Switch'; import useAppParams from 'lib/hooks/useAppParams'; import { showAlert } from 'lib/errorHandling'; @@ -43,7 +43,7 @@ const SendMessage: React.FC<{ closeSidebar: () => void }> = ({ const sendMessage = useSendMessage({ clusterName, topicName }); const defaultValues = React.useMemo(() => getDefaultValues(serdes), [serdes]); - const partitionOptions: SelectOption[] = React.useMemo( + const partitionOptions = React.useMemo( () => getPartitionOptions(topic?.partitions || []), [topic] ); diff --git a/frontend/src/components/Topics/Topic/SendMessage/utils.ts b/frontend/src/components/Topics/Topic/SendMessage/utils.ts index c8161b0c8..46d9e1278 100644 --- a/frontend/src/components/Topics/Topic/SendMessage/utils.ts +++ b/frontend/src/components/Topics/Topic/SendMessage/utils.ts @@ -4,7 +4,6 @@ import { TopicSerdeSuggestion, } from 'generated-sources'; import jsf from 'json-schema-faker'; -import { compact } from 'lodash'; import Ajv, { DefinedError } from 'ajv/dist/2020'; import addFormats from 'ajv-formats'; import upperFirst from 'lodash/upperFirst'; @@ -46,12 +45,12 @@ export const getPartitionOptions = (partitions: Partition[]) => })); export const getSerdeOptions = (items: SerdeDescription[]) => { - const options = items.map(({ name }) => { - if (!name) return undefined; - return { label: name, value: name }; - }); - - return compact(options); + return items.reduce<{ label: string; value: string }[]>((acc, { name }) => { + if (name) { + acc.push({ value: name, label: name }); + } + return acc; + }, []); }; export const validateBySchema = ( diff --git a/frontend/src/components/Topics/Topic/Topic.tsx b/frontend/src/components/Topics/Topic/Topic.tsx index 5a639f0c4..b5bcf8d52 100644 --- a/frontend/src/components/Topics/Topic/Topic.tsx +++ b/frontend/src/components/Topics/Topic/Topic.tsx @@ -17,7 +17,6 @@ import { ActionDropdownItem, } from 'components/common/ActionComponent'; import Navbar from 'components/common/Navigation/Navbar.styled'; -import { useAppDispatch } from 'lib/hooks/redux'; import useAppParams from 'lib/hooks/useAppParams'; import { Dropdown, DropdownItemHint } from 'components/common/Dropdown'; import { @@ -26,7 +25,6 @@ import { useRecreateTopic, useTopicDetails, } from 'lib/hooks/api/topics'; -import { resetTopicMessages } from 'redux/reducers/topicMessages/topicMessagesSlice'; import { Action, CleanUpPolicy, ResourceType } from 'generated-sources'; import PageLoader from 'components/common/PageLoader/PageLoader'; import SlidingSidebar from 'components/common/SlidingSidebar'; @@ -41,7 +39,6 @@ import Edit from './Edit/Edit'; import SendMessage from './SendMessage/SendMessage'; const Topic: React.FC = () => { - const dispatch = useAppDispatch(); const { value: isSidebarOpen, setFalse: closeSidebar, @@ -62,16 +59,12 @@ const Topic: React.FC = () => { navigate(clusterTopicsPath(clusterName)); }; - React.useEffect(() => { - return () => { - dispatch(resetTopicMessages()); - }; - }, []); const clearMessages = useClearTopicMessages(clusterName); const clearTopicMessagesHandler = async () => { await clearMessages.mutateAsync(topicName); }; const canCleanup = data?.cleanUpPolicy === CleanUpPolicy.DELETE; + return ( <> Promise; } -const CleanupPolicyOptions: Array = [ +const CleanupPolicyOptions = [ { value: 'delete', label: 'Delete' }, { value: 'compact', label: 'Compact' }, { value: 'compact,delete', label: 'Compact,Delete' }, @@ -39,7 +39,7 @@ const CleanupPolicyOptions: Array = [ export const getCleanUpPolicyValue = (cleanUpPolicy?: string) => { if (!cleanUpPolicy) return undefined; - return CleanupPolicyOptions.find((option: SelectOption) => { + return CleanupPolicyOptions.find((option) => { return ( option.value.toString().replace(/,/g, '_') === cleanUpPolicy?.toLowerCase() @@ -47,7 +47,7 @@ export const getCleanUpPolicyValue = (cleanUpPolicy?: string) => { })?.value.toString(); }; -const RetentionBytesOptions: Array = [ +const RetentionBytesOptions = [ { value: NOT_SET, label: 'Not Set' }, { value: BYTES_IN_GB, label: '1 GB' }, { value: BYTES_IN_GB * 10, label: '10 GB' }, @@ -75,7 +75,7 @@ const TopicForm: React.FC = ({ getCleanUpPolicyValue(cleanUpPolicy) || CleanupPolicyOptions[0].value; const getRetentionBytes = - RetentionBytesOptions.find((option: SelectOption) => { + RetentionBytesOptions.find((option) => { return option.value === retentionBytes; })?.value || RetentionBytesOptions[0].value; diff --git a/frontend/src/components/Topics/shared/Form/__tests__/TimeToRetainBtn.spec.tsx b/frontend/src/components/Topics/shared/Form/__tests__/TimeToRetainBtn.spec.tsx index f09eec6c5..8fd700c74 100644 --- a/frontend/src/components/Topics/shared/Form/__tests__/TimeToRetainBtn.spec.tsx +++ b/frontend/src/components/Topics/shared/Form/__tests__/TimeToRetainBtn.spec.tsx @@ -4,7 +4,7 @@ import { screen } from '@testing-library/react'; import TimeToRetainBtn, { Props, } from 'components/Topics/shared/Form/TimeToRetainBtn'; -import { useForm, FormProvider } from 'react-hook-form'; +import { FormProvider, useForm } from 'react-hook-form'; import { theme } from 'theme/theme'; import userEvent from '@testing-library/user-event'; @@ -61,7 +61,7 @@ describe('TimeToRetainBtn', () => { SetUpComponent({ value: 604800000 }); const buttonElement = screen.getByRole('button'); expect(buttonElement).toHaveStyle( - `background-color:${theme.button.secondary.invertedColors.normal}` + `background-color:${theme.chips.backgroundColor.active}` ); expect(buttonElement).toHaveStyle(`border:none`); }); diff --git a/frontend/src/components/common/ActionComponent/ActionSelect/ActionSelect.tsx b/frontend/src/components/common/ActionComponent/ActionSelect/ActionSelect.tsx index 31c73982a..030546187 100644 --- a/frontend/src/components/common/ActionComponent/ActionSelect/ActionSelect.tsx +++ b/frontend/src/components/common/ActionComponent/ActionSelect/ActionSelect.tsx @@ -8,15 +8,15 @@ import { useActionTooltip } from 'lib/hooks/useActionTooltip'; import { usePermission } from 'lib/hooks/usePermission'; import * as S from 'components/common/ActionComponent/ActionComponent.styled'; -interface Props extends SelectProps, ActionComponentProps {} +interface Props extends SelectProps, ActionComponentProps {} -const ActionSelect: React.FC = ({ +const ActionSelect = ({ message = getDefaultActionMessage(), permission, placement = 'bottom', disabled, ...props -}) => { +}: Props) => { const canDoAction = usePermission( permission.resource, permission.action, diff --git a/frontend/src/components/common/Button/Button.styled.ts b/frontend/src/components/common/Button/Button.styled.ts index a436d01e7..649719987 100644 --- a/frontend/src/components/common/Button/Button.styled.ts +++ b/frontend/src/components/common/Button/Button.styled.ts @@ -3,7 +3,6 @@ import styled from 'styled-components'; export interface ButtonProps { buttonType: 'primary' | 'secondary' | 'danger'; buttonSize: 'S' | 'M' | 'L'; - isInverted?: boolean; } const StyledButton = styled.button` @@ -11,44 +10,32 @@ const StyledButton = styled.button` flex-direction: row; align-items: center; justify-content: center; - padding: 0 12px; + padding: ${({ buttonSize }) => (buttonSize === 'S' ? '0 8px' : '0 12px')}; border: none; border-radius: 4px; white-space: nowrap; - background: ${({ isInverted, buttonType, theme }) => - isInverted - ? 'transparent' - : theme.button[buttonType].backgroundColor.normal}; - color: ${({ isInverted, buttonType, theme }) => - isInverted - ? theme.button[buttonType].invertedColors.normal - : theme.button[buttonType].color.normal}; + background: ${({ buttonType, theme }) => + theme.button[buttonType].backgroundColor.normal}; + + color: ${({ buttonType, theme }) => theme.button[buttonType].color.normal}; + height: ${({ theme, buttonSize }) => theme.button.height[buttonSize]}; font-size: ${({ theme, buttonSize }) => theme.button.fontSize[buttonSize]}; font-weight: 500; - height: ${({ theme, buttonSize }) => theme.button.height[buttonSize]}; &:hover:enabled { - background: ${({ isInverted, buttonType, theme }) => - isInverted - ? 'transparent' - : theme.button[buttonType].backgroundColor.hover}; - color: ${({ isInverted, buttonType, theme }) => - isInverted - ? theme.button[buttonType].invertedColors.hover - : theme.button[buttonType].color}; + background: ${({ buttonType, theme }) => + theme.button[buttonType].backgroundColor.hover}; + color: ${({ buttonType, theme }) => theme.button[buttonType].color.normal}; cursor: pointer; } + &:active:enabled { - background: ${({ isInverted, buttonType, theme }) => - isInverted - ? 'transparent' - : theme.button[buttonType].backgroundColor.active}; - color: ${({ isInverted, buttonType, theme }) => - isInverted - ? theme.button[buttonType].invertedColors.active - : theme.button[buttonType].color}; + background: ${({ buttonType, theme }) => + theme.button[buttonType].backgroundColor.active}; + color: ${({ buttonType, theme }) => theme.button[buttonType].color.normal}; } + &:disabled { opacity: 0.5; cursor: not-allowed; @@ -59,11 +46,11 @@ const StyledButton = styled.button` } & a { - color: ${({ theme }) => theme.button.primary.color}; + color: ${({ theme }) => theme.button.primary.color.normal}; } & svg { - margin-right: 7px; + margin-right: 4px; fill: ${({ theme, disabled, buttonType }) => disabled ? theme.button[buttonType].color.disabled diff --git a/frontend/src/components/common/Button/Button.tsx b/frontend/src/components/common/Button/Button.tsx index fe330a5e4..46bf120fe 100644 --- a/frontend/src/components/common/Button/Button.tsx +++ b/frontend/src/components/common/Button/Button.tsx @@ -1,10 +1,9 @@ -import StyledButton, { - ButtonProps, -} from 'components/common/Button/Button.styled'; import React from 'react'; import { Link } from 'react-router-dom'; import Spinner from 'components/common/Spinner/Spinner'; +import StyledButton, { ButtonProps } from './Button.styled'; + export interface Props extends React.ButtonHTMLAttributes, ButtonProps { @@ -12,24 +11,27 @@ export interface Props inProgress?: boolean; } -export const Button: React.FC = ({ to, ...props }) => { +export const Button: React.FC = ({ + to, + children, + disabled, + inProgress, + ...props +}) => { if (to) { return ( - - {props.children} + + {children} ); } + return ( - - {props.children}{' '} - {props.inProgress ? ( + + {children}{' '} + {inProgress ? ( ) : null} diff --git a/frontend/src/components/common/Button/__tests__/Button.spec.tsx b/frontend/src/components/common/Button/__tests__/Button.spec.tsx index 21919eb0d..ced6d0af4 100644 --- a/frontend/src/components/common/Button/__tests__/Button.spec.tsx +++ b/frontend/src/components/common/Button/__tests__/Button.spec.tsx @@ -50,14 +50,6 @@ describe('Button', () => { ); }); - it('renders inverted color Button', () => { - render( diff --git a/frontend/src/components/common/FlexBox/FlexBox.tsx b/frontend/src/components/common/FlexBox/FlexBox.tsx new file mode 100644 index 000000000..2a767ad56 --- /dev/null +++ b/frontend/src/components/common/FlexBox/FlexBox.tsx @@ -0,0 +1,39 @@ +import React, { CSSProperties, ReactNode } from 'react'; +import styled from 'styled-components'; + +interface FlexboxProps { + flexDirection?: CSSProperties['flexDirection']; + alignItems?: CSSProperties['alignItems']; + alignSelf?: CSSProperties['alignSelf']; + justifyContent?: CSSProperties['justifyContent']; + justifyItems?: CSSProperties['justifyItems']; + gap?: CSSProperties['gap']; + margin?: CSSProperties['margin']; + padding?: CSSProperties['padding']; + color?: CSSProperties['color']; + flexGrow?: CSSProperties['flexGrow']; + flexWrap?: CSSProperties['flexWrap']; + width?: CSSProperties['width']; + children: ReactNode; +} + +const FlexboxContainer = styled.div` + display: flex; + flex-direction: ${(props) => props.flexDirection || 'row'}; + align-items: ${(props) => props.alignItems}; + align-self: ${(props) => props.alignSelf}; + justify-content: ${(props) => props.justifyContent}; + justify-items: ${(props) => props.justifyItems}; + gap: ${(props) => props.gap}; + margin: ${(props) => props.margin}; + padding: ${(props) => props.padding}; + flex-grow: ${(props) => props.flexGrow}; + width: ${(props) => props.width}; + color ${(props) => props.color}; +`; + +const Flexbox: React.FC = ({ children, ...rest }) => { + return {children}; +}; + +export default Flexbox; diff --git a/frontend/src/components/common/Icons/GitIcon.tsx b/frontend/src/components/common/Icons/GitHubIcon.tsx similarity index 90% rename from frontend/src/components/common/Icons/GitIcon.tsx rename to frontend/src/components/common/Icons/GitHubIcon.tsx index daecb611f..e9132c76f 100644 --- a/frontend/src/components/common/Icons/GitIcon.tsx +++ b/frontend/src/components/common/Icons/GitHubIcon.tsx @@ -1,7 +1,7 @@ import React from 'react'; import styled from 'styled-components'; -const GitIcon: React.FC<{ className?: string }> = ({ className }) => ( +const GitHubIcon: React.FC<{ className?: string }> = ({ className }) => ( = ({ className }) => ( ); -export default styled(GitIcon)``; +export default styled(GitHubIcon)``; diff --git a/frontend/src/components/common/Icons/ProductHuntIcon.tsx b/frontend/src/components/common/Icons/ProductHuntIcon.tsx new file mode 100644 index 000000000..b0f660491 --- /dev/null +++ b/frontend/src/components/common/Icons/ProductHuntIcon.tsx @@ -0,0 +1,21 @@ +import React from 'react'; +import styled from 'styled-components'; + +const ProductHuntIcon: React.FC<{ className?: string }> = ({ className }) => ( + + + + +); + +export default styled(ProductHuntIcon)``; diff --git a/frontend/src/components/common/NewTable/__test__/Table.spec.tsx b/frontend/src/components/common/NewTable/__test__/Table.spec.tsx index b22e7c0ec..266276379 100644 --- a/frontend/src/components/common/NewTable/__test__/Table.spec.tsx +++ b/frontend/src/components/common/NewTable/__test__/Table.spec.tsx @@ -7,7 +7,7 @@ import Table, { LinkCell, TagCell, } from 'components/common/NewTable'; -import { screen, waitFor } from '@testing-library/dom'; +import { screen } from '@testing-library/dom'; import { ColumnDef, Row } from '@tanstack/react-table'; import userEvent from '@testing-library/user-event'; import { formatTimestamp } from 'lib/dateTimeHelpers'; @@ -94,7 +94,7 @@ const columns: ColumnDef[] = [ const ExpandedRow: React.FC = () =>
I am expanded row
; -interface Props extends TableProps { +interface Props extends TableProps { path?: string; } @@ -276,7 +276,7 @@ describe('Table', () => { describe('Sorting', () => { it('sort rows', async () => { - await renderComponent({ + renderComponent({ path: '/?sortBy=text&&sortDirection=desc', enableSorting: true, }); @@ -293,7 +293,7 @@ describe('Table', () => { expect(rows[1].textContent?.indexOf('sit')).toBeGreaterThan(-1); // Disable sorting by text column - await waitFor(() => userEvent.click(th)); + await userEvent.click(th); rows = screen.getAllByRole('row'); expect(rows[1].textContent?.indexOf('lorem')).toBeGreaterThan(-1); expect(rows[2].textContent?.indexOf('ipsum')).toBeGreaterThan(-1); @@ -301,7 +301,7 @@ describe('Table', () => { expect(rows[4].textContent?.indexOf('sit')).toBeGreaterThan(-1); // Sort by text column ascending - await waitFor(() => userEvent.click(th)); + await userEvent.click(th); rows = screen.getAllByRole('row'); expect(rows[1].textContent?.indexOf('dolor')).toBeGreaterThan(-1); expect(rows[2].textContent?.indexOf('ipsum')).toBeGreaterThan(-1); diff --git a/frontend/src/components/common/Search/Search.tsx b/frontend/src/components/common/Search/Search.tsx index 72b5d1d54..3fa7d27ac 100644 --- a/frontend/src/components/common/Search/Search.tsx +++ b/frontend/src/components/common/Search/Search.tsx @@ -1,4 +1,4 @@ -import React, { useRef } from 'react'; +import React, { ComponentRef, useRef } from 'react'; import { useDebouncedCallback } from 'use-debounce'; import Input from 'components/common/Input/Input'; import { useSearchParams } from 'react-router-dom'; @@ -29,7 +29,8 @@ const Search: React.FC = ({ onChange, }) => { const [searchParams, setSearchParams] = useSearchParams(); - const ref = useRef(null); + const ref = useRef>(null); + const handleChange = useDebouncedCallback((e) => { if (ref.current != null) { ref.current.value = e.target.value; @@ -44,8 +45,11 @@ const Search: React.FC = ({ setSearchParams(searchParams); } }, 500); + const clearSearchValue = () => { - if (searchParams.get('q')) { + if (onChange) { + onChange(''); + } else if (searchParams.get('q')) { searchParams.set('q', ''); setSearchParams(searchParams); } diff --git a/frontend/src/components/common/Select/ControlledSelect.tsx b/frontend/src/components/common/Select/ControlledSelect.tsx index 1ea90c356..54174fd4c 100644 --- a/frontend/src/components/common/Select/ControlledSelect.tsx +++ b/frontend/src/components/common/Select/ControlledSelect.tsx @@ -6,24 +6,24 @@ import { ErrorMessage } from '@hookform/error-message'; import Select, { SelectOption } from './Select'; -interface ControlledSelectProps { +interface ControlledSelectProps { name: string; label: React.ReactNode; hint?: string; - options: SelectOption[]; - onChange?: (val: string | number) => void; + options: SelectOption[]; + onChange?: (val: T) => void; disabled?: boolean; placeholder?: string; } -const ControlledSelect: React.FC = ({ +const ControlledSelect = ({ name, label, onChange, options, disabled = false, placeholder, -}) => { +}: ControlledSelectProps) => { const id = React.useId(); return ( diff --git a/frontend/src/components/common/Select/Select.tsx b/frontend/src/components/common/Select/Select.tsx index a72660d2c..72bf358ba 100644 --- a/frontend/src/components/common/Select/Select.tsx +++ b/frontend/src/components/common/Select/Select.tsx @@ -3,123 +3,113 @@ import useClickOutside from 'lib/hooks/useClickOutside'; import DropdownArrowIcon from 'components/common/Icons/DropdownArrowIcon'; import * as S from './Select.styled'; -import LiveIcon from './LiveIcon.styled'; -export interface SelectProps { - options?: Array; +export interface SelectProps { + options?: SelectOption[]; id?: string; name?: string; selectSize?: 'M' | 'L'; - isLive?: boolean; minWidth?: string; - value?: string | number; - defaultValue?: string | number; + value?: T; + defaultValue?: T; placeholder?: string; disabled?: boolean; - onChange?: (option: string | number) => void; + onChange?: (option: T) => void; isThemeMode?: boolean; } -export interface SelectOption { +export interface SelectOption { label: string | number | React.ReactElement; - value: string | number; + value: T; disabled?: boolean; - isLive?: boolean; } -const Select = React.forwardRef( - ( - { - options = [], - value, - defaultValue, - selectSize = 'L', - placeholder = '', - isLive, - disabled = false, - onChange, - isThemeMode, - ...props - }, - ref - ) => { - const [selectedOption, setSelectedOption] = useState(value); - const [showOptions, setShowOptions] = useState(false); +// Use the generic type T for forwardRef +const Select = ( + { + options = [], + value, + defaultValue, + selectSize = 'L', + placeholder = '', + disabled = false, + onChange, + isThemeMode, + ...props + }: SelectProps, + ref?: React.Ref +) => { + const [selectedOption, setSelectedOption] = useState(value); + const [showOptions, setShowOptions] = useState(false); - const showOptionsHandler = () => { - if (!disabled) setShowOptions(!showOptions); - }; + const showOptionsHandler = () => { + if (!disabled) setShowOptions(!showOptions); + }; - const selectContainerRef = useRef(null); - const clickOutsideHandler = () => setShowOptions(false); - useClickOutside(selectContainerRef, clickOutsideHandler); + const selectContainerRef = useRef(null); + const clickOutsideHandler = () => setShowOptions(false); + useClickOutside(selectContainerRef, clickOutsideHandler); - const updateSelectedOption = (option: SelectOption) => { - if (!option.disabled) { - setSelectedOption(option.value); + const updateSelectedOption = (option: SelectOption) => { + if (!option.disabled) { + setSelectedOption(option.value); - if (onChange) { - onChange(option.value); - } - - setShowOptions(false); + if (onChange) { + onChange(option.value); } - }; - React.useEffect(() => { - setSelectedOption(value); - }, [isLive, value]); + setShowOptions(false); + } + }; - return ( -
- - - {isLive && } - - {options.find( - (option) => option.value === (defaultValue || selectedOption) - )?.label || placeholder} - - - {showOptions && ( - - {options?.map((option) => ( - updateSelectedOption(option)} - tabIndex={0} - role="option" - > - {option.isLive && } - {option.label} - - ))} - - )} - - -
- ); - } -); + return ( +
+ + + + {options.find( + (option) => option.value === (defaultValue || selectedOption) + )?.label || placeholder} + + + {showOptions && ( + + {options?.map((option) => ( + updateSelectedOption(option)} + tabIndex={0} + role="option" + > + {option.label} + + ))} + + )} + + +
+ ); +}; Select.displayName = 'Select'; -export default Select; +export default React.forwardRef(Select) as ( + props: SelectProps & React.RefAttributes +) => React.ReactElement; diff --git a/frontend/src/components/common/Select/__tests__/Select.spec.tsx b/frontend/src/components/common/Select/__tests__/Select.spec.tsx index f05d9e126..5d9a8a97f 100644 --- a/frontend/src/components/common/Select/__tests__/Select.spec.tsx +++ b/frontend/src/components/common/Select/__tests__/Select.spec.tsx @@ -1,7 +1,4 @@ -import Select, { - SelectOption, - SelectProps, -} from 'components/common/Select/Select'; +import Select, { SelectProps } from 'components/common/Select/Select'; import React from 'react'; import { render } from 'lib/testHelpers'; import { screen } from '@testing-library/react'; @@ -13,12 +10,12 @@ jest.mock('react-hook-form', () => ({ }), })); -const options: Array = [ +const options = [ { label: 'test-label1', value: 'test-value1', disabled: false }, { label: 'test-label2', value: 'test-value2', disabled: true }, ]; -const renderComponent = (props?: Partial) => +const renderComponent = (props?: Partial>) => render(); - expect(screen.getByRole('listbox')).toBeInTheDocument(); - }); - }); }); diff --git a/frontend/src/components/common/SlidingSidebar/SlidingSidebar.styled.ts b/frontend/src/components/common/SlidingSidebar/SlidingSidebar.styled.ts index 4ccb4ae0b..ef33a386b 100644 --- a/frontend/src/components/common/SlidingSidebar/SlidingSidebar.styled.ts +++ b/frontend/src/components/common/SlidingSidebar/SlidingSidebar.styled.ts @@ -22,6 +22,26 @@ export const Wrapper = styled.div<{ $open?: boolean }>( ` ); +export const HeaderText = styled.div` + ${({ theme }) => theme.heading?.base}; + font-size: 24px; + line-height: 16px; +`; + +export const Header = styled.div` + display: flex; + justify-content: space-between; + width: 100%; + align-items: center; + padding: 20px; +`; + +export const CloseIconButtonWrapper = styled.button` + background: transparent; + border: none; + cursor: pointer; +`; + export const Content = styled.div<{ $open?: boolean }>( ({ theme }) => ` background-color: ${theme.default.backgroundColor}; diff --git a/frontend/src/components/common/SlidingSidebar/SlidingSidebar.tsx b/frontend/src/components/common/SlidingSidebar/SlidingSidebar.tsx index 40ef062d5..7fab10845 100644 --- a/frontend/src/components/common/SlidingSidebar/SlidingSidebar.tsx +++ b/frontend/src/components/common/SlidingSidebar/SlidingSidebar.tsx @@ -1,6 +1,5 @@ import React, { PropsWithChildren } from 'react'; -import Heading from 'components/common/heading/Heading.styled'; -import { Button } from 'components/common/Button/Button'; +import CloseCircleIcon from 'components/common/Icons/CloseCircleIcon'; import * as S from './SlidingSidebar.styled'; @@ -18,12 +17,12 @@ const SlidingSidebar: React.FC = ({ }) => { return ( - - {title} - - + + {title} + + + + {children} ); diff --git a/frontend/src/components/common/Tooltip/Tooltip.tsx b/frontend/src/components/common/Tooltip/Tooltip.tsx index 0764320f5..6b74391ca 100644 --- a/frontend/src/components/common/Tooltip/Tooltip.tsx +++ b/frontend/src/components/common/Tooltip/Tooltip.tsx @@ -1,9 +1,9 @@ import React, { useState } from 'react'; import { + Placement, useFloating, useHover, useInteractions, - Placement, } from '@floating-ui/react'; import * as S from './Tooltip.styled'; @@ -12,9 +12,15 @@ interface TooltipProps { value: React.ReactNode; content: string; placement?: Placement; + showTooltip?: boolean; } -const Tooltip: React.FC = ({ value, content, placement }) => { +const Tooltip: React.FC = ({ + value, + content, + placement, + showTooltip = true, +}) => { const [open, setOpen] = useState(false); const { x, y, refs, strategy, context } = useFloating({ open, @@ -28,7 +34,7 @@ const Tooltip: React.FC = ({ value, content, placement }) => {
{value}
- {open && ( + {showTooltip && open && ( void; - setContent: React.Dispatch>; - setConfirm: React.Dispatch void) | undefined>>; + setContent: Dispatch>; + setConfirm: Dispatch void) | undefined>>; cancel: () => void; dangerButton: boolean; - setDangerButton: React.Dispatch>; + setDangerButton: Dispatch>; + isConfirming: boolean; + setIsConfirming: Dispatch>; } -export const ConfirmContext = React.createContext( - null -); +export const ConfirmContext = createContext(null); -export const ConfirmContextProvider: React.FC< - React.PropsWithChildren -> = ({ children }) => { - const [content, setContent] = useState(null); +export const ConfirmContextProvider: FC = ({ children }) => { + const [content, setContent] = useState(null); const [confirm, setConfirm] = useState<(() => void) | undefined>(undefined); const [dangerButton, setDangerButton] = useState(false); + const [isConfirming, setIsConfirming] = useState(false); const cancel = () => { setContent(null); @@ -36,6 +43,8 @@ export const ConfirmContextProvider: React.FC< cancel, dangerButton, setDangerButton, + isConfirming, + setIsConfirming, }} > {children} diff --git a/frontend/src/components/contexts/TopicMessagesContext.ts b/frontend/src/components/contexts/TopicMessagesContext.ts deleted file mode 100644 index 3ca2ca655..000000000 --- a/frontend/src/components/contexts/TopicMessagesContext.ts +++ /dev/null @@ -1,14 +0,0 @@ -import React from 'react'; -import { SeekDirection } from 'generated-sources'; - -export interface ContextProps { - seekDirection: SeekDirection; - changeSeekDirection(val: string): void; - isLive: boolean; -} - -const TopicMessagesContext = React.createContext( - {} as ContextProps -); - -export default TopicMessagesContext; diff --git a/frontend/src/lib/constants.ts b/frontend/src/lib/constants.ts index 833ad2faf..102b79faa 100644 --- a/frontend/src/lib/constants.ts +++ b/frontend/src/lib/constants.ts @@ -1,4 +1,3 @@ -import { SelectOption } from 'components/common/Select/Select'; import { ConfigurationParameters, ConsumerGroupState } from 'generated-sources'; declare global { @@ -75,7 +74,7 @@ export const QUERY_REFETCH_OFF_OPTIONS = { }; // Cluster Form Constants -export const AUTH_OPTIONS: SelectOption[] = [ +export const AUTH_OPTIONS = [ { value: 'SASL/JAAS', label: 'SASL/JAAS' }, { value: 'SASL/GSSAPI', label: 'SASL/GSSAPI' }, { value: 'SASL/OAUTHBEARER', label: 'SASL/OAUTHBEARER' }, @@ -88,11 +87,11 @@ export const AUTH_OPTIONS: SelectOption[] = [ { value: 'mTLS', label: 'mTLS' }, ]; -export const SECURITY_PROTOCOL_OPTIONS: SelectOption[] = [ +export const SECURITY_PROTOCOL_OPTIONS = [ { value: 'SASL_SSL', label: 'SASL_SSL' }, { value: 'SASL_PLAINTEXT', label: 'SASL_PLAINTEXT' }, ]; -export const METRICS_OPTIONS: SelectOption[] = [ +export const METRICS_OPTIONS = [ { value: 'JMX', label: 'JMX' }, { value: 'PROMETHEUS', label: 'PROMETHEUS' }, ]; diff --git a/frontend/src/lib/hooks/api/topicMessages.tsx b/frontend/src/lib/hooks/api/topicMessages.tsx index 886b2979c..cf22148ac 100644 --- a/frontend/src/lib/hooks/api/topicMessages.tsx +++ b/frontend/src/lib/hooks/api/topicMessages.tsx @@ -1,103 +1,118 @@ -import React from 'react'; +import React, { useCallback, useRef } from 'react'; import { fetchEventSource } from '@microsoft/fetch-event-source'; import { BASE_PARAMS, MESSAGES_PER_PAGE } from 'lib/constants'; -import { ClusterName } from 'redux/interfaces'; +import { ClusterName, TopicName } from 'redux/interfaces'; import { GetSerdesRequest, - SeekDirection, - SeekType, + PollingMode, TopicMessage, TopicMessageConsuming, TopicMessageEvent, TopicMessageEventTypeEnum, } from 'generated-sources'; import { showServerError } from 'lib/errorHandling'; -import toast from 'react-hot-toast'; -import { useQuery } from '@tanstack/react-query'; +import { useMutation, useQuery } from '@tanstack/react-query'; import { messagesApiClient } from 'lib/api'; -import { StopLoading } from 'components/Topics/Topic/Messages/Messages.styled'; +import { useSearchParams } from 'react-router-dom'; +import { MessagesFilterKeys } from 'lib/hooks/useMessagesFilters'; +import { convertStrToPollingMode } from 'lib/hooks/filterUtils'; +import { useMessageFiltersStore } from 'lib/hooks/useMessageFiltersStore'; interface UseTopicMessagesProps { clusterName: ClusterName; - topicName: string; - searchParams: URLSearchParams; + topicName: TopicName; } -type ConsumingMode = - | 'live' - | 'oldest' - | 'newest' - | 'fromOffset' // from 900 -> 1000 - | 'toOffset' // from 900 -> 800 - | 'sinceTime' // from 10:15 -> 11:15 - | 'untilTime'; // from 10:15 -> 9:15 - export const useTopicMessages = ({ clusterName, topicName, - searchParams, }: UseTopicMessagesProps) => { + const [searchParams] = useSearchParams(); const [messages, setMessages] = React.useState([]); const [phase, setPhase] = React.useState(); - const [meta, setMeta] = React.useState(); - const [isFetching, setIsFetching] = React.useState(false); - const abortController = new AbortController(); + const [consumptionStats, setConsumptionStats] = + React.useState(); + const [isFetching, setIsFetching] = React.useState(false); + const abortController = useRef(new AbortController()); + const prevReqUrl = useRef(''); // get initial properties - const mode = searchParams.get('m') as ConsumingMode; - const limit = searchParams.get('perPage') || MESSAGES_PER_PAGE; - const seekTo = searchParams.get('seekTo') || '0-0'; + + const abortFetchData = useCallback(() => { + if (abortController.current.signal.aborted) return; + + setIsFetching(false); + abortController.current.abort(); + abortController.current = new AbortController(); + }, []); React.useEffect(() => { + const mode = convertStrToPollingMode( + searchParams.get(MessagesFilterKeys.mode) || '' + ); + const fetchData = async () => { setIsFetching(true); + const url = `${BASE_PARAMS.basePath}/api/clusters/${encodeURIComponent( clusterName - )}/topics/${topicName}/messages`; + )}/topics/${topicName}/messages/v2`; + const requestParams = new URLSearchParams({ - limit, - seekTo: seekTo.replaceAll('-', '::').replaceAll('.', ','), - q: searchParams.get('q') || '', - keySerde: searchParams.get('keySerde') || '', - valueSerde: searchParams.get('valueSerde') || '', + limit: searchParams.get(MessagesFilterKeys.limit) || MESSAGES_PER_PAGE, + mode: searchParams.get(MessagesFilterKeys.mode) || '', + }); + + [ + MessagesFilterKeys.stringFilter, + MessagesFilterKeys.keySerde, + MessagesFilterKeys.smartFilterId, + MessagesFilterKeys.valueSerde, + ].forEach((item) => { + const value = searchParams.get(item); + if (value) { + requestParams.set(item, value); + } }); switch (mode) { - case 'live': - requestParams.set('seekDirection', SeekDirection.TAILING); - requestParams.set('seekType', SeekType.LATEST); + case PollingMode.TO_TIMESTAMP: + case PollingMode.FROM_TIMESTAMP: + requestParams.set( + MessagesFilterKeys.timestamp, + searchParams.get(MessagesFilterKeys.timestamp) || '0' + ); break; - case 'oldest': - requestParams.set('seekType', SeekType.BEGINNING); - requestParams.set('seekDirection', SeekDirection.FORWARD); - break; - case 'newest': - requestParams.set('seekType', SeekType.LATEST); - requestParams.set('seekDirection', SeekDirection.BACKWARD); - break; - case 'fromOffset': - requestParams.set('seekType', SeekType.OFFSET); - requestParams.set('seekDirection', SeekDirection.FORWARD); - break; - case 'toOffset': - requestParams.set('seekType', SeekType.OFFSET); - requestParams.set('seekDirection', SeekDirection.BACKWARD); - break; - case 'sinceTime': - requestParams.set('seekType', SeekType.TIMESTAMP); - requestParams.set('seekDirection', SeekDirection.FORWARD); - break; - case 'untilTime': - requestParams.set('seekType', SeekType.TIMESTAMP); - requestParams.set('seekDirection', SeekDirection.BACKWARD); + case PollingMode.TO_OFFSET: + case PollingMode.FROM_OFFSET: + requestParams.set( + MessagesFilterKeys.offset, + searchParams.get(MessagesFilterKeys.offset) || '0' + ); break; default: - break; } + searchParams.getAll(MessagesFilterKeys.partitions).forEach((value) => { + requestParams.append(MessagesFilterKeys.partitions, value); + }); + + const { nextCursor, setNextCursor } = useMessageFiltersStore.getState(); + + const tempCompareUrl = new URLSearchParams(requestParams); + tempCompareUrl.delete(MessagesFilterKeys.cursor); + + const tempToString = tempCompareUrl.toString(); + + // filters stay the say and we have cursor set cursor + if (nextCursor && tempToString === prevReqUrl.current) { + requestParams.set(MessagesFilterKeys.cursor, nextCursor); + } + + prevReqUrl.current = tempToString; await fetchEventSource(`${url}?${requestParams.toString()}`, { method: 'GET', - signal: abortController.signal, + signal: abortController.current.signal, openWhenHidden: true, async onopen(response) { const { ok, status } = response; @@ -110,13 +125,17 @@ export const useTopicMessages = ({ }, onmessage(event) { const parsedData: TopicMessageEvent = JSON.parse(event.data); - const { message, consuming } = parsedData; + const { message, consuming, cursor } = parsedData; + + if (useMessageFiltersStore.getState().nextCursor !== cursor?.id) { + setNextCursor(cursor?.id || undefined); + } switch (parsedData.type) { case TopicMessageEventTypeEnum.MESSAGE: if (message) { setMessages((prevMessages) => { - if (mode === 'live') { + if (mode === PollingMode.TAILING) { return [message, ...prevMessages]; } return [...prevMessages, message]; @@ -127,59 +146,36 @@ export const useTopicMessages = ({ if (parsedData.phase?.name) setPhase(parsedData.phase.name); break; case TopicMessageEventTypeEnum.CONSUMING: - if (consuming) setMeta(consuming); + if (consuming) setConsumptionStats(consuming); break; default: } }, onclose() { setIsFetching(false); + abortController.current = new AbortController(); }, onerror(err) { + setNextCursor(undefined); setIsFetching(false); + abortController.current = new AbortController(); showServerError(err); }, }); }; - const abortFetchData = () => { - setIsFetching(false); - abortController.abort(); - }; - if (mode === 'live') { - toast.promise( - fetchData(), - { - loading: ( - <> -
Consuming messages...
-   - Abort - - ), - success: 'Cancelled', - error: 'Something went wrong. Please try again.', - }, - { - id: 'messages', - position: 'top-center', - // eslint-disable-next-line @typescript-eslint/ban-ts-comment - // @ts-ignore - missing type for icon - success: { duration: 10, icon: false }, - } - ); - } else { - fetchData(); - } + abortFetchData(); + fetchData(); return abortFetchData; - }, [searchParams]); + }, [searchParams, abortFetchData]); return { phase, messages, - meta, + consumptionStats, isFetching, + abortFetchData, }; }; @@ -196,3 +192,19 @@ export function useSerdes(props: GetSerdesRequest) { } ); } + +export function useRegisterSmartFilter({ + clusterName, + topicName, +}: { + clusterName: ClusterName; + topicName: TopicName; +}) { + return useMutation((payload: { filterCode: string }) => { + return messagesApiClient.registerFilter({ + clusterName, + topicName, + messageFilterRegistration: { filterCode: payload.filterCode }, + }); + }); +} diff --git a/frontend/src/lib/hooks/api/topics.ts b/frontend/src/lib/hooks/api/topics.ts index 00d08bc66..3f68a1709 100644 --- a/frontend/src/lib/hooks/api/topics.ts +++ b/frontend/src/lib/hooks/api/topics.ts @@ -318,6 +318,7 @@ export function useAnalyzeTopic(props: GetTopicDetailsRequest) { }, }); } + export function useCancelTopicAnalysis(props: GetTopicDetailsRequest) { const client = useQueryClient(); return useMutation(() => api.cancelTopicAnalysis(props), { diff --git a/frontend/src/lib/hooks/filterUtils.ts b/frontend/src/lib/hooks/filterUtils.ts new file mode 100644 index 000000000..87482992e --- /dev/null +++ b/frontend/src/lib/hooks/filterUtils.ts @@ -0,0 +1,17 @@ +import { PollingMode } from 'generated-sources'; + +export const ModeOptions = [ + { value: PollingMode.LATEST, label: 'Oldest' }, + { value: PollingMode.EARLIEST, label: 'Newest' }, + { value: PollingMode.TAILING, label: 'Live' }, + { value: PollingMode.FROM_OFFSET, label: 'From offset' }, + { value: PollingMode.TO_OFFSET, label: 'To offset' }, + { value: PollingMode.FROM_TIMESTAMP, label: 'Since time' }, + { value: PollingMode.TO_TIMESTAMP, label: 'To time' }, +]; + +export function convertStrToPollingMode( + value: string | number +): PollingMode | undefined { + return PollingMode[value.toString() as keyof typeof PollingMode]; +} diff --git a/frontend/src/lib/hooks/useConfirm.ts b/frontend/src/lib/hooks/useConfirm.ts index baac856c5..117db2401 100644 --- a/frontend/src/lib/hooks/useConfirm.ts +++ b/frontend/src/lib/hooks/useConfirm.ts @@ -1,17 +1,22 @@ import { ConfirmContext } from 'components/contexts/ConfirmContext'; -import React, { useContext } from 'react'; +import { type ReactNode, useContext } from 'react'; export const useConfirm = (danger = false) => { const context = useContext(ConfirmContext); - return ( - message: React.ReactNode, - callback: () => void | Promise - ) => { + + return (message: ReactNode, callback: () => void | Promise) => { context?.setDangerButton(danger); context?.setContent(message); + context?.setIsConfirming(false); context?.setConfirm(() => async () => { - await callback(); - context?.cancel(); + context?.setIsConfirming(true); + + try { + await callback(); + } finally { + context?.setIsConfirming(false); + context?.cancel(); + } }); }; }; diff --git a/frontend/src/lib/hooks/useMessageFiltersStore.ts b/frontend/src/lib/hooks/useMessageFiltersStore.ts index 8397d41d2..fae45589b 100644 --- a/frontend/src/lib/hooks/useMessageFiltersStore.ts +++ b/frontend/src/lib/hooks/useMessageFiltersStore.ts @@ -2,40 +2,81 @@ import { LOCAL_STORAGE_KEY_PREFIX } from 'lib/constants'; import create from 'zustand'; import { persist } from 'zustand/middleware'; -interface AdvancedFilter { - name: string; +export type AdvancedFiltersType = Record; + +export interface AdvancedFilter { + id: string; value: string; + filterCode: string; } interface MessageFiltersState { - filters: AdvancedFilter[]; - activeFilter?: AdvancedFilter; + filters: AdvancedFiltersType; + notPersistedFilter: AdvancedFilter | undefined; save: (filter: AdvancedFilter) => void; - apply: (filter: AdvancedFilter) => void; - remove: (name: string) => void; - update: (name: string, filter: AdvancedFilter) => void; + nextCursor: string | undefined; + setNextCursor: (str: string | undefined) => void; + replace: (filterId: string, filter: AdvancedFilter) => void; + commit: (filter: AdvancedFilter | undefined) => void; + remove: (id: string) => void; + removeAll: () => void; } +export const selectFilter = + (id?: string) => + ({ filters, notPersistedFilter }: MessageFiltersState) => { + if (!id) return undefined; + + if (filters[id]) return filters[id]; + + if (notPersistedFilter?.id === id) return notPersistedFilter; + + return undefined; + }; + export const useMessageFiltersStore = create()( persist( (set) => ({ - filters: [], + filters: {}, + nextCursor: undefined, + notPersistedFilter: undefined, save: (filter) => set((state) => ({ - filters: [...state.filters, filter], - })), - apply: (filter) => set(() => ({ activeFilter: filter })), - remove: (name) => - set((state) => ({ - filters: state.filters.filter((f) => f.name !== name), + filters: { ...state.filters, [filter.id]: filter }, })), - update: (name, filter) => - set((state) => ({ - filters: state.filters.map((f) => (f.name === name ? filter : f)), + replace: (filterId, filter) => + set((state) => { + const newFilters = { ...state.filters }; + + if (filterId !== filter.id) { + delete newFilters[filterId]; + } + + newFilters[filter.id] = filter; + + return { filters: newFilters }; + }), + commit: (filter) => + set(() => ({ + notPersistedFilter: filter, })), + remove: (id) => + set((state) => { + const filters = { ...state.filters }; + delete filters[id]; + + if (state.notPersistedFilter?.id === id) { + return { filters, notPersistedFilter: undefined }; + } + + return { filters }; + }), + removeAll: () => set(() => ({ filters: {} })), + setNextCursor: (cursor) => set(() => ({ nextCursor: cursor })), }), { name: `${LOCAL_STORAGE_KEY_PREFIX}-message-filters`, + partialize: (state) => ({ filters: state.filters }), } ) ); diff --git a/frontend/src/lib/hooks/useMessagesFilters.ts b/frontend/src/lib/hooks/useMessagesFilters.ts new file mode 100644 index 000000000..fee0b4fdf --- /dev/null +++ b/frontend/src/lib/hooks/useMessagesFilters.ts @@ -0,0 +1,249 @@ +import { useSearchParams } from 'react-router-dom'; +import { PollingMode } from 'generated-sources'; +import { useEffect } from 'react'; +import { Option } from 'react-multi-select-component'; +import { ObjectValues } from 'lib/types'; + +import { convertStrToPollingMode, ModeOptions } from './filterUtils'; +import { + AdvancedFilter, + selectFilter, + useMessageFiltersStore, +} from './useMessageFiltersStore'; + +/** + * @description !! Note !! + * Key value should match + * */ +export const MessagesFilterKeys = { + mode: 'mode', + timestamp: 'timestamp', + keySerde: 'keySerde', + valueSerde: 'valueSerde', + limit: 'limit', + offset: 'offset', + stringFilter: 'stringFilter', + partitions: 'partitions', + smartFilterId: 'smartFilterId', + activeFilterId: 'activeFilterId', + activeFilterNPId: 'activeFilterNPId', // not persisted filter name to indicate the refresh + cursor: 'cursor', + r: 'r', // used tp force refresh of the data +} as const; + +export type MessagesFilterKeysTypes = ObjectValues; + +const PER_PAGE = 100; + +const defaultModeValue = ModeOptions[0].value; + +export function useRefreshData(initSearchParams?: URLSearchParams) { + const [, setSearchParams] = useSearchParams(initSearchParams); + return () => { + setSearchParams((params) => { + if (params.get(MessagesFilterKeys.r)) { + params.delete(MessagesFilterKeys.r); + } else { + params.set(MessagesFilterKeys.r, 'r'); + } + + return params; + }); + }; +} + +export function useMessagesFilters() { + const [searchParams, setSearchParams] = useSearchParams(); + const refreshData = useRefreshData(searchParams); + + useEffect(() => { + setSearchParams((params) => { + params.set(MessagesFilterKeys.limit, PER_PAGE.toString()); + + if (!params.get(MessagesFilterKeys.mode)) { + params.set(MessagesFilterKeys.mode, defaultModeValue); + } + + if (params.get(MessagesFilterKeys.activeFilterNPId)) { + params.delete(MessagesFilterKeys.activeFilterNPId); + params.delete(MessagesFilterKeys.smartFilterId); + } + return params; + }); + }, []); + + /** + * @description + * Params getter + * */ + const mode = + convertStrToPollingMode(searchParams.get(MessagesFilterKeys.mode) || '') || + defaultModeValue; + + const dateParams = searchParams.get(MessagesFilterKeys.timestamp); + + const date = dateParams ? new Date(parseFloat(dateParams)) : null; + + const keySerde = searchParams.get(MessagesFilterKeys.keySerde) || undefined; + const valueSerde = + searchParams.get(MessagesFilterKeys.valueSerde) || undefined; + + const offset = searchParams.get(MessagesFilterKeys.offset) || undefined; + + const search = searchParams.get(MessagesFilterKeys.stringFilter) || ''; + + const partitions = searchParams.getAll(MessagesFilterKeys.partitions); + + const smartFilterId = + searchParams.get(MessagesFilterKeys.activeFilterId) || + searchParams.get(MessagesFilterKeys.activeFilterNPId) || + ''; + + const smartFilter = useMessageFiltersStore(selectFilter(smartFilterId)); + + /** + * @description + * Params setters + * */ + const setMode = (newMode: PollingMode) => { + setSearchParams((params) => { + params.set(MessagesFilterKeys.mode, newMode); + + params.delete(MessagesFilterKeys.offset); + params.delete(MessagesFilterKeys.timestamp); + return params; + }); + }; + + const setTimeStamp = (newDate: Date | null) => { + if (newDate === null) { + setSearchParams((params) => { + params.delete(MessagesFilterKeys.timestamp); + return params; + }); + return; + } + + setSearchParams((params) => { + params.set(MessagesFilterKeys.timestamp, newDate.getTime().toString()); + return params; + }); + }; + + const setKeySerde = (newKeySerde: string) => { + setSearchParams((params) => { + params.set(MessagesFilterKeys.keySerde, newKeySerde); + return params; + }); + }; + + const setValueSerde = (newValueSerde: string) => { + setSearchParams((params) => { + params.set(MessagesFilterKeys.valueSerde, newValueSerde); + return params; + }); + }; + + const setOffsetValue = (newOffsetValue: string) => { + setSearchParams((params) => { + params.set(MessagesFilterKeys.offset, newOffsetValue); + return params; + }); + }; + + const setSearch = (value: string) => { + setSearchParams((params) => { + if (value) { + params.set(MessagesFilterKeys.stringFilter, value); + } else { + params.delete(MessagesFilterKeys.stringFilter); + } + return params; + }); + }; + + const setPartition = (values: Option[]) => { + setSearchParams((params) => { + params.delete(MessagesFilterKeys.partitions); + + values.forEach((option) => { + params.append(MessagesFilterKeys.partitions, option.value); + }); + + return params; + }); + }; + + const setSmartFilter = ( + newFilter: AdvancedFilter | null, + persisted = true + ) => { + if (newFilter === null) { + setSearchParams((params) => { + params.delete(MessagesFilterKeys.smartFilterId); + params.delete(MessagesFilterKeys.activeFilterId); + params.delete(MessagesFilterKeys.activeFilterNPId); + return params; + }); + return; + } + + const { id } = newFilter; + // callback should always capture the latest states not rely on rendering + + const filter = selectFilter(newFilter.id)( + useMessageFiltersStore.getState() + ); + + // setting something that is not in the state + if (!filter) return; + + setSearchParams((params) => { + params.set(MessagesFilterKeys.smartFilterId, filter.filterCode); + params.set( + persisted + ? MessagesFilterKeys.activeFilterId + : MessagesFilterKeys.activeFilterNPId, + id + ); + return params; + }); + }; + + return { + mode, + setMode, + date, + setTimeStamp, + keySerde, + setKeySerde, + valueSerde, + setValueSerde, + offset, + setOffsetValue, + search, + setSearch, + partitions, + setPartition, + smartFilter, + setSmartFilter, + refreshData, + }; +} + +export function useIsMessagesSmartFilterPersisted( + initSearchParams?: URLSearchParams +) { + const [searchParams] = useSearchParams(initSearchParams); + + return !!searchParams.get(MessagesFilterKeys.activeFilterId); +} + +export function useIsLiveMode(initSearchParams?: URLSearchParams) { + const [searchParams] = useSearchParams(initSearchParams); + + return ( + convertStrToPollingMode(searchParams.get(MessagesFilterKeys.mode) || '') === + PollingMode.TAILING + ); +} diff --git a/frontend/src/lib/types.ts b/frontend/src/lib/types.ts new file mode 100644 index 000000000..7bc715840 --- /dev/null +++ b/frontend/src/lib/types.ts @@ -0,0 +1,4 @@ +export type ObjectValues> = T[keyof T]; + +export type WithPartialKey = Omit & + Partial>>; diff --git a/frontend/src/react-app-env.d.ts b/frontend/src/react-app-env.d.ts deleted file mode 100644 index 1f42c255e..000000000 --- a/frontend/src/react-app-env.d.ts +++ /dev/null @@ -1,3 +0,0 @@ -/// -/// -/// diff --git a/frontend/src/redux/reducers/index.ts b/frontend/src/redux/reducers/index.ts index aa5cb69cf..6385e0710 100644 --- a/frontend/src/redux/reducers/index.ts +++ b/frontend/src/redux/reducers/index.ts @@ -1,10 +1,8 @@ import { combineReducers } from '@reduxjs/toolkit'; import loader from 'redux/reducers/loader/loaderSlice'; import schemas from 'redux/reducers/schemas/schemasSlice'; -import topicMessages from 'redux/reducers/topicMessages/topicMessagesSlice'; export default combineReducers({ loader, - topicMessages, schemas, }); diff --git a/frontend/src/redux/reducers/topicMessages/__test__/fixtures.ts b/frontend/src/redux/reducers/topicMessages/__test__/fixtures.ts deleted file mode 100644 index 8f8bce2c2..000000000 --- a/frontend/src/redux/reducers/topicMessages/__test__/fixtures.ts +++ /dev/null @@ -1,29 +0,0 @@ -import { - TopicMessage, - TopicMessageConsuming, - TopicMessageTimestampTypeEnum, -} from 'generated-sources'; - -export const topicMessagePayload: TopicMessage = { - partition: 29, - offset: 14, - timestamp: new Date('2021-07-21T23:25:14.865Z'), - timestampType: TopicMessageTimestampTypeEnum.CREATE_TIME, - key: 'schema-registry', - headers: {}, - content: - '{"host":"schemaregistry1","port":8085,"master_eligibility":true,"scheme":"http","version":1}', -}; - -export const topicMessagePayloadV2: TopicMessage = { - ...topicMessagePayload, - partition: 28, - offset: 88, -}; - -export const topicMessagesMetaPayload: TopicMessageConsuming = { - bytesConsumed: 1830, - elapsedMs: 440, - messagesConsumed: 2301, - isCancelled: false, -}; diff --git a/frontend/src/redux/reducers/topicMessages/__test__/reducer.spec.ts b/frontend/src/redux/reducers/topicMessages/__test__/reducer.spec.ts deleted file mode 100644 index b9fa4cd4e..000000000 --- a/frontend/src/redux/reducers/topicMessages/__test__/reducer.spec.ts +++ /dev/null @@ -1,79 +0,0 @@ -import reducer, { - addTopicMessage, - resetTopicMessages, - updateTopicMessagesMeta, - updateTopicMessagesPhase, -} from 'redux/reducers/topicMessages/topicMessagesSlice'; - -import { - topicMessagePayload, - topicMessagePayloadV2, - topicMessagesMetaPayload, -} from './fixtures'; - -describe('TopicMessages reducer', () => { - it('Adds new message', () => { - const state = reducer( - undefined, - addTopicMessage({ message: topicMessagePayload }) - ); - expect(state.messages.length).toEqual(1); - }); - - it('Adds new message with live tailing one', () => { - const state = reducer( - undefined, - addTopicMessage({ message: topicMessagePayload }) - ); - const modifiedState = reducer( - state, - addTopicMessage({ message: topicMessagePayloadV2, prepend: true }) - ); - expect(modifiedState.messages.length).toEqual(2); - expect(modifiedState.messages).toEqual([ - topicMessagePayloadV2, - topicMessagePayload, - ]); - }); - - it('Adds new message with live tailing off', () => { - const state = reducer( - undefined, - addTopicMessage({ message: topicMessagePayload }) - ); - const modifiedState = reducer( - state, - addTopicMessage({ message: topicMessagePayloadV2 }) - ); - expect(modifiedState.messages.length).toEqual(2); - expect(modifiedState.messages).toEqual([ - topicMessagePayload, - topicMessagePayloadV2, - ]); - }); - - it('reset messages', () => { - const state = reducer( - undefined, - addTopicMessage({ message: topicMessagePayload }) - ); - expect(state.messages.length).toEqual(1); - - const newState = reducer(state, resetTopicMessages()); - expect(newState.messages.length).toEqual(0); - }); - - it('Updates Topic Messages Phase', () => { - const phase = 'Polling'; - - const state = reducer(undefined, updateTopicMessagesPhase(phase)); - expect(state.phase).toEqual(phase); - }); - it('Updates Topic Messages Meta', () => { - const state = reducer( - undefined, - updateTopicMessagesMeta(topicMessagesMetaPayload) - ); - expect(state.meta).toEqual(topicMessagesMetaPayload); - }); -}); diff --git a/frontend/src/redux/reducers/topicMessages/__test__/selectors.spec.ts b/frontend/src/redux/reducers/topicMessages/__test__/selectors.spec.ts deleted file mode 100644 index d324a2d5e..000000000 --- a/frontend/src/redux/reducers/topicMessages/__test__/selectors.spec.ts +++ /dev/null @@ -1,62 +0,0 @@ -import { store } from 'redux/store'; -import * as selectors from 'redux/reducers/topicMessages/selectors'; -import { - initialState, - addTopicMessage, - updateTopicMessagesMeta, - updateTopicMessagesPhase, -} from 'redux/reducers/topicMessages/topicMessagesSlice'; - -import { topicMessagePayload, topicMessagesMetaPayload } from './fixtures'; - -const newTopicMessagePayload = { - ...topicMessagePayload, - timestamp: topicMessagePayload.timestamp.toString(), -}; -describe('TopicMessages selectors', () => { - describe('Initial state', () => { - it('returns empty message array', () => { - expect(selectors.getTopicMessges(store.getState())).toEqual([]); - }); - - it('returns undefined phase', () => { - expect(selectors.getTopicMessgesPhase(store.getState())).toBeUndefined(); - }); - - it('returns initial vesrion of meta', () => { - expect(selectors.getTopicMessgesMeta(store.getState())).toEqual( - initialState.meta - ); - }); - }); - - describe('state', () => { - beforeAll(() => { - store.dispatch( - addTopicMessage({ - message: newTopicMessagePayload, - }) - ); - store.dispatch(updateTopicMessagesPhase('consuming')); - store.dispatch(updateTopicMessagesMeta(topicMessagesMetaPayload)); - }); - - it('returns messages', () => { - expect(selectors.getTopicMessges(store.getState())).toEqual([ - newTopicMessagePayload, - ]); - }); - - it('returns phase', () => { - expect(selectors.getTopicMessgesPhase(store.getState())).toEqual( - 'consuming' - ); - }); - - it('returns ordered versions of schema', () => { - expect(selectors.getTopicMessgesMeta(store.getState())).toEqual( - topicMessagesMetaPayload - ); - }); - }); -}); diff --git a/frontend/src/redux/reducers/topicMessages/selectors.ts b/frontend/src/redux/reducers/topicMessages/selectors.ts deleted file mode 100644 index b2636cdf2..000000000 --- a/frontend/src/redux/reducers/topicMessages/selectors.ts +++ /dev/null @@ -1,30 +0,0 @@ -import { createSelector } from '@reduxjs/toolkit'; -import { RootState, TopicMessagesState } from 'redux/interfaces'; - -const topicMessagesState = ({ topicMessages }: RootState): TopicMessagesState => - topicMessages; - -export const getTopicMessges = createSelector( - topicMessagesState, - ({ messages }) => messages -); - -export const getTopicMessgesPhase = createSelector( - topicMessagesState, - ({ phase }) => phase -); - -export const getTopicMessgesMeta = createSelector( - topicMessagesState, - ({ meta }) => meta -); - -export const getIsTopicMessagesFetching = createSelector( - topicMessagesState, - ({ isFetching }) => isFetching -); - -export const getIsTopicMessagesType = createSelector( - topicMessagesState, - ({ messageEventType }) => messageEventType -); diff --git a/frontend/src/redux/reducers/topicMessages/topicMessagesSlice.ts b/frontend/src/redux/reducers/topicMessages/topicMessagesSlice.ts deleted file mode 100644 index 530a37811..000000000 --- a/frontend/src/redux/reducers/topicMessages/topicMessagesSlice.ts +++ /dev/null @@ -1,57 +0,0 @@ -import { createSlice } from '@reduxjs/toolkit'; -import { TopicMessagesState } from 'redux/interfaces'; -import { TopicMessage } from 'generated-sources'; - -export const initialState: TopicMessagesState = { - messages: [], - meta: { - bytesConsumed: 0, - elapsedMs: 0, - messagesConsumed: 0, - isCancelled: false, - }, - messageEventType: '', - isFetching: false, -}; - -const topicMessagesSlice = createSlice({ - name: 'topicMessages', - initialState, - reducers: { - addTopicMessage: (state, action) => { - const messages: TopicMessage[] = action.payload.prepend - ? [action.payload.message, ...state.messages] - : [...state.messages, action.payload.message]; - - return { - ...state, - messages, - }; - }, - resetTopicMessages: () => initialState, - updateTopicMessagesPhase: (state, action) => { - state.phase = action.payload; - }, - updateTopicMessagesMeta: (state, action) => { - state.meta = action.payload; - }, - setTopicMessagesFetchingStatus: (state, action) => { - state.isFetching = action.payload; - }, - - setMessageEventType: (state, action) => { - state.messageEventType = action.payload; - }, - }, -}); - -export const { - addTopicMessage, - resetTopicMessages, - updateTopicMessagesPhase, - updateTopicMessagesMeta, - setTopicMessagesFetchingStatus, - setMessageEventType, -} = topicMessagesSlice.actions; - -export default topicMessagesSlice.reducer; diff --git a/frontend/src/theme/theme.ts b/frontend/src/theme/theme.ts index 44e19eb5b..084e50f4e 100644 --- a/frontend/src/theme/theme.ts +++ b/frontend/src/theme/theme.ts @@ -35,15 +35,21 @@ const Colors = { '60': '#29A352', }, brand: { + '0': '#FFFFFF', + '3': '#F9FAFA', '5': '#F1F2F3', '10': '#E3E6E8', '15': '#D5DADD', '20': '#C7CED1', '30': '#ABB5BA', '40': '#8F9CA3', - '50': '#2F3639', - '60': '#22282A', - '70': '#171A1C', + '50': '#73848C', + '60': '#5C6970', + '70': '#454F54', + '80': '#2F3639', + '85': '#22282A', + '90': '#171A1C', + '95': '#0B0D0E', }, red: { '10': '#FAD1D1', @@ -203,14 +209,19 @@ const baseTheme = { closeModalIcon: Colors.neutral[25], savedIcon: Colors.brand[50], dropdownArrowIcon: Colors.neutral[50], - git: { + github: { hover: Colors.neutral[90], active: Colors.neutral[70], }, discord: { normal: Colors.neutral[20], hover: Colors.blue[45], - active: Colors.brand[15], + active: '#B8BEF9', + }, + producthunt: { + normal: Colors.neutral[20], + hover: '#FF6154', + active: '#FFBDB8', }, }, textArea: { @@ -335,6 +346,10 @@ export const theme = { color: Colors.brand[50], hoverColor: Colors.brand[60], }, + user: { + color: Colors.brand[70], + hoverColor: Colors.brand[50], + }, hr: { backgroundColor: Colors.neutral[5], }, @@ -391,38 +406,26 @@ export const theme = { button: { primary: { backgroundColor: { - normal: Colors.brand[50], - hover: Colors.brand[70], - active: Colors.brand[60], - disabled: Colors.neutral[5], + normal: Colors.brand[80], + hover: Colors.brand[90], + active: Colors.brand[70], + disabled: Colors.brand[50], }, color: { - normal: Colors.neutral[0], - disabled: Colors.neutral[30], - }, - invertedColors: { - normal: Colors.brand[50], - hover: Colors.brand[60], - active: Colors.brand[60], + normal: Colors.brand[0], + disabled: Colors.brand[30], }, }, secondary: { backgroundColor: { normal: Colors.brand[5], hover: Colors.brand[10], - active: Colors.brand[30], - disabled: Colors.neutral[5], + active: Colors.brand[15], + disabled: Colors.brand[5], }, color: { - normal: Colors.neutral[90], - disabled: Colors.neutral[30], - }, - isActiveColor: Colors.neutral[0], - invertedColors: { - normal: Colors.neutral[50], - hover: Colors.neutral[70], - active: Colors.neutral[90], - disabled: Colors.neutral[75], + normal: Colors.brand[90], + disabled: Colors.brand[30], }, }, danger: { @@ -433,13 +436,8 @@ export const theme = { disabled: Colors.red[20], }, color: { - normal: Colors.neutral[0], - disabled: Colors.neutral[0], - }, - invertedColors: { - normal: Colors.brand[50], - hover: Colors.brand[60], - active: Colors.brand[60], + normal: Colors.brand[0], + disabled: Colors.red[10], }, }, height: { @@ -452,11 +450,6 @@ export const theme = { M: '14px', L: '16px', }, - border: { - normal: Colors.neutral[50], - hover: Colors.neutral[70], - active: Colors.neutral[90], - }, }, chips: { backgroundColor: { @@ -781,6 +774,10 @@ export const darkTheme: ThemeType = { color: Colors.brand[50], hoverColor: Colors.brand[30], }, + user: { + color: Colors.brand[20], + hoverColor: Colors.brand[50], + }, hr: { backgroundColor: Colors.neutral[80], }, @@ -838,37 +835,25 @@ export const darkTheme: ThemeType = { primary: { backgroundColor: { normal: Colors.brand[10], - hover: Colors.brand[5], + hover: Colors.brand[0], active: Colors.brand[20], - disabled: Colors.brand[60], + disabled: Colors.brand[50], }, color: { - normal: Colors.neutral[70], - disabled: Colors.neutral[60], - }, - invertedColors: { - normal: Colors.brand[30], - hover: Colors.brand[60], - active: Colors.brand[60], + normal: Colors.brand[90], + disabled: Colors.brand[70], }, }, secondary: { backgroundColor: { - normal: Colors.brand[50], + normal: Colors.brand[80], hover: Colors.brand[70], active: Colors.brand[60], - disabled: Colors.neutral[75], + disabled: Colors.brand[80], }, color: { - normal: Colors.neutral[0], - disabled: Colors.neutral[60], - }, - isActiveColor: Colors.neutral[90], - invertedColors: { - normal: Colors.neutral[50], - hover: Colors.neutral[70], - active: Colors.neutral[90], - disabled: Colors.neutral[75], + normal: Colors.brand[0], + disabled: Colors.brand[70], }, }, danger: { @@ -879,13 +864,8 @@ export const darkTheme: ThemeType = { disabled: Colors.red[20], }, color: { - normal: Colors.neutral[0], - disabled: Colors.neutral[0], - }, - invertedColors: { - normal: Colors.brand[50], - hover: Colors.brand[60], - active: Colors.brand[60], + normal: Colors.brand[0], + disabled: Colors.red[10], }, }, height: { @@ -898,11 +878,6 @@ export const darkTheme: ThemeType = { M: '14px', L: '16px', }, - border: { - normal: Colors.neutral[50], - hover: Colors.neutral[70], - active: Colors.neutral[90], - }, }, chips: { backgroundColor: { @@ -1202,15 +1177,19 @@ export const darkTheme: ThemeType = { sunIcon: Colors.neutral[0], infoIcon: Colors.neutral[70], savedIcon: Colors.brand[30], - git: { - ...baseTheme.icons.git, + github: { + ...baseTheme.icons.github, hover: Colors.neutral[70], - active: Colors.neutral[90], + active: Colors.neutral[85], }, discord: { ...baseTheme.icons.discord, normal: Colors.neutral[30], }, + producthunt: { + ...baseTheme.icons.producthunt, + normal: Colors.neutral[5], + }, }, textArea: { ...baseTheme.textArea, diff --git a/serde-api/pom.xml b/serde-api/pom.xml index 9452dd3ac..3c2e3b3a1 100644 --- a/serde-api/pom.xml +++ b/serde-api/pom.xml @@ -1,128 +1,129 @@ + xmlns="http://maven.apache.org/POM/4.0.0" + xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> - 4.0.0 - jar + 4.0.0 + jar - - 17 - 17 - + + 17 + 17 + - - - ossrh - https://s01.oss.sonatype.org/content/repositories/snapshots - - - ossrh - https://s01.oss.sonatype.org/service/local/staging/deploy/maven2/ - - + + + ossrh + https://s01.oss.sonatype.org/content/repositories/snapshots + + + ossrh + https://s01.oss.sonatype.org/service/local/staging/deploy/maven2/ + + - kafbat-ui-serde-api - kafbat-ui-serde-api - http://github.com/kafbat/kafka-ui - - - The Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt - - - - - Kafbat - maintainers@kafbat.io - Kafbat - https://kafbat.io/ - - - - scm:git:git://github.com/kafbat/kafka-ui.git - scm:git:ssh://github.com:kafbat/kafka-ui.git - https://github.com/kafbat/kafka-ui - - io.kafbat.ui - serde-api - 1.0.0 + kafbat-ui-serde-api + kafbat-ui-serde-api + http://github.com/kafbat/kafka-ui + + + The Apache License, Version 2.0 + http://www.apache.org/licenses/LICENSE-2.0.txt + + + + + Kafbat + maintainers@kafbat.io + Kafbat + https://kafbat.io/ + + + + scm:git:git://github.com/kafbat/kafka-ui.git + scm:git:ssh://github.com:kafbat/kafka-ui.git + https://github.com/kafbat/kafka-ui + + io.kafbat.ui + serde-api + 1.0.0 - - - - - org.apache.maven.plugins - maven-install-plugin - 2.5.2 - - - org.apache.maven.plugins - maven-jar-plugin - 3.3.0 - - - org.sonatype.plugins - nexus-staging-maven-plugin - 1.6.13 - true - - ossrh - https://s01.oss.sonatype.org/ - true - - - - org.apache.maven.plugins - maven-source-plugin - 2.2.1 - - - attach-sources - - jar-no-fork - - - - - - org.apache.maven.plugins - maven-javadoc-plugin - - 8 - - 3.5.0 - - - attach-javadocs - - jar - - - - - - org.apache.maven.plugins - maven-gpg-plugin - - - sign-artifacts - verify - - sign - - - - - - - --pinentry-mode - loopback - - - - - - + + + + + org.apache.maven.plugins + maven-install-plugin + 2.5.2 + + + org.apache.maven.plugins + maven-jar-plugin + 3.3.0 + + + org.sonatype.central + central-publishing-maven-plugin + 0.4.0 + true + + central + true + true + + + + org.apache.maven.plugins + maven-source-plugin + 2.2.1 + + + attach-sources + + jar-no-fork + + + + + + org.apache.maven.plugins + maven-javadoc-plugin + + 8 + + 3.5.0 + + + attach-javadocs + + jar + + + + + + org.apache.maven.plugins + maven-gpg-plugin + 3.1.0 + + + sign-artifacts + verify + + sign + + + + + + + --pinentry-mode + loopback + + + + + + diff --git a/settings.xml b/settings.xml index 7935a5f53..e3dd1641d 100644 --- a/settings.xml +++ b/settings.xml @@ -1,7 +1,7 @@ - ossrh + central ${server.username} ${server.password} @@ -14,4 +14,4 @@ - \ No newline at end of file +